
Co-authored-by: fkuner <784819644@qq.com> Co-authored-by: fforkboat <fforkboat@gmail.com>
44053 lines
2.1 MiB
Executable File
44053 lines
2.1 MiB
Executable File
/**
|
|
* Copyright (c) 2021 OceanBase
|
|
* OceanBase CE is licensed under Mulan PubL v2.
|
|
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
|
* You may obtain a copy of Mulan PubL v2 at:
|
|
* http://license.coscl.org.cn/MulanPubL-2.0
|
|
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
|
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
|
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
|
* See the Mulan PubL v2 for more details.
|
|
*/
|
|
|
|
#define USING_LOG_PREFIX RS
|
|
#include "rootserver/ob_ddl_service.h"
|
|
|
|
#include "lib/oblog/ob_log.h"
|
|
#include "lib/time/ob_time_utility.h"
|
|
#include "lib/string/ob_strings.h"
|
|
#include "lib/string/ob_sql_string.h"
|
|
#include "lib/hash/ob_placement_hashset.h"
|
|
#include "lib/hash/ob_placement_hashmap.h"
|
|
#include "lib/hash/ob_hashmap.h"
|
|
#include "lib/worker.h"
|
|
#include "lib/container/ob_array_iterator.h"
|
|
#include "lib/mysqlclient/ob_mysql_transaction.h"
|
|
#include "share/ob_srv_rpc_proxy.h"
|
|
#include "common/ob_timeout_ctx.h"
|
|
#include "common/rowkey/ob_rowkey.h"
|
|
#include "share/ob_define.h"
|
|
#include "share/inner_table/ob_inner_table_schema.h"
|
|
#include "share/schema/ob_table_schema.h"
|
|
#include "share/schema/ob_multi_version_schema_service.h"
|
|
#include "share/schema/ob_part_mgr_util.h"
|
|
#include "share/schema/ob_schema_printer.h"
|
|
#include "share/schema/ob_schema_utils.h"
|
|
#include "share/schema/ob_ddl_sql_service.h"
|
|
#include "share/schema/ob_security_audit_sql_service.h"
|
|
#include "share/schema/ob_user_sql_service.h"
|
|
#include "share/schema/ob_schema_service_sql_impl.h"
|
|
#include "share/ob_autoincrement_service.h"
|
|
#include "share/ob_tablet_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_fts_index_builder_util.h"
|
|
#include "share/sequence/ob_sequence_ddl_proxy.h"
|
|
#include "share/ob_schema_status_proxy.h"
|
|
#include "share/ob_tenant_mgr.h"
|
|
#include "lib/worker.h"
|
|
#include "share/ob_schema_status_proxy.h"
|
|
#include "share/ob_global_stat_proxy.h"
|
|
#include "share/ob_freeze_info_proxy.h"
|
|
#include "share/ob_service_epoch_proxy.h"
|
|
#include "rootserver/standby/ob_standby_service.h" // ObStandbyService
|
|
#include "sql/resolver/ob_stmt_type.h"
|
|
#include "sql/resolver/ddl/ob_ddl_resolver.h"
|
|
#include "sql/resolver/expr/ob_raw_expr_modify_column_name.h"
|
|
#include "sql/printer/ob_raw_expr_printer.h"
|
|
#include "share/ob_all_server_tracer.h"
|
|
#include "ob_zone_manager.h"
|
|
#include "rootserver/ob_schema2ddl_sql.h"
|
|
#include "rootserver/ob_unit_manager.h"
|
|
#include "rootserver/ob_index_builder.h"
|
|
#include "rootserver/ob_zone_unit_provider.h"
|
|
#include "rootserver/ob_rs_job_table_operator.h"
|
|
#include "rootserver/ob_ddl_sql_generator.h"
|
|
#include "rootserver/ddl_task/ob_ddl_task.h"
|
|
#include "rootserver/ob_ddl_help.h"
|
|
#include "rootserver/ob_locality_util.h"
|
|
#include "rootserver/ob_root_service.h"
|
|
#include "rootserver/ob_vertical_partition_builder.h"
|
|
#include "rootserver/ddl_task/ob_constraint_task.h"
|
|
#include "rootserver/ddl_task/ob_ddl_retry_task.h"
|
|
#include "share/ob_freeze_info_manager.h"
|
|
#include "rootserver/freeze/ob_major_freeze_helper.h"
|
|
#include "rootserver/ob_alter_primary_zone_checker.h"
|
|
#include "rootserver/ob_tenant_thread_helper.h"//get_zone_priority
|
|
#include "lib/utility/ob_tracepoint.h"
|
|
#include "observer/ob_server_struct.h"
|
|
#include "storage/tx/ob_ts_mgr.h"
|
|
#include "storage/tx/ob_i_ts_source.h"
|
|
#include "sql/engine/px/ob_px_util.h"
|
|
#include "share/ob_unit_replica_counter.h"
|
|
#include "sql/ob_sql_utils.h"
|
|
#include "observer/omt/ob_tenant_timezone_mgr.h"
|
|
#include "share/schema/ob_schema_mgr.h"
|
|
#include "rootserver/ob_tablet_creator.h"
|
|
#include "rootserver/ob_table_creator.h"
|
|
#include "rootserver/ob_balance_group_ls_stat_operator.h"
|
|
#include "share/ob_share_util.h"
|
|
#include "share/ob_leader_election_waiter.h"
|
|
#include "rootserver/ob_tablet_drop.h"
|
|
#include "share/schema/ob_context_mgr.h"
|
|
#include "share/schema/ob_context_ddl_proxy.h"
|
|
#include "share/ob_global_context_operator.h"
|
|
#include "share/ls/ob_ls_creator.h"
|
|
#include "share/ls/ob_ls_operator.h"
|
|
#include "ob_lob_meta_builder.h"
|
|
#include "ob_lob_piece_builder.h"
|
|
#include "share/ls/ob_ls_life_manager.h"//ObLSLifeAgentManager
|
|
#include "share/restore/ob_physical_restore_table_operator.h"//ObPhysicalRestoreTableOperator
|
|
#include "storage/tablelock/ob_table_lock_rpc_client.h"
|
|
#include "storage/ddl/ob_ddl_lock.h"
|
|
#include "rootserver/restore/ob_restore_util.h"//insert_user_tenant_restore_job
|
|
#include "logservice/palf/palf_base_info.h"//PalfBaseInfo
|
|
#include "logservice/data_dictionary/ob_data_dict_storager.h" // ObDataDictStorage
|
|
#include "share/scn.h"
|
|
#include "share/backup/ob_backup_config.h" // ObBackupConfigParserMgr
|
|
#include "share/schema/ob_mlog_info.h"
|
|
#ifdef OB_BUILD_ARBITRATION
|
|
#include "share/arbitration_service/ob_arbitration_service_table_operator.h"
|
|
#include "share/arbitration_service/ob_arbitration_service_utils.h" // ObArbitrationServiceUtils
|
|
#endif
|
|
#ifdef OB_BUILD_ORACLE_PL
|
|
#include "pl/sys_package/ob_dbms_audit_mgmt.h" // ObDbmsAuditMgmt
|
|
#endif
|
|
#include "share/backup/ob_log_restore_config.h"//ObLogRestoreSourceServiceConfigParser
|
|
#include "storage/tx_storage/ob_ls_map.h"
|
|
#include "storage/tx_storage/ob_ls_service.h"
|
|
#include "storage/tablelock/ob_lock_inner_connection_util.h"
|
|
#include "storage/compaction/ob_compaction_schedule_util.h"
|
|
#include "share/schema/ob_mview_info.h"
|
|
#include "storage/mview/ob_mview_sched_job_utils.h"
|
|
#include "storage/vector_index/ob_vector_index_sched_job_utils.h"
|
|
#include "rootserver/restore/ob_tenant_clone_util.h"
|
|
#include "rootserver/ob_split_partition_helper.h"
|
|
#include "rootserver/mview/ob_mview_dependency_service.h"
|
|
#include "rootserver/parallel_ddl/ob_ddl_helper.h"
|
|
#include "storage/ddl/ob_ddl_alter_auto_part_attr.h"
|
|
#include "share/tablet/ob_tablet_to_ls_operator.h"
|
|
#include "share/tablet/ob_tablet_to_table_history_operator.h"
|
|
#include "src/share/ob_vec_index_builder_util.h"
|
|
#include "share/vector_index/ob_vector_index_util.h"
|
|
#include "rootserver/direct_load/ob_direct_load_partition_exchange.h"
|
|
#include "storage/column_store/ob_column_store_replica_util.h"
|
|
|
|
namespace oceanbase
|
|
{
|
|
using namespace common;
|
|
using namespace share;
|
|
using namespace obrpc;
|
|
using namespace storage;
|
|
using namespace palf;
|
|
namespace rootserver
|
|
{
|
|
#define MODIFY_LOCALITY_NOT_ALLOWED() \
|
|
do { \
|
|
ret = OB_OP_NOT_ALLOW; \
|
|
LOG_WARN("modify locality is not allowed", K(ret)); \
|
|
} while (0)
|
|
// The input of value must be a string
|
|
#define SET_TENANT_VARIABLE(sysvar_id, value) \
|
|
if (OB_SUCC(ret)) {\
|
|
int64_t store_idx = OB_INVALID_INDEX; \
|
|
if (OB_FAIL(ObSysVarFactory::calc_sys_var_store_idx(sysvar_id, store_idx))) { \
|
|
LOG_WARN("failed to calc sys var store idx", KR(ret), K(sysvar_id)); \
|
|
} else if (OB_UNLIKELY(store_idx < 0 \
|
|
|| store_idx >= ObSysVarFactory::ALL_SYS_VARS_COUNT)) { \
|
|
ret = OB_ERR_UNEXPECTED; \
|
|
LOG_WARN("got store_idx is invalid", K(ret), K(store_idx)); \
|
|
} else if (OB_FAIL(sys_params[store_idx].init( \
|
|
sys_variable_schema.get_tenant_id(),\
|
|
ObSysVariables::get_name(store_idx),\
|
|
ObSysVariables::get_type(store_idx),\
|
|
value,\
|
|
ObSysVariables::get_min(store_idx),\
|
|
ObSysVariables::get_max(store_idx),\
|
|
ObSysVariables::get_info(store_idx),\
|
|
ObSysVariables::get_flags(store_idx)))) {\
|
|
LOG_WARN("failed to set tenant variable", \
|
|
KR(ret), K(value), K(sysvar_id), K(store_idx));\
|
|
}\
|
|
}
|
|
// Convert macro integer to string for setting into system variable
|
|
#define VAR_INT_TO_STRING(buf, value) \
|
|
if (OB_SUCC(ret)) {\
|
|
if (OB_FAIL(databuff_printf(buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%d", static_cast<int>(value)))) {\
|
|
LOG_WARN("failed to print value in buf", K(value), K(ret));\
|
|
}\
|
|
}
|
|
#define VAR_UINT_TO_STRING(buf, value) \
|
|
if (OB_SUCC(ret)) {\
|
|
if (OB_FAIL(databuff_printf(buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%lu", static_cast<uint64_t>(value)))) {\
|
|
LOG_WARN("failed to print value in buf", K(value), K(ret));\
|
|
}\
|
|
}
|
|
|
|
#define GRANT_SYS_ROLE_NUM 2 /* len of role array is 2 */
|
|
#define GRANT_ROLE_MIN_ROLE_NUM 3 /* min len of role array is 3 */
|
|
|
|
ObDDLService::ObDDLService()
|
|
: inited_(false),
|
|
stopped_(false),
|
|
rpc_proxy_(NULL),
|
|
common_rpc_(NULL),
|
|
sql_proxy_(NULL),
|
|
schema_service_(NULL),
|
|
lst_operator_(NULL),
|
|
zone_mgr_(NULL),
|
|
unit_mgr_(NULL),
|
|
snapshot_mgr_(NULL),
|
|
ddl_lock_(),
|
|
index_name_checker_(),
|
|
non_partitioned_tablet_allocator_()
|
|
{
|
|
}
|
|
|
|
int ObDDLService::init(obrpc::ObSrvRpcProxy &rpc_proxy,
|
|
obrpc::ObCommonRpcProxy &common_rpc,
|
|
common::ObMySQLProxy &sql_proxy,
|
|
share::schema::ObMultiVersionSchemaService &schema_service,
|
|
share::ObLSTableOperator &lst_operator,
|
|
ObZoneManager &zone_mgr,
|
|
ObUnitManager &unit_mgr,
|
|
ObSnapshotInfoManager &snapshot_mgr)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (inited_) {
|
|
ret = OB_INIT_TWICE;
|
|
LOG_WARN("init twice", KR(ret));
|
|
} else if (OB_FAIL(index_name_checker_.init(sql_proxy))) {
|
|
LOG_WARN("fail to init index name checker", KR(ret));
|
|
} else if (OB_FAIL(non_partitioned_tablet_allocator_.init(sql_proxy))) {
|
|
LOG_WARN("fail to init non partitioned tablet allocator", KR(ret));
|
|
} else {
|
|
rpc_proxy_ = &rpc_proxy;
|
|
common_rpc_ = &common_rpc;
|
|
sql_proxy_ = &sql_proxy;
|
|
schema_service_ = &schema_service;
|
|
lst_operator_ = &lst_operator;
|
|
zone_mgr_ = &zone_mgr;
|
|
unit_mgr_ = &unit_mgr;
|
|
snapshot_mgr_ = &snapshot_mgr;
|
|
stopped_ = false;
|
|
inited_ = true;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_zones_in_region(
|
|
const common::ObRegion ®ion,
|
|
const common::ObIArray<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;
|
|
}
|
|
|
|
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_restore = false;
|
|
bool use_local = false;
|
|
int64_t version_in_inner_table = OB_INVALID_VERSION;
|
|
ObRefreshSchemaStatus schema_status;
|
|
if (OB_INVALID_TENANT_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is null", K(ret));
|
|
} else if (OB_FAIL(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_restore && OB_SYS_TENANT_ID != tenant_id) {
|
|
ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_;
|
|
if (OB_ISNULL(schema_status_proxy)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_status_proxy is null", KR(ret));
|
|
} else if (OB_FAIL(schema_status_proxy->get_refresh_schema_status(tenant_id, schema_status))) {
|
|
LOG_WARN("failed to get tenant refresh schema status", KR(ret), K(tenant_id));
|
|
} else if (OB_INVALID_VERSION == schema_status.readable_schema_version_) {
|
|
// The second of physical recovery, after reset schema status, modify_schema can be modified
|
|
use_local = false;
|
|
} else if (is_restore) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("tenant is still restoring, ddl not supported", KR(ret), K(tenant_id), K(schema_status));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (use_local) {
|
|
// Only for the failover/switchover stage of the standalone cluster
|
|
if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else {
|
|
LOG_INFO("use local tenant schema guard", K(ret), K(tenant_id));
|
|
}
|
|
} else {
|
|
// 1. the normal tenants do DDL in primary cluster.
|
|
// 2. restore tenant is in modify_schema stage in primary cluster.
|
|
schema_status.tenant_id_ = tenant_id;
|
|
// Ensure that the user tenant schema is updated to the latest
|
|
if (OB_FAIL(schema_service_->get_schema_version_in_inner_table(*sql_proxy_, schema_status, version_in_inner_table))) {
|
|
LOG_WARN("fail to get latest schema version in inner table", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard, version_in_inner_table))) {
|
|
if (OB_SCHEMA_EAGAIN == ret) {
|
|
int t_ret = OB_SUCCESS;
|
|
ObArray<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::get_tenant_schema_guard_with_version_in_inner_table(
|
|
const uint64_t src_tenant_id,
|
|
const uint64_t dst_tenant_id,
|
|
share::schema::ObSchemaGetterGuard &hold_buf_src_tenant_schema_guard,
|
|
share::schema::ObSchemaGetterGuard &hold_buf_dst_tenant_schema_guard,
|
|
share::schema::ObSchemaGetterGuard *&src_tenant_schema_guard,
|
|
share::schema::ObSchemaGetterGuard *&dst_tenant_schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
src_tenant_schema_guard = nullptr;
|
|
dst_tenant_schema_guard = nullptr;
|
|
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == src_tenant_id || OB_INVALID_TENANT_ID == dst_tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", K(ret), K(src_tenant_id), K(dst_tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(dst_tenant_id, hold_buf_dst_tenant_schema_guard))) {
|
|
LOG_WARN("get tenant schema guard failed", K(dst_tenant_id));
|
|
} else if (src_tenant_id == dst_tenant_id) {
|
|
src_tenant_schema_guard = &hold_buf_dst_tenant_schema_guard;
|
|
dst_tenant_schema_guard = &hold_buf_dst_tenant_schema_guard;
|
|
} else {
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(src_tenant_id, hold_buf_src_tenant_schema_guard))) {
|
|
LOG_WARN("get tenant schema guard failed", K(src_tenant_id));
|
|
} else {
|
|
src_tenant_schema_guard = &hold_buf_src_tenant_schema_guard;
|
|
dst_tenant_schema_guard = &hold_buf_dst_tenant_schema_guard;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_UNLIKELY(nullptr == src_tenant_schema_guard || nullptr == dst_tenant_schema_guard)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", K(ret), K(src_tenant_id), K(dst_tenant_id), KP(src_tenant_schema_guard), KP(dst_tenant_schema_guard));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_tenant_in_alter_locality(
|
|
const uint64_t tenant_id,
|
|
bool &in_alter_locality)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
share::schema::ObSchemaGetterGuard schema_guard;
|
|
const share::schema::ObTenantSchema *tenant_schema = NULL;
|
|
in_alter_locality = false;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("not init", K(ret));
|
|
} else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("tenant not exists", K(ret), K(tenant_id));
|
|
} else if (OB_UNLIKELY(NULL == tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", K(ret), K(tenant_id));
|
|
} else if (!tenant_schema->get_previous_locality_str().empty()) {
|
|
// previous locality is not null, alter locality is processing
|
|
in_alter_locality = true;
|
|
} else {
|
|
// do nothing
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_user_tables(
|
|
const bool if_not_exist,
|
|
const ObString &ddl_stmt_str,
|
|
const ObErrorInfo &error_info,
|
|
ObIArray<ObTableSchema> &table_schemas,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const obrpc::ObSequenceDDLArg &sequence_ddl_arg,
|
|
const uint64_t last_replay_log_id,
|
|
const ObIArray<ObDependencyInfo> *dep_infos,
|
|
ObIArray<ObMockFKParentTableSchema> &mock_fk_parent_table_schema_array,
|
|
int64_t &ddl_task_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ddl_task_id = 0;
|
|
RS_TRACE(create_user_tables_begin);
|
|
uint64_t tenant_id = OB_INVALID_TENANT_ID;
|
|
bool have_duplicate_table = false;
|
|
bool is_compatible = false;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("not init", K(ret));
|
|
} else if (table_schemas.count() < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schemas have no element", K(ret));
|
|
} else {
|
|
tenant_id = table_schemas.at(0).get_tenant_id();
|
|
have_duplicate_table = table_schemas.at(0).is_duplicate_table();
|
|
// 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 (!have_duplicate_table) {
|
|
// do nothing
|
|
} else if (OB_FAIL(ObShareUtil::check_compat_version_for_readonly_replica(tenant_id, is_compatible))) {
|
|
LOG_WARN("fail to check compat version for duplicate log stream", KR(ret), K(tenant_id));
|
|
} else if (!is_compatible) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("duplicate table is not supported below 4.2", KR(ret), K(tenant_id));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "create duplicate table below 4.2");
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
//do nothing
|
|
} else if (OB_FAIL(create_tables_in_trans(if_not_exist, ddl_stmt_str, error_info, table_schemas,
|
|
sequence_ddl_arg,
|
|
last_replay_log_id, dep_infos, mock_fk_parent_table_schema_array,
|
|
ddl_task_id))) {
|
|
LOG_WARN("create_tables_in_trans failed", K(ret));
|
|
}
|
|
RS_TRACE_EXT(create_user_tables_end, OB_ID(ret), ret);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_inner_expr_index(ObMySQLTransaction &trans,
|
|
const ObTableSchema &orig_table_schema,
|
|
const uint64_t tenant_data_version,
|
|
ObTableSchema &new_table_schema,
|
|
ObIArray<ObColumnSchemaV2*> &new_columns,
|
|
ObTableSchema &index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_table_exist(index_schema))) {
|
|
if (OB_ERR_TABLE_EXIST != ret) {
|
|
LOG_WARN("check_table_exist failed", K(index_schema), K(ret));
|
|
}
|
|
} else {
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
uint64_t new_table_id = index_schema.get_table_id();
|
|
if (OB_UNLIKELY(NULL == schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_WARN("schema service must be not null", K(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(
|
|
index_schema.get_tenant_id(), new_table_id))) {
|
|
LOG_WARN("failed to fetch_new_table_id", K(ret));
|
|
} else {
|
|
index_schema.set_table_id(new_table_id);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_columns.count(); ++i) {
|
|
ObColumnSchemaV2 *new_column_schema = new_columns.at(i);
|
|
if (OB_ISNULL(new_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("new column schema is null");
|
|
} else if (OB_FAIL(ddl_operator.insert_single_column(
|
|
trans, new_table_schema, *new_column_schema))) {
|
|
LOG_WARN("failed to create table schema, ", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ddl_operator.alter_table_options(schema_guard,
|
|
new_table_schema,
|
|
orig_table_schema,
|
|
false,
|
|
trans))) {
|
|
LOG_WARN("alter table options failed", K(ret), K(new_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.create_table(
|
|
index_schema, trans, nullptr/*ddl_stmt_str*/, true, false))) {
|
|
// record the create index operation when index enables rather than schema generates.
|
|
LOG_WARN("failed to create index schema", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("fail to get last schema_version", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans,
|
|
tenant_id, index_schema.get_table_id(), last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!", K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && index_schema.has_tablet()
|
|
&& OB_FAIL(create_index_tablet(index_schema, trans, schema_guard,
|
|
true/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("fail to create_index_tablet", KR(ret), K(index_schema));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_global_index(
|
|
ObMySQLTransaction &trans,
|
|
const obrpc::ObCreateIndexArg &arg,
|
|
const share::schema::ObTableSchema &table_schema,
|
|
const uint64_t tenant_data_version,
|
|
share::schema::ObTableSchema &index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(table_schema.check_create_index_on_hidden_primary_key(index_schema))) {
|
|
LOG_WARN("fail to check create global index on table", K(ret), K(index_schema));
|
|
} else if (OB_FAIL(create_index_table(arg, tenant_data_version, index_schema, trans))) {
|
|
LOG_WARN("fail to create global index", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_global_inner_expr_index(
|
|
ObMySQLTransaction &trans,
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
const uint64_t tenant_data_version,
|
|
share::schema::ObTableSchema &new_table_schema,
|
|
common::ObIArray<share::schema::ObColumnSchemaV2*> &new_columns,
|
|
share::schema::ObTableSchema &index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(create_inner_expr_index(trans, orig_table_schema,
|
|
tenant_data_version, new_table_schema, new_columns, index_schema))) {
|
|
LOG_WARN("fail to create inner expr index", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// create_index_table is used by create index
|
|
int ObDDLService::create_index_table(
|
|
const obrpc::ObCreateIndexArg &arg,
|
|
const uint64_t tenant_data_version,
|
|
ObTableSchema &table_schema,
|
|
ObMySQLTransaction &sql_trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
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(schema_service->fetch_new_table_id(
|
|
table_schema.get_tenant_id(), new_table_id))) {
|
|
LOG_WARN("fail to fetch new table id", K(ret));
|
|
} else {
|
|
table_schema.set_table_id(new_table_id);
|
|
// zone_list, resource_pool_list, primary_zone not set, copy from all_database_schema
|
|
if (OB_FAIL(schema_guard.get_database_schema(
|
|
tenant_id, table_schema.get_database_id(), database_schema))) {
|
|
LOG_WARN("get_database_schema failed", K(tenant_id),
|
|
K(table_schema.get_database_id()), K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database schema should not be null", K(ret));
|
|
} else if (!arg.is_inner_ && database_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not create index in recyclebin", K(ret), K(*database_schema));
|
|
} else {} // no more to do
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// For create index operation, generate ddl_stmt_str when index enables, but
|
|
// for alter table add index operation, keep generating ddl_stmt_str same as 3.x while generating index schema.
|
|
if (OB_FAIL(create_index_or_mlog_table_in_trans(table_schema,
|
|
nullptr/* ddl_stmt_str */, &sql_trans, schema_guard, true/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("create_table_in_trans failed", KR(ret), K(arg), K(table_schema));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_mlog_table(
|
|
ObMySQLTransaction &sql_trans,
|
|
const obrpc::ObCreateMLogArg &arg,
|
|
const uint64_t tenant_data_version,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObTableSchema &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t new_table_id = table_schema.get_table_id();
|
|
uint64_t tenant_id = table_schema.get_tenant_id();
|
|
ObSchemaService *schema_service = nullptr;
|
|
const ObDatabaseSchema *database_schema = nullptr;
|
|
bool is_oracle_mode = false;
|
|
ObArray<ObSchemaType> conflict_schema_types;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(check_table_exist(table_schema))) {
|
|
if (OB_ERR_TABLE_EXIST != ret) {
|
|
LOG_WARN("failed to check table exist", KR(ret), K(table_schema));
|
|
} else {
|
|
ret = OB_ERR_MLOG_EXIST;
|
|
LOG_WARN("a materialized view log already exists on table",
|
|
KR(ret), K(arg.table_name_));
|
|
LOG_USER_ERROR(OB_ERR_MLOG_EXIST, to_cstring(arg.table_name_));
|
|
}
|
|
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
|
|
tenant_id, is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode with tenant id", KR(ret), K(tenant_id));
|
|
} else if (is_oracle_mode && OB_FAIL(schema_guard.check_oracle_object_exist(
|
|
tenant_id, table_schema.get_database_id(), table_schema.get_table_name_str(),
|
|
TABLE_SCHEMA, INVALID_ROUTINE_TYPE, false/*if_not_exist*/, conflict_schema_types))) {
|
|
LOG_WARN("failed to check oracle object exist", KR(ret));
|
|
} else if (conflict_schema_types.count() > 0) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("mlog name is already used by an existing object in oralce mode",
|
|
KR(ret), K(table_schema.get_table_name_str()), K(conflict_schema_types));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(
|
|
tenant_id, table_schema.get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get database schema",
|
|
KR(ret), K(tenant_id), K(table_schema.get_database_id()));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database schema should not be null",
|
|
KR(ret), K(tenant_id), K(table_schema.get_database_id()));
|
|
} else if (database_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("cannot create mlog in recyclebin", KR(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())){
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema sevice cannot be null", KR(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(
|
|
table_schema.get_tenant_id(), new_table_id))) {
|
|
LOG_WARN("failed to fetch new table id", KR(ret));
|
|
} else if (OB_FALSE_IT(table_schema.set_table_id(new_table_id))) {
|
|
} else if (is_oracle_mode) {
|
|
// add pk constraint
|
|
ObArenaAllocator allocator("CreateMlog");
|
|
ObString cst_name;
|
|
ObConstraint cst;
|
|
if (OB_FAIL(ObTableSchema::create_cons_name_automatically(
|
|
cst_name,
|
|
table_schema.get_table_name_str(),
|
|
allocator,
|
|
CONSTRAINT_TYPE_PRIMARY_KEY,
|
|
is_oracle_mode))) {
|
|
LOG_WARN("failed to create cons name automatically", KR(ret));
|
|
} else if (cst_name.length() > OB_MAX_CONSTRAINT_NAME_LENGTH_ORACLE) {
|
|
ret = OB_ERR_TOO_LONG_IDENT;
|
|
LOG_WARN("constraint_name length overflow", KR(ret), K(cst_name.length()));
|
|
} else if (OB_FAIL(cst.set_constraint_name(cst_name))) {
|
|
LOG_WARN("failed to set constraint name", KR(ret));
|
|
} else {
|
|
uint64_t new_cst_id = OB_INVALID_ID;
|
|
bool cst_name_exist = false;
|
|
cst.set_name_generated_type(GENERATED_TYPE_SYSTEM);
|
|
cst.set_constraint_type(CONSTRAINT_TYPE_PRIMARY_KEY);
|
|
cst.set_tenant_id(tenant_id);
|
|
cst.set_table_id(table_schema.get_table_id());
|
|
if (OB_FAIL(schema_service->fetch_new_constraint_id(tenant_id, new_cst_id))) {
|
|
LOG_WARN("failed to fetch new constraint id", K(ret));
|
|
} else if (FALSE_IT(cst.set_constraint_id(new_cst_id))) {
|
|
} else if (cst.get_constraint_name_str().empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("cst name is empty", KR(ret));
|
|
} else if (OB_FAIL(check_constraint_name_is_exist(
|
|
schema_guard, table_schema, cst.get_constraint_name_str(), false, cst_name_exist))) {
|
|
LOG_WARN("failed to check constraint name is exist", KR(ret), K(cst.get_constraint_name_str()));
|
|
} else if (cst_name_exist) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
if (!is_oracle_mode) {
|
|
LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE,
|
|
cst.get_constraint_name_str().length(), cst.get_constraint_name_str().ptr());
|
|
}
|
|
LOG_WARN("cst name is duplicate", KR(ret), K(cst.get_constraint_name_str()));
|
|
} else if (OB_FAIL(table_schema.add_constraint(cst))) {
|
|
LOG_WARN("failed to add constraint", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(create_index_or_mlog_table_in_trans(
|
|
table_schema,
|
|
&arg.ddl_stmt_str_,
|
|
&sql_trans,
|
|
schema_guard,
|
|
true /*need_check_tablet_cnt*/,
|
|
tenant_data_version))) {
|
|
LOG_WARN("failed to create index or mlog table in trans", KR(ret), K(arg.ddl_stmt_str_), K(table_schema));
|
|
} else if (OB_FAIL(add_mlog(sql_trans, arg, schema_guard, table_schema))) {
|
|
LOG_WARN("failed to add mlog", KR(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_mlog_tablet(
|
|
ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &mlog_schema,
|
|
const bool need_check_tablet_cnt,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t tenant_id = mlog_schema.get_tenant_id();
|
|
SCN frozen_scn;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat error", K(is_inited()), KR(ret));
|
|
} else if (!mlog_schema.is_mlog_table() || tenant_data_version <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("arg must be materialized view log table", KR(ret), K(tenant_id), K(tenant_data_version), K(mlog_schema));
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObTableCreator table_creator(tenant_id, frozen_scn, trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(tenant_id, schema_guard, sql_proxy_);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
if (OB_FAIL(table_creator.init(need_check_tablet_cnt))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
} else {
|
|
const ObTableSchema *data_table_schema = NULL;
|
|
const uint64_t data_table_id = mlog_schema.get_data_table_id();
|
|
ObSEArray<const share::schema::ObTableSchema*, 1> schemas;
|
|
ObSEArray<bool, 1> need_create_empty_majors;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, data_table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("data table schema not exists", KR(ret), K(data_table_id));
|
|
} else if (OB_FAIL(schemas.push_back(&mlog_schema))
|
|
|| OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("failed to push back mlog schema", KR(ret), K(mlog_schema));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.prepare_like(*data_table_schema))) {
|
|
LOG_WARN("failed to prepare like data table schema", KR(ret), KPC(data_table_schema));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) {
|
|
LOG_WARN("failed to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_local_aux_tables_arg(
|
|
schemas, data_table_schema, ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors))) {
|
|
LOG_WARN("failed to add create tablets of local aux tables arg", KR(ret));
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("failed to execute create tablet", KR(ret));
|
|
}
|
|
}
|
|
|
|
// finishing is always invoked for new table tablet allocator
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) {
|
|
LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_mlog(
|
|
ObMySQLTransaction &trans,
|
|
const obrpc::ObCreateMLogArg &arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &mlog_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = mlog_schema.get_tenant_id();
|
|
const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id);
|
|
const uint64_t mlog_table_id = mlog_schema.get_table_id();
|
|
const obrpc::ObCreateMLogArg::PurgeOptions &purge_options = arg.purge_options_;
|
|
ObMLogPurgeMode purge_mode = purge_options.purge_mode_;
|
|
int64_t purge_start = purge_options.start_datetime_expr_.get_timestamp();
|
|
ObString purge_next = purge_options.next_datetime_expr_;
|
|
ObString purge_job;
|
|
ObArenaAllocator allocator("mlog_sched");
|
|
|
|
if (!purge_options.start_datetime_expr_.is_null()
|
|
|| !purge_options.next_datetime_expr_.empty()) {
|
|
const ObDatabaseSchema *database = NULL;
|
|
ObString job_prefix(ObMLogInfo::MLOG_PURGE_JOB_PREFIX);
|
|
int64_t job_id = OB_INVALID_ID;
|
|
// job_name is generated as "job_prefix+job_id"
|
|
if (OB_FAIL(ObMViewSchedJobUtils::generate_job_id(tenant_id, job_id))) {
|
|
LOG_WARN("failed to generate mview job id", KR(ret));
|
|
} else if (OB_FAIL(ObMViewSchedJobUtils::generate_job_name(
|
|
allocator, job_id, job_prefix, purge_job))) {
|
|
LOG_WARN("failed to generate mview job name",
|
|
KR(ret), K(tenant_id), K(job_prefix));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(
|
|
tenant_id, mlog_schema.get_database_id(), database))) {
|
|
LOG_WARN("failed to get database_schema", KR(ret),
|
|
K(tenant_id), "database_id", mlog_schema.get_database_id());
|
|
} else if (OB_ISNULL(database)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database_schema is null", KR(ret),
|
|
"database_id", mlog_schema.get_database_id());
|
|
} else {
|
|
const ObString mlog_purge_func("DBMS_MVIEW.purge_log");
|
|
const ObString &db_name = database->get_database_name_str();
|
|
const ObString &table_name = arg.table_name_;
|
|
ObString job_action;
|
|
// job_action is generated as "mlog_purge_func('db_name.table_name')"
|
|
if (OB_FAIL(ObMViewSchedJobUtils::generate_job_action(
|
|
allocator,
|
|
mlog_purge_func,
|
|
db_name,
|
|
table_name,
|
|
job_action))) {
|
|
LOG_WARN("failed to generate mview job action", KR(ret));
|
|
} else if (OB_FAIL(ObMViewSchedJobUtils::add_scheduler_job(
|
|
trans,
|
|
tenant_id,
|
|
job_id,
|
|
purge_job,
|
|
job_action,
|
|
purge_options.start_datetime_expr_,
|
|
purge_next,
|
|
arg.purge_options_.exec_env_))) {
|
|
LOG_WARN("failed to add mview scheduler job", KR(ret), K(purge_job),
|
|
K(job_action), K(purge_options.start_datetime_expr_), K(purge_next));
|
|
} else {
|
|
LOG_INFO("succeed to add purge mlog scheduler job",
|
|
K(purge_options.start_datetime_expr_), K(purge_next));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObMLogInfo mlog_info;
|
|
mlog_info.set_tenant_id(tenant_id);
|
|
mlog_info.set_mlog_id(mlog_table_id);
|
|
mlog_info.set_purge_mode(purge_options.purge_mode_);
|
|
if (!purge_options.start_datetime_expr_.is_null()) {
|
|
mlog_info.set_purge_start(purge_options.start_datetime_expr_.get_timestamp());
|
|
}
|
|
mlog_info.set_schema_version(mlog_schema.get_schema_version());
|
|
if (OB_FAIL(mlog_info.set_purge_next(purge_next))) {
|
|
LOG_WARN("fail to set purge next", KR(ret));
|
|
} else if (OB_FAIL(mlog_info.set_purge_job(purge_job))) {
|
|
LOG_WARN("fail to set purge job", KR(ret));
|
|
} else if (OB_FAIL(ObMLogInfo::insert_mlog_info(trans, mlog_info))) {
|
|
LOG_WARN("fail to insert mlog info", KR(ret), K(mlog_info));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_create_with_db_id(ObDatabaseSchema &schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t db_id = schema.get_database_id();
|
|
if (OB_INVALID_ID != db_id) {
|
|
const bool enable_sys_table_ddl = common::ObServerConfig::get_instance().enable_sys_table_ddl;
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE];
|
|
if (!enable_sys_table_ddl) { //Only when the configuration item switch is turned on can the internal table be created
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s", "create database with database_id");
|
|
} else if (!is_inner_db(db_id)) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s %lu", "create inner db with invalid database_id: ", db_id);
|
|
} else {
|
|
schema.set_database_id(db_id);
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::replace_table_schema_type(ObTableSchema &schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t table_id = schema.get_table_id();
|
|
if (OB_INVALID_ID != table_id) {
|
|
if (!schema.is_user_table() && !schema.is_view_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("should not reach here");
|
|
} else {
|
|
const bool enable_sys_table_ddl = common::ObServerConfig::get_instance().enable_sys_table_ddl;
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE];
|
|
if (!enable_sys_table_ddl) { // Only when the configuration item switch is turned on can the internal table is created
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s", "create table with table_id");
|
|
} else if (!is_inner_table(table_id)) { // Cannot specify table_id to create non-internal table
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s %lu", "create inner table with invalid table_id: ", table_id);
|
|
} else if (schema.is_user_table() && is_sys_view(table_id)) { // table_id of table cannot fall into the system view id range
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s", "create table with table_id which fall into sys_view interval");
|
|
} else if (schema.is_user_view() && !is_sys_view(table_id)) { // table_id of view cannot fall into the non-system view range
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s", "create view with table_id which fall into non-sys_view interval");
|
|
} else if (is_reserved_table_id(table_id)) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s", "table_id which fall into reserved interval");
|
|
} else {
|
|
ObTableType type = get_inner_table_type_by_id(table_id);
|
|
if (MAX_TABLE_TYPE != type) {
|
|
schema.set_table_type(type);
|
|
schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
|
|
schema.set_table_id(table_id);
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("type is error", K(type), K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
if (OB_OP_NOT_ALLOW == ret) {
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg);
|
|
} else {
|
|
LOG_USER_ERROR(OB_ERR_UNEXPECTED, err_msg);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_object_id_for_partition_schemas(
|
|
ObIArray<ObTableSchema> &partition_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool gen_subpart_only = false;
|
|
int64_t total_object_cnt = 0;
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < partition_schemas.count(); i++) {
|
|
const ObTableSchema &partition_schema = partition_schemas.at(i);
|
|
int64_t object_cnt = 0;
|
|
if (OB_FAIL(calc_partition_object_id_cnt_(partition_schema, gen_subpart_only, object_cnt))) {
|
|
LOG_WARN("fail to calc partition object id cnt", KR(ret));
|
|
} else {
|
|
total_object_cnt += object_cnt;
|
|
}
|
|
} // end for
|
|
|
|
if (OB_SUCC(ret) && total_object_cnt > 0) {
|
|
ObIDGenerator id_generator;
|
|
ObObjectID max_object_id = OB_INVALID_ID;
|
|
ObObjectID min_object_id = OB_INVALID_ID;
|
|
const uint64_t tenant_id = partition_schemas.at(0).get_tenant_id();
|
|
if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is empty", KR(ret), KP_(schema_service));
|
|
} else if (OB_FAIL(schema_service_->get_schema_service()
|
|
->fetch_new_partition_ids(tenant_id, total_object_cnt, max_object_id))) {
|
|
LOG_WARN("fail to get max object id", KR(ret), K(tenant_id), K(total_object_cnt));
|
|
} else if (OB_UNLIKELY(0 >= (min_object_id = max_object_id - total_object_cnt + 1))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("min_object_id should be greator than 0",
|
|
KR(ret), K(min_object_id), K(max_object_id), K(total_object_cnt));
|
|
} else if (OB_FAIL(id_generator.init(1/*step*/, min_object_id, max_object_id))) {
|
|
LOG_WARN("fail to init id generator", KR(ret), K(min_object_id), K(max_object_id));
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < partition_schemas.count(); i++) {
|
|
ObTableSchema &partition_schema = partition_schemas.at(i);
|
|
if (OB_FAIL(generate_object_id_for_partition_schema(partition_schema, gen_subpart_only, &id_generator))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret));
|
|
}
|
|
} // end for
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::calc_partition_object_id_cnt_(
|
|
const ObPartitionSchema &partition_schema,
|
|
const bool gen_subpart_only,
|
|
int64_t &object_cnt)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t schema_id = partition_schema.get_table_id();
|
|
const ObPartitionLevel part_level = partition_schema.get_part_level();
|
|
const int64_t partition_num = partition_schema.get_partition_num();
|
|
const int64_t first_part_num = partition_schema.get_first_part_num();
|
|
object_cnt = 0;
|
|
if (OB_UNLIKELY(part_level >= PARTITION_LEVEL_MAX)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part level is unexpected", KR(ret), K(part_level));
|
|
} else if (is_inner_table(schema_id)
|
|
|| is_sys_tablegroup_id(schema_id)
|
|
|| PARTITION_LEVEL_ZERO == part_level) {
|
|
// 1. For non-partitioned schemas'(include view、virtual table):
|
|
// object_id is equal to its schema_id.
|
|
// 2. For partitioned virtual table(list columns only):
|
|
// object_id for its partition is hard code by schema.
|
|
// For the above reasons, we won't allocate object_id for table/tablegroup
|
|
object_cnt = 0;
|
|
} else if (OB_UNLIKELY(first_part_num != partition_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("first_part_num is not equal to partition_num", KR(ret), K(partition_schema));
|
|
} else if (OB_UNLIKELY(0 >= (object_cnt = partition_schema.get_all_part_num()))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid partition num", KR(ret), K(object_cnt), K(partition_schema));
|
|
} else if (PARTITION_LEVEL_TWO == part_level && !gen_subpart_only) {
|
|
object_cnt += first_part_num;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// Actually, object_id can be allocated within the same ddl trans.
|
|
// To avoid refactor more codes, generate_object_id_for_partition_schema() should be called with generate_tablet_id().
|
|
//
|
|
// [@input]gen_subpart_only:
|
|
// - True : for add/drop subpartition situations, part_id is valid and should not be generated again.
|
|
// - False : other situations.
|
|
int ObDDLService::generate_object_id_for_partition_schema(
|
|
ObPartitionSchema &partition_schema,
|
|
const bool gen_subpart_only /* = false */,
|
|
ObIDGenerator *batch_id_generator /* = NULL */)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = partition_schema.get_tenant_id();
|
|
// table_id/tablegroup_id
|
|
const int64_t schema_id = partition_schema.get_table_id();
|
|
ObPartitionLevel part_level = partition_schema.get_part_level();
|
|
ObPartition** part_array = partition_schema.get_part_array();
|
|
int64_t partition_num = partition_schema.get_partition_num();
|
|
int64_t object_cnt = 0;
|
|
if (OB_FAIL(calc_partition_object_id_cnt_(partition_schema, gen_subpart_only, object_cnt))) {
|
|
LOG_WARN("fail to calc partition object id cnt", KR(ret), K(partition_schema));
|
|
} else if (0 == object_cnt) {
|
|
// skip
|
|
} else if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_array is empty", KR(ret), KP(part_array));
|
|
} else {
|
|
ObIDGenerator tmp_id_generator;
|
|
if (OB_ISNULL(batch_id_generator)) {
|
|
ObObjectID max_object_id = OB_INVALID_ID;
|
|
ObObjectID min_object_id = OB_INVALID_ID;
|
|
if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is empty", KR(ret), KP_(schema_service));
|
|
} else if (OB_FAIL(schema_service_->get_schema_service()
|
|
->fetch_new_partition_ids(tenant_id, object_cnt, max_object_id))) {
|
|
LOG_WARN("fail to get max object id", KR(ret), K(tenant_id), K(object_cnt));
|
|
} else if (OB_UNLIKELY(0 >= (min_object_id = max_object_id - object_cnt + 1))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("min_object_id should be greator than 0",
|
|
KR(ret), K(min_object_id), K(max_object_id), K(object_cnt));
|
|
} else if (OB_FAIL(tmp_id_generator.init(1/*step*/, min_object_id, max_object_id))) {
|
|
LOG_WARN("fail to init id generator", KR(ret), K(min_object_id), K(max_object_id));
|
|
}
|
|
}
|
|
|
|
ObObjectID object_id = OB_INVALID_ID;
|
|
ObIDGenerator &id_generator = OB_ISNULL(batch_id_generator) ?
|
|
tmp_id_generator :
|
|
*batch_id_generator;
|
|
// 1. generate object_id for partitions
|
|
if (OB_SUCC(ret) && !gen_subpart_only) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < partition_num; i++) {
|
|
ObPartition* part = part_array[i];
|
|
if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part is null", KR(ret), K(i));
|
|
} else if (OB_FAIL(id_generator.next(object_id))) {
|
|
LOG_WARN("fail to get next object id", KR(ret));
|
|
} else {
|
|
part->set_part_id(object_id);
|
|
}
|
|
} // end for
|
|
}
|
|
// 2. generate object_id for subpartitions
|
|
if (OB_SUCC(ret) && PARTITION_LEVEL_TWO == part_level) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < partition_num; i++) {
|
|
ObPartition* part = part_array[i];
|
|
if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part is null", KR(ret), K(i));
|
|
} else if (OB_UNLIKELY(part->get_part_id() <= 0)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_id is invalid", KR(ret), KPC(part));
|
|
} else if (OB_ISNULL(part->get_subpart_array())
|
|
|| part->get_subpartition_num() <= 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sub_part_array is null or invalid subpartition num", KR(ret), KPC(part));
|
|
} else {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < part->get_subpartition_num(); j++) {
|
|
ObSubPartition *subpart = part->get_subpart_array()[j];
|
|
if (OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("subpart is null", KR(ret), KPC(part), K(j));
|
|
} else if (OB_FAIL(id_generator.next(object_id))) {
|
|
LOG_WARN("fail to get next object id", KR(ret));
|
|
} else {
|
|
subpart->set_part_id(part->get_part_id());
|
|
subpart->set_sub_part_id(object_id);
|
|
}
|
|
} // end for subpart
|
|
}
|
|
} // end for part
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_tables_tablet_id(ObIArray<ObTableSchema> &table_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t total_normal_tablet_cnt = 0;
|
|
int64_t total_extended_tablet_cnt = 0;
|
|
ObIDGenerator normal_tablet_id_generator;
|
|
ObIDGenerator extended_tablet_id_generator;
|
|
uint64_t tenant_id = OB_INVALID_TENANT_ID;
|
|
|
|
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 (table_schemas.count() > 0) {
|
|
tenant_id = table_schemas.at(0).get_tenant_id();
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
const ObTableSchema &table_schema = table_schemas.at(i);
|
|
uint64_t tablet_cnt = 0;
|
|
if (OB_FAIL(calc_table_tablet_id_cnt_(table_schema, tablet_cnt))) {
|
|
LOG_WARN("fail to calc tablet_id cnt", KR(ret));
|
|
} else if (table_schema.gen_normal_tablet()) {
|
|
total_normal_tablet_cnt += tablet_cnt;
|
|
} else {
|
|
total_extended_tablet_cnt += tablet_cnt;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && total_normal_tablet_cnt > 0) {
|
|
uint64_t min_tablet_id = OB_INVALID_ID;
|
|
uint64_t max_tablet_id = OB_INVALID_ID;
|
|
if (OB_FAIL(schema_service_->get_schema_service()->fetch_new_tablet_ids(
|
|
tenant_id, true /*gen_normal_tablet*/, total_normal_tablet_cnt, min_tablet_id))) {
|
|
LOG_WARN("fail to fetch new tablet id", KR(ret), K(total_normal_tablet_cnt));
|
|
} else if (FALSE_IT(max_tablet_id = min_tablet_id + total_normal_tablet_cnt - 1)) {
|
|
} else if (OB_FAIL(normal_tablet_id_generator.init(1/*step*/, min_tablet_id, max_tablet_id))) {
|
|
LOG_WARN("fail to init id generator", KR(ret), K(min_tablet_id), K(max_tablet_id));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && total_extended_tablet_cnt> 0) {
|
|
uint64_t min_tablet_id = OB_INVALID_ID;
|
|
uint64_t max_tablet_id = OB_INVALID_ID;
|
|
if (OB_FAIL(schema_service_->get_schema_service()->fetch_new_tablet_ids(
|
|
tenant_id, false/*gen_normal_tablet*/, total_extended_tablet_cnt, min_tablet_id))) {
|
|
LOG_WARN("fail to fetch new tablet id", KR(ret), K(total_extended_tablet_cnt));
|
|
} else if (FALSE_IT(max_tablet_id = min_tablet_id + total_extended_tablet_cnt - 1)) {
|
|
} else if (OB_FAIL(extended_tablet_id_generator.init(1/*step*/, min_tablet_id, max_tablet_id))) {
|
|
LOG_WARN("fail to init id generator", KR(ret), K(min_tablet_id), K(max_tablet_id));
|
|
}
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
ObTableSchema &table_schema = table_schemas.at(i);
|
|
ObIDGenerator &id_generator = table_schema.gen_normal_tablet() ?
|
|
normal_tablet_id_generator :
|
|
extended_tablet_id_generator;
|
|
if (OB_FAIL(generate_tablet_id(table_schema, &id_generator))) {
|
|
LOG_WARN("fail to generator tablet_id", KR(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::calc_table_tablet_id_cnt_(
|
|
const ObTableSchema &table_schema,
|
|
uint64_t &tablet_cnt)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
tablet_cnt = 0;
|
|
if (is_sys_table(table_schema.get_table_id())
|
|
|| table_schema.is_vir_table()
|
|
|| table_schema.is_view_table()
|
|
|| table_schema.is_external_table()) {
|
|
// 1. sys table use table_id as tablet_id
|
|
// 2. virtual table/view/external table don't have tablet_id
|
|
} else if (OB_UNLIKELY((tablet_cnt = table_schema.get_all_part_num()) <= 0)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get tablet cnt", KR(ret), K(tablet_cnt), K(table_schema));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_tablet_id(
|
|
ObTableSchema &table_schema,
|
|
ObIDGenerator *batch_id_generator /* = NULL*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tablet_num = OB_INVALID_ID;
|
|
if (is_sys_table(table_schema.get_table_id())) {
|
|
table_schema.set_tablet_id(table_schema.get_table_id());
|
|
} else if (table_schema.is_external_table()) {
|
|
//skip
|
|
} else if (OB_FAIL(calc_table_tablet_id_cnt_(table_schema, tablet_num))) {
|
|
LOG_WARN("fail to calc tablet num", KR(ret), K(table_schema));
|
|
} else if (0 == tablet_num) {
|
|
// skip
|
|
} else {
|
|
ObIDGenerator tmp_id_generator;
|
|
if (OB_ISNULL(batch_id_generator)) {
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
uint64_t min_tablet_id = OB_INVALID_ID;
|
|
uint64_t max_tablet_id = OB_INVALID_ID;
|
|
if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_ERROR("schema service is null", KR(ret), KP_(schema_service));
|
|
} else if (OB_FAIL(schema_service_->get_schema_service()->fetch_new_tablet_ids(
|
|
tenant_id, table_schema.gen_normal_tablet(), tablet_num, min_tablet_id))) {
|
|
LOG_WARN("fail to fetch new tablet id", KR(ret), K(tablet_num));
|
|
} else if (FALSE_IT(max_tablet_id = min_tablet_id + tablet_num - 1)) {
|
|
} else if (OB_FAIL(tmp_id_generator.init(1/*step*/, min_tablet_id, max_tablet_id))) {
|
|
LOG_WARN("fail to init id generator", KR(ret), K(min_tablet_id), K(max_tablet_id));
|
|
}
|
|
}
|
|
|
|
ObIDGenerator &id_generator = OB_ISNULL(batch_id_generator) ?
|
|
tmp_id_generator :
|
|
*batch_id_generator;
|
|
uint64_t new_tablet_id = OB_INVALID_ID;
|
|
ObPartitionLevel part_level = table_schema.get_part_level();
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_UNLIKELY(part_level >= PARTITION_LEVEL_MAX)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part level is unexpected", KR(ret), K(part_level));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
if (OB_FAIL(id_generator.next(new_tablet_id))) {
|
|
LOG_WARN("fail to get next tablet_id", KR(ret));
|
|
} else {
|
|
(void) table_schema.set_tablet_id(new_tablet_id);
|
|
}
|
|
} else {
|
|
ObPartition **part_array = table_schema.get_part_array();
|
|
int64_t part_num = table_schema.get_partition_num();
|
|
if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", KR(ret), KP(part_array));
|
|
} else {
|
|
for (int64_t i = 0; i < part_num && OB_SUCC(ret); i++) {
|
|
if (OB_ISNULL(part_array[i])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret), K(i));
|
|
} else if (PARTITION_LEVEL_ONE == part_level) {
|
|
if (OB_FAIL(id_generator.next(new_tablet_id))) {
|
|
LOG_WARN("fail to get next tablet_id", KR(ret));
|
|
} else {
|
|
part_array[i]->set_tablet_id(new_tablet_id);
|
|
}
|
|
} else if (PARTITION_LEVEL_TWO == part_level) {
|
|
ObSubPartition **sub_part_array = part_array[i]->get_subpart_array();
|
|
int64_t sub_part_num = part_array[i]->get_subpartition_num();
|
|
if (OB_ISNULL(sub_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sub part array is null", KR(ret), K(i));
|
|
} else {
|
|
for (int64_t j = 0; j < sub_part_num && OB_SUCC(ret); j++) {
|
|
if (OB_ISNULL(sub_part_array[j])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret), K(i), K(j));
|
|
} else if (OB_FAIL(id_generator.next(new_tablet_id))) {
|
|
LOG_WARN("fail to get next tablet_id", KR(ret));
|
|
} else {
|
|
sub_part_array[j]->set_tablet_id(new_tablet_id);
|
|
}
|
|
} // end for
|
|
}
|
|
} else {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("4.0 not support part type", KR(ret), K(part_level));
|
|
}
|
|
} // end for
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// For create table/tablegroup
|
|
// 1. Add missing partition/subpartition schema.
|
|
// 2. Reorganize part_idx/subpart_idx.
|
|
int ObDDLService::try_format_partition_schema(ObPartitionSchema &partition_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// 1. generate missing partition/subpartition.
|
|
bool generated = false;
|
|
if (partition_schema.is_external_table()) {
|
|
//do nothing
|
|
} else if (OB_FAIL(partition_schema.try_generate_hash_part())) {
|
|
LOG_WARN("fail to generate hash part", KR(ret), K(partition_schema));
|
|
} else if (OB_FAIL(partition_schema.try_generate_hash_subpart(generated))) {
|
|
LOG_WARN("fail to generate hash part", KR(ret), K(partition_schema));
|
|
} else if (generated) {
|
|
// skip
|
|
} else if (OB_FAIL(partition_schema.try_generate_subpart_by_template(generated))) {
|
|
LOG_WARN("fail to generate_subpart_by_template", KR(ret), K(partition_schema));
|
|
}
|
|
if (OB_SUCC(ret) && generated) {
|
|
partition_schema.set_sub_part_template_def_valid();
|
|
LOG_INFO("convert schema to nontemplate", K(partition_schema));
|
|
}
|
|
// 2. generate part_idx/subpart_idx.
|
|
if (FAILEDx(partition_schema.try_init_partition_idx())) {
|
|
LOG_WARN("fail to init partition idx", KR(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/* generate_schema is called when creating a data table,
|
|
* IndexBuilder::generate_schema is called when create a index table.
|
|
*/
|
|
int ObDDLService::generate_schema(
|
|
const ObCreateTableArg &arg,
|
|
ObTableSchema &schema)
|
|
{
|
|
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();
|
|
uint64_t compat_version = 0;
|
|
ObSchemaService *schema_service = NULL;
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
const ObTablespaceSchema *tablespace_schema = NULL;
|
|
bool is_oracle_mode = false;
|
|
ObSchemaGetterGuard guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("fail to get data version", K(ret), K(tenant_id));
|
|
} else if (not_compat_for_queuing_mode(compat_version) && arg.schema_.is_new_queuing_table_mode()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN(QUEUING_MODE_NOT_COMPAT_WARN_STR, K(ret), K(tenant_id), K(compat_version), K(arg));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, QUEUING_MODE_NOT_COMPAT_USER_ERROR_STR);
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else {
|
|
schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service must not null", K(ret));
|
|
}
|
|
}
|
|
|
|
// support to create inner table if enable_sys_table_ddl is opened
|
|
// -- system view ----> table_type will be TABLE_TYPE_VIEW
|
|
// -- other(virtual table or core table) ----> table_type will be TABLE_TYPE_SYS
|
|
if (OB_SUCC(ret) && OB_FAIL(replace_table_schema_type(schema))) {
|
|
LOG_WARN("not supported operator", K(ret));
|
|
}
|
|
|
|
// set basic schema info, will be checked by check_table_exist
|
|
if (OB_SUCC(ret)) {
|
|
LOG_DEBUG("hualong schema is ", K(schema));
|
|
if (OB_FAIL(set_tablegroup_id(schema))) {
|
|
LOG_WARN("set_tablegroup_id failed", "tablegroup name",
|
|
schema.get_tablegroup_name(), K(ret));
|
|
} else if (OB_FAIL(guard.get_database_schema(
|
|
tenant_id, schema.get_database_id(), database_schema))) {
|
|
LOG_WARN("get_database_schema failed", K(tenant_id), "database_id",
|
|
schema.get_database_id(), K(ret));
|
|
} else if (NULL == database_schema) {
|
|
ret = OB_ERR_NO_DB_SELECTED;
|
|
LOG_WARN("not find this database schema", K(schema.get_database_id()), K(ret));
|
|
} else if (!arg.is_inner_ && database_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not create table in recyclebin", K(ret), K(schema));
|
|
} else if (OB_FAIL(guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id));
|
|
} else if (OB_UNLIKELY(NULL == tenant_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tenant schema is null", K(ret), KP(tenant_schema));
|
|
} else if (OB_INVALID_ID != new_table_id && is_inner_table(new_table_id)) {
|
|
// Specify table_id to create a system table, it may create a tenant-level system table, skip
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(
|
|
schema.get_tenant_id(), new_table_id))) {
|
|
LOG_WARN("fail to fetch new table id", K(ret));
|
|
} else if (FALSE_IT(schema.set_table_id(new_table_id))) {
|
|
} else if (OB_FAIL(try_format_partition_schema(schema))) {
|
|
LOG_WARN("fail to try_format_partition_schema", K(schema), KR(ret));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(schema));
|
|
} else if (OB_FAIL(generate_tablet_id(schema))) {
|
|
LOG_WARN("fail to fetch new table id", K(schema), K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && OB_INVALID_ID != schema.get_tablespace_id()) {
|
|
if (OB_FAIL(guard.get_tablespace_schema(tenant_id, schema.get_tablespace_id(), tablespace_schema))) {
|
|
LOG_WARN("fail to get tablespace schema", K(schema), K(ret));
|
|
} else if (OB_UNLIKELY(NULL == tablespace_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tablespace schema is null", K(ret), K(schema));
|
|
} else if (OB_FAIL(schema.set_encrypt_key(tablespace_schema->get_encrypt_key()))) {
|
|
LOG_WARN("fail to set encrypt key", K(ret), K(schema));
|
|
} else {
|
|
schema.set_master_key_id(tablespace_schema->get_master_key_id());
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (schema.has_partition()) { // include table of standalone and binding
|
|
common::ObArray<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 {} // has no partition
|
|
}
|
|
|
|
// constraints
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < constraints.count(); ++i) {
|
|
uint64_t new_cst_id = OB_INVALID_ID;
|
|
ObConstraint &cst = const_cast<ObConstraint &>(constraints.at(i));
|
|
cst.set_tenant_id(tenant_id);
|
|
cst.set_table_id(schema.get_table_id());
|
|
if (OB_FAIL(schema_service->fetch_new_constraint_id(tenant_id, new_cst_id))) {
|
|
LOG_WARN("failed to fetch new constraint id", K(ret));
|
|
} else if (FALSE_IT(cst.set_constraint_id(new_cst_id))) {
|
|
} else if (cst.get_constraint_name_str().empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("cst name is empty", K(ret));
|
|
} else {
|
|
// Check whether the name of the constraint is repeated
|
|
bool is_constraint_name_exist = false;
|
|
if (OB_FAIL(check_constraint_name_is_exist(guard, schema, cst.get_constraint_name_str(), false, is_constraint_name_exist))) {
|
|
LOG_WARN("fail to check constraint name is exist or not", K(ret), K(cst.get_constraint_name_str()));
|
|
} else if (is_constraint_name_exist) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
if (!is_oracle_mode) {
|
|
LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst.get_constraint_name_str().length(), cst.get_constraint_name_str().ptr());
|
|
}
|
|
LOG_WARN("cst name is duplicate", K(ret), K(cst.get_constraint_name_str()));
|
|
}
|
|
}
|
|
// The process that check whether the constraint name is repeated is end
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(schema.add_constraint(cst))) {
|
|
LOG_WARN("add constraint failed", K(ret), K(i));
|
|
}
|
|
}
|
|
}
|
|
|
|
// check udt id invalid
|
|
if (OB_SUCC(ret) && is_oracle_mode) {
|
|
if (OB_FAIL(check_table_udt_id_is_exist(guard, schema, tenant_id))) {
|
|
LOG_WARN("check udt id failed", K(ret));
|
|
}
|
|
}
|
|
|
|
// fill table schema for interval part
|
|
if (OB_SUCC(ret) && schema.has_partition()
|
|
&& schema.is_interval_part()) {
|
|
int64_t part_num = schema.get_part_option().get_part_num();
|
|
ObPartition **part_array = schema.get_part_array();
|
|
const ObRowkey *transition_point = NULL;
|
|
if (PARTITION_LEVEL_TWO == schema.get_part_level()
|
|
&& !schema.has_sub_part_template_def()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("interval part of composited-partitioned table not support", K(ret), K(schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval part of composited-partitioned table without template");
|
|
} else if (1 != schema.get_partition_key_column_num()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("more than one partition key not support", K(ret), K(schema));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "more than one partition key");
|
|
} else if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("range part array is null", K(ret), K(schema));
|
|
} else if (0 == part_num) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_INFO("range part num is 0", K(ret), K(schema));
|
|
} else if (FALSE_IT(transition_point = &part_array[part_num - 1]->get_high_bound_val())) {
|
|
} else if (OB_FAIL(ObPartitionUtils::check_interval_partition_table(*transition_point, schema.get_interval_range()))) {
|
|
LOG_WARN("fail to check_interval_partition_table", K(ret), K(schema));
|
|
} else if (OB_FAIL(schema.set_transition_point(*transition_point))) {
|
|
LOG_WARN("fail to set transition point", K(ret), K(schema));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (schema.is_materialized_view()) {
|
|
if (OB_FAIL(ObResolverUtils::check_schema_valid_for_mview(schema))) {
|
|
LOG_WARN("failed to check schema valid for mview", KR(ret), K(schema));
|
|
} else if (arg.mv_ainfo_.count() <= 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("there should be mv ainfo", KR(ret), K(arg.mv_ainfo_.count()));
|
|
} else {
|
|
schema.get_view_schema().set_mv_refresh_info(&(arg.mv_ainfo_.at(0).mv_refresh_info_));
|
|
}
|
|
}
|
|
}
|
|
|
|
// check auto_partition validity
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(schema.check_validity_for_auto_partition())) {
|
|
LOG_WARN("fail to check auto partition setting", KR(ret), K(schema), K(arg));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
schema.set_micro_index_clustered(arg.schema_.get_micro_index_clustered());
|
|
}
|
|
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 j = 0; OB_SUCC(ret) && j < index_info.get_size(); ++j) {
|
|
if (OB_ISNULL(index_col = index_table_schema.get_column_schema(index_info.get_column(j)->column_id_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get index column schema failed", K(ret));
|
|
} else if (index_col->is_hidden() || index_col->is_shadow_column()) { // do nothing
|
|
} else if (OB_FAIL(uk_columns.push_back(index_col->get_column_name()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("push back index column failed", K(ret));
|
|
} else {} // do nothing
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
const ObIArray<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_table_udt_id_is_exist(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObTableSchema &table_schema,
|
|
const uint64_t tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema::const_column_iterator tmp_begin = table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator tmp_end = table_schema.column_end();
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) {
|
|
ObColumnSchemaV2 *col = (*tmp_begin);
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get column schema failed", K(ret));
|
|
} else if (col->get_meta_type().is_user_defined_sql_type()) {
|
|
// delete hidden primary key
|
|
if (OB_FAIL(check_udt_id_is_exist(schema_guard, *col, tenant_id))) {
|
|
LOG_WARN("fail to check column udt id", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_udt_id_is_exist(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObColumnSchemaV2 &col_schema,
|
|
const uint64_t tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t udt_id = col_schema.get_sub_data_type();
|
|
const ObUDTTypeInfo *udt_info = NULL;
|
|
|
|
if (OB_FAIL(schema_guard.get_udt_info(tenant_id, udt_id, udt_info))) {
|
|
LOG_WARN("get udt info failed", K(ret), K(tenant_id), K(udt_id));
|
|
} else if (OB_NOT_NULL(udt_info)) {
|
|
// do nothing
|
|
} else if (OB_FAIL(schema_guard.get_udt_info(OB_SYS_TENANT_ID, udt_id, udt_info))) {
|
|
LOG_WARN("get udt info failed", K(ret), K(OB_SYS_TENANT_ID), K(udt_id));
|
|
} else if (OB_NOT_NULL(udt_info)) {
|
|
// do nothing
|
|
} else {
|
|
ret = OB_ERR_INVALID_DATATYPE;
|
|
LOG_WARN("invalid column udt id", K(ret), K(udt_id));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// In oracle mode, check constraints and foreign key constraints are in the same namespace.
|
|
// So we need to check if the new constraint name dup with all kinds of constraints in oracle mode.
|
|
// In mysql mode, check constraints and foreign key constraints are in the separate namespaces.
|
|
// So we need to check if the new check constraint name dup with check constraints and fk constraint name dup with fk constraints in mysql mode.
|
|
int ObDDLService::check_constraint_name_is_exist(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObTableSchema &table_schema,
|
|
const common::ObString &constraint_name,
|
|
const bool is_foreign_key, // this param is only effective in mysql mode
|
|
bool &is_constraint_name_exist)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t constraint_id = OB_INVALID_ID;
|
|
bool is_oracle_mode = false;
|
|
is_constraint_name_exist = false;
|
|
|
|
if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("check if oracle compat mode failed", K(ret));
|
|
} else {
|
|
// check if fk name is exist
|
|
if (is_oracle_mode || is_foreign_key) {
|
|
if (OB_FAIL(schema_guard.get_foreign_key_id(table_schema.get_tenant_id(),
|
|
table_schema.get_database_id(),
|
|
constraint_name,
|
|
constraint_id))) {
|
|
LOG_WARN("get foreign key id failed", K(ret),
|
|
K(table_schema.get_tenant_id()),
|
|
K(table_schema.get_database_id()),
|
|
K(constraint_name));
|
|
} else if (OB_INVALID_ID != constraint_id) {
|
|
is_constraint_name_exist = true;
|
|
}
|
|
}
|
|
// check if cst name is exist
|
|
if (OB_SUCC(ret) && !is_constraint_name_exist && (is_oracle_mode || !is_foreign_key)) {
|
|
if (table_schema.is_mysql_tmp_table()) {
|
|
// tmp table in mysql mode, do nothing
|
|
} else if (OB_FAIL(schema_guard.get_constraint_id(table_schema.get_tenant_id(),
|
|
table_schema.get_database_id(),
|
|
constraint_name,
|
|
constraint_id))) {
|
|
LOG_WARN("get constraint id failed", K(ret),
|
|
K(table_schema.get_tenant_id()),
|
|
K(table_schema.get_database_id()),
|
|
K(constraint_name));
|
|
} else if (OB_INVALID_ID != constraint_id) {
|
|
is_constraint_name_exist = true;
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::deal_with_cst_for_alter_table(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const ObTableSchema *orig_table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObMockFKParentTableSchema &mock_fk_parent_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const ObTableSchema *table_schema = NULL;
|
|
bool is_oracle_mode = false;
|
|
if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table schema is null", KR(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
alter_table_schema.get_origin_database_name(),
|
|
alter_table_schema.get_origin_table_name(),
|
|
false,
|
|
table_schema))) {
|
|
LOG_WARN("get table schema failed", K(ret), K(alter_table_schema.get_origin_database_name()), K(alter_table_schema.get_origin_table_name()));
|
|
} else if (OB_FAIL(table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
}
|
|
// check check constraint info
|
|
if (OB_FAIL(ret)) {
|
|
} else if (alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::ADD_CONSTRAINT) {
|
|
ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin();
|
|
for (; OB_SUCC(ret) && iter != alter_table_schema.constraint_end(); iter++) {
|
|
if (OB_ISNULL(iter) || OB_ISNULL(*iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("constraint is null", K(ret));
|
|
} else if ((*iter)->get_constraint_name_str().empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("check constraint name is null", K(ret));
|
|
} else {
|
|
bool is_check_constraint_name_exist = true;
|
|
if (OB_FAIL(check_constraint_name_is_exist(
|
|
schema_guard,
|
|
*table_schema,
|
|
(*iter)->get_constraint_name_str(),
|
|
false,
|
|
is_check_constraint_name_exist))) {
|
|
LOG_WARN("fail to check check constraint name is exist or not",
|
|
K(ret), K((*iter)->get_constraint_name_str()));
|
|
} else if (is_check_constraint_name_exist) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_WARN("check constraint name is duplicate",
|
|
K(ret), K((*iter)->get_constraint_name_str()));
|
|
if (!is_oracle_mode) {
|
|
LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, (*iter)->get_constraint_name_str().length(), (*iter)->get_constraint_name_str().ptr());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// check foreign key info
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.foreign_key_arg_list_.count(); i++) {
|
|
const ObCreateForeignKeyArg &foreign_key_arg = alter_table_arg.foreign_key_arg_list_.at(i);
|
|
ObForeignKeyInfo foreign_key_info;
|
|
// Check for duplicate foreign key constraint names
|
|
if (foreign_key_arg.foreign_key_name_.empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fk name is empty", K(ret));
|
|
} else {
|
|
bool is_foreign_key_name_exist = true;
|
|
if (OB_FAIL(check_constraint_name_is_exist(schema_guard, *table_schema, foreign_key_arg.foreign_key_name_, true, is_foreign_key_name_exist))) {
|
|
LOG_WARN("fail to check foreign key name is exist or not", K(ret), K(foreign_key_arg.foreign_key_name_));
|
|
} else if (is_foreign_key_name_exist) {
|
|
if (foreign_key_arg.is_modify_fk_state_) {
|
|
// Check whether the constraint is a foreign key constraint
|
|
uint64_t fk_constraint_id = OB_INVALID_ID;
|
|
if (OB_FAIL(schema_guard.get_foreign_key_id(table_schema->get_tenant_id(),
|
|
table_schema->get_database_id(),
|
|
foreign_key_arg.foreign_key_name_,
|
|
fk_constraint_id))) {
|
|
LOG_WARN("get foreign key id failed", K(ret),
|
|
K(table_schema->get_tenant_id()),
|
|
K(table_schema->get_database_id()),
|
|
K(foreign_key_arg.foreign_key_name_));
|
|
} else if (OB_INVALID_ID != fk_constraint_id) {
|
|
// There is a corresponding foreign key, do nothing
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("no such fk constraint", K(ret), K(foreign_key_arg.foreign_key_name_));
|
|
}
|
|
} else if (is_oracle_mode) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_WARN("fk name is duplicate", K(ret), K(foreign_key_arg.foreign_key_name_));
|
|
} else { // mysql mode
|
|
ret = OB_ERR_DUP_KEY;
|
|
LOG_USER_ERROR(OB_ERR_DUP_KEY, table_schema->get_table_name_str().length(),
|
|
table_schema->get_table_name_str().ptr());
|
|
}
|
|
}
|
|
}
|
|
// Check whether the foreign key constraint name is repeated end
|
|
if (OB_SUCC(ret) && foreign_key_arg.is_modify_fk_state_) {
|
|
bool is_found = false;
|
|
const ObIArray<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 (foreign_key_arg.is_parent_table_mock_) {
|
|
if (OB_NOT_NULL(parent_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("parent_schema is exist", K(ret), KPC(parent_schema));
|
|
} else if (OB_FAIL(gen_mock_fk_parent_table_for_create_fk(schema_guard,
|
|
table_schema->get_tenant_id(), foreign_key_arg, NULL, foreign_key_info,
|
|
mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to generate_mock_fk_parent_table_schema", K(ret), K(table_schema->get_tenant_id()), K(foreign_key_arg));
|
|
}
|
|
} else if (OB_ISNULL(parent_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("parent table is not exist", K(ret), K(foreign_key_arg));
|
|
} else if (parent_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("parent table is in recyclebin", K(ret), K(foreign_key_arg));
|
|
} else if (parent_schema->get_table_id() != table_schema->get_table_id()) {
|
|
// If the reference table is itself, there is no need to update sync_versin_for_cascade_table
|
|
if (OB_FAIL(alter_table_schema.add_depend_table_id(parent_schema->get_table_id()))) {
|
|
LOG_WARN("failed to add depend table id", K(ret), K(foreign_key_arg));
|
|
}
|
|
}
|
|
}
|
|
// get child column schema.
|
|
if (OB_SUCC(ret)) {
|
|
foreign_key_info.child_table_id_ = table_schema->get_table_id();
|
|
foreign_key_info.parent_table_id_ = foreign_key_arg.is_parent_table_mock_ ? mock_fk_parent_table_schema.get_mock_fk_parent_table_id() : parent_schema->get_table_id();
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.child_columns_.count(); j++) {
|
|
const ObString &column_name = foreign_key_arg.child_columns_.at(j);
|
|
const ObColumnSchemaV2 *column_schema = table_schema->get_column_schema(column_name);
|
|
if (OB_ISNULL(column_schema)) {
|
|
ret = OB_ERR_COLUMN_NOT_FOUND;
|
|
LOG_WARN("child column is not exist", K(ret), K(column_name));
|
|
} else if (OB_FAIL(foreign_key_info.child_column_ids_.push_back(column_schema->get_column_id()))) {
|
|
LOG_WARN("failed to push child column id", K(ret), K(column_name));
|
|
}
|
|
}
|
|
}
|
|
// get parent column schema.
|
|
if (OB_SUCC(ret) && !foreign_key_arg.is_parent_table_mock_) {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.parent_columns_.count(); j++) {
|
|
const ObString &column_name = foreign_key_arg.parent_columns_.at(j);
|
|
const ObColumnSchemaV2 *column_schema = parent_schema->get_column_schema(column_name);
|
|
if (OB_ISNULL(column_schema)) {
|
|
ret = OB_ERR_COLUMN_NOT_FOUND;
|
|
LOG_WARN("parent column is not exist", K(ret), K(column_name));
|
|
} else if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(column_schema->get_column_id()))) {
|
|
LOG_WARN("failed to push parent column id", K(ret), K(column_name));
|
|
}
|
|
}
|
|
}
|
|
// get reference option and foreign key name.
|
|
if (OB_SUCC(ret)) {
|
|
foreign_key_info.update_action_ = foreign_key_arg.update_action_;
|
|
foreign_key_info.delete_action_ = foreign_key_arg.delete_action_;
|
|
foreign_key_info.foreign_key_name_ = foreign_key_arg.foreign_key_name_;
|
|
foreign_key_info.enable_flag_ = foreign_key_arg.enable_flag_;
|
|
foreign_key_info.validate_flag_ = foreign_key_arg.validate_flag_;
|
|
foreign_key_info.rely_flag_ = foreign_key_arg.rely_flag_;
|
|
foreign_key_info.ref_cst_type_ = foreign_key_arg.ref_cst_type_;
|
|
foreign_key_info.ref_cst_id_ = foreign_key_arg.ref_cst_id_;
|
|
foreign_key_info.is_parent_table_mock_ = foreign_key_arg.is_parent_table_mock_;
|
|
foreign_key_info.name_generated_type_ = foreign_key_arg.name_generated_type_;
|
|
}
|
|
// add foreign key info.
|
|
if (OB_SUCC(ret)) {
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_FAIL(schema_service->fetch_new_constraint_id(table_schema->get_tenant_id(),
|
|
foreign_key_info.foreign_key_id_))) {
|
|
LOG_WARN("failed to fetch new foreign key id", K(ret), K(foreign_key_arg));
|
|
} else if (OB_FAIL(alter_table_schema.add_foreign_key_info(foreign_key_info))) {
|
|
LOG_WARN("failed to push foreign key info", K(ret), K(foreign_key_info));
|
|
}
|
|
}
|
|
}
|
|
} // for
|
|
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_cst_name_dup_for_rename_table_mysql(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObTableSchema *from_table_schema,
|
|
const uint64_t to_database_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(from_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to check if oracle compat mode", K(ret));
|
|
} else if (is_oracle_mode) {
|
|
// do nothing, not support rename table to another db in oracle mode
|
|
} else if (from_table_schema->get_database_id() != to_database_id) {
|
|
// check if constraint/foreign key name is exist when rename table to another database in mysql mode
|
|
ObTableSchema tmp_schema;
|
|
bool is_constraint_name_exist = true;
|
|
tmp_schema.set_table_id(from_table_schema->get_table_id());
|
|
tmp_schema.set_tenant_id(from_table_schema->get_tenant_id());
|
|
tmp_schema.set_table_type(from_table_schema->get_table_type());
|
|
tmp_schema.set_database_id(to_database_id);
|
|
// for check constraint
|
|
ObTableSchema::const_constraint_iterator iter = from_table_schema->constraint_begin();
|
|
for (;OB_SUCC(ret) && iter != from_table_schema->constraint_end(); ++iter) {
|
|
if (OB_FAIL(check_constraint_name_is_exist(schema_guard, tmp_schema, (*iter)->get_constraint_name_str(), false, is_constraint_name_exist))) {
|
|
LOG_WARN("fail to check check constraint name is exist or not", K(ret), K((*iter)->get_constraint_name_str()));
|
|
} else if (is_constraint_name_exist) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, (*iter)->get_constraint_name_str().length(), (*iter)->get_constraint_name_str().ptr());
|
|
LOG_WARN("check constraint name is duplicate", K(ret), K((*iter)->get_constraint_name_str()));
|
|
}
|
|
}
|
|
// for foreign key constraint
|
|
if (OB_SUCC(ret)) {
|
|
const common::ObIArray<ObForeignKeyInfo>& foreign_keys = from_table_schema->get_foreign_key_infos();
|
|
for (int i = 0; OB_SUCC(ret) && i < foreign_keys.count(); ++i) {
|
|
if (OB_FAIL(check_constraint_name_is_exist(
|
|
schema_guard, tmp_schema, foreign_keys.at(i).foreign_key_name_, true, is_constraint_name_exist))) {
|
|
LOG_WARN("fail to check foreign key name is exist or not", K(ret), K(foreign_keys.at(i).foreign_key_name_));
|
|
} else if (is_constraint_name_exist) {
|
|
ret = OB_ERR_DUP_KEY;
|
|
LOG_USER_ERROR(OB_ERR_DUP_KEY, foreign_keys.at(i).foreign_key_name_.length(), foreign_keys.at(i).foreign_key_name_.ptr());
|
|
LOG_WARN("foreign key name is duplicate", K(ret), K(foreign_keys.at(i).foreign_key_name_));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_database_exist(
|
|
const uint64_t tenant_id,
|
|
const common::ObString &database_name,
|
|
uint64_t &database_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool exist = false;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(schema_service_->check_database_exist(
|
|
tenant_id, database_name, database_id, exist))) {
|
|
LOG_WARN("check database exist failed", "database", database_name, K(ret));
|
|
} else {
|
|
if (!exist) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr());
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_table_exist(ObTableSchema &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
bool is_table_name_exist = false;
|
|
bool is_table_id_exist = false;
|
|
const int64_t table_schema_version = OB_INVALID_VERSION; // get the latest local schema_guard
|
|
if (OB_FAIL(schema_service_->check_table_exist(
|
|
table_schema.get_tenant_id(),
|
|
table_schema.get_database_id(),
|
|
table_schema.get_table_name_str(),
|
|
table_schema.is_index_table(),
|
|
table_schema_version,
|
|
is_table_name_exist))) {
|
|
LOG_WARN("failed to check is table exist",
|
|
K(table_schema.get_tenant_id()),
|
|
K(table_schema.get_database_id()),
|
|
K(table_schema.is_index_table()),
|
|
K(table_schema.get_table_name()), K(ret));
|
|
} else if (OB_INVALID_ID != table_schema.get_table_id()
|
|
&& OB_FAIL(schema_service_->check_table_exist(
|
|
table_schema.get_tenant_id(),
|
|
table_schema.get_table_id(),
|
|
table_schema_version,
|
|
is_table_id_exist))) {
|
|
LOG_WARN("failed to check is table exist",
|
|
K(table_schema.get_tenant_id()),
|
|
K(table_schema.get_table_id()), K(ret));
|
|
} else if (is_table_name_exist || is_table_id_exist) {
|
|
ret = OB_ERR_TABLE_EXIST;
|
|
LOG_WARN("table is exist, cannot create it twice,",
|
|
K(table_schema.get_tenant_id()),
|
|
K(table_schema.get_database_id()),
|
|
K(table_schema.get_table_id()),
|
|
K(table_schema.get_table_name()), K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_inner_stat() const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (!inited_) {
|
|
ret = OB_NOT_INIT;
|
|
LOG_WARN("not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service_) || OB_ISNULL(sql_proxy_)
|
|
|| OB_ISNULL(rpc_proxy_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service_,sql_proxy_ or rpc_proxy_ is null", K(ret));
|
|
} else if (OB_ISNULL(lst_operator_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("lst_operator_ is null", KR(ret));
|
|
} else if (OB_ISNULL(unit_mgr_) || OB_ISNULL(zone_mgr_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unit_mgr_ or zone_mgr_ is null", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_sample_table_schema(
|
|
common::ObIArray<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 != table_schemas.at(i)->get_table_id()) {
|
|
sample_table_schema = table_schemas.at(i);
|
|
} else {} // go on find
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::set_tablegroup_id(ObTableSchema &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
uint64_t tablegroup_id = OB_INVALID_ID;
|
|
uint64_t tenant_id = table_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (table_schema.get_tablegroup_name().empty()) {
|
|
table_schema.set_tablegroup_id(OB_INVALID_ID);
|
|
} else if (OB_FAIL(schema_guard.get_tablegroup_id(table_schema.get_tenant_id(),
|
|
table_schema.get_tablegroup_name(), tablegroup_id))) {
|
|
LOG_WARN("get_tablegroup_id failed", "tenant_id", table_schema.get_tenant_id(),
|
|
"tablegroup_name", table_schema.get_tablegroup_name(), K(ret));
|
|
} else if (OB_INVALID_ID == tablegroup_id) {
|
|
ret = OB_TABLEGROUP_NOT_EXIST;
|
|
LOG_WARN("group name not exist ", K(ret));
|
|
} else {
|
|
table_schema.set_tablegroup_id(tablegroup_id);
|
|
}
|
|
// if table's tablegroup does not exist, use database's default tablegroup
|
|
if (OB_SUCC(ret) && OB_INVALID_ID == table_schema.get_tablegroup_id()) {
|
|
const ObDatabaseSchema *db_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, table_schema.get_database_id(), db_schema))) {
|
|
LOG_WARN("fail to get database schema", K(ret), K(tenant_id), "db_id", table_schema.get_database_id());
|
|
} else if (OB_UNLIKELY(NULL == db_schema)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("fail to get database schema", K(ret), K(tenant_id), "db_id", table_schema.get_database_id());
|
|
} else {
|
|
table_schema.set_tablegroup_id(db_schema->get_default_tablegroup_id());
|
|
}
|
|
}
|
|
// if database's default_tablegroup_id does not exist, use tenant's default tablegroup
|
|
if (OB_SUCC(ret) && OB_INVALID_ID == table_schema.get_tablegroup_id()) {
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_tenant_info(table_schema.get_tenant_id(), tenant_schema))) {
|
|
LOG_WARN("fail to get tenant schema", K(ret), "tenant_id", table_schema.get_tenant_id());
|
|
} else if (OB_UNLIKELY(NULL == tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("fail to get tenant schema", K(ret), "tenant_id", table_schema.get_tenant_id());
|
|
} else {
|
|
table_schema.set_tablegroup_id(tenant_schema->get_default_tablegroup_id());
|
|
}
|
|
}
|
|
|
|
// TODO: (2019.6.24 wendu) Cannot add replicated table to tablegroup
|
|
if (OB_SUCC(ret)) {
|
|
if (ObDuplicateScope::DUPLICATE_SCOPE_NONE != table_schema.get_duplicate_scope()
|
|
&& OB_INVALID_ID != table_schema.get_tablegroup_id()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("duplicated 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, "duplicated table in tablegroup");
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t table_id = table_schema.get_table_id();
|
|
if (!(is_inner_table(table_id)
|
|
|| table_schema.is_user_table()
|
|
|| table_schema.is_mysql_tmp_table())) {
|
|
table_schema.set_tablegroup_id(OB_INVALID_ID);
|
|
}
|
|
}
|
|
|
|
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::print_view_expanded_definition(
|
|
const ObTableSchema &table_schema,
|
|
ObString &ddl_stmt_str,
|
|
common::ObIAllocator &allocator,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
bool if_not_exist)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
char *buf = NULL;
|
|
int64_t buf_len = OB_MAX_VARCHAR_LENGTH;
|
|
int64_t pos = 0;
|
|
bool is_oracle_mode;
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const int64_t database_id = table_schema.get_database_id();
|
|
|
|
if (OB_ISNULL(buf = static_cast<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(tenant_id, database_id, database_schema))) {
|
|
LOG_WARN("failed to get database schema", K(ret), K(tenant_id), K(database_id));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database not exist", K(ret), K(database_id));
|
|
} else if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos,
|
|
is_oracle_mode ? "CREATE%s %sVIEW \"%s\".\"%s\" AS %.*s" : "CREATE%s %sVIEW `%s`.`%s` AS %.*s",
|
|
if_not_exist ? " OR REPLACE" : "",
|
|
table_schema.is_materialized_view() ? "MATERIALIZED " : "",
|
|
database_schema->get_database_name(),
|
|
table_schema.get_table_name(),
|
|
table_schema.get_view_schema().get_view_definition_str().length(),
|
|
table_schema.get_view_schema().get_view_definition_str().ptr()))) {
|
|
LOG_WARN("fail to print view definition", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, ";"))) {
|
|
LOG_WARN("fail to print end", K(ret));
|
|
} else {
|
|
ddl_stmt_str.assign_ptr(buf, static_cast<int32_t>(pos));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_obj_privs_ora(const uint64_t tenant_id,
|
|
const uint64_t obj_id,
|
|
const uint64_t obj_type,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObIArray<ObObjPriv> &obj_privs) {
|
|
int ret = OB_SUCCESS;
|
|
ObArray<const ObObjPriv*> obj_privs_pointer;
|
|
if (OB_FAIL(schema_guard.get_obj_priv_with_obj_id(tenant_id,
|
|
obj_id,
|
|
obj_type,
|
|
obj_privs_pointer,
|
|
true))) {
|
|
LOG_WARN("fail to get_obj_priv_with_obj_id", KR(ret), K(tenant_id), K(obj_id), K(obj_type));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < obj_privs_pointer.count(); ++i) {
|
|
if (OB_ISNULL(obj_privs_pointer.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("obj_privs_pointer contains NULL", KR(ret), K(i));
|
|
} else if(OB_FAIL(obj_privs.push_back(*(obj_privs_pointer.at(i))))) {
|
|
LOG_WARN("obj_privs fail to push back", KR(ret), K(i));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_tablets_in_trans_for_mv_(ObIArray<ObTableSchema> &table_schemas,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
SCN frozen_scn;
|
|
uint64_t tenant_id = OB_INVALID_ID;
|
|
if (table_schemas.count() < 2) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table_schemas count should not be smaller than 2", KR(ret), K(table_schemas.count()));
|
|
} else {
|
|
tenant_id = table_schemas.at(0).get_tenant_id();
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (OB_FAIL(frozen_scn.convert_for_tx(
|
|
compaction::ObBasicMergeScheduler::INIT_COMPACTION_SCN))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObTableCreator table_creator(
|
|
tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(
|
|
tenant_id,
|
|
schema_guard,
|
|
sql_proxy_);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
if (OB_FAIL(table_creator.init(true/*need_tablet_cnt_check*/))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
}
|
|
|
|
ObArray<const ObTableSchema*> tablet_schemas;
|
|
ObArray<bool> need_create_empty_majors;
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
// create tablets for container table and lob tables
|
|
for (int64_t i = 0; OB_SUCC(ret) && (i < table_schemas.count()); ++i) {
|
|
const ObTableSchema &this_table = table_schemas.at(i);
|
|
const int64_t table_id = this_table.get_table_id();
|
|
if (OB_INVALID_VERSION == last_schema_version
|
|
&& OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("get last schema version failed", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_ori_schema_version(
|
|
trans, tenant_id, table_id, last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!",
|
|
KR(ret), K(tenant_id), K(table_id), K(last_schema_version));
|
|
} else if (!this_table.is_materialized_view()) {
|
|
// no need to create tablets for mv because its data is stored in container table
|
|
if (OB_FAIL(tablet_schemas.push_back(&this_table))) {
|
|
LOG_WARN("failed to push back", KR(ret), K(this_table));
|
|
} else if (OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("failed to push_back", KR(ret), K(this_table));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
const ObTableSchema &container_table = table_schemas.at(1);
|
|
const ObTablegroupSchema *container_tablegroup_schema = NULL; // keep NULL if no tablegroup
|
|
if (OB_INVALID_ID != container_table.get_tablegroup_id()) {
|
|
if (OB_FAIL(schema_guard.get_tablegroup_schema(
|
|
container_table.get_tenant_id(),
|
|
container_table.get_tablegroup_id(),
|
|
container_tablegroup_schema))) {
|
|
LOG_WARN("get tablegroup schema failed", KR(ret), K(container_table));
|
|
} else if (OB_ISNULL(container_tablegroup_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("container_tablegroup_schema is null", KR(ret), K(container_table));
|
|
}
|
|
}
|
|
if (FAILEDx(new_table_tablet_allocator.prepare(trans, container_table, container_tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare ls for container table schema tablets", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg(
|
|
tablet_schemas,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors/*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table partitions failed", KR(ret), K(tablet_schemas));
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("execute create partition failed", KR(ret));
|
|
}
|
|
}
|
|
|
|
// finishing is always invoked for new table tablet allocator
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) {
|
|
LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::create_tablets_in_trans_(ObIArray<ObTableSchema> &table_schemas,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
SCN frozen_scn;
|
|
share::schema::ObTableSchema *first_table = nullptr;
|
|
uint64_t tenant_id = OB_INVALID_ID;
|
|
if (table_schemas.count() <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table_schemas should not be emtpy", KR(ret), K(table_schemas.count()));
|
|
} else {
|
|
first_table = &table_schemas.at(0);
|
|
tenant_id = first_table->get_tenant_id();
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (OB_ISNULL(first_table)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("first_table is null", KR(ret));
|
|
} else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObTableCreator table_creator(
|
|
tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(
|
|
tenant_id,
|
|
schema_guard,
|
|
sql_proxy_);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
const ObTablegroupSchema *data_tablegroup_schema = NULL; // keep NULL if no tablegroup
|
|
if (OB_FAIL(table_creator.init(true/*need_tablet_cnt_check*/))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
} else if (OB_INVALID_ID != first_table->get_tablegroup_id()) {
|
|
if (OB_FAIL(schema_guard.get_tablegroup_schema(
|
|
first_table->get_tenant_id(),
|
|
first_table->get_tablegroup_id(),
|
|
data_tablegroup_schema))) {
|
|
LOG_WARN("get tablegroup_schema failed", KR(ret), KPC(first_table));
|
|
} else if (OB_ISNULL(data_tablegroup_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("data_tablegroup_schema is null", KR(ret), KPC(first_table));
|
|
}
|
|
}
|
|
|
|
ObArray<const ObTableSchema*> schemas;
|
|
ObArray<bool> need_create_empty_majors;
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
const share::schema::ObTableSchema &this_table = table_schemas.at(i);
|
|
const int64_t table_id = this_table.get_table_id();
|
|
if (!this_table.has_tablet()) {
|
|
} else if (!this_table.is_global_index_table()) {
|
|
if (OB_FAIL(schemas.push_back(&this_table)) || OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("failed to push_back", KR(ret), K(this_table));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(new_table_tablet_allocator.prepare(trans, this_table, data_tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare ls for index schema tablets", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(
|
|
ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_table_arg(
|
|
this_table,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
true /*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table partitions failed", KR(ret), K(this_table));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_INVALID_VERSION == last_schema_version
|
|
&& OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("get last schema version failed", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_ori_schema_version(
|
|
trans, tenant_id, table_id, last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!",
|
|
KR(ret), K(tenant_id), K(table_id), K(last_schema_version));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (schemas.count() <= 0) {
|
|
// virtual tablet and view skip
|
|
} else if (OB_FAIL(new_table_tablet_allocator.prepare(trans, *first_table, data_tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare ls for first table", KR(ret), KPC(first_table));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(
|
|
ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg(
|
|
schemas,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors/*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table partitions failed", KR(ret), KPC(first_table),
|
|
K(last_schema_version));
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("execute create partition failed", KR(ret));
|
|
}
|
|
|
|
// finishing is always invoked for new table tablet allocator
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) {
|
|
LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_tables_in_trans(const bool if_not_exist,
|
|
const ObString &ddl_stmt_str,
|
|
const ObErrorInfo &error_info,
|
|
ObIArray<ObTableSchema> &table_schemas,
|
|
const obrpc::ObSequenceDDLArg &sequence_ddl_arg,
|
|
const uint64_t last_replay_log_id,
|
|
const ObIArray<ObDependencyInfo> *dep_infos,
|
|
ObIArray<ObMockFKParentTableSchema> &mock_fk_parent_table_schema_array,
|
|
int64_t &ddl_task_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
RS_TRACE(create_tables_in_trans_begin);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (table_schemas.count() < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schemas have no element", K(ret));
|
|
} else if (OB_INVALID_ID == last_replay_log_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(last_replay_log_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
uint64_t tenant_id = table_schemas.at(0).get_tenant_id();
|
|
share::schema::ObTableSchema *first_table = &table_schemas.at(0);
|
|
uint64_t data_table_id = first_table->get_table_id();
|
|
ObArray<ObObjPriv> orig_obj_privs_ora;
|
|
const ObTableSchema *old_view_schema = NULL;
|
|
bool is_oracle_mode = false;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObDDLTaskRecord task_record;
|
|
ddl_task_id = 0;
|
|
ObMViewInfo mview_info;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans", KR(ret), K(tenant_id),
|
|
K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(first_table->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", KR(ret), KPC(first_table));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", 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();
|
|
if (OB_FAIL(schema_guard.get_table_schema(table_schema.get_tenant_id(),
|
|
table_schema.get_database_id(),
|
|
view_name,
|
|
false, //is index
|
|
old_view_schema))) {
|
|
LOG_WARN("failed to get table schema", K(view_name), K(ret));
|
|
} else if (OB_ISNULL(old_view_schema)) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
if (!is_oracle_mode) {
|
|
// no need to store obj privs
|
|
} else if (OB_FAIL(get_obj_privs_ora(table_schema.get_tenant_id(),
|
|
old_view_schema->get_table_id(),
|
|
static_cast<uint64_t>(ObObjectType::TABLE),
|
|
schema_guard,
|
|
orig_obj_privs_ora))) {
|
|
LOG_WARN("fial to get obj privs ora", KR(ret), K(table_schema.get_tenant_id()),
|
|
K(old_view_schema->get_table_id()));
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(drop_trigger_in_drop_table(trans, ddl_operator, schema_guard,
|
|
*old_view_schema, false))) {
|
|
// 兼容oracle,create or replace view时drop trigger, 且不进回收站
|
|
LOG_WARN("failed to drop trigger", KR(ret), K(old_view_schema->get_table_id()));
|
|
} else if (OB_FAIL(ddl_operator.drop_table(*old_view_schema, trans))) {
|
|
LOG_WARN("failed to drop old view schema", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
RS_TRACE(operator_create_table_begin);
|
|
|
|
// Fill index/lob table for system table
|
|
if (OB_SUCC(ret)
|
|
&& ObSysTableChecker::is_sys_table_has_index(data_table_id)) {
|
|
if (OB_FAIL(ObSysTableChecker::append_sys_table_index_schemas(
|
|
tenant_id, data_table_id, table_schemas))) {
|
|
LOG_WARN("fail to add sys table index", KR(ret), K(tenant_id), "table_id", data_table_id);
|
|
} else {
|
|
first_table = &table_schemas.at(0);
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && is_system_table(data_table_id)) {
|
|
HEAP_VARS_2((ObTableSchema, lob_meta_schema), (ObTableSchema, lob_piece_schema)) {
|
|
if (OB_FAIL(add_sys_table_lob_aux(tenant_id, data_table_id, lob_meta_schema, lob_piece_schema))) {
|
|
LOG_WARN("fail to get sys table lob aux schema", KR(ret), K(data_table_id));
|
|
} else if (OB_FAIL(table_schemas.push_back(lob_meta_schema))) {
|
|
LOG_WARN("fail to push back lob meta table", KR(ret), K(lob_meta_schema));
|
|
} else if (OB_FAIL(table_schemas.push_back(lob_piece_schema))) {
|
|
LOG_WARN("fail to push back lob piece table", KR(ret), K(lob_piece_schema));
|
|
} else {
|
|
first_table = &table_schemas.at(0);
|
|
}
|
|
}
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
ObTableSchema &table_schema = table_schemas.at(i);
|
|
if (OB_FAIL(ddl_operator.create_sequence_in_create_table(table_schema,
|
|
trans,
|
|
schema_guard,
|
|
&sequence_ddl_arg))) {
|
|
LOG_WARN("create sequence in create table fail", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.create_table(table_schema,
|
|
trans,
|
|
0 == i ? &tmp_ddl_stmt_str : NULL,
|
|
i == table_schemas.count() - 1))) {
|
|
LOG_WARN("failed to create table schema, ", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, table_schema))) {
|
|
LOG_WARN("failed to insert_temp_table_info!", K(ret));
|
|
} else if (table_schema.is_view_table() && dep_infos != nullptr && 0 == i) {
|
|
for (int64_t i = 0 ; OB_SUCC(ret) && i < dep_infos->count(); ++i) {
|
|
ObDependencyInfo dep;
|
|
if (OB_FAIL(dep.assign(dep_infos->at(i)))) {
|
|
LOG_WARN("failed to assign dependency info", K(ret));
|
|
} else {
|
|
dep.set_tenant_id(tenant_id);
|
|
dep.set_dep_obj_id(table_schema.get_table_id());
|
|
dep.set_dep_obj_owner_id(table_schema.get_table_id());
|
|
dep.set_schema_version(table_schema.get_schema_version());
|
|
OZ (dep.insert_schema_object_dependency(trans));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && (0 == i) && table_schema.is_view_table() &&
|
|
OB_NOT_NULL(old_view_schema) && is_oracle_mode) {
|
|
const uint64_t db_id = table_schema.get_database_id();
|
|
const ObSimpleDatabaseSchema *db_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, db_id, db_schema))) {
|
|
LOG_WARN("failed to get database schema", KR(ret), K(db_id));
|
|
} else if (OB_ISNULL(db_schema)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("db schema is NULL", KR(ret), K(tenant_id), K(db_id));
|
|
} else if (OB_FAIL(restore_obj_privs_for_table(table_schema.get_table_id(),
|
|
db_schema->get_database_name_str(),
|
|
table_schema.get_table_name_str(),
|
|
ddl_operator,
|
|
trans,
|
|
orig_obj_privs_ora))) {
|
|
LOG_WARN("restore_obj_privs_for_table failed", KR(ret), K(table_schema.get_table_id()),
|
|
K(db_schema->get_database_name_str()), K(table_schema.get_table_name_str()));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && (0 == i) && table_schema.is_materialized_view()) {
|
|
const uint64_t db_id = table_schema.get_database_id();
|
|
const ObSimpleDatabaseSchema *db_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, db_id, db_schema))) {
|
|
LOG_WARN("failed to get database schema", KR(ret), K(db_id));
|
|
} else if (OB_ISNULL(db_schema)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("db schema is NULL", KR(ret), K(tenant_id), K(db_id));
|
|
} else if (OB_FAIL(ObMViewSchedJobUtils::add_mview_info_and_refresh_job(trans,
|
|
tenant_id,
|
|
table_schema.get_table_id(),
|
|
db_schema->get_database_name_str(),
|
|
table_schema.get_table_name_str(),
|
|
table_schema.get_view_schema().get_mv_refresh_info(),
|
|
table_schema.get_schema_version(),
|
|
mview_info))) {
|
|
LOG_WARN("fail to start mview refresh job", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
// add error info for create force view
|
|
if (OB_SUCC(ret) && 1 == table_schemas.count() && first_table->is_user_view()) {
|
|
if (OB_LIKELY(ERROR_STATUS_HAS_ERROR != error_info.get_error_status())) {
|
|
/* do nothing */
|
|
} else if (OB_UNLIKELY(!is_oracle_mode)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected compat mode add create view error info", K(ret), K(is_oracle_mode));
|
|
} else {
|
|
ObErrorInfo tmp_error_info;
|
|
if (OB_FAIL(tmp_error_info.assign(error_info))) {
|
|
LOG_WARN("failed to assign error info", K(ret));
|
|
} else {
|
|
tmp_error_info.set_obj_id(first_table->get_table_id());
|
|
tmp_error_info.set_obj_type(static_cast<uint64_t>(ObObjectType::VIEW));
|
|
tmp_error_info.set_database_id(first_table->get_database_id());
|
|
tmp_error_info.set_tenant_id(first_table->get_tenant_id());
|
|
tmp_error_info.set_schema_version(first_table->get_schema_version());
|
|
if (OB_FAIL(tmp_error_info.handle_error_info(trans, NULL))) {
|
|
LOG_WARN("insert create error info failed.", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (FAILEDx(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(tenant_id));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (first_table->is_materialized_view()) {
|
|
if (OB_FAIL(create_tablets_in_trans_for_mv_(table_schemas, ddl_operator, trans, schema_guard, tenant_data_version))) {
|
|
LOG_WARN("fail to create tablets in trans for mv", KR(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(create_tablets_in_trans_(table_schemas, ddl_operator, trans, schema_guard, tenant_data_version))) {
|
|
LOG_WARN("fail to create tablets in trans", KR(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& first_table->is_materialized_view()
|
|
&& table_schemas.count() >= 2) {
|
|
ObTableSchema &mview_schema = table_schemas.at(0);
|
|
ObTableSchema &container_table_schema = table_schemas.at(1);
|
|
if (OB_FAIL(start_mview_complete_refresh_task(trans,
|
|
schema_guard,
|
|
mview_schema,
|
|
container_table_schema,
|
|
dep_infos,
|
|
allocator,
|
|
tenant_data_version,
|
|
mview_info,
|
|
task_record))) {
|
|
LOG_WARN("failed to start mview complete refresh task", KR(ret));
|
|
} else {
|
|
ddl_task_id = task_record.task_id_;
|
|
}
|
|
}
|
|
}
|
|
RS_TRACE(operator_create_table_end);
|
|
}
|
|
DEBUG_SYNC(BEFORE_CREATE_TABLE_TRANS_COMMIT);
|
|
if (OB_SUCC(ret)
|
|
&& THIS_WORKER.is_timeout_ts_valid()
|
|
&& THIS_WORKER.is_timeout()) {
|
|
ret = OB_TIMEOUT;
|
|
LOG_WARN("already timeout", KR(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = OB_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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
RS_TRACE(public_schema_begin);
|
|
if (OB_SUCC(ret)) {
|
|
DEBUG_SYNC(CREATE_TABLE_BEFORE_PUBLISH_SCHEMA);
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", KR(ret), K(tenant_id));
|
|
} else {
|
|
RS_TRACE(public_schema_end);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& first_table->is_materialized_view()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(GCTX.root_service_->get_ddl_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", KR(tmp_ret), K(task_record));
|
|
}
|
|
}
|
|
}
|
|
RS_TRACE(create_tables_in_trans_end);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::start_mview_complete_refresh_task(
|
|
ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &mview_schema,
|
|
const ObTableSchema &container_table_schema,
|
|
const ObIArray<ObDependencyInfo> *dep_infos,
|
|
common::ObIAllocator &allocator,
|
|
const uint64_t tenant_data_version,
|
|
const ObMViewInfo &mview_info,
|
|
ObDDLTaskRecord &task_record)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t max_dependency_version = 0;
|
|
uint64_t tenant_id = mview_schema.get_tenant_id();
|
|
uint64_t compat_version = 0;
|
|
const ObMVRefreshInfo *mv_refresh_info = mview_schema.get_view_schema().get_mv_refresh_info();
|
|
ObFixedLengthString<common::OB_MAX_TIMESTAMP_TZ_LENGTH> time_zone;
|
|
const ObSysVarSchema *data_format_schema = nullptr;
|
|
const ObSysVarSchema *nls_timestamp_format = nullptr;
|
|
const ObSysVarSchema *nls_timestamp_tz_format = nullptr;
|
|
obrpc::ObMViewCompleteRefreshArg arg;
|
|
arg.tenant_id_ = tenant_id;
|
|
arg.table_id_ = mview_schema.get_table_id();
|
|
arg.consumer_group_id_ = THIS_WORKER.get_group_id();
|
|
arg.session_id_ = 100;// FIXME
|
|
arg.exec_tenant_id_ = tenant_id;
|
|
if (OB_UNLIKELY(nullptr == dep_infos || nullptr == mv_refresh_info)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("dep_infos is nullptr", KR(ret) , KP(dep_infos), KP(mv_refresh_info));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("fail to get data version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(arg.last_refresh_scn_.convert_for_inner_table_field(mview_info.get_last_refresh_scn()))) {
|
|
LOG_WARN("fail to covert for inner table field", KR(ret), K(mview_info));
|
|
} else if (OB_FAIL(share::ObBackupUtils::get_tenant_sys_time_zone_wrap(tenant_id,
|
|
time_zone,
|
|
arg.tz_info_wrap_))) {
|
|
LOG_WARN("failed to get tenant sys timezoen wrap", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id,
|
|
share::SYS_VAR_NLS_DATE_FORMAT,
|
|
data_format_schema))) {
|
|
LOG_WARN("fail to get tenant system variable", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id,
|
|
share::SYS_VAR_NLS_TIMESTAMP_FORMAT,
|
|
nls_timestamp_format))) {
|
|
LOG_WARN("fail to get tenant system variable", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id,
|
|
share::SYS_VAR_NLS_TIMESTAMP_TZ_FORMAT,
|
|
nls_timestamp_tz_format))) {
|
|
LOG_WARN("fail to get tenant system variable", K(ret));
|
|
} else if (OB_ISNULL(data_format_schema) || OB_ISNULL(nls_timestamp_format) || OB_ISNULL(nls_timestamp_tz_format)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("var schema must not be null", K(ret));
|
|
} else {
|
|
arg.parallelism_ = mv_refresh_info->parallel_;
|
|
arg.tz_info_ = arg.tz_info_wrap_.get_tz_info_offset();
|
|
arg.nls_formats_[ObNLSFormatEnum::NLS_DATE] = data_format_schema->get_value();
|
|
arg.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP] = nls_timestamp_format->get_value();
|
|
arg.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP_TZ] = nls_timestamp_tz_format->get_value();
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < dep_infos->count(); ++i) {
|
|
const ObDependencyInfo &dep = dep_infos->at(i);
|
|
const ObSimpleTableSchemaV2 *base_table_schema = nullptr;
|
|
if ((compat_version < DATA_VERSION_4_3_5_0) && OB_UNLIKELY(ObObjectType::TABLE != dep.get_ref_obj_type())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("ref obj type is not table, not supported", KR(ret), K(dep));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "the ref obj type of materialized view not user table is");
|
|
} else if ((compat_version >= DATA_VERSION_4_3_5_0)
|
|
&& OB_UNLIKELY((ObObjectType::TABLE != dep.get_ref_obj_type())
|
|
&& ObObjectType::VIEW != dep.get_ref_obj_type())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("ref obj type is not table or mview, not supported", KR(ret), K(dep));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "the ref obj type of materialized view not user table or mview is");
|
|
} else if (OB_FAIL(
|
|
schema_guard.get_simple_table_schema(tenant_id, dep.get_ref_obj_id(), base_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(dep));
|
|
} else if (OB_ISNULL(base_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("base table schema is nullptr", K(ret));
|
|
} else {
|
|
ObBasedSchemaObjectInfo base_info;
|
|
base_info.schema_id_ = dep.get_ref_obj_id();
|
|
base_info.schema_type_ = ObSchemaType::TABLE_SCHEMA;
|
|
base_info.schema_version_ = base_table_schema->get_schema_version();
|
|
base_info.schema_tenant_id_ = tenant_id;
|
|
max_dependency_version = MAX(max_dependency_version, base_table_schema->get_schema_version());
|
|
if (OB_FAIL(arg.based_schema_object_infos_.push_back(base_info))) {
|
|
LOG_WARN("fail to push back base info", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_UNLIKELY(max_dependency_version > mview_schema.get_max_dependency_version())) {
|
|
ret = OB_OLD_SCHEMA_VERSION;
|
|
LOG_WARN("base table schema version is old",
|
|
KR(ret), K(max_dependency_version), KPC(dep_infos),
|
|
"old_max_dependency_version", mview_schema.get_max_dependency_version());
|
|
} else {
|
|
ObCreateDDLTaskParam param(tenant_id,
|
|
ObDDLType::DDL_CREATE_MVIEW,
|
|
&mview_schema,
|
|
&container_table_schema,
|
|
0/*object_id*/,
|
|
container_table_schema.get_schema_version(),
|
|
arg.parallelism_,
|
|
arg.consumer_group_id_,
|
|
&allocator,
|
|
&arg);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (OB_FAIL(GCTX.root_service_->get_ddl_task_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit create index ddl task failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
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_index_or_mlog_table_in_trans(
|
|
ObTableSchema &table_schema,
|
|
const ObString *ddl_stmt_str,
|
|
ObMySQLTransaction *sql_trans,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const bool need_check_tablet_cnt,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
ObDDLSQLTransaction tmp_trans(schema_service_);
|
|
ObMySQLTransaction &trans = OB_ISNULL(sql_trans) ? tmp_trans : *sql_trans;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(sql_trans)
|
|
&& OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.create_table(table_schema,
|
|
trans,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("failed to create table schema, ", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, table_schema))) {
|
|
LOG_WARN("failed to insert temp table info!", KR(ret));
|
|
} else {
|
|
LOG_INFO("succeed to insert table schema in schema tables",
|
|
K(table_schema.get_tenant_id()),
|
|
K(table_schema.get_database_id()),
|
|
K(table_schema.get_table_id()),
|
|
K(table_schema.get_table_name()));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("fail to get last schema version", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans, tenant_id, table_schema.get_table_id(), last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!", KR(ret), K(tenant_id), K(last_schema_version));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && table_schema.has_tablet()) {
|
|
if (table_schema.is_mlog_table()) {
|
|
if (OB_FAIL(create_mlog_tablet(trans, schema_guard, table_schema, need_check_tablet_cnt, tenant_data_version))) {
|
|
LOG_WARN("failed to create_mlog_tablet", KR(ret), K(table_schema));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(create_index_tablet(table_schema, trans, schema_guard, need_check_tablet_cnt, tenant_data_version))) {
|
|
LOG_WARN("fail to create_index_tablet", KR(ret), K(table_schema));
|
|
}
|
|
}
|
|
}
|
|
if (OB_ISNULL(sql_trans) && trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
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("duplicated 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, "duplicated table in tablegroup");
|
|
} else {
|
|
if (OB_SUCC(ret)
|
|
&& alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::TABLEGROUP_NAME)) {
|
|
ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(helper.check_table_alter_tablegroup(
|
|
schema_guard, NULL, orig_table_schema, new_table_schema))) {
|
|
LOG_WARN("fail to check table schema in tablegroup", K(ret));
|
|
} else {} // good
|
|
}
|
|
bool alter_collation = alter_table_schema.alter_option_bitset_.has_member(
|
|
obrpc::ObAlterTableArg::COLLATION_TYPE);
|
|
bool alter_charset = alter_table_schema.alter_option_bitset_.has_member(
|
|
obrpc::ObAlterTableArg::CHARSET_TYPE);
|
|
if (OB_SUCC(ret) && (alter_collation || alter_charset)) {
|
|
ObCharsetType charset_type = alter_table_schema.get_charset_type();
|
|
ObCollationType collation_type = alter_table_schema.get_collation_type();
|
|
if (alter_collation && alter_charset) {
|
|
if (!ObCharset::is_valid_collation(charset_type, collation_type)) {
|
|
ret = OB_ERR_COLLATION_MISMATCH;
|
|
const char *cs_name = ObCharset::charset_name(charset_type);
|
|
const char *coll_name = ObCharset::collation_name(collation_type);
|
|
ObString charset = ObString::make_string(cs_name);
|
|
ObString collation = ObString::make_string(coll_name);
|
|
LOG_USER_ERROR(OB_ERR_COLLATION_MISMATCH, collation.length(), collation.ptr(),
|
|
charset.length(), charset.ptr());
|
|
} else {
|
|
new_table_schema.set_collation_type(collation_type);
|
|
new_table_schema.set_charset_type(charset_type);
|
|
}
|
|
} else if (alter_collation) {
|
|
new_table_schema.set_collation_type(collation_type);
|
|
new_table_schema.set_charset_type(ObCharset::charset_type_by_coll(collation_type));
|
|
} else if (alter_charset) {
|
|
new_table_schema.set_collation_type(ObCharset::get_default_collation(charset_type));
|
|
new_table_schema.set_charset_type(charset_type);
|
|
}
|
|
}
|
|
}
|
|
LOG_DEBUG("set new table options", K(alter_table_arg), K(alter_table_schema),
|
|
K(tenant_schema), K(new_table_schema), K(orig_table_schema));
|
|
return ret;
|
|
}
|
|
|
|
//set new table options to new table schema
|
|
int ObDDLService::set_raw_table_options(
|
|
const AlterTableSchema &alter_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
bool &need_update_index_table)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
//replace alter options
|
|
need_update_index_table = false;
|
|
uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
for (int32_t i = ObAlterTableArg::AUTO_INCREMENT;
|
|
OB_SUCC(ret) && i < ObAlterTableArg::MAX_OPTION; ++i) {
|
|
if (alter_table_schema.alter_option_bitset_.has_member(i)) {
|
|
switch (i) {
|
|
case ObAlterTableArg::TABLE_DOP: {
|
|
new_table_schema.set_dop(alter_table_schema.get_dop());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::AUTO_INCREMENT: {
|
|
new_table_schema.set_auto_increment(alter_table_schema.get_auto_increment());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::BLOCK_SIZE: {
|
|
need_update_index_table = true;
|
|
new_table_schema.set_block_size(alter_table_schema.get_block_size());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::CHARSET_TYPE: {
|
|
new_table_schema.set_charset_type(alter_table_schema.get_charset_type());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::COLLATION_TYPE: {
|
|
new_table_schema.set_collation_type(alter_table_schema.get_collation_type());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::STORE_FORMAT: {
|
|
new_table_schema.set_row_store_type(alter_table_schema.get_row_store_type());
|
|
new_table_schema.set_store_format(alter_table_schema.get_store_format());
|
|
need_update_index_table = true;
|
|
break;
|
|
}
|
|
case ObAlterTableArg::STORAGE_FORMAT_VERSION: {
|
|
new_table_schema.set_storage_format_version(alter_table_schema.get_storage_format_version());
|
|
need_update_index_table = true;
|
|
break;
|
|
}
|
|
case ObAlterTableArg::COMPRESS_METHOD: {
|
|
ret = new_table_schema.set_compress_func_name(alter_table_schema.get_compress_func_name());
|
|
need_update_index_table = true;
|
|
break;
|
|
}
|
|
case ObAlterTableArg::COMMENT: {
|
|
ret = new_table_schema.set_comment(alter_table_schema.get_comment_str());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::EXPIRE_INFO: {
|
|
ret = new_table_schema.set_expire_info(alter_table_schema.get_expire_info());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::PRIMARY_ZONE: {
|
|
LOG_INFO("changing table's primay_zone does not take effect");
|
|
ret = OB_SUCCESS; // do nothing
|
|
break;
|
|
}
|
|
case ObAlterTableArg::REPLICA_NUM: {
|
|
// ignore alter replica num
|
|
break;
|
|
}
|
|
case ObAlterTableArg::TABLET_SIZE: {
|
|
new_table_schema.set_tablet_size(alter_table_schema.get_tablet_size());
|
|
need_update_index_table = true;
|
|
break;
|
|
}
|
|
case ObAlterTableArg::PCTFREE: {
|
|
new_table_schema.set_pctfree(alter_table_schema.get_pctfree());
|
|
need_update_index_table = true;
|
|
break;
|
|
}
|
|
case ObAlterTableArg::PROGRESSIVE_MERGE_NUM: {
|
|
need_update_index_table = true;
|
|
new_table_schema.set_progressive_merge_num(alter_table_schema.get_progressive_merge_num());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::PROGRESSIVE_MERGE_ROUND: {
|
|
need_update_index_table = true;
|
|
new_table_schema.set_progressive_merge_round(alter_table_schema.get_progressive_merge_round());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::TABLE_NAME: {
|
|
//check table rename
|
|
const ObString &new_table_name = alter_table_schema.get_table_name_str();
|
|
const ObString &origin_table_name = alter_table_schema.get_origin_table_name();
|
|
const ObString &new_database_name = alter_table_schema.get_database_name();
|
|
const ObString &origin_database_name = alter_table_schema.get_origin_database_name();
|
|
ObNameCaseMode mode = OB_NAME_CASE_INVALID;
|
|
bool is_oracle_mode = 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(schema_guard.get_tenant_name_case_mode(tenant_id, mode))) {
|
|
LOG_WARN("fail to get tenant name case mode", K(tenant_id), K(ret));
|
|
} else if (OB_FAIL(orig_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get compat mode", K(ret), K(tenant_id));
|
|
} else {
|
|
//TODO rename database_name need to update all index table @hualong
|
|
ObString database_name;
|
|
if (!is_oracle_mode) {
|
|
// mysql mode
|
|
if (!new_database_name.empty() && !ObCharset::case_mode_equal(mode, new_database_name, origin_database_name)) {
|
|
database_name = new_database_name;
|
|
} else {
|
|
database_name = origin_database_name;
|
|
}
|
|
} else {
|
|
// oracle mode
|
|
if (!new_database_name.empty() && !ObCharset::case_sensitive_equal(new_database_name, origin_database_name)) {
|
|
database_name = new_database_name;
|
|
} else {
|
|
database_name = origin_database_name;
|
|
}
|
|
}
|
|
ObString table_name;
|
|
uint64_t database_id = OB_INVALID_ID;
|
|
if (!is_oracle_mode) {
|
|
// mysql mode
|
|
if (!new_table_name.empty() && !ObCharset::case_mode_equal(mode, new_table_name, origin_table_name)) {
|
|
table_name = new_table_name;
|
|
} else {
|
|
table_name = origin_table_name;
|
|
}
|
|
} else {
|
|
// oracle mode
|
|
if (!new_table_name.empty() && !ObCharset::case_sensitive_equal(new_table_name, origin_table_name)) {
|
|
table_name = new_table_name;
|
|
} else {
|
|
table_name = origin_table_name;
|
|
}
|
|
}
|
|
if (OB_FAIL(schema_guard.get_database_id(tenant_id, database_name, database_id))) {
|
|
LOG_WARN("fail to get database id", K(tenant_id), K(database_name), K(ret));
|
|
} else if (database_name != origin_database_name || table_name != origin_table_name) {
|
|
if (OB_UNLIKELY(OB_RECYCLEBIN_SCHEMA_ID == database_id
|
|
|| OB_PUBLIC_SCHEMA_ID == database_id)) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("rename table to hidden database is not allowd",
|
|
KR(ret), K(database_id), K(database_name), K(table_name));
|
|
} else if (!is_oracle_mode) {
|
|
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", KR(ret), K(tenant_id), K(database_name), K(table_name));
|
|
} else if (OB_NOT_NULL(synonym_info)) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", KR(ret), K(database_name), K(table_name));
|
|
} 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", KR(ret), K(tenant_id), K(database_name), K(table_name));
|
|
} else if (OB_ISNULL(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());
|
|
}
|
|
} else {
|
|
ObArray<ObSchemaType> conflict_schema_types;
|
|
if (OB_FAIL(schema_guard.check_oracle_object_exist(
|
|
tenant_id, database_id, table_name, TABLE_SCHEMA,
|
|
INVALID_ROUTINE_TYPE, false /*if_not_eixst*/, conflict_schema_types))) {
|
|
LOG_WARN("fail to check oracle object exist", KR(ret), K(tenant_id),
|
|
K(database_id), K(database_name), K(table_name));
|
|
} else if (conflict_schema_types.count() > 0) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object in oralce mode",
|
|
KR(ret), K(tenant_id), K(database_id), K(database_name),
|
|
K(table_name), K(conflict_schema_types));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_table_schema.set_table_name(table_name))) {
|
|
LOG_WARN("fail to set table name", K(table_name), K(ret));
|
|
} else {
|
|
need_update_index_table = true;
|
|
new_table_schema.set_database_id(database_id);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& OB_FAIL(check_cst_name_dup_for_rename_table_mysql(
|
|
schema_guard, orig_table_schema, new_table_schema.get_database_id()))) {
|
|
LOG_WARN("fail to check cst name dup for rename table mysql", K(ret));
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::TABLEGROUP_NAME: {
|
|
const ObString &tablegroup_name = alter_table_schema.get_tablegroup_name();
|
|
if (!tablegroup_name.empty()) {
|
|
uint64_t tablegroup_id = OB_INVALID_ID;
|
|
uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
//tablegroup_id not set in resolver, only record tablegroup name
|
|
if (OB_FAIL(schema_guard.get_tablegroup_id(tenant_id,
|
|
tablegroup_name,
|
|
tablegroup_id))) {
|
|
LOG_WARN("failed to get tablegroup id", K(ret), K(tablegroup_name));
|
|
} else if (OB_INVALID_ID == tablegroup_id) {
|
|
ret = OB_TABLEGROUP_NOT_EXIST;
|
|
LOG_WARN("invalid tablegroup name", K(ret), K(tablegroup_name));
|
|
} else {
|
|
new_table_schema.set_tablegroup_id(tablegroup_id);
|
|
need_update_index_table = true;
|
|
}
|
|
} else {
|
|
new_table_schema.set_tablegroup_id(OB_INVALID_ID);
|
|
need_update_index_table = true;
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::SEQUENCE_COLUMN_ID: {
|
|
//TODO
|
|
break;
|
|
}
|
|
case ObAlterTableArg::USE_BLOOM_FILTER: {
|
|
new_table_schema.set_is_use_bloomfilter(alter_table_schema.is_use_bloomfilter());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::READ_ONLY: {
|
|
new_table_schema.set_read_only(alter_table_schema.is_read_only());
|
|
need_update_index_table = true;
|
|
break;
|
|
}
|
|
case ObAlterTableArg::LOCALITY: {
|
|
if (is_sys_table(new_table_schema.get_table_id())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("change system table's locality is not allowed", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "change system table's locality");
|
|
} else {
|
|
LOG_INFO("changing table's locality does not take effect");
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::SESSION_ID: {
|
|
bool is_index = false;
|
|
const ObString &table_name = new_table_schema.get_table_name_str();
|
|
const uint64_t database_id = new_table_schema.get_database_id();
|
|
uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
const ObTableSchema *found_table_schema = NULL;
|
|
if (0 == new_table_schema.get_session_id()
|
|
|| alter_table_schema.get_session_id() == new_table_schema.get_session_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table session id is reset already", K(ret), K(new_table_schema), K(alter_table_schema));
|
|
} else {
|
|
// bug18197606, In the process of querying or building a table,
|
|
// another session creates a table with the same name,
|
|
// Modify the first two can lead to the same name table and session_id = 0;
|
|
// It is necessary to check whether there is a normal table with session_id = 0
|
|
// in the last step of querying the table creation. If it is error CTAS will be rolled back.
|
|
uint64_t org_session_id = schema_guard.get_session_id();
|
|
schema_guard.set_session_id(alter_table_schema.get_session_id());
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
database_id,
|
|
table_name,
|
|
is_index,
|
|
found_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id),
|
|
K(database_id), K(table_name), K(is_index), K(ret));
|
|
} else if (OB_NOT_NULL(found_table_schema)) {
|
|
ret = OB_ERR_TABLE_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_TABLE_EXIST, table_name.length(), table_name.ptr());
|
|
} else {
|
|
new_table_schema.set_session_id(alter_table_schema.get_session_id());
|
|
}
|
|
schema_guard.set_session_id(org_session_id);
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::SESSION_ACTIVE_TIME: {
|
|
new_table_schema.set_sess_active_time(alter_table_schema.get_sess_active_time());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::DUPLICATE_SCOPE: {
|
|
// alter table duplicate scope not allowed in master now
|
|
new_table_schema.set_duplicate_attribute(alter_table_schema.get_duplicate_scope(),
|
|
alter_table_schema.get_duplicate_read_consistency());
|
|
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: {
|
|
uint64_t compat_version = 0;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (not_compat_for_queuing_mode(compat_version) && alter_table_schema.is_new_queuing_table_mode()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN(QUEUING_MODE_NOT_COMPAT_WARN_STR, K(ret), K(compat_version), K(alter_table_schema.get_table_mode()));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, QUEUING_MODE_NOT_COMPAT_USER_ERROR_STR);
|
|
} else {
|
|
new_table_schema.set_table_mode(alter_table_schema.get_table_mode());
|
|
need_update_index_table = true;
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::INCREMENT_MODE : {
|
|
new_table_schema.set_table_auto_increment_mode(
|
|
alter_table_schema.get_table_auto_increment_mode());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::INCREMENT_CACHE_SIZE : {
|
|
uint64_t data_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (data_version < MOCK_DATA_VERSION_4_2_3_0 ||
|
|
(data_version >= DATA_VERSION_4_3_0_0 && data_version < DATA_VERSION_4_3_2_0)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("table auto_increment_cache_size less than 4.2.3 not support", K(ret),
|
|
K(data_version));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "table auto_increment_cache_size less than 4.2.3");
|
|
} else {
|
|
new_table_schema.set_auto_increment_cache_size(
|
|
alter_table_schema.get_auto_increment_cache_size());
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::ENABLE_EXTENDED_ROWID: {
|
|
new_table_schema.set_table_rowid_mode(alter_table_schema.get_table_rowid_mode());
|
|
break;
|
|
}
|
|
case ObAlterTableArg::ENCRYPTION: {
|
|
if (OB_FAIL(new_table_schema.set_encryption_str(alter_table_schema.get_encryption_str()))) {
|
|
LOG_WARN("fail to set encryption_str", K(ret), K(alter_table_schema.get_encryption_str()));
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::TABLESPACE_ID: {
|
|
new_table_schema.set_tablespace_id(alter_table_schema.get_tablespace_id());
|
|
if (OB_FAIL(new_table_schema.set_encryption_str(alter_table_schema.get_encryption_str()))) {
|
|
LOG_WARN("fail to set encryption_str", K(ret), K(alter_table_schema.get_encryption_str()));
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::TTL_DEFINITION: {
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (compat_version < DATA_VERSION_4_2_1_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("ttl definition less than 4.2.1 not support", K(ret), K(compat_version));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "ttl definition less than 4.2.1");
|
|
} else if (OB_FAIL(new_table_schema.set_ttl_definition(alter_table_schema.get_ttl_definition()))) {
|
|
LOG_WARN("fail to set ttl definition", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::KV_ATTRIBUTES: {
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (compat_version < DATA_VERSION_4_2_1_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("kv attributes less than 4.2.1 not support", K(ret), K(compat_version));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "kv attributes less than 4.2");
|
|
} else if (OB_FAIL(new_table_schema.set_kv_attributes(alter_table_schema.get_kv_attributes()))) {
|
|
LOG_WARN("fail to set kv attributes", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case ObAlterTableArg::LOB_INROW_THRESHOLD: {
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (compat_version < DATA_VERSION_4_2_1_2) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("lob inrow threshold less than 4.2.1.2 not support", K(ret), K(compat_version));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "lob inrow threshold less than 4.2.1.2");
|
|
} else {
|
|
new_table_schema.set_lob_inrow_threshold(alter_table_schema.get_lob_inrow_threshold());
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Unknown option!", K(i));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_locality_compatible_(
|
|
ObTenantSchema &schema,
|
|
const bool for_create_tenant)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
common::ObArray<share::ObZoneReplicaAttrSet> zone_locality;
|
|
const uint64_t tenant_id = for_create_tenant ? OB_SYS_TENANT_ID : schema.get_tenant_id();
|
|
bool is_compatible_with_readonly_replica = false;
|
|
bool is_compatible_with_columnstore_replica = false;
|
|
if (OB_FAIL(ObShareUtil::check_compat_version_for_readonly_replica(
|
|
tenant_id, is_compatible_with_readonly_replica))) {
|
|
LOG_WARN("fail to check compatible with readonly replica", KR(ret), K(schema));
|
|
} else if (OB_FAIL(ObShareUtil::check_compat_version_for_columnstore_replica(
|
|
tenant_id, is_compatible_with_columnstore_replica))) {
|
|
LOG_WARN("fail to check compatible with columnstore replica", KR(ret), K(schema));
|
|
} else if (OB_FAIL(schema.get_zone_replica_attr_array(zone_locality))) {
|
|
LOG_WARN("fail to get locality from schema", K(ret), K(schema));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < zone_locality.count(); ++i) {
|
|
const share::ObZoneReplicaAttrSet &this_set = zone_locality.at(i);
|
|
if (this_set.zone_set_.count() <= 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("zone set count unexpected", K(ret), "zone_set_cnt", this_set.zone_set_.count());
|
|
} else if (! is_compatible_with_readonly_replica
|
|
&& 0 != this_set.get_readonly_replica_num()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can not create tenant with read-only replica below data version 4.2", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Create tenant with R-replica in locality below data version 4.2");
|
|
} else if (! is_compatible_with_columnstore_replica
|
|
&& 0 != this_set.get_columnstore_replica_num()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can not create tenant with column-store replica below data version 4.3.3", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Create tenant with C-replica in locality below data version 4.3.3");
|
|
} else if (GCTX.is_shared_storage_mode()
|
|
&& 0 != this_set.get_columnstore_replica_num()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can not create tenant with column-store replica in shared-storage mode", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "In shared-storage mode, C-replica is");
|
|
}
|
|
}
|
|
}
|
|
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::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::check_hidden_index_exist(
|
|
const uint64 tenant_id,
|
|
const uint64_t database_id,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const ObString &index_name,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
bool &is_exist)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
is_exist = false;
|
|
ObArray<const ObSimpleTableSchemaV2 *> table_schemas;
|
|
if (OB_FAIL(hidden_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("check if oracle mode failed", K(ret), K(hidden_table_schema));
|
|
} else if (is_oracle_mode) {
|
|
// in oracle mode, we need iterate hidden table map within database
|
|
if (OB_FAIL(schema_guard.get_table_schemas_in_database(tenant_id, database_id, table_schemas))) {
|
|
LOG_WARN("get table schemas in database", K(ret));
|
|
}
|
|
} else {
|
|
// is mysql mode, just check index_name exists in hidden_table_schema
|
|
if (OB_FAIL(schema_guard.get_index_schemas_with_data_table_id(tenant_id, hidden_table_schema.get_table_id(), table_schemas))) {
|
|
LOG_WARN("get index schemas failed", K(ret));
|
|
}
|
|
}
|
|
|
|
// check if the index name exist in index_schemas array
|
|
if (OB_SUCC(ret)) {
|
|
ObString tmp_index_name;
|
|
ObCompareNameWithTenantID column_name_cmp(hidden_table_schema.get_tenant_id());
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count() && !is_exist; ++i) {
|
|
const ObSimpleTableSchemaV2 *table_schema = table_schemas.at(i);
|
|
if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, index schema must not be nullptr", K(ret));
|
|
} else if (table_schema->is_index_table() && OB_FAIL(table_schema->get_index_name(tmp_index_name))) {
|
|
LOG_WARN("get index name failed", K(ret));
|
|
} else {
|
|
is_exist = 0 == column_name_cmp.compare(tmp_index_name, index_name);
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// Used for alter table xxx drop primary key.
|
|
// reset origin rowkey info and add heap table hidden pk column.
|
|
int ObDDLService::drop_primary_key(
|
|
ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// step1: clear origin primary key.
|
|
const ObRowkeyInfo &rowkey_info = new_table_schema.get_rowkey_info();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_info.get_size(); i++) {
|
|
const ObRowkeyColumn *rowkey_column = rowkey_info.get_column(i);
|
|
ObColumnSchemaV2 *col = nullptr;
|
|
if (OB_ISNULL(col = new_table_schema.get_column_schema(rowkey_column->column_id_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is nullptr", K(ret), K(rowkey_column->column_id_), K(new_table_schema));
|
|
} else if (OB_HIDDEN_SESSION_ID_COLUMN_ID == col->get_column_id()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Dropping primary key of global temporary table");
|
|
} else {
|
|
col->set_rowkey_position(0);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// step2: set table pk mode and origanization mode.
|
|
new_table_schema.set_table_pk_mode(ObTablePKMode::TPKM_TABLET_SEQ_PK);
|
|
new_table_schema.set_table_organization_mode(ObTableOrganizationMode::TOM_HEAP_ORGANIZED);
|
|
// step3: add hidden pk column.
|
|
new_table_schema.reset_rowkey_info();
|
|
int32_t rowkey_position = 1;
|
|
ObColumnSchemaV2 hidden_pk;
|
|
hidden_pk.reset();
|
|
hidden_pk.set_column_id(OB_HIDDEN_PK_INCREMENT_COLUMN_ID);
|
|
hidden_pk.set_data_type(ObUInt64Type);
|
|
hidden_pk.set_nullable(false);
|
|
hidden_pk.set_is_hidden(true);
|
|
hidden_pk.set_charset_type(CHARSET_BINARY);
|
|
hidden_pk.set_collation_type(CS_TYPE_BINARY);
|
|
if (OB_FAIL(hidden_pk.set_column_name(OB_HIDDEN_PK_INCREMENT_COLUMN_NAME))) {
|
|
LOG_WARN("failed to set column name", K(ret));
|
|
} else {
|
|
hidden_pk.set_rowkey_position(rowkey_position);
|
|
if (OB_FAIL(new_table_schema.add_column(hidden_pk))) {
|
|
LOG_WARN("add column to table_schema failed", K(ret), K(hidden_pk));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_primary_key(const ObIArray<ObString> &pk_column_names, ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// step1: clear origin primary key
|
|
ObTableSchema::const_column_iterator tmp_begin = new_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator tmp_end = new_table_schema.column_end();
|
|
ObColumnSchemaV2 *del_hidden_pk_column = nullptr;
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) {
|
|
ObColumnSchemaV2 *col = (*tmp_begin);
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
} else if (OB_HIDDEN_PK_INCREMENT_COLUMN_ID == col->get_column_id()) {
|
|
del_hidden_pk_column = col;
|
|
} else if (OB_HIDDEN_SESSION_ID_COLUMN_ID == col->get_column_id()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Adding primary key to global temporary table");
|
|
} else {
|
|
col->set_rowkey_position(0);
|
|
}
|
|
new_table_schema.set_table_pk_mode(ObTablePKMode::TPKM_OLD_NO_PK);
|
|
new_table_schema.set_table_organization_mode(ObTableOrganizationMode::TOM_INDEX_ORGANIZED);
|
|
}
|
|
|
|
if (OB_SUCC(ret) && nullptr != del_hidden_pk_column) {
|
|
// delete hidden primary key
|
|
if (OB_FAIL(new_table_schema.delete_column(del_hidden_pk_column->get_column_name_str()))) {
|
|
LOG_WARN("fail to delete hidden primary key", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
// step2: set new primary key rowkey_position
|
|
int64_t rowkey_position = 1;
|
|
new_table_schema.reset_rowkey_info();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < pk_column_names.count(); i++) {
|
|
ObColumnSchemaV2 *col_schema = nullptr;
|
|
const ObString &col_name = pk_column_names.at(i);
|
|
if (OB_ISNULL(col_schema = new_table_schema.get_column_schema(col_name))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected err, can not find column", K(ret), K(col_name), K(new_table_schema));
|
|
} else {
|
|
col_schema->set_rowkey_position(rowkey_position++);
|
|
col_schema->set_nullable(false);
|
|
if (OB_FAIL(new_table_schema.set_rowkey_info(*col_schema))) {
|
|
LOG_WARN("set rowkey info failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
LOG_WARN("fail to check primary key cover partition column", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_hidden_table_with_pk_changed(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObSArray<ObString> &index_columns,
|
|
const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObArenaAllocator &allocator,
|
|
const ObIndexArg::IndexActionType &index_action_type,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool bind_tablets = false;
|
|
ObString index_name("");
|
|
const bool is_drop_pk = ObIndexArg::DROP_PRIMARY_KEY == index_action_type;
|
|
const bool is_add_or_alter_pk = (ObIndexArg::ADD_PRIMARY_KEY == index_action_type) || (ObIndexArg::ALTER_PRIMARY_KEY == index_action_type);
|
|
// For add primary key and modify column in one sql, create user hidden table when modifing column.
|
|
const bool create_user_hidden_table_now = !(ObIndexArg::ADD_PRIMARY_KEY == index_action_type && alter_table_arg.is_alter_columns_);
|
|
if ((!is_drop_pk && !is_add_or_alter_pk)
|
|
|| (is_drop_pk && 0 != index_columns.count())
|
|
|| (is_add_or_alter_pk && 0 == index_columns.count())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(index_action_type), K(index_columns));
|
|
} else if (is_add_or_alter_pk && OB_FAIL(add_primary_key(index_columns, new_table_schema))) {
|
|
LOG_WARN("failed to add pk", K(ret), K(index_columns), K(new_table_schema));
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (new_table_schema.is_auto_partitioned_table()) {
|
|
bool is_match_rowkey_prefix = true;
|
|
if (is_drop_pk) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to drop primary keys of auto-partitioned table", KR(ret), K(index_action_type),
|
|
K(new_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop primary keys of auto-partitioning table is");
|
|
} else if (new_table_schema.get_part_level() == PARTITION_LEVEL_ZERO) {
|
|
if (OB_FAIL(new_table_schema.is_presetting_partition_key_match_rowkey_prefix(is_match_rowkey_prefix))){
|
|
LOG_WARN("fail to check whether presetting partition key matches rowkey prefix", KR(ret), K(new_table_schema));
|
|
}
|
|
} else if (OB_FAIL(new_table_schema.is_partition_key_match_rowkey_prefix(is_match_rowkey_prefix))) {
|
|
LOG_WARN("fail to check whether partition key matches rowkey prefix", KR(ret), K(new_table_schema));
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!is_match_rowkey_prefix) {
|
|
// auto-partitioning require that partition key must be equal to primary key prefix.
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("mismatching between primary key prefix and partition key", KR(ret), K(index_action_type),
|
|
K(new_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "mismatching between primary key prefix and partition key is");
|
|
} else if (!new_table_schema.is_partitioned_table() &&
|
|
origin_table_schema.get_part_option().get_part_func_expr_str().empty()) {
|
|
// for auto-partitioned non-partitioned table, empty part_func_expr means the presetting partition key
|
|
// of the data_table will always be "current" primary key.
|
|
// thus, when modify primary key, we need to check whether the part_func_type is need to be modified
|
|
const bool is_single_pk_column_new = new_table_schema.get_rowkey_column_num() == 1;
|
|
ObPartitionFuncType part_func_type = PARTITION_FUNC_TYPE_RANGE;
|
|
if (is_single_pk_column_new) {
|
|
ObObjMeta type;
|
|
ObRowkeyColumn row_key_col;
|
|
const common::ObRowkeyInfo &row_key_info = new_table_schema.get_rowkey_info();
|
|
if (OB_FAIL(row_key_info.get_column(0/*since there is only one row key, we only need to check the first one*/, row_key_col))) {
|
|
LOG_WARN("get row key column failed", K(ret), K(row_key_info));
|
|
} else {
|
|
type = row_key_col.get_meta_type();
|
|
}
|
|
if (ObResolverUtils::is_partition_range_column_type(type.get_type())) {
|
|
part_func_type = PARTITION_FUNC_TYPE_RANGE_COLUMNS;
|
|
}
|
|
} else {
|
|
part_func_type = PARTITION_FUNC_TYPE_RANGE_COLUMNS;
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
new_table_schema.get_part_option().set_part_func_type(part_func_type);
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_drop_pk && OB_FAIL(drop_primary_key(new_table_schema))) {
|
|
LOG_WARN("failed to add hidden pk column for heap table", K(ret));
|
|
} else if (!create_user_hidden_table_now) {
|
|
} else if (OB_FAIL(adjust_cg_for_offline(new_table_schema))) {
|
|
LOG_WARN("failed to adjust for create hiddent table with pk changed", K(ret));
|
|
} else if (OB_FAIL(get_add_pk_index_name(origin_table_schema,
|
|
new_table_schema,
|
|
index_action_type,
|
|
alter_table_arg.index_arg_list_,
|
|
schema_guard,
|
|
index_name))) {
|
|
LOG_WARN("fail to rename hidden table's pk constraint", K(ret));
|
|
} else if (OB_FAIL(create_user_hidden_table(origin_table_schema,
|
|
new_table_schema,
|
|
&alter_table_arg.sequence_ddl_arg_,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
allocator,
|
|
tenant_data_version,
|
|
index_name))) {
|
|
LOG_WARN("failed to alter table offline", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_add_pk_index_name(const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
const ObIndexArg::IndexActionType &index_action_type,
|
|
const ObIArray<ObIndexArg *> &index_arg_list,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObString &index_name)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
bool is_exist = false;
|
|
|
|
if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check oracle mode", K(ret));
|
|
} else if (is_oracle_mode && index_action_type == ObIndexArg::ADD_PRIMARY_KEY) {
|
|
// find pk name;
|
|
ObIndexArg *tmp_index_arg = nullptr;
|
|
bool found = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && !found && i < index_arg_list.count(); i++) {
|
|
if (OB_ISNULL(tmp_index_arg = index_arg_list.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get index arg", K(ret));
|
|
} else if (tmp_index_arg->index_action_type_ == ObIndexArg::ADD_PRIMARY_KEY) {
|
|
found = true;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && found && tmp_index_arg->index_name_.length() != 0) {
|
|
index_name.assign_ptr(tmp_index_arg->index_name_.ptr(),
|
|
tmp_index_arg->index_name_.length());
|
|
// check constraint name is not duplicated.
|
|
if (OB_FAIL(check_constraint_name_is_exist(schema_guard,
|
|
origin_table_schema,
|
|
index_name,
|
|
false,
|
|
is_exist))) {
|
|
LOG_WARN("fail to check constraint exist", K(ret));
|
|
} else if (is_exist) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_WARN("check constraint name is duplicate", K(ret), K(index_name));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_change_column_order(
|
|
const ObTableSchema &table_schema,
|
|
const AlterColumnSchema &alter_column_schema,
|
|
bool &is_change_column_order) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObString &this_name = alter_column_schema.get_origin_column_name();
|
|
const ObString &next_name = alter_column_schema.get_next_column_name();
|
|
const ObString &prev_name = alter_column_schema.get_prev_column_name();
|
|
const bool is_first = alter_column_schema.is_first_;
|
|
const bool is_before = !next_name.empty();
|
|
const bool is_after = !prev_name.empty();
|
|
const int flag_cnt = static_cast<int>(is_first) + static_cast<int>(is_before) + static_cast<int>(is_after);
|
|
const ObSchemaOperationType op_type = alter_column_schema.alter_type_;
|
|
if (OB_UNLIKELY(1 < flag_cnt)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("change column order flag conflict", K(ret), K(is_first), K(is_before), K(is_after));
|
|
} else {
|
|
is_change_column_order = 0 != flag_cnt;
|
|
}
|
|
if (OB_SUCC(ret) && is_change_column_order) {
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get oracle mode", K(ret));
|
|
} else if (OB_DDL_CHANGE_COLUMN == op_type || OB_DDL_MODIFY_COLUMN == op_type) {
|
|
// same order iff no non-hidden column exists in (left, right)
|
|
const ObString &left_name = is_before ? this_name : prev_name;
|
|
const ObString &right_name = is_before ? next_name : this_name;
|
|
bool same_order = false;
|
|
ObColumnIterByPrevNextID iter(table_schema);
|
|
const ObColumnSchemaV2 *column = nullptr;
|
|
// iter to left column, for is_first the left column is just iter start
|
|
while (OB_SUCC(ret) && !same_order && !is_first) {
|
|
if (OB_FAIL(iter.next(column))) {
|
|
} else if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
} else if ((is_oracle_mode && left_name == column->get_column_name_str())
|
|
|| (!is_oracle_mode && 0 == left_name.case_compare(column->get_column_name_str()))) {
|
|
if (left_name == right_name) {
|
|
same_order = true;
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
// iter from left to first non-shadow non-hidden column
|
|
while (OB_SUCC(ret) && !same_order) {
|
|
if (OB_FAIL(iter.next(column))) {
|
|
} else if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
} else if (column->is_shadow_column() || column->is_hidden()) {
|
|
// skip
|
|
} else {
|
|
if ((is_oracle_mode && right_name == column->get_column_name_str())
|
|
|| (!is_oracle_mode && 0 == right_name.case_compare(column->get_column_name_str()))) {
|
|
same_order = true;
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
if (same_order) {
|
|
is_change_column_order = false;
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
if (ret == OB_ITER_END) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("failed to check is change column order", K(ret));
|
|
}
|
|
}
|
|
} else if (OB_DDL_ADD_COLUMN == op_type) {
|
|
const ObColumnSchemaV2 *column = nullptr;
|
|
if (is_after && nullptr != (column = table_schema.get_column_schema(prev_name))
|
|
&& column->get_next_column_id() == BORDER_COLUMN_ID) {
|
|
// add trailing column won't affect the order of old columns
|
|
is_change_column_order = false;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_column_is_offline(
|
|
const ObTableSchema &orig_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObColumnSchemaV2 &orig_column_schema,
|
|
AlterColumnSchema &alter_column_schema,
|
|
bool &is_offline) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_change_column_order = false;
|
|
bool need_rewrite_data = false;
|
|
bool add_pk = false;
|
|
if (OB_FAIL(check_is_change_column_order(orig_table_schema, alter_column_schema, is_change_column_order))) {
|
|
LOG_WARN("failed to check is change column order", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.check_alter_column_is_offline(
|
|
&orig_column_schema, &alter_column_schema, schema_guard, need_rewrite_data))) {
|
|
LOG_WARN("fail to check column can be altered", K(ret));
|
|
} else {
|
|
add_pk = orig_table_schema.is_heap_table() && alter_column_schema.is_primary_key_;
|
|
is_offline = is_change_column_order || need_rewrite_data || add_pk;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check wheter the hidden table of offline ddl contains stored generated column.
|
|
int ObDDLService::check_exist_stored_gen_col(
|
|
const ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
bool &is_exist)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_exist = false;
|
|
int64_t column_cnt = 0;
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
ObTableSchema::const_column_iterator it_begin = orig_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = orig_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
ObColumnSchemaV2 *column_schema = (*it_begin);
|
|
if (OB_ISNULL(column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error", K(ret));
|
|
} else if (column_schema->is_stored_generated_column()) {
|
|
column_cnt++;
|
|
} else {/* do nothing. */}
|
|
}
|
|
it_begin = alter_table_schema.column_begin();
|
|
it_end = alter_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && !is_exist && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else if (alter_column_schema->is_stored_generated_column()) {
|
|
if (OB_DDL_DROP_COLUMN == alter_column_schema->alter_type_) {
|
|
column_cnt--;
|
|
} else if (OB_DDL_ADD_COLUMN == alter_column_schema->alter_type_) {
|
|
is_exist = true;
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected ddl type found here", K(ret), K(*alter_column_schema));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!is_exist && column_cnt > 0) {
|
|
is_exist = true;
|
|
} else {/* do nothing. */}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_add_column_online(const ObTableSchema &table_schema,
|
|
const AlterColumnSchema &alter_column_schema,
|
|
ObDDLType &tmp_ddl_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
tmp_ddl_type = ObDDLType::DDL_INVALID;
|
|
bool is_change_column_order = false;
|
|
if (OB_DDL_ADD_COLUMN != alter_column_schema.alter_type_) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error", K(ret), K(alter_column_schema));
|
|
} else if (alter_column_schema.is_autoincrement_ || alter_column_schema.is_primary_key_ || alter_column_schema.has_not_null_constraint()) {
|
|
tmp_ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else if (nullptr != table_schema.get_column_schema(alter_column_schema.get_column_name())) {
|
|
tmp_ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else if (OB_FAIL(check_is_change_column_order(table_schema, alter_column_schema, is_change_column_order))) {
|
|
LOG_WARN("fail to check is change column order", K(ret));
|
|
} else if (is_change_column_order || alter_column_schema.is_stored_generated_column()) {
|
|
tmp_ddl_type = ObDDLType::DDL_ADD_COLUMN_OFFLINE;
|
|
} else {
|
|
tmp_ddl_type = ObDDLType::DDL_ADD_COLUMN_ONLINE;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_modify_partition_key(const ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
bool &is_modify_partition_key)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_modify_partition_key = false;
|
|
const common::ObPartitionKeyInfo &partition_keys = orig_table_schema.get_partition_key_info();
|
|
const common::ObPartitionKeyInfo &subpartition_keys = orig_table_schema.get_subpartition_key_info();
|
|
ObTableSchema::const_column_iterator iter = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator iter_end = alter_table_schema.column_end();
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
for(; OB_SUCC(ret) && !is_modify_partition_key && iter != iter_end; iter++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*iter))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("iter is NULL", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.is_column_in_partition_key(alter_column_schema->get_column_id(),
|
|
is_modify_partition_key))) {
|
|
LOG_WARN("fail to check if column in partition key", K(ret), "column_id", alter_column_schema->get_column_id());
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_change_cst_column_name(const ObTableSchema &table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
bool &change_cst_column_name)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
change_cst_column_name = false;
|
|
ObTableSchema::const_column_iterator iter = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator iter_end = alter_table_schema.column_end();
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
for(; OB_SUCC(ret) && !change_cst_column_name && iter != iter_end; iter++) {
|
|
const ObColumnSchemaV2 *col_schema = nullptr;
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*iter))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("iter is NULL", K(ret));
|
|
} else if (OB_ISNULL(col_schema = table_schema.get_column_schema(alter_column_schema->get_column_id()))) {
|
|
} else if (col_schema->get_column_name_str() != alter_column_schema->get_column_name_str()) {
|
|
change_cst_column_name = true;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_alter_decimal_int_offline(const share::ObDDLType &ddl_type,
|
|
const ObTableSchema &table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
bool &is_alter_decimal_int_offline)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_alter_decimal_int_offline = false;
|
|
if (DDL_MODIFY_COLUMN == ddl_type) {
|
|
ObTableSchema::const_column_iterator iter = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator iter_end = alter_table_schema.column_end();
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
for(; OB_SUCC(ret) && iter != iter_end; iter++) {
|
|
const ObColumnSchemaV2 *col_schema = nullptr;
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*iter))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("iter is NULL", K(ret));
|
|
} else if (OB_DDL_MODIFY_COLUMN != alter_column_schema->alter_type_ ||
|
|
OB_ISNULL(col_schema = table_schema.get_column_schema(alter_column_schema->get_column_id()))) {
|
|
is_alter_decimal_int_offline = false;
|
|
break;
|
|
} else if (ob_is_decimal_int_tc(col_schema->get_data_type()) &&
|
|
ob_is_number_or_decimal_int_tc(alter_column_schema->get_data_type())) {
|
|
is_alter_decimal_int_offline = true;
|
|
} else {
|
|
is_alter_decimal_int_offline = false;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_table_column(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const bool is_oracle_mode,
|
|
const uint64_t tenant_data_version,
|
|
ObDDLType &ddl_type,
|
|
bool &ddl_need_retry_at_executor)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_modify_partition_key = false;
|
|
common::ObIAllocator &allocator = alter_table_arg.allocator_;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0};
|
|
AlterColumnSchema *alter_column_schema = NULL;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
ddl_need_retry_at_executor = false;
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
const ObColumnSchemaV2 *orig_column_schema = orig_table_schema.get_column_schema(orig_column_name);
|
|
const ObSchemaOperationType op_type = alter_column_schema->alter_type_;
|
|
switch (op_type) {
|
|
case OB_DDL_ADD_COLUMN: {
|
|
ObDDLType tmp_ddl_type = ObDDLType::DDL_INVALID;
|
|
if (OB_FAIL(check_is_add_column_online(orig_table_schema, *alter_column_schema, tmp_ddl_type))) {
|
|
LOG_WARN("fail to check is add column online", K(ret));
|
|
} else if (tmp_ddl_type == ObDDLType::DDL_ADD_COLUMN_ONLINE) {
|
|
if (ObDDLType::DDL_INVALID == ddl_type
|
|
|| ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_ADD_COLUMN_ONLINE;
|
|
} else if (ObDDLType::DDL_NORMAL_TYPE == ddl_type) {
|
|
// ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
} else if (ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type) {
|
|
// ddl_type = ObDDLType::DDL_ADD_COLUMN_OFFLINE;
|
|
} else if (ObDDLType::DDL_DROP_COLUMN == ddl_type
|
|
|| ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_COLUMN_REDEFINITION;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
} else if (tmp_ddl_type == ObDDLType::DDL_TABLE_REDEFINITION) {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else {
|
|
if (ObDDLType::DDL_INVALID == ddl_type
|
|
|| ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type
|
|
|| ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_ADD_COLUMN_OFFLINE;
|
|
} else if (ObDDLType::DDL_DROP_COLUMN == ddl_type
|
|
|| ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_COLUMN_REDEFINITION;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_DROP_COLUMN: {
|
|
if (ObDDLType::DDL_INVALID == ddl_type
|
|
|| ObDDLType::DDL_DROP_COLUMN == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_DROP_COLUMN;
|
|
} else if (ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type
|
|
|| ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type
|
|
|| ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_COLUMN_REDEFINITION;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_CHANGE_COLUMN:
|
|
case OB_DDL_MODIFY_COLUMN: {
|
|
bool is_offline = false;
|
|
bool add_pk = false;
|
|
if (OB_FAIL(fill_column_collation(alter_table_schema.get_sql_mode(),
|
|
is_oracle_mode,
|
|
orig_table_schema,
|
|
allocator,
|
|
*alter_column_schema))) {
|
|
LOG_WARN("failed to fill column collation", K(ret));
|
|
} else if (OB_ISNULL(orig_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid orig column schema", K(ret));
|
|
} else if (OB_FAIL(check_alter_column_is_offline(
|
|
orig_table_schema, schema_guard, *orig_column_schema, *alter_column_schema, is_offline))) {
|
|
LOG_WARN("failed to check is offline", K(ret));
|
|
} else if (is_offline) {
|
|
if (alter_column_schema->is_primary_key_) {
|
|
if (orig_table_schema.get_rowkey_column_num() > 0) {
|
|
if (!orig_table_schema.is_heap_table()) {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
RS_LOG(WARN, "multiple primary key defined", K(ret));
|
|
} else {
|
|
add_pk = true;
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rowkey_column_num must be greater than 0", K(ret),
|
|
K(orig_table_schema.get_rowkey_column_num()));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (add_pk) {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else if (is_offline) {
|
|
if (ObDDLType::DDL_INVALID == ddl_type
|
|
|| ObDDLType::DDL_MODIFY_COLUMN == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_MODIFY_COLUMN;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
} else {
|
|
if (alter_column_schema->is_autoincrement() && !orig_column_schema->is_autoincrement()) {
|
|
if (orig_column_schema->is_generated_column()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Changing the STORED status for generated columns");
|
|
} else if (orig_table_schema.get_autoinc_column_id() == 0) {
|
|
if (orig_column_schema->is_nullable()) {
|
|
// if the original table has null, we need to do double write to fill the nulls
|
|
if (ObDDLType::DDL_INVALID == ddl_type || ObDDLType::DDL_MODIFY_COLUMN == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_MODIFY_COLUMN;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
ddl_need_retry_at_executor = true;
|
|
} else {
|
|
if (ObDDLType::DDL_INVALID == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_MODIFY_AUTO_INCREMENT;
|
|
} else if (ObDDLType::DDL_MODIFY_COLUMN == ddl_type) {
|
|
// ddl_type = ObDDLType::DDL_MODIFY_COLUMN;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
}
|
|
|
|
} else {
|
|
ret = OB_ERR_WRONG_AUTO_KEY;
|
|
LOG_USER_ERROR(OB_ERR_WRONG_AUTO_KEY);
|
|
}
|
|
} else if (ObDDLType::DDL_INVALID == ddl_type
|
|
|| ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type
|
|
|| ObDDLType::DDL_NORMAL_TYPE == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_ALTER_COLUMN: {
|
|
if (ObDDLType::DDL_INVALID == ddl_type
|
|
|| ObDDLType::DDL_ADD_COLUMN_ONLINE == ddl_type
|
|
|| ObDDLType::DDL_NORMAL_TYPE == ddl_type) {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("unhandled operator type!", K_(alter_column_schema->alter_type));
|
|
break;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& ObDDLType::DDL_TABLE_REDEFINITION == ddl_type
|
|
&& alter_column_schema->is_identity_column()
|
|
&& OB_DDL_DROP_COLUMN != op_type) {
|
|
if (nullptr == orig_column_schema) {
|
|
ddl_need_retry_at_executor = true;
|
|
} else if (orig_column_schema->is_nullable()) {
|
|
// only nullable column need generate value from sequence
|
|
ddl_need_retry_at_executor = true;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (ObDDLType::DDL_DROP_COLUMN == ddl_type
|
|
|| ObDDLType::DDL_ADD_COLUMN_OFFLINE == ddl_type
|
|
|| ObDDLType::DDL_COLUMN_REDEFINITION == ddl_type) {
|
|
bool is_exist_stored_gen_col = false; // whether the target table contain stored generated column.
|
|
bool is_column_group_store = false;
|
|
bool need_process_cs_replica = false;
|
|
if (OB_FAIL(ObCODDLUtil::need_column_group_store(orig_table_schema, is_column_group_store))) {
|
|
LOG_WARN("fail to check schema is column group store", K(ret));
|
|
} else if (is_column_group_store) {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else if (OB_FAIL(check_exist_stored_gen_col(orig_table_schema,
|
|
alter_table_schema,
|
|
is_exist_stored_gen_col))) {
|
|
LOG_WARN("fail to check exist stored generated column", K(ret));
|
|
} else if (is_exist_stored_gen_col) {
|
|
// column redefinition cannot handle stored gen column, use table redefinition instead
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else if (OB_FAIL(ObCSReplicaUtil::check_need_process_cs_replica_for_offline_ddl(orig_table_schema, need_process_cs_replica))) {
|
|
LOG_WARN("fail to check need process cs replica", K(ret));
|
|
} else if (need_process_cs_replica) {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else {/* do nothing. */}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_double_table_long_running_ddl(ddl_type)
|
|
&& OB_FAIL(check_is_modify_partition_key(orig_table_schema,
|
|
alter_table_schema,
|
|
is_modify_partition_key))) {
|
|
LOG_WARN("fail to check is modify partition key", K(ret));
|
|
} else if (is_modify_partition_key) {
|
|
ddl_type = ObDDLType::DDL_ALTER_PARTITION_BY;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// Check whether alter primary key and alter column in one sql is allowed.
|
|
// Currently, support MODIFY column and ADD PRIMARY KEY in one sql.
|
|
int ObDDLService::check_support_alter_pk_and_columns(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const obrpc::ObIndexArg::IndexActionType &index_action_type,
|
|
bool &is_support)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_support = true;
|
|
if (ObIndexArg::ADD_PRIMARY_KEY != index_action_type) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(index_action_type), K(alter_table_arg));
|
|
} else if (!alter_table_arg.is_alter_columns_) {
|
|
// without column operation in the sql.
|
|
} else {
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
for (ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
OB_SUCC(ret) && is_support && it_begin != alter_table_schema.column_end(); it_begin++) {
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else if (OB_DDL_MODIFY_COLUMN != alter_column_schema->alter_type_) {
|
|
is_support = false;
|
|
} else if (alter_column_schema->is_primary_key_) {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret), KPC(alter_column_schema));
|
|
} else {
|
|
// do nothing.
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_table_index(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObDDLType &ddl_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0};
|
|
ObIndexArg::IndexActionType last_type = ObIndexArg::INVALID_ACTION;
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) {
|
|
ObIndexArg *index_arg = const_cast<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 {
|
|
const ObIndexArg::IndexActionType type = index_arg->index_action_type_;
|
|
switch(type) {
|
|
case ObIndexArg::DROP_PRIMARY_KEY: {
|
|
if (!is_invalid_ddl_type(ddl_type)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment",
|
|
ObIndexArg::to_type_str(last_type), ObIndexArg::to_type_str(type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
} else {
|
|
ddl_type = DDL_DROP_PRIMARY_KEY;
|
|
last_type = type;
|
|
}
|
|
break;
|
|
}
|
|
case ObIndexArg::ADD_PRIMARY_KEY: {
|
|
bool is_support = true;
|
|
if (ObDDLType::DDL_INVALID == ddl_type) {
|
|
ddl_type = DDL_ADD_PRIMARY_KEY;
|
|
last_type = type;
|
|
} else if (OB_FAIL(check_support_alter_pk_and_columns(alter_table_arg, type, is_support))) {
|
|
LOG_WARN("check support column operation and add primary key in one sql failed", K(ret), K(alter_table_arg));
|
|
} else if (is_support) {
|
|
ddl_type = DDL_TABLE_REDEFINITION;
|
|
last_type = type;
|
|
} else {
|
|
ret = OB_NOT_SUPPORTED;
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment",
|
|
ObIndexArg::to_type_str(last_type), ObIndexArg::to_type_str(type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
LOG_WARN("Not Supported DDL", K(ret), K(ddl_type), K(last_type), K(type));
|
|
}
|
|
break;
|
|
}
|
|
case ObIndexArg::ALTER_PRIMARY_KEY: {
|
|
if (!is_invalid_ddl_type(ddl_type)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment",
|
|
ObIndexArg::to_type_str(last_type), ObIndexArg::to_type_str(type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
} else {
|
|
ddl_type = DDL_ALTER_PRIMARY_KEY;
|
|
last_type = type;
|
|
}
|
|
break;
|
|
}
|
|
case ObIndexArg::ADD_INDEX:
|
|
case ObIndexArg::REBUILD_INDEX:
|
|
case ObIndexArg::DROP_INDEX:
|
|
case ObIndexArg::ALTER_INDEX:
|
|
case ObIndexArg::ALTER_INDEX_PARALLEL:
|
|
case ObIndexArg::RENAME_INDEX:
|
|
case ObIndexArg::ALTER_INDEX_TABLESPACE: {
|
|
// offline ddl cannot appear at the same time with other ddl
|
|
if ((DDL_MODIFY_COLUMN == ddl_type || DDL_ADD_COLUMN_OFFLINE == ddl_type
|
|
|| DDL_ADD_COLUMN_ONLINE == ddl_type || DDL_TABLE_REDEFINITION == ddl_type)
|
|
&& ObIndexArg::ADD_INDEX == type) {
|
|
// TODO(shuangcan): distinguish simple table and double table ddl
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
last_type = type;
|
|
} else if (is_long_running_ddl(ddl_type)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment",
|
|
ddl_type_str(ddl_type), ObIndexArg::to_type_str(type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
last_type = type;
|
|
}
|
|
break;
|
|
}
|
|
case ObIndexArg::DROP_FOREIGN_KEY: {
|
|
if (DDL_DROP_COLUMN == ddl_type) {
|
|
// In oracle mode, we support to drop foreign key implicitly caused by drop column.
|
|
} else if (is_long_running_ddl(ddl_type)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and %s in single statment",
|
|
ddl_type_str(ddl_type), ObIndexArg::to_type_str(type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
last_type = type;
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Unknown index action type!", K(type), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_convert_to_character(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObDDLType &ddl_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// alter table table_name CONVERT TO CHARACTER SET charset_name [COLLATE collation_name]
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
ObCharsetType charset_type = alter_table_schema.get_charset_type();
|
|
ObCollationType collation_type = alter_table_schema.get_collation_type();
|
|
if (CS_TYPE_INVALID == collation_type) {
|
|
// If collation_type is not given, the default collation_type of charset_type is used
|
|
collation_type = ObCharset::get_default_collation(charset_type);
|
|
alter_table_schema.set_collation_type(collation_type);
|
|
alter_table_schema.set_charset_type(charset_type);
|
|
} else if (!ObCharset::is_valid_collation(charset_type, collation_type)) {
|
|
ret = OB_ERR_COLLATION_MISMATCH;
|
|
const char *cs_name = ObCharset::charset_name(charset_type);
|
|
const char *coll_name = ObCharset::collation_name(collation_type);
|
|
ObString charset = ObString::make_string(cs_name);
|
|
ObString collation = ObString::make_string(coll_name);
|
|
LOG_USER_ERROR(OB_ERR_COLLATION_MISMATCH, collation.length(), collation.ptr(),
|
|
charset.length(), charset.ptr());
|
|
}
|
|
// This is to do a performance optimization. If the collation_type of the original table is
|
|
// equivalent to the new collation_type, do nothing
|
|
if (OB_SUCC(ret) && orig_table_schema.get_collation_type() != collation_type) {
|
|
if (is_long_running_ddl(ddl_type)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement");
|
|
} else {
|
|
bool can_convert = false;
|
|
bool convert_partition_key = false;
|
|
ObTableSchema::const_column_iterator tmp_begin = orig_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator tmp_end = orig_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) {
|
|
ObColumnSchemaV2 *col = (*tmp_begin);
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is NULL", K(ret));
|
|
} else if (ObDDLUtil::check_can_convert_character(col->get_meta_type())) {
|
|
if (orig_table_schema.is_column_in_foreign_key(col->get_column_id())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Alter column charset or collation with foreign key");
|
|
} else if (OB_FAIL(orig_table_schema.is_column_in_partition_key(col->get_column_id(),
|
|
convert_partition_key))) {
|
|
LOG_WARN("fail to check if column in partition key", K(ret), "column id", col->get_column_id());
|
|
}
|
|
}
|
|
}
|
|
// change charset of a partitioned table can lead to repartition, we should handle it seperately.
|
|
if (convert_partition_key) {
|
|
if (is_long_running_ddl(ddl_type)) {
|
|
// override the ret code here is by design.
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement");
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_ALTER_PARTITION_BY;
|
|
}
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_CONVERT_TO_CHARACTER;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::check_is_add_identity_column(const share::schema::ObTableSchema &orig_table_schema,
|
|
const share::schema::ObTableSchema &hidden_table_schema,
|
|
bool &is_add_identity_column)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema::const_column_iterator iter = orig_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator end = orig_table_schema.column_end();
|
|
ObTableSchema::const_column_iterator hidden_iter = hidden_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator hidden_end = hidden_table_schema.column_end();
|
|
is_add_identity_column = false;
|
|
for (; OB_SUCC(ret) && hidden_iter != hidden_end; ++hidden_iter) {
|
|
const ObColumnSchemaV2 *column = *hidden_iter;
|
|
if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid column schema", K(column));
|
|
} else if (column->is_identity_column()) {
|
|
if (is_add_identity_column) {
|
|
ret = OB_ERR_IDENTITY_COLUMN_COUNT_EXCE_LIMIT;
|
|
LOG_USER_ERROR(OB_ERR_IDENTITY_COLUMN_COUNT_EXCE_LIMIT);
|
|
LOG_WARN("add more than one identity column is not allowed");
|
|
} else {
|
|
is_add_identity_column = true;
|
|
}
|
|
}
|
|
}
|
|
for (; OB_SUCC(ret) && iter != end; ++iter) {
|
|
const ObColumnSchemaV2 *column = *iter;
|
|
if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid column schema", K(column));
|
|
} else if (column->is_identity_column()) {
|
|
is_add_identity_column = false;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_table_partition(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const bool is_oracle_mode,
|
|
ObDDLType &ddl_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
const uint64_t tablegroup_id = orig_table_schema.get_tablegroup_id();
|
|
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (obrpc::ObAlterTableArg::REPARTITION_TABLE == alter_table_arg.alter_part_type_) {
|
|
if (is_oracle_mode && PARTITION_LEVEL_ZERO != part_level) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "re-partition a patitioned table");
|
|
} else if (OB_INVALID_ID != tablegroup_id && compat_version < DATA_VERSION_4_2_0_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "can't modify table partition in tablegroup");
|
|
} else {
|
|
if (is_long_running_ddl(ddl_type)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement");
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_ALTER_PARTITION_BY;
|
|
}
|
|
}
|
|
} else {
|
|
if (is_long_running_ddl(ddl_type)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement");
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_column_group(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const share::schema::ObTableSchema &ori_table_schema,
|
|
share::schema::ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (alter_table_arg.alter_table_schema_.get_column_group_count() <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("alter table arg has no column groups", K(ret), K(alter_table_arg));
|
|
} else if (alter_table_arg.based_schema_object_infos_.count() <= 0) {
|
|
/* based schema object infos is checked in the alter column group, here: only check count*/
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("based info object count <=0 cannot promise consist", K(ret));
|
|
} else {
|
|
ObTableSchema::const_column_group_iterator iter_begin =
|
|
alter_table_arg.alter_table_schema_.column_group_begin();
|
|
ObTableSchema::const_column_group_iterator iter_end =
|
|
alter_table_arg.alter_table_schema_.column_group_end();
|
|
|
|
for (; OB_SUCC(ret) && iter_begin != iter_end; iter_begin++) {
|
|
bool cg_exist = false;
|
|
ObColumnGroupSchema *column_group = *iter_begin;
|
|
|
|
if (OB_ISNULL(column_group)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column group should not be null", K(ret), K(alter_table_arg));
|
|
} else if (column_group->get_column_group_id() <= new_table_schema.get_max_used_column_group_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("added column group should have greater id than used column id",
|
|
K(ret), K(new_table_schema.get_max_used_column_group_id()),
|
|
K(column_group->get_column_group_id()));
|
|
} else if (OB_FAIL(new_table_schema.add_column_group(*column_group))) {
|
|
if (OB_HASH_EXIST == ret) {
|
|
ret = OB_ERR_COLUMN_GROUP_DUPLICATE;
|
|
LOG_WARN("fail to add column group, column group duplicate", K(ret), K(new_table_schema));
|
|
char err_msg[OB_MAX_COLUMN_GROUP_NAME_LENGTH] = {'\0'};
|
|
ObString err_msg_str(OB_MAX_COLUMN_GROUP_NAME_LENGTH, 0 /*length*/, err_msg);
|
|
int tmp_ret = column_group->get_column_group_type_name(err_msg_str);
|
|
if (tmp_ret != OB_SUCCESS) {
|
|
LOG_WARN("fail to get readable column group name", K(tmp_ret), KPC(column_group));
|
|
} else {
|
|
LOG_USER_ERROR(OB_ERR_COLUMN_GROUP_DUPLICATE, err_msg_str.length(), err_msg_str.ptr());
|
|
}
|
|
} else {
|
|
LOG_WARN("fail to add column group to table schema", K(ret), K(new_table_schema), KPC(column_group));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
/* note must alter rowkey cg first, else will affect default cg*/
|
|
if (OB_FAIL(ObSchemaUtils::alter_rowkey_column_group(new_table_schema))) {
|
|
LOG_WARN("fail to adjust rowkey column group when add column group", K(ret));
|
|
} else if (OB_FAIL(ObSchemaUtils::alter_default_column_group(new_table_schema))) {
|
|
LOG_WARN("fail to alter default column group", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_column_group(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const share::schema::ObTableSchema &ori_table_schema,
|
|
share::schema::ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (alter_table_arg.alter_table_schema_.get_column_group_count() <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("alter table arg has no column group", K(ret), K(alter_table_arg.alter_table_schema_));
|
|
} else if (alter_table_arg.based_schema_object_infos_.count() <= 0) {
|
|
/* based schema object infos is checked in the alter column group, here only check count*/
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("based schema object info count <= 0, cannot promise column consist", K(ret));
|
|
} else {
|
|
ObTableSchema::const_column_group_iterator iter_begin =
|
|
alter_table_arg.alter_table_schema_.column_group_begin();
|
|
ObTableSchema::const_column_group_iterator iter_end =
|
|
alter_table_arg.alter_table_schema_.column_group_end();
|
|
|
|
for (; OB_SUCC(ret) && iter_begin != iter_end; iter_begin++) {
|
|
const ObColumnGroupSchema *column_group = *iter_begin;
|
|
ObColumnGroupSchema *ori_column_group = nullptr;
|
|
/* drop column group use column group name to get real column*/
|
|
if (OB_ISNULL(column_group)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column group in origin table should not be null", K(ret));
|
|
} else if (OB_FAIL(ori_table_schema.get_column_group_by_name(column_group->get_column_group_name(),
|
|
ori_column_group))) {
|
|
/* if not exist in origin*/
|
|
if (OB_HASH_NOT_EXIST == ret) {
|
|
ret = OB_COLUMN_GROUP_NOT_FOUND;
|
|
LOG_WARN("cannot found column group", KPC(column_group));
|
|
char err_msg[OB_MAX_COLUMN_GROUP_NAME_LENGTH] = {'\0'};
|
|
ObString err_msg_str(OB_MAX_COLUMN_GROUP_NAME_LENGTH, 0, err_msg);
|
|
int tmp_ret = column_group->get_column_group_type_name(err_msg_str);
|
|
if (tmp_ret != OB_SUCCESS){
|
|
LOG_WARN("fail to get readable column group name");
|
|
} else {
|
|
LOG_USER_ERROR(OB_COLUMN_GROUP_NOT_FOUND, err_msg_str.length(), err_msg_str.ptr());
|
|
}
|
|
} else {
|
|
LOG_WARN("fail to get column group by name", K(ret), K(ori_table_schema), KPC(column_group));
|
|
}
|
|
} else if (OB_ISNULL(ori_column_group)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column group should not be null", K(ret), KPC(column_group));
|
|
} else if (OB_FAIL(new_table_schema.remove_column_group(ori_column_group->get_column_group_id()))) {
|
|
LOG_WARN("fail to remove column group from new table schema", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ObSchemaUtils::alter_rowkey_column_group(new_table_schema))) {
|
|
LOG_WARN("fail to alter rowkey column group", K(ret));
|
|
} else if (OB_FAIL(ObSchemaUtils::alter_default_column_group(new_table_schema))) {
|
|
LOG_WARN("fail to alter default column group", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::alter_column_group(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
share::schema::ObTableSchema &new_table_schema,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool bind_tablets = false;
|
|
uint64_t compat_version = 0;
|
|
if (alter_table_arg.alter_table_schema_.get_column_group_count() == 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("alter table arg has no column group", K(ret), K(alter_table_arg.alter_table_schema_));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(orig_table_schema.get_tenant_id(), compat_version))) {
|
|
LOG_WARN("fail to get compat version", K(ret), K(orig_table_schema), K(compat_version));
|
|
} else if (compat_version < DATA_VERSION_4_3_0_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("compat version not support", K(ret), K(compat_version));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant data version is less than 4.3, alter column group");
|
|
} else if (alter_table_arg.based_schema_object_infos_.count() <= 0) {
|
|
/* alter_table() has use check_parallel_ddl_conflict() before
|
|
so here only need to check count
|
|
*/
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("there is no schema object infos to promise consit", K(ret));
|
|
} else {
|
|
new_table_schema.set_column_store(true);
|
|
switch (alter_table_arg.alter_table_schema_.alter_type_) {
|
|
case share::schema::OB_DDL_ADD_COLUMN_GROUP: {
|
|
if (OB_FAIL(add_column_group(alter_table_arg, orig_table_schema, new_table_schema))) {
|
|
LOG_WARN("fail to add column group to new table schema", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case share::schema::OB_DDL_DROP_COLUMN_GROUP: {
|
|
if (OB_FAIL(drop_column_group(alter_table_arg, orig_table_schema, new_table_schema))) {
|
|
LOG_WARN("fail to dorp column in new table schema", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("recevive unexpected alter table actions", K(ret),
|
|
K(alter_table_arg.alter_table_schema_.alter_type_));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::adjust_cg_for_offline(ObTableSchema &new_table_schema)
|
|
{
|
|
/* do adjustment on column group when add or drop column/primary key*/
|
|
int ret = OB_SUCCESS;
|
|
bool is_each_cg_exist = false;
|
|
bool is_all_cg_exist = false;
|
|
if (!new_table_schema.is_column_store_supported()) {
|
|
/*skip*/
|
|
} else if (OB_FAIL(new_table_schema.is_column_group_exist(OB_ALL_COLUMN_GROUP_NAME, is_all_cg_exist))) {
|
|
LOG_WARN("fail to check is all column group exist", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.is_column_group_exist(OB_EACH_COLUMN_GROUP_NAME, is_each_cg_exist))) {
|
|
LOG_WARN("fail to check is each column group exist", K(ret));
|
|
} else {
|
|
/* for double_table_ddl reset all column groups*/
|
|
new_table_schema.reset_column_group_info();
|
|
/* add each column group*/
|
|
ObTableSchema::const_column_iterator col_iter = new_table_schema.column_begin();
|
|
for (; OB_SUCC(ret) && is_each_cg_exist && col_iter != new_table_schema.column_end(); col_iter++) {
|
|
ObColumnSchemaV2 *col = *col_iter;
|
|
ObColumnGroupSchema new_single_cg;
|
|
new_single_cg.reset();
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column group pointer should not be null", K(ret));
|
|
} else if (col->is_virtual_generated_column()) {
|
|
/* skip virtual column group*/
|
|
} else if (OB_FAIL(ObSchemaUtils::build_single_column_group(new_table_schema, col, new_table_schema.get_tenant_id(),
|
|
new_table_schema.get_max_used_column_group_id() +1,
|
|
new_single_cg))) {
|
|
LOG_WARN("fail to build single column group", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.add_column_group(new_single_cg))) {
|
|
LOG_WARN("fail to add new column group to table schema", K(ret));
|
|
}
|
|
}
|
|
|
|
/* add all column group*/
|
|
if (OB_SUCC(ret) &&is_all_cg_exist) {
|
|
ObColumnGroupSchema new_cg;
|
|
new_cg.reset();
|
|
if (OB_FAIL(ObSchemaUtils::build_all_column_group(
|
|
new_table_schema, new_table_schema.get_tenant_id(),
|
|
new_table_schema.get_max_used_column_group_id() +1, new_cg))) {
|
|
LOG_WARN("fail to build new all column group schema", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.add_column_group(new_cg))) {
|
|
LOG_WARN("fail to add new column group to table schema", K(ret));
|
|
}
|
|
}
|
|
/* adjust rowkey & default column group*/
|
|
if (OB_SUCC(ret)) {
|
|
ObArray<uint64_t> column_ids;
|
|
ObColumnGroupSchema default_cg;
|
|
default_cg.reset();
|
|
if (OB_FAIL(ObSchemaUtils::build_column_group(new_table_schema, new_table_schema.get_tenant_id(),
|
|
ObColumnGroupType::DEFAULT_COLUMN_GROUP,
|
|
OB_DEFAULT_COLUMN_GROUP_NAME, column_ids,
|
|
DEFAULT_TYPE_COLUMN_GROUP_ID, default_cg))) {
|
|
LOG_WARN("fail to build column group", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.add_column_group(default_cg))) {
|
|
LOG_WARN("failt to add default column group", K(ret));
|
|
} else if (OB_FAIL(ObSchemaUtils::alter_rowkey_column_group(new_table_schema))) {
|
|
LOG_WARN("fail to alter rowkey column group", K(ret));
|
|
} else if (OB_FAIL(ObSchemaUtils::alter_default_column_group(new_table_schema))) {
|
|
LOG_WARN("fail to alter default column grouop schema", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_alter_partition_new_table_schema_offline(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const AlterTableSchema & alter_table_schema,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
if (OB_FAIL(new_table_schema.assign(orig_table_schema))) {
|
|
LOG_WARN("fail to assign table schema", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.assign_partition_schema(alter_table_schema))) {
|
|
LOG_WARN("fail to assign table partition schema", K(ret));
|
|
} else {
|
|
new_table_schema.reset_column_info();
|
|
new_table_schema.reset_column_part_key_info();
|
|
|
|
ObTableSchema::const_column_iterator tmp_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator tmp_end = alter_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) {
|
|
ObColumnSchemaV2 *col = (*tmp_begin);
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
} else if (OB_FAIL(new_table_schema.add_column(*col))) {
|
|
LOG_WARN("fail to add column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
OZ (fill_interval_info_for_offline(alter_table_schema, new_table_schema), new_table_schema);
|
|
if (OB_SUCC(ret) && alter_table_arg.is_alter_partitions_) {
|
|
// In the alter table partition by range(xxx) (partitions...) case,
|
|
// it is necessary to reset attributes of old table related to automatic partitioning here
|
|
new_table_schema.get_part_option().assign_auto_partition_attr(orig_table_schema.get_part_option());
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_partition_with_tablegroup(const ObTableSchema *orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tablegroup_id = new_table_schema.get_tablegroup_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_INVALID_ID != tablegroup_id) {
|
|
ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(helper.check_table_partition_in_tablegroup(orig_table_schema, new_table_schema, schema_guard))) {
|
|
LOG_WARN("check partition option failed", KR(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_table_partition_by(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool bind_tablets = false;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
OZ (gen_alter_partition_new_table_schema_offline(
|
|
alter_table_arg, alter_table_schema, orig_table_schema, new_table_schema));
|
|
if (FAILEDx(new_table_schema.check_validity_for_auto_partition())) {
|
|
LOG_WARN("fail to check enable auto partitioning", KR(ret), K(new_table_schema));
|
|
}
|
|
OZ (create_user_hidden_table(orig_table_schema,
|
|
new_table_schema,
|
|
&alter_table_arg.sequence_ddl_arg_,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version));
|
|
return ret;
|
|
}
|
|
|
|
// convert character set for ObBasePartition, inluding high_bound_val and list_row_values.
|
|
int ObDDLService::convert_to_character_for_partition(
|
|
const ObCollationType &to_collation,
|
|
ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObPartitionLevel part_level = new_table_schema.get_part_level();
|
|
const ObPartitionFuncType part_func_type = new_table_schema.get_part_option().get_part_func_type();
|
|
const ObPartitionFuncType subpart_func_type = new_table_schema.get_sub_part_option().get_sub_part_func_type();
|
|
if (PARTITION_LEVEL_MAX <= part_level) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(part_level), K(new_table_schema));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
// non-partitioned, do nothing.
|
|
} else {
|
|
const int64_t part_num = new_table_schema.get_partition_num();
|
|
ObPartition **part_array = new_table_schema.get_part_array();
|
|
if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null first part array", K(ret), K(part_level), K(part_num), K(part_func_type));
|
|
}
|
|
// for the first-level part.
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < part_num; i++) {
|
|
ObPartition *partition = part_array[i];
|
|
if (OB_ISNULL(partition)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected err", K(ret));
|
|
} else if (PARTITION_FUNC_TYPE_RANGE_COLUMNS == part_func_type
|
|
&& OB_FAIL(partition->convert_character_for_range_columns_part(to_collation))) {
|
|
LOG_WARN("convert charset failed", K(ret), K(to_collation));
|
|
} else if (PARTITION_FUNC_TYPE_LIST_COLUMNS == part_func_type
|
|
&& OB_FAIL(partition->convert_character_for_list_columns_part(to_collation))) {
|
|
LOG_WARN("convert charset failed", K(ret), K(to_collation));
|
|
} else if (PARTITION_LEVEL_TWO == part_level) {
|
|
// for the second-level part.
|
|
const int64_t subpart_num = partition->get_subpartition_num();
|
|
ObSubPartition **subpart_array = partition->get_subpart_array();
|
|
if (OB_ISNULL(subpart_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", K(ret));
|
|
} else if (subpart_num < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sub_part_num less than 1", K(ret));
|
|
} else {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < subpart_num; j++) {
|
|
ObSubPartition *subpart = subpart_array[j];
|
|
if (OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected err", K(ret));
|
|
} else if (PARTITION_FUNC_TYPE_RANGE_COLUMNS == subpart_func_type
|
|
&& OB_FAIL(subpart->convert_character_for_range_columns_part(to_collation))) {
|
|
LOG_WARN("convert charset failed", K(ret), K(to_collation));
|
|
} else if (PARTITION_FUNC_TYPE_LIST_COLUMNS == subpart_func_type
|
|
&& OB_FAIL(subpart->convert_character_for_list_columns_part(to_collation))) {
|
|
LOG_WARN("convert charset failed", K(ret), K(to_collation));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// for def subpartition array.
|
|
if (OB_SUCC(ret) && new_table_schema.has_sub_part_template_def()) {
|
|
ObSubPartition **def_subpart_array = new_table_schema.get_def_subpart_array();
|
|
const int64_t def_subpart_num = new_table_schema.get_def_sub_part_num();
|
|
if (OB_ISNULL(def_subpart_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected err, def subpart arr is null", K(ret), K(new_table_schema));
|
|
} else {
|
|
ObSubPartition *subpart_info = nullptr;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < def_subpart_num; i++) {
|
|
if (OB_ISNULL(subpart_info = def_subpart_array[i])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sub part is nullptr", K(ret));
|
|
} else if (PARTITION_FUNC_TYPE_RANGE_COLUMNS == subpart_func_type
|
|
&& OB_FAIL(subpart_info->convert_character_for_range_columns_part(to_collation))) {
|
|
LOG_WARN("convert charset failed", K(ret), K(to_collation));
|
|
} else if (PARTITION_FUNC_TYPE_LIST_COLUMNS == subpart_func_type
|
|
&& OB_FAIL(subpart_info->convert_character_for_list_columns_part(to_collation))) {
|
|
LOG_WARN("convert charset failed", K(ret), K(to_collation));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::convert_to_character(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool bind_tablets = false;
|
|
bool can_convert = false;
|
|
const ObSQLMode sql_mode = alter_table_arg.alter_table_schema_.get_sql_mode();
|
|
bool is_oracle_mode = false;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
ObCollationType collation_type = alter_table_schema.get_collation_type();
|
|
new_table_schema.set_collation_type(collation_type);
|
|
new_table_schema.set_charset_type(ObCharset::charset_type_by_coll(collation_type));
|
|
ObTableSchema::const_column_iterator tmp_begin = orig_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator tmp_end = orig_table_schema.column_end();
|
|
if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get oracle mode", K(ret));
|
|
} else if (is_oracle_mode) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected compat mode", K(ret), K(orig_table_schema));
|
|
} else {
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) {
|
|
ObColumnSchemaV2 *orig_col = (*tmp_begin);
|
|
if (OB_ISNULL(orig_col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is NULL", K(ret));
|
|
} else if (ObDDLUtil::check_can_convert_character(orig_col->get_meta_type())) {
|
|
ObColumnSchemaV2 *col = new_table_schema.get_column_schema(orig_col->get_column_name());
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is NULL", K(ret));
|
|
} else {
|
|
col->set_collation_type(collation_type);
|
|
col->set_charset_type(ObCharset::charset_type_by_coll(collation_type));
|
|
if (OB_FAIL(fill_column_collation(sql_mode,
|
|
is_oracle_mode,
|
|
new_table_schema,
|
|
alter_table_arg.allocator_,
|
|
*col))) {
|
|
LOG_WARN("failed to fill column collation", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// convert character set for partition.
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(convert_to_character_for_partition(collation_type, new_table_schema))) {
|
|
LOG_WARN("convert collation type for partition failed", K(ret));
|
|
}
|
|
}
|
|
|
|
OZ (create_user_hidden_table(orig_table_schema,
|
|
new_table_schema,
|
|
&alter_table_arg.sequence_ddl_arg_,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check whether drop primary key is allowed.
|
|
int ObDDLService::check_can_drop_primary_key(const ObTableSchema &origin_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
const ObIArray<ObForeignKeyInfo> &fk_infos = origin_table_schema.get_foreign_key_infos();
|
|
ObPartitionFuncType part_func_type = origin_table_schema.get_part_option().get_part_func_type();
|
|
// disallow to drop pk if the table is implicit key partition table.
|
|
if (origin_table_schema.is_heap_table()) {
|
|
const ObString pk_name = "PRIMAY";
|
|
ret = OB_ERR_CANT_DROP_FIELD_OR_KEY;
|
|
LOG_WARN("can't DROP 'PRIMARY', check primary key exists", K(ret), K(origin_table_schema));
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, pk_name.length(), pk_name.ptr());
|
|
} else if (share::schema::PARTITION_FUNC_TYPE_KEY_IMPLICIT == part_func_type) {
|
|
ret = OB_ERR_FIELD_NOT_FOUND_PART;
|
|
LOG_WARN("can't drop primary key if table is implicit key partition table to be compatible with mysql mode", K(ret));
|
|
} else if (fk_infos.empty()) {
|
|
// allowed to drop primary key.
|
|
} else if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check if oracle compat mode", K(ret));
|
|
} else if (is_oracle_mode) {
|
|
// disallowed to drop pk if the table is parent table.
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < fk_infos.count(); i++) {
|
|
if (fk_infos.at(i).parent_table_id_ == origin_table_schema.get_table_id()) {
|
|
ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK;
|
|
LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK);
|
|
LOG_WARN("drop primary key is disallowed, due to referenced foreign key", K(ret));
|
|
}
|
|
}
|
|
} else {
|
|
// disallow to drop pk if,
|
|
// 1. the table is parent table,
|
|
// 2. the table is child table and the primary key columns contain foreign key refernce columns.
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < fk_infos.count(); i++) {
|
|
if (fk_infos.at(i).parent_table_id_ == origin_table_schema.get_table_id()) {
|
|
ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK;
|
|
LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK);
|
|
LOG_WARN("drop primary key is disallowed, due to referenced foreign key", K(ret));
|
|
} else {
|
|
bool is_all_rowkey_columns = true; // check the refernce columns is all part of rowkey columns.
|
|
FOREACH_CNT_X(col_id, fk_infos.at(i).child_column_ids_, OB_SUCC(ret) && is_all_rowkey_columns) {
|
|
const ObColumnSchemaV2 *col_schema = nullptr;
|
|
if (OB_ISNULL(col_schema = origin_table_schema.get_column_schema(*col_id))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("child column of foreign key is nullptr", K(ret), K(*col_id));
|
|
} else if (!col_schema->is_rowkey_column()) {
|
|
is_all_rowkey_columns = false;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && is_all_rowkey_columns) {
|
|
ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK;
|
|
LOG_USER_ERROR(OB_ERR_ATLER_TABLE_ILLEGAL_FK);
|
|
LOG_WARN("drop primary key is disallowed, due to referenced foreign key", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_table_primary_key(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
common::ObArenaAllocator &allocator,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t index_count = new_table_schema.get_index_tid_count();
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) {
|
|
ObIndexArg *index_arg = const_cast<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 {
|
|
const ObIndexArg::IndexActionType type = index_arg->index_action_type_;
|
|
switch(type) {
|
|
case ObIndexArg::ADD_INDEX: {
|
|
break;
|
|
}
|
|
case ObIndexArg::DROP_PRIMARY_KEY: {
|
|
ObCreateIndexArg *create_index_arg = static_cast<ObCreateIndexArg *>(index_arg);
|
|
ObSArray<ObString> index_columns;
|
|
index_columns.reset();
|
|
if (OB_FAIL(check_can_drop_primary_key(origin_table_schema))) {
|
|
LOG_WARN("fail to check whether to allow to drop primary key", K(ret));
|
|
} else if (OB_FAIL(create_hidden_table_with_pk_changed(alter_table_arg,
|
|
index_columns,
|
|
origin_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
allocator,
|
|
type,
|
|
tenant_data_version))) {
|
|
LOG_WARN("failed to add hidden primary key for heap table", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case ObIndexArg::ADD_PRIMARY_KEY:
|
|
case ObIndexArg::ALTER_PRIMARY_KEY: {
|
|
if (ObIndexArg::ADD_PRIMARY_KEY == type) {
|
|
if (OB_MAX_INDEX_PER_TABLE <= index_count) {
|
|
ret = OB_ERR_TOO_MANY_KEYS;
|
|
LOG_USER_ERROR(OB_ERR_TOO_MANY_KEYS, OB_MAX_INDEX_PER_TABLE);
|
|
LOG_WARN("too many index for table!", K(index_count), K(OB_MAX_INDEX_PER_TABLE));
|
|
} else if (!new_table_schema.is_heap_table()) {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret));
|
|
}
|
|
} else {
|
|
if (new_table_schema.is_heap_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("primary key does not exist!", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObCreateIndexArg *create_index_arg = static_cast<ObCreateIndexArg *>(index_arg);
|
|
ObSArray<ObString> index_columns;
|
|
for (int32_t i = 0; OB_SUCC(ret)
|
|
&& i < create_index_arg->index_columns_.count(); ++i) {
|
|
const ObColumnSortItem &col_item = create_index_arg->index_columns_.at(i);
|
|
if (OB_FAIL(index_columns.push_back(col_item.column_name_))) {
|
|
LOG_WARN("failed to add index column name", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) &&
|
|
OB_FAIL(create_hidden_table_with_pk_changed(alter_table_arg,
|
|
index_columns,
|
|
origin_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
allocator,
|
|
type,
|
|
tenant_data_version))) {
|
|
LOG_WARN("failed to add primary key", K(ret));
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("offline ddl cannot appear at the same time with other ddl", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_table_constraint(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
share::ObDDLType &ddl_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0};
|
|
const ObAlterTableArg::AlterConstraintType type = alter_table_arg.alter_constraint_type_;
|
|
bool change_cst_column_name = false;
|
|
bool is_alter_decimal_int_offline = false;
|
|
bool is_column_group_store = false;
|
|
switch(type) {
|
|
case obrpc::ObAlterTableArg::ADD_CONSTRAINT:
|
|
case obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE: {
|
|
if (OB_FAIL(check_is_change_cst_column_name(orig_table_schema,
|
|
alter_table_arg.alter_table_schema_,
|
|
change_cst_column_name))) {
|
|
LOG_WARN("failed to check change cst column name", K(ret));
|
|
} else if ((ObDDLType::DDL_TABLE_REDEFINITION == ddl_type || ObDDLType::DDL_MODIFY_COLUMN == ddl_type)
|
|
&& !change_cst_column_name) {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
} else if (is_long_running_ddl(ddl_type)) {
|
|
// if modify auto_increment and constraint together, treat it as normal modify column
|
|
ret = OB_NOT_SUPPORTED;
|
|
} else if (change_cst_column_name) {
|
|
ddl_type = ObDDLType::DDL_CHANGE_COLUMN_NAME;
|
|
ret = OB_NOT_SUPPORTED;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
}
|
|
break;
|
|
}
|
|
// to avoid ddl type being modified from DROP_COLUMN to NORMAL_TYPE
|
|
case obrpc::ObAlterTableArg::DROP_CONSTRAINT: {
|
|
bool is_drop_col_only = false;
|
|
if (ObDDLType::DDL_DROP_COLUMN == ddl_type) {
|
|
// In oracle mode, we support to drop constraint implicitly caused by drop column.
|
|
} else if (OB_FAIL(ObCODDLUtil::need_column_group_store(orig_table_schema, is_column_group_store))) {
|
|
LOG_WARN("fail to check schema is column group store", K(ret));
|
|
} else if (OB_FAIL(ObSchemaUtils::is_drop_column_only(alter_table_arg.alter_table_schema_, is_drop_col_only))) {
|
|
LOG_WARN("fail to check is drop column only", K(ret), K(alter_table_arg.alter_table_schema_));
|
|
} else if (ObDDLType::DDL_TABLE_REDEFINITION == ddl_type && is_drop_col_only && is_column_group_store) {
|
|
// for column store, drop column is table redefinition
|
|
} else if (OB_FAIL(check_is_alter_decimal_int_offline(ddl_type,
|
|
orig_table_schema,
|
|
alter_table_arg.alter_table_schema_,
|
|
is_alter_decimal_int_offline))) {
|
|
LOG_WARN("fail to check is alter decimal int offline ddl", K(ret));
|
|
} else if (is_long_running_ddl(ddl_type) && !is_alter_decimal_int_offline) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
} else if (is_alter_decimal_int_offline) {
|
|
ddl_type = ObDDLType::DDL_MODIFY_COLUMN;
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Unknown alter constraint action type!", K(ret), K(type));
|
|
}
|
|
}
|
|
if (OB_NOT_SUPPORTED == ret) {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and alter constraint in single statement", ddl_type_str(ddl_type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_constraint_caused_by_drop_column(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(check_can_alter_table_constraints(obrpc::ObAlterTableArg::DROP_CONSTRAINT, schema_guard, orig_table_schema, alter_table_schema))) {
|
|
LOG_WARN("fail to check can alter constraints", K(ret), K(alter_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.drop_table_constraints(orig_table_schema, alter_table_schema, new_table_schema, trans))) {
|
|
LOG_WARN("failed to drop table constraints", K(ret), K(alter_table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_all_dropped_udt_hidden_column_ids(const ObTableSchema &orig_table_schema, const ObColumnSchemaV2 &orig_column_schema,
|
|
common::ObIArray<int64_t> &drop_cols_id_arr, int64_t &columns_cnt_in_new_table)
|
|
{
|
|
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 oracle compat mode", K(ret));
|
|
} else if (is_oracle_mode && orig_column_schema.is_xmltype()) {
|
|
ObSEArray<ObColumnSchemaV2 *, 1> hidden_cols;
|
|
if (OB_FAIL(orig_table_schema.get_column_schema_in_same_col_group(orig_column_schema.get_column_id(),
|
|
orig_column_schema.get_udt_set_id(),
|
|
hidden_cols))) {
|
|
LOG_WARN("failed to get column schema", K(ret));
|
|
} else {
|
|
for (int i = 0; i < hidden_cols.count() && OB_SUCC(ret); i++) {
|
|
if (OB_FAIL(drop_cols_id_arr.push_back(hidden_cols.at(i)->get_column_id()))) {
|
|
LOG_WARN("fail to push back column id", K(ret), KPC(hidden_cols.at(i)));
|
|
} else {
|
|
columns_cnt_in_new_table--;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_all_dropped_column_ids(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
common::ObIArray<int64_t> &drop_cols_id_arr,
|
|
int64_t *new_table_cols_cnt)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t columns_cnt_in_new_table = orig_table_schema.get_column_count();
|
|
ObTableSchema::const_column_iterator it = nullptr;
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
for (it = alter_table_schema.column_begin(); OB_SUCC(ret) && it != alter_table_schema.column_end(); it++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("alter_column_schema is NULL", K(ret), K(alter_table_schema));
|
|
} else if (OB_DDL_ADD_COLUMN == alter_column_schema->alter_type_) {
|
|
columns_cnt_in_new_table++;
|
|
} else if (OB_DDL_DROP_COLUMN == alter_column_schema->alter_type_) {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
const ObColumnSchemaV2 *orig_column_schema = orig_table_schema.get_column_schema(orig_column_name);
|
|
if (OB_ISNULL(orig_column_schema)) {
|
|
ret = OB_ERR_CANT_DROP_FIELD_OR_KEY;
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, orig_column_name.length(), orig_column_name.ptr());
|
|
LOG_WARN("fail to find old column schema!", K(ret), K(orig_column_name), KPC(orig_column_schema));
|
|
} else if (OB_FAIL(drop_cols_id_arr.push_back(orig_column_schema->get_column_id()))) {
|
|
LOG_WARN("fail to push back column id", K(ret), KPC(orig_column_schema));
|
|
} else {
|
|
columns_cnt_in_new_table--;
|
|
if (OB_FAIL(get_all_dropped_udt_hidden_column_ids(orig_table_schema, *orig_column_schema,
|
|
drop_cols_id_arr, columns_cnt_in_new_table))) {
|
|
LOG_WARN("fail to push back udt hidden column id", K(ret), KPC(orig_column_schema));
|
|
}
|
|
}
|
|
} else {/* do nothing. */}
|
|
}
|
|
if (OB_SUCC(ret) && nullptr != new_table_cols_cnt) {
|
|
*new_table_cols_cnt = columns_cnt_in_new_table;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_can_drop_column(
|
|
const ObString &orig_column_name,
|
|
const ObColumnSchemaV2 *orig_column_schema,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &new_table_schema,
|
|
const int64_t new_table_cols_cnt,
|
|
ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t column_count = new_table_cols_cnt;
|
|
if (OB_ISNULL(orig_column_schema) || OB_ISNULL(new_table_schema.get_column_schema(orig_column_name))) {
|
|
ret = OB_ERR_CANT_DROP_FIELD_OR_KEY;
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, orig_column_name.length(), orig_column_name.ptr());
|
|
LOG_WARN("fail to find old column schema!", K(ret), K(orig_column_name), KP(orig_column_schema),
|
|
K(new_table_schema));
|
|
} else if (orig_table_schema.is_oracle_tmp_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop oracle temporary table column is");
|
|
LOG_WARN("oracle temporary table column is not allowed to be dropped", K(ret));
|
|
} else if (orig_column_schema->has_generated_column_deps()) {
|
|
bool has_func_idx_col_deps = false;
|
|
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 oracle compat mode", K(ret));
|
|
} else if (!is_oracle_mode && OB_FAIL(orig_table_schema.check_functional_index_columns_depend(*orig_column_schema, schema_guard, has_func_idx_col_deps))) {
|
|
LOG_WARN("fail to check if column has functional index dependency.", K(ret));
|
|
} else if (!has_func_idx_col_deps) {
|
|
ret = OB_ERR_DEPENDENT_BY_GENERATED_COLUMN;
|
|
LOG_USER_ERROR(OB_ERR_DEPENDENT_BY_GENERATED_COLUMN, orig_column_name.length(), orig_column_name.ptr());
|
|
LOG_WARN("Dropping column has generated column deps", K(ret), K(orig_column_name));
|
|
} else {
|
|
ret = OB_ERR_DEPENDENT_BY_FUNCTIONAL_INDEX;
|
|
LOG_USER_ERROR(OB_ERR_DEPENDENT_BY_FUNCTIONAL_INDEX, orig_column_name.length(), orig_column_name.ptr());
|
|
LOG_WARN("Dropping column has functional index column deps", K(ret), K(orig_column_name));
|
|
}
|
|
} else if (OB_FAIL(check_is_drop_partition_key(orig_table_schema, *orig_column_schema, schema_guard))) {
|
|
LOG_WARN("check drop partition column failed", K(ret));
|
|
} else if ((!new_table_schema.is_heap_table() && column_count <= ObTableSchema::MIN_COLUMN_COUNT_WITH_PK_TABLE)
|
|
|| (new_table_schema.is_heap_table() && column_count <= ObTableSchema::MIN_COLUMN_COUNT_WITH_HEAP_TABLE)) {
|
|
ret = OB_CANT_REMOVE_ALL_FIELDS;
|
|
LOG_USER_ERROR(OB_CANT_REMOVE_ALL_FIELDS);
|
|
LOG_WARN("Can not delete all columns in table", K(ret));
|
|
} else if (orig_column_schema->is_rowkey_column()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop rowkey column is");
|
|
LOG_WARN("rowkey column is not allowed to be dropped", K(ret), K(orig_column_schema->get_column_name_str()));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_drop_partition_key(
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
const ObColumnSchemaV2 &to_drop_column,
|
|
ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_tbl_partition_key = false;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
const uint64_t to_drop_column_id = to_drop_column.get_column_id();
|
|
if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get index infos failed", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.is_tbl_partition_key(to_drop_column_id, is_tbl_partition_key,
|
|
false /* ignore_presetting_key */))) {
|
|
LOG_WARN("fail to check tbl partition key", K(ret), K(to_drop_column), K(orig_table_schema));
|
|
} else {
|
|
// to check whether the column is the partition key of the global index.
|
|
for (int64_t idx = 0; OB_SUCC(ret) && !is_tbl_partition_key && idx < simple_index_infos.count(); idx++) {
|
|
const ObTableSchema *index_schema = nullptr;
|
|
const uint64_t index_tid = simple_index_infos.at(idx).table_id_;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, index_tid, index_schema))) {
|
|
LOG_WARN("get index schema failed", K(ret), K(tenant_id), K(index_tid));
|
|
} else if (OB_ISNULL(index_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("index not exist", K(ret), K(tenant_id), K(index_tid));
|
|
} else if (index_schema->is_global_index_table()
|
|
&& OB_FAIL(index_schema->is_tbl_partition_key(to_drop_column_id, is_tbl_partition_key,
|
|
false /* ignore_presetting_key */))) {
|
|
LOG_WARN("check column in part key failed", K(ret), K(index_tid));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && is_tbl_partition_key) {
|
|
ret = OB_ERR_DEPENDENT_BY_PARTITION_FUNC;
|
|
LOG_USER_ERROR(OB_ERR_DEPENDENT_BY_PARTITION_FUNC,
|
|
to_drop_column.get_column_name_str().length(),
|
|
to_drop_column.get_column_name_str().ptr());
|
|
LOG_WARN("drop column has table part key deps", K(ret), "column_name", to_drop_column.get_column_name_str());
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// to check whether the dropped column is related to constraint, and check can drop the column.
|
|
int ObDDLService::check_drop_column_with_drop_constraint(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &orig_table_schema,
|
|
const common::ObIArray<int64_t> &drop_cols_id_arr)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const ObAlterTableArg::AlterConstraintType type = alter_table_arg.alter_constraint_type_;
|
|
if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check if oracle compat mode", K(ret));
|
|
} else if (OB_FAIL(check_can_alter_table_constraints(type, schema_guard, orig_table_schema, alter_table_schema))) {
|
|
LOG_WARN("fail to check can alter constraints", K(ret), K(type), K(alter_table_schema));
|
|
} else {
|
|
FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret)) {
|
|
for (ObTableSchema::const_constraint_iterator iter = orig_table_schema.constraint_begin(); OB_SUCC(ret) &&
|
|
iter != orig_table_schema.constraint_end(); iter++) {
|
|
if (CONSTRAINT_TYPE_CHECK != (*iter)->get_constraint_type()) {
|
|
} else if (0 == (*iter)->get_column_cnt()) {
|
|
} else {
|
|
const ObString &cst_name = (*iter)->get_constraint_name_str();
|
|
for (ObConstraint::const_cst_col_iterator cst_col_iter = (*iter)->cst_col_begin();
|
|
OB_SUCC(ret) && (cst_col_iter != (*iter)->cst_col_end()); ++cst_col_iter) {
|
|
if (*cst_col_iter == *dropped_col) {
|
|
// the dropped column is related to check constraint.
|
|
const ObString &dropped_column_name = orig_table_schema.get_column_schema(*dropped_col)->get_column_name_str();
|
|
ObConstraint* const* res = std::find_if(alter_table_schema.constraint_begin(), alter_table_schema.constraint_end(), [&cst_name](const ObConstraint* cst)
|
|
{ return 0 == cst_name.case_compare(cst->get_constraint_name_str()); });
|
|
if (alter_table_schema.constraint_end() == res) {
|
|
LOG_WARN("the column is related to check constraint, can not be dropped", K(ret), K(cst_name), K(dropped_column_name));
|
|
if (is_oracle_mode) {
|
|
ret = OB_ERR_MODIFY_OR_DROP_MULTI_COLUMN_CONSTRAINT;
|
|
LOG_USER_ERROR(OB_ERR_DROP_COL_REFERENCED_MULTI_COLS_CONSTRAINT,
|
|
dropped_column_name.length(), dropped_column_name.ptr(),
|
|
cst_name.length(), cst_name.ptr());
|
|
} else {
|
|
ret = OB_ERR_DROP_COL_REFERENCED_MULTI_COLS_CONSTRAINT;
|
|
LOG_USER_ERROR(OB_ERR_DROP_COL_REFERENCED_MULTI_COLS_CONSTRAINT,
|
|
cst_name.length(), cst_name.ptr(), dropped_column_name.length(),
|
|
dropped_column_name.ptr());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// To be compatible with Mysql 5.6 and 8.0, follwing behavior on child table are allowed on OB 4.0:
|
|
// 1. drop foreign key non-related columns and drop any foreign key in single stmt;
|
|
// 2. drop the foreign key and its' some/all related columns in single stmt.
|
|
// Notice that, drop fk related column on parent table has been processed in phase ddl resolver.
|
|
// Here, only need to report OB_ERR_ALTER_COLUMN_FK if drop foreign key related columns without drop the fk.
|
|
int ObDDLService::check_drop_column_with_drop_foreign_key(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const common::ObIArray<int64_t> &drop_cols_id_arr)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret)) {
|
|
// 1. to iter all foreign keys related to the dropped column.
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = orig_table_schema.get_foreign_key_infos();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
const ObForeignKeyInfo &fk_info = foreign_key_infos.at(i);
|
|
const ObString &curr_fk_name = fk_info.foreign_key_name_;
|
|
if (fk_info.child_table_id_ == orig_table_schema.get_table_id()) {
|
|
bool is_drop_child_col = false;
|
|
FOREACH_CNT_X(col_id, fk_info.child_column_ids_, OB_SUCC(ret) && !is_drop_child_col) {
|
|
if (*dropped_col == *col_id) {
|
|
is_drop_child_col = true;
|
|
}
|
|
}
|
|
if (is_drop_child_col) {
|
|
// 2. to check whether to drop the related foreign key.
|
|
bool is_drop_curr_fk = false;
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
for (int64_t i = 0; OB_SUCC(ret) && !is_drop_curr_fk && i < index_arg_list.size(); ++i) {
|
|
ObIndexArg *index_arg = const_cast<ObIndexArg *>(index_arg_list.at(i));
|
|
if (OB_ISNULL(index_arg)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument, index arg should not be null", K(ret), K(index_arg_list));
|
|
} else if (ObIndexArg::DROP_FOREIGN_KEY != index_arg->index_action_type_) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(index_arg));
|
|
} else {
|
|
const ObDropForeignKeyArg *drop_foreign_key_arg = static_cast<ObDropForeignKeyArg *>(index_arg);
|
|
if (0 == curr_fk_name.case_compare(drop_foreign_key_arg->foreign_key_name_)) {
|
|
is_drop_curr_fk = true;
|
|
}
|
|
}
|
|
}
|
|
// 3. drop child column of fk, but the fk is not dropped, should report error.
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!is_drop_curr_fk) {
|
|
const ObString &column_name = orig_table_schema.get_column_schema(*dropped_col)->get_column_name_str();
|
|
ret = OB_ERR_ALTER_COLUMN_FK;
|
|
LOG_USER_ERROR(OB_ERR_ALTER_COLUMN_FK, column_name.length(), column_name.ptr());
|
|
LOG_WARN("the column is related to foreign key, and can not be dropped", K(ret), K(column_name), K(curr_fk_name));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::delete_column_from_schema(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObSchemaGetterGuard schema_guard;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
const ObString &orig_database_name = alter_table_schema.get_origin_database_name();
|
|
const ObString &orig_table_name = alter_table_schema.get_origin_table_name();
|
|
ObTableSchema new_table_schema;
|
|
const ObTableSchema *orig_table_schema = nullptr;
|
|
const ObTenantSchema *tenant_schema = nullptr;
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(tenant_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tenant schema is null", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
orig_database_name,
|
|
orig_table_name,
|
|
false, // is_index
|
|
orig_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(orig_database_name), K(orig_table_name), KP(orig_table_schema));
|
|
} else if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("err unexpected", K(ret), K(orig_database_name), K(orig_table_name));
|
|
} else if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
new_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(drop_constraint_caused_by_drop_column(alter_table_arg, schema_guard, *orig_table_schema, new_table_schema, ddl_operator, trans))) {
|
|
LOG_WARN("fail to drop constraint caused by drop column", K(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(delete_column_from_schema_in_trans(alter_table_schema, schema_guard, *orig_table_schema, new_table_schema,
|
|
ddl_operator, trans))) {
|
|
LOG_WARN("fail to set column to no minor", K(ret), K(new_table_schema));
|
|
} else {
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema,
|
|
trans,
|
|
operation_type,
|
|
&alter_table_arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(delete_auto_increment_attribute(orig_table_schema, new_table_schema, alter_table_schema))) {
|
|
LOG_WARN("fail to delete auto-increment attri", K(ret), K(new_table_schema), K(alter_table_schema));
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("fail to publish schema", K(ret), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::delete_column_from_schema_in_trans(
|
|
const AlterTableSchema &alter_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema::const_column_iterator it = nullptr;
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
for (it = alter_table_schema.column_begin(); OB_SUCC(ret) && it != alter_table_schema.column_end(); it++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("alter_column_schema is NULL", K(ret), K(alter_table_schema));
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
const ObColumnSchemaV2 *orig_column_schema = orig_table_schema.get_column_schema(orig_column_name);
|
|
if (OB_ISNULL(orig_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ObColumnSchemaV2 should not be null", K(ret), K(*it));
|
|
} else if (OB_FAIL(ddl_operator.drop_sequence_in_drop_column(
|
|
*orig_column_schema,
|
|
trans,
|
|
schema_guard))) {
|
|
LOG_WARN("alter table drop identity column fail", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_prev_id_for_delete_column(orig_table_schema,
|
|
new_table_schema, *orig_column_schema, trans))) {
|
|
LOG_WARN("failed to update column previous id for delele column", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.delete_single_column(trans, new_table_schema, orig_column_name))) {
|
|
LOG_WARN("fail to delete column", K(ret), K(alter_column_schema));
|
|
} else {
|
|
LOG_INFO("delete column from schema", K(orig_column_name));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::delete_auto_increment_attribute(
|
|
const ObTableSchema *orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
AlterTableSchema &alter_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
uint64_t table_id = new_table_schema.get_table_id();
|
|
uint64_t part_num = new_table_schema.get_all_part_num();
|
|
// drop auto-increment attr in drop column
|
|
ObAutoincrementService &autoinc_service = ObAutoincrementService::get_instance();
|
|
if (0 != orig_table_schema->get_autoinc_column_id()
|
|
&& 0 == new_table_schema.get_autoinc_column_id()) {
|
|
LOG_INFO("begin to clear all auto-increment cache",
|
|
K(tenant_id), K(table_id), K(part_num),
|
|
K(orig_table_schema->get_autoinc_column_id()));
|
|
if (OB_FAIL(autoinc_service.clear_autoinc_cache_all(tenant_id,
|
|
table_id,
|
|
orig_table_schema->get_autoinc_column_id(),
|
|
orig_table_schema->is_order_auto_increment_mode()))) {
|
|
LOG_WARN("failed to clear auto-increment cache", K(tenant_id), K(table_id));
|
|
}
|
|
}
|
|
// sync sync_value(auto_increment)
|
|
uint64_t auto_increment = alter_table_schema.get_auto_increment();
|
|
if (OB_SUCC(ret)) {
|
|
if (new_table_schema.get_autoinc_column_id() > 0 && auto_increment > 0) {
|
|
LOG_INFO("begin to sync auto_increment",
|
|
K(tenant_id), K(table_id), K(part_num), K(auto_increment));
|
|
if (OB_FAIL(autoinc_service.sync_auto_increment_all(tenant_id,
|
|
table_id,
|
|
new_table_schema.get_autoinc_column_id(),
|
|
0 == auto_increment ? 0 : auto_increment - 1))) {
|
|
LOG_WARN("failed to sync auto_increment", K(tenant_id), K(table_id), K(auto_increment));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// rename and remap all index tables.
|
|
int ObDDLService::remap_index_tablets_to_new_indexs(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObSArray<ObTableSchema> &table_schemas,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObSEArray<ObAuxTableMetaInfo, 16>simple_index_infos;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not inited", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("fail to get simple index infos failed", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = nullptr;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("fail to get index_table_schema", K(ret), K(tenant_id), "table_id", simple_index_infos.at(i).table_id_);
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("index table schema should not be null", K(ret), "table_id", simple_index_infos.at(i).table_id_);
|
|
} else {
|
|
ObString new_index_table_name;
|
|
HEAP_VAR(ObTableSchema, new_index_table_schema) {
|
|
if (OB_FAIL(new_index_table_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign new index table schema", K(ret));
|
|
} else if (OB_FAIL(gen_new_index_table_name(index_table_schema->get_table_name_str(),
|
|
orig_table_schema.get_table_id(),
|
|
hidden_table_schema.get_table_id(),
|
|
alter_table_arg.allocator_,
|
|
new_index_table_name))) {
|
|
LOG_WARN("fail to build new index table name!", K(ret), K(hidden_table_schema.get_table_id()), K(new_index_table_name));
|
|
} else {
|
|
uint64_t new_index_tid = OB_INVALID_ID;
|
|
const ObString *ddl_stmt_str = nullptr;
|
|
const bool need_sync_schema_version = true;
|
|
const bool is_truncate_table = false;
|
|
if (OB_FAIL(schema_service->fetch_new_table_id(tenant_id, new_index_tid))) {
|
|
LOG_WARN("fail to fetch new table id", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema version", K(ret), K(tenant_id));
|
|
} else {
|
|
new_index_table_schema.set_table_id(new_index_tid);
|
|
new_index_table_schema.set_data_table_id(hidden_table_schema.get_table_id());
|
|
new_index_table_schema.set_table_name(new_index_table_name);
|
|
new_index_table_schema.set_schema_version(new_schema_version);
|
|
new_index_table_schema.set_tablet_id(index_table_schema->get_tablet_id());
|
|
new_index_table_schema.set_in_offline_ddl_white_list(true);
|
|
new_index_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL);
|
|
if (OB_FAIL(schema_service->get_table_sql_service().create_table(new_index_table_schema,
|
|
trans,
|
|
ddl_stmt_str,
|
|
need_sync_schema_version,
|
|
is_truncate_table))) {
|
|
LOG_WARN("fail to create table", K(ret), K(need_sync_schema_version), K(is_truncate_table));
|
|
} else if (OB_FAIL(table_schemas.push_back(new_index_table_schema))) {
|
|
LOG_WARN("fail to push back table schmea", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// remap and rename index tables, and then take effect.
|
|
int ObDDLService::swap_orig_and_hidden_table_state(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
ObSArray<ObTableSchema> &table_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
HEAP_VARS_2((ObTableSchema, new_orig_table_schema),
|
|
(ObTableSchema, new_hidden_table_schema)) {
|
|
ObSEArray<ObAuxTableMetaInfo, 16> orig_simple_index_infos;
|
|
if (OB_FAIL(new_orig_table_schema.assign(orig_table_schema))
|
|
|| OB_FAIL(new_hidden_table_schema.assign(hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.get_simple_index_infos(
|
|
orig_simple_index_infos))) {
|
|
LOG_WARN("fail to get simple index infos", K(ret));
|
|
} else {
|
|
// if the original table is a parent table, after swap the status of the two tables
|
|
// need to rename the original fk name to the newly generated hidden fk name , and then
|
|
// rename the newly generated hidden fk name to the original fk name on the child table
|
|
if (orig_table_schema.is_parent_table()) {
|
|
if (OB_FAIL(swap_all_child_table_fk_name(orig_table_schema,
|
|
hidden_table_schema,
|
|
schema_guard,
|
|
trans,
|
|
alter_table_arg.allocator_))) {
|
|
LOG_WARN("fail to modify child table fk name", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.need_rebuild_trigger_) {
|
|
if (OB_FAIL(rebuild_triggers_on_hidden_table(orig_table_schema,
|
|
hidden_table_schema,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("fail to create triggers on hidden table", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObTableSchema tmp_schema;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_simple_index_infos.count(); i++) {
|
|
tmp_schema.reset();
|
|
const ObTableSchema *orig_index_table_schema = nullptr;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
orig_simple_index_infos.at(i).table_id_, orig_index_table_schema))) {
|
|
LOG_WARN("fail to get orig index table schema", K(ret),
|
|
K(tenant_id), "table_id", orig_simple_index_infos.at(i).table_id_);
|
|
} else if (OB_ISNULL(orig_index_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("orig index table schema is null", K(ret), "table_id", orig_simple_index_infos.at(i).table_id_);
|
|
} else if (OB_FAIL(tmp_schema.assign(*orig_index_table_schema))) {
|
|
LOG_WARN("fail to assign orig index table schema", K(ret));
|
|
} else {
|
|
tmp_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL);
|
|
if (OB_FAIL(table_schemas.push_back(tmp_schema))) {
|
|
LOG_WARN("fail to add table schema", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
new_orig_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL);
|
|
new_hidden_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL);
|
|
new_orig_table_schema.set_table_name(hidden_table_schema.get_table_name_str());
|
|
new_hidden_table_schema.set_table_name(orig_table_schema.get_table_name_str());
|
|
if (OB_FAIL(table_schemas.push_back(new_orig_table_schema))
|
|
|| OB_FAIL(table_schemas.push_back(new_hidden_table_schema))) {
|
|
LOG_WARN("fail to add table schema!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
const ObString *ddl_stmt_str = nullptr;
|
|
ObTableSchema &tmp_schema = table_schemas.at(i);
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
tmp_schema.set_in_offline_ddl_white_list(true);
|
|
if (tmp_schema.get_table_id() == orig_table_schema.get_table_id()) {
|
|
ddl_stmt_str = &alter_table_arg.ddl_stmt_str_;
|
|
}
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(tmp_schema,
|
|
trans,
|
|
operation_type,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::remap_index_tablets_and_take_effect(
|
|
obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSArray<uint64_t> index_ids;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
const uint64_t dest_tenant_id = alter_table_arg.exec_tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSArray<ObTableSchema> table_schemas;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObSchemaGetterGuard dest_schema_guard;
|
|
const ObTableSchema *orig_table_schema = nullptr;
|
|
const ObTableSchema *hidden_table_schema = nullptr;
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
int64_t refreshed_schema_version = 0;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(dest_tenant_id, dest_schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(dest_tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
dest_schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("fail to get orig and hidden table schema", K(ret));
|
|
} else if (OB_FAIL(remap_index_tablets_to_new_indexs(alter_table_arg,
|
|
*orig_table_schema,
|
|
*hidden_table_schema,
|
|
schema_guard,
|
|
table_schemas,
|
|
trans))) {
|
|
LOG_WARN("fail to remap all index tables to the hidden table", K(ret));
|
|
} else if (OB_FAIL(swap_orig_and_hidden_table_state(alter_table_arg,
|
|
*orig_table_schema,
|
|
*hidden_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
table_schemas))) {
|
|
LOG_WARN("fail to swap origin and hidden table state", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::switch_index_name_and_status_for_vec_index_table(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
LOG_DEBUG("switch_index_name_and_status_for_vec_index_table", K(alter_table_arg));
|
|
|
|
const int64_t old_index_id = alter_table_arg.table_id_;
|
|
const int64_t new_index_id = alter_table_arg.hidden_table_id_;
|
|
const int64_t tenant_id = alter_table_arg.alter_table_schema_.get_tenant_id();
|
|
common::ObIAllocator &allocator = alter_table_arg.allocator_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (old_index_id == OB_INVALID_ID || new_index_id == OB_INVALID_ID
|
|
|| tenant_id == OB_INVALID_TENANT_ID) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(old_index_id), K(new_index_id), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const ObTableSchema *old_table_schema = NULL;
|
|
const ObTableSchema *new_table_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, old_index_id, old_table_schema))) {
|
|
LOG_WARN("fail to get old index table schema", K(ret), K(old_index_id));
|
|
} else if (OB_ISNULL(old_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpectd null pointer", K(ret));
|
|
} else if (!old_table_schema->is_vec_index() || !old_table_schema->is_vec_delta_buffer_type()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("old index table must be vec delta_buffer_table", K(ret), K(old_table_schema));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, new_index_id, new_table_schema))) {
|
|
LOG_WARN("fail to get new index table schema", K(ret), K(new_index_id));
|
|
} else if (OB_ISNULL(new_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpectd null pointer", K(ret));
|
|
} else if (!new_table_schema->is_vec_index() || !new_table_schema->is_vec_delta_buffer_type()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("new index table must be vec delta_buffer_table", K(ret), K(new_table_schema));
|
|
} else if (old_table_schema->get_database_id() != new_table_schema->get_database_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("old table and new table should in same database", K(ret));
|
|
} else if (old_table_schema->get_data_table_id() != new_table_schema->get_data_table_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("old table and new table should in same data table", K(ret));
|
|
} else {
|
|
const int64_t database_id = old_table_schema->get_database_id();
|
|
const int64_t data_table_id = old_table_schema->get_data_table_id();
|
|
const bool is_index = true;
|
|
ObString old_domain_index_name = old_table_schema->get_table_name_str();
|
|
ObString new_domain_index_name = new_table_schema->get_table_name_str();
|
|
ObArray<ObString> new_table_names;
|
|
ObArray<ObString> old_table_names;
|
|
const int64_t unused_id = 1; // not set OB_INVALID_ID for sava check
|
|
|
|
SMART_VAR(ObSArray<ObTableSchema>, table_schemas) {
|
|
// ObSArray<ObTableSchema> table_schemas;
|
|
const int64_t EXPECTED_UPDATE_TABLE_CNT = 7; // 3 old index table + 3 new index table + 1 data data
|
|
if (OB_FAIL(ObVectorIndexUtil::generate_switch_index_names(old_domain_index_name,
|
|
new_domain_index_name,
|
|
allocator,
|
|
old_table_names,
|
|
new_table_names))) {
|
|
LOG_WARN("fail to generate switch index names");
|
|
} else if (OB_FAIL(ObVectorIndexUtil::update_index_tables_status(tenant_id,
|
|
database_id,
|
|
old_table_names,
|
|
new_table_names,
|
|
ddl_operator,
|
|
schema_guard,
|
|
trans,
|
|
table_schemas))) {
|
|
LOG_WARN("fail to update index table status", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ObVectorIndexUtil::update_index_tables_attributes(tenant_id,
|
|
database_id,
|
|
data_table_id,
|
|
EXPECTED_UPDATE_TABLE_CNT,
|
|
old_table_names,
|
|
new_table_names,
|
|
ddl_operator,
|
|
schema_guard,
|
|
trans,
|
|
table_schemas))) {
|
|
LOG_WARN("fail to update index table attribute", K(ret), K(tenant_id));
|
|
}
|
|
} // end smart_var table_schemas
|
|
}
|
|
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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_autoinc_schema(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObSchemaGetterGuard schema_guard;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
uint64_t table_id = alter_table_schema.get_table_id();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
const ObTableSchema *curr_table_schema;
|
|
ObTableSchema new_table_schema;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObAutoincrementService &auto_inc_service = ObAutoincrementService::get_instance();
|
|
|
|
if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, curr_table_schema))) {
|
|
LOG_WARN("get data table schema failed", K(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_ISNULL(curr_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("cannot find orig table", K(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_UNLIKELY(curr_table_schema->get_table_state_flag() != ObTableStateFlag::TABLE_STATE_OFFLINE_DDL)) {
|
|
ret = OB_NO_NEED_UPDATE;
|
|
LOG_WARN("already updated", K(ret), K(tenant_id), K(table_id), K(curr_table_schema->get_schema_version()));
|
|
} else if (OB_FAIL(new_table_schema.assign(*curr_table_schema))) {
|
|
LOG_WARN("fail to assign table schema", K(ret));
|
|
} else {
|
|
ObTableSchema::const_column_iterator iter = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator iter_end = alter_table_schema.column_end();
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
ObColumnSchemaV2 *new_column_schema = nullptr;
|
|
int64_t alter_column_num = 0;
|
|
new_table_schema.set_in_offline_ddl_white_list(true);
|
|
new_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL);
|
|
for(; OB_SUCC(ret) && iter != iter_end; iter++) {
|
|
if (alter_column_num != 0 || OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*iter))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected alter_column_num or iter is NULL", K(ret), K(alter_column_num));
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
const ObColumnSchemaV2 *curr_column_schema = curr_table_schema->get_column_schema(orig_column_name);
|
|
new_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
if (OB_ISNULL(new_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("alter column schema is null", KR(ret), K(new_table_schema.get_table_id()),
|
|
K(orig_column_name));
|
|
} else if (OB_ISNULL(curr_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("cur column schema is null", KR(ret), K(curr_table_schema->get_table_id()),
|
|
K(orig_column_name));
|
|
} else {
|
|
new_column_schema->set_autoincrement(alter_column_schema->is_autoincrement());
|
|
new_column_schema->set_nullable(alter_column_schema->is_nullable());
|
|
new_table_schema.set_auto_increment(alter_table_schema.get_auto_increment());
|
|
new_table_schema.set_autoinc_column_id(alter_column_schema->get_column_id());
|
|
|
|
// we need clear inner autoinc when add autoinc attribute bug/53305960
|
|
if (new_column_schema->is_autoincrement() && !curr_column_schema->is_autoincrement()) {
|
|
if (OB_FAIL(ddl_operator.try_reinit_autoinc_row(new_table_schema, trans))) {
|
|
LOG_WARN("fail to reinit autoinc row", KR(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
if (FAILEDx(ddl_operator.update_single_column(trans,
|
|
*curr_table_schema,
|
|
new_table_schema,
|
|
*new_column_schema))) {
|
|
LOG_WARN("update single column failed", K(ret), K(*new_column_schema));
|
|
}
|
|
}
|
|
alter_column_num++;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
const bool is_commit = alter_column_num == 1;
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema,
|
|
trans,
|
|
operation_type,
|
|
is_commit ? &alter_table_arg.ddl_stmt_str_ : nullptr))) {
|
|
LOG_WARN("update table attribute failed", K(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_NO_NEED_UPDATE == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_partitions(ObMySQLTransaction &trans,
|
|
const ObTableSchema &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTabletIDArray tablet_ids;
|
|
const int64_t tenant_id = table_schema.get_tenant_id();
|
|
const int64_t table_id = table_schema.get_table_id();
|
|
// skip those type table for lock table
|
|
if (!table_schema.has_tablet()
|
|
|| table_schema.is_aux_table()
|
|
|| table_schema.is_mlog_table()
|
|
|| table_schema.is_sys_table()) {
|
|
} else if (OB_FAIL(table_schema.get_tablet_ids(tablet_ids))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(table_schema));
|
|
} else if (OB_FAIL(lock_tablets(trans, tenant_id, table_id, tablet_ids))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(table_schema), K(table_schema));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_tablets(ObMySQLTransaction &trans,
|
|
const int64_t tenant_id,
|
|
const int64_t table_id,
|
|
const ObTabletIDArray &tablet_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t timeout = 0;
|
|
|
|
observer::ObInnerSQLConnection *conn = NULL;
|
|
if (OB_ISNULL(conn = dynamic_cast<observer::ObInnerSQLConnection *>
|
|
(trans.get_connection()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("conn_ is NULL", KR(ret));
|
|
} else {
|
|
LOG_INFO("lock tablet", KR(ret), K(tablet_ids), K(table_id), K(tenant_id), KPC(conn));
|
|
if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_2_0_0) {
|
|
for (int i = 0; i < tablet_ids.count() && OB_SUCC(ret); i++) {
|
|
if (OB_FAIL(ObInnerConnectionLockUtil::lock_tablet(tenant_id,
|
|
table_id,
|
|
tablet_ids.at(i),
|
|
EXCLUSIVE,
|
|
timeout,
|
|
conn))) {
|
|
LOG_WARN("lock dest table failed", KR(ret), K(table_id), K(tenant_id));
|
|
}
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ObInnerConnectionLockUtil::lock_tablet(tenant_id,
|
|
table_id,
|
|
tablet_ids,
|
|
EXCLUSIVE,
|
|
timeout,
|
|
conn))) {
|
|
LOG_WARN("lock dest table failed", KR(ret), K(table_id), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_table(ObMySQLTransaction &trans,
|
|
const ObSimpleTableSchemaV2 &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
const uint64_t table_id = table_schema.get_table_id();
|
|
const int64_t tenant_id = table_schema.get_tenant_id();
|
|
const int64_t timeout = 0;
|
|
|
|
observer::ObInnerSQLConnection *conn = NULL;
|
|
// skip those type table for lock table
|
|
if (!table_schema.has_tablet()
|
|
|| table_schema.is_aux_table()
|
|
|| table_schema.is_mlog_table()
|
|
|| table_schema.is_sys_table()) {
|
|
} else if (OB_ISNULL(conn = dynamic_cast<observer::ObInnerSQLConnection *>
|
|
(trans.get_connection()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("conn_ is NULL", KR(ret));
|
|
} else {
|
|
LOG_INFO("lock table", KR(ret), K(table_id), K(tenant_id), KPC(conn));
|
|
if (OB_FAIL(ObInnerConnectionLockUtil::lock_table(tenant_id,
|
|
table_id,
|
|
EXCLUSIVE,
|
|
timeout,
|
|
conn))) {
|
|
LOG_WARN("lock dest table failed", KR(ret), K(table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_mview(ObMySQLTransaction &trans, const ObSimpleTableSchemaV2 &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t tenant_id = table_schema.get_tenant_id();
|
|
const uint64_t mview_id = table_schema.get_table_id();
|
|
observer::ObInnerSQLConnection *conn = nullptr;
|
|
uint64_t data_version = 0;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
|
|
LOG_WARN("fail to get data version", KR(ret), K(tenant_id));
|
|
} else if (OB_UNLIKELY(data_version < DATA_VERSION_4_3_0_0)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("version lower than 4.3 does not support this operation", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant's data version is below 4.3.0.0, mview is ");
|
|
} else if (OB_UNLIKELY(!table_schema.is_materialized_view())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid args", KR(ret), K(table_schema));
|
|
} else if (OB_ISNULL(conn = dynamic_cast<observer::ObInnerSQLConnection *>
|
|
(trans.get_connection()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("conn_ is NULL", KR(ret));
|
|
} else {
|
|
LOG_INFO("lock mview", KR(ret), K(mview_id), K(tenant_id), KPC(conn));
|
|
ObLockObjRequest lock_arg;
|
|
lock_arg.obj_type_ = ObLockOBJType::OBJ_TYPE_MATERIALIZED_VIEW;
|
|
lock_arg.obj_id_ = mview_id;
|
|
lock_arg.owner_id_ = ObTableLockOwnerID::default_owner();
|
|
lock_arg.lock_mode_ = EXCLUSIVE;
|
|
lock_arg.op_type_ = ObTableLockOpType::IN_TRANS_COMMON_LOCK;
|
|
lock_arg.timeout_us_ = 0;
|
|
if (OB_FAIL(ObInnerConnectionLockUtil::lock_obj(tenant_id, lock_arg, conn))) {
|
|
LOG_WARN("fail to lock mview obj", KR(ret), K(tenant_id), K(lock_arg), KPC(conn));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_tables_of_database(const ObDatabaseSchema &database_schema,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const uint64_t tenant_id = database_schema.get_tenant_id();
|
|
const uint64_t database_id = database_schema.get_database_id();
|
|
ObArray<const ObSimpleTableSchemaV2*> table_schemas;
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is null", KR(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schemas_in_database(tenant_id,
|
|
database_id,
|
|
table_schemas))) {
|
|
LOG_WARN("fail to get table ids in database", K(tenant_id), K(database_id), K(ret));
|
|
} else {
|
|
const ObSimpleTableSchemaV2 *table_schema = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
table_schema = table_schemas.at(i);
|
|
if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (table_schema->is_materialized_view() && OB_FAIL(lock_mview(trans, *table_schema))) {
|
|
LOG_WARN("fail to lock mview", KR(ret), KPC(table_schema));
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
table_schema = table_schemas.at(i);
|
|
if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (!table_schema->check_can_do_ddl()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("offline ddl is being executed, other ddl operations are not allowed",
|
|
K(table_schema->get_table_id()), K(ret));
|
|
} else if (OB_FAIL(lock_table(trans, *table_schema))) {
|
|
LOG_WARN("fail to lock_table", KR(ret), KPC(table_schema));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_aux_index_schema_exist(
|
|
const uint64_t tenant_id,
|
|
const obrpc::ObCreateIndexArg &arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema *data_schema,
|
|
bool &is_exist,
|
|
const ObTableSchema *&index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_exist = false;
|
|
index_schema = nullptr;
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
ObIndexType index_type = arg.index_type_;
|
|
ObString index_table_name;
|
|
if (tenant_id == OB_INVALID_ID ||
|
|
!arg.is_valid() ||
|
|
OB_ISNULL(data_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(arg), KPC(data_schema));
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator,
|
|
data_schema->get_table_id(),
|
|
arg.index_name_,
|
|
index_table_name))) {
|
|
LOG_WARN("failed to construct index table name", K(ret),
|
|
K(arg.index_name_));
|
|
} else if (share::schema::is_fts_index(index_type)) {
|
|
/* is_fts_index means: rowkey-doc, doc-rowkey, fts, word-doc multivalue-index, fts-index all run here*/
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
data_schema->get_database_id(),
|
|
index_table_name,
|
|
true/*is_index*/,
|
|
index_schema,
|
|
false/*with_hidden_flag*/,
|
|
share::schema::is_built_in_fts_index(index_type)))) {
|
|
if (OB_TABLE_NOT_EXIST == ret) {
|
|
is_exist = false;
|
|
index_schema = nullptr;
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("failed to get index schema", K(ret), K(tenant_id), K(index_table_name));
|
|
}
|
|
} else if (OB_NOT_NULL(index_schema)) {
|
|
is_exist = true;
|
|
LOG_INFO("fts index aux table already exist, no need to generate",
|
|
K(index_table_name));
|
|
}
|
|
} else if (share::schema::is_vec_index(index_type)) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
data_schema->get_database_id(),
|
|
index_table_name,
|
|
true/*is_index*/,
|
|
index_schema,
|
|
false/*with_hidden_flag*/,
|
|
share::schema::is_built_in_vec_index(index_type)))) {
|
|
if (OB_TABLE_NOT_EXIST == ret) {
|
|
is_exist = false;
|
|
index_schema = nullptr;
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("failed to get index schema", K(ret), K(tenant_id), K(index_table_name));
|
|
}
|
|
} else if (OB_NOT_NULL(index_schema)) {
|
|
is_exist = true;
|
|
LOG_INFO("vec index aux table already exist, no need to generate",
|
|
K(index_table_name));
|
|
}
|
|
} else if (share::schema::is_multivalue_index_aux(index_type)) {
|
|
/* only multivalue index 3rd table run here */
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
data_schema->get_database_id(),
|
|
index_table_name,
|
|
true/*is_index*/,
|
|
index_schema,
|
|
false/*with_hidden_flag*/,
|
|
share::schema::is_built_in_multivalue_index(index_type)))) {
|
|
if (OB_TABLE_NOT_EXIST == ret) {
|
|
is_exist = false;
|
|
index_schema = nullptr;
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("failed to get index schema", K(ret), K(tenant_id), K(index_table_name));
|
|
}
|
|
} else if (OB_NOT_NULL(index_schema)) {
|
|
is_exist = true;
|
|
LOG_INFO("multivalue index aux table already exist, no need to generate",
|
|
K(index_table_name));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected index type", K(ret), K(index_type));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_aux_index_schema_(
|
|
const uint64_t tenant_id,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObCreateIndexArg &create_index_arg,
|
|
ObTableSchema &nonconst_data_schema,
|
|
const ObTableSchema *data_schema,
|
|
ObIArray<ObColumnSchemaV2*> &gen_columns,
|
|
ObDDLSQLTransaction &trans,
|
|
const uint64_t tenant_data_version,
|
|
ObTableSchema &index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (!create_index_arg.is_valid() || OB_INVALID_TENANT_ID == tenant_id ||
|
|
OB_ISNULL(data_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(create_index_arg), KPC(data_schema));
|
|
} else {
|
|
ObIndexBuilder index_builder(*this);
|
|
const bool global_index_without_column_info = true;
|
|
if (create_index_arg.is_rebuild_index_) {
|
|
if (OB_FAIL(ObVectorIndexUtil::generate_index_schema_from_exist_table(tenant_id,
|
|
schema_guard,
|
|
*this,
|
|
create_index_arg,
|
|
*data_schema,
|
|
index_schema))) {
|
|
LOG_WARN("fail to generate index schema from exist table", K(ret), K(tenant_id), K(create_index_arg));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!create_index_arg.is_rebuild_index_ &&
|
|
OB_FAIL(index_builder.generate_schema(create_index_arg,
|
|
nonconst_data_schema,
|
|
global_index_without_column_info,
|
|
true/*generate_id*/,
|
|
index_schema))) {
|
|
LOG_WARN("fail to generate schema", K(ret), K(create_index_arg));
|
|
} else if (OB_FAIL(nonconst_data_schema.check_create_index_on_hidden_primary_key(index_schema))) {
|
|
LOG_WARN("failed to check create index on table", K(ret), K(index_schema));
|
|
} else if (gen_columns.empty()) {
|
|
if (OB_FAIL(create_index_table(create_index_arg,
|
|
tenant_data_version,
|
|
index_schema,
|
|
trans))) {
|
|
LOG_WARN("fail to create index", K(ret), K(index_schema));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(create_inner_expr_index(trans,
|
|
*data_schema,
|
|
tenant_data_version,
|
|
nonconst_data_schema,
|
|
gen_columns,
|
|
index_schema))) {
|
|
LOG_WARN("fail to create inner expr index", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_aux_index_task_(
|
|
const ObTableSchema *data_schema,
|
|
const ObTableSchema *idx_schema,
|
|
ObCreateIndexArg &create_index_arg,
|
|
ObArenaAllocator &allocator,
|
|
const int64_t parent_task_id,
|
|
const uint64_t tenant_data_version,
|
|
ObDDLSQLTransaction &trans,
|
|
ObDDLTaskRecord &task_record)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_ISNULL(data_schema) ||
|
|
OB_ISNULL(idx_schema) ||
|
|
OB_ISNULL(GCTX.root_service_) ||
|
|
OB_ISNULL(GCTX.sql_proxy_) ||
|
|
!create_index_arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KPC(data_schema), KPC(idx_schema),
|
|
KP(GCTX.root_service_), KP(GCTX.sql_proxy_), K(create_index_arg));
|
|
} else {
|
|
bool need_partitioned = ((DATA_VERSION_4_2_2_0 <= tenant_data_version &&
|
|
tenant_data_version < DATA_VERSION_4_3_0_0) ||
|
|
tenant_data_version >= DATA_VERSION_4_3_2_0) &&
|
|
idx_schema->is_storage_local_index_table() &&
|
|
idx_schema->is_partitioned_table();
|
|
ObCreateDDLTaskParam param(data_schema->get_tenant_id(),
|
|
need_partitioned ? ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX : ObDDLType::DDL_CREATE_INDEX,
|
|
data_schema,
|
|
idx_schema,
|
|
0/*object_id*/,
|
|
idx_schema->get_schema_version(),
|
|
create_index_arg.parallelism_,
|
|
create_index_arg.consumer_group_id_,
|
|
&allocator,
|
|
&create_index_arg,
|
|
parent_task_id);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (OB_FAIL(GCTX.root_service_->get_ddl_task_scheduler().
|
|
create_ddl_task(param, trans, task_record))) {
|
|
if (OB_ENTRY_EXIST == ret) {
|
|
trans.reset_last_error();
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("submit create index ddl task failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_aux_index(
|
|
const obrpc::ObCreateAuxIndexArg &arg,
|
|
obrpc::ObCreateAuxIndexRes &result)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const uint64_t data_table_id = arg.data_table_id_;
|
|
int64_t index_status = ObIndexStatus::INDEX_STATUS_NOT_FOUND;
|
|
ObArenaAllocator allocator(lib::ObLabel("DdlTaskTmp"));
|
|
SMART_VARS_2((obrpc::ObCreateIndexArg, create_index_arg),
|
|
(ObTableSchema, nonconst_data_schema)) {
|
|
ObDDLSQLTransaction trans(&get_schema_service());
|
|
int64_t refreshed_schema_version = 0;
|
|
uint64_t tenant_data_version = 0;
|
|
bool schema_already_exist = false;
|
|
uint64_t index_table_id = OB_INVALID_ID;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *data_schema = nullptr;
|
|
const ObTableSchema *idx_schema = nullptr;
|
|
ObSEArray<ObColumnSchemaV2 *, 1> gen_columns;
|
|
ObArenaAllocator allocator(lib::ObLabel("DdlTaskTmp"));
|
|
ObDDLTaskRecord task_record;
|
|
if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(arg));
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is nullptr", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id,
|
|
schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id,
|
|
refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(&get_sql_proxy(),
|
|
tenant_id,
|
|
refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id),
|
|
K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
data_table_id,
|
|
data_schema))) {
|
|
LOG_WARN("get table schema failed", K(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(data_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, table schema is nullptr", K(ret), K(data_table_id));
|
|
} else if (OB_FAIL(nonconst_data_schema.assign(*data_schema))) {
|
|
LOG_WARN("failed to assign to nonconst data schema", K(ret));
|
|
} else if (OB_FAIL(create_index_arg.assign(arg.create_index_arg_))) {
|
|
LOG_WARN("fail to assign create index arg", K(ret));
|
|
/* is_fts_index means: rowkey-doc, doc-rowkey, fts, word-doc multivalue-index, fts-index all run here*/
|
|
} else if (share::schema::is_fts_index(create_index_arg.index_type_)
|
|
&& OB_FAIL(ObFtsIndexBuilderUtil::adjust_fts_args(create_index_arg,
|
|
nonconst_data_schema,
|
|
allocator,
|
|
gen_columns))) {
|
|
LOG_WARN("fail to adjust create index args", K(ret), K(create_index_arg));
|
|
} else if (share::schema::is_multivalue_index_aux(create_index_arg.index_type_) /* only multivalue index 3rd table run here */
|
|
&& OB_FAIL(ObMulValueIndexBuilderUtil::adjust_mulvalue_index_args(create_index_arg,
|
|
nonconst_data_schema,
|
|
allocator,
|
|
gen_columns))) {
|
|
LOG_WARN("fail to adjust create index args", K(ret), K(create_index_arg));
|
|
} else if (!create_index_arg.is_rebuild_index_
|
|
&& share::schema::is_vec_index(create_index_arg.index_type_)
|
|
&& OB_FAIL(ObVecIndexBuilderUtil::adjust_vec_args(create_index_arg,
|
|
nonconst_data_schema,
|
|
allocator,
|
|
gen_columns))) {
|
|
LOG_WARN("fail to adjust expr index args", K(ret));
|
|
} else if (!(share::schema::is_fts_index(create_index_arg.index_type_) ||
|
|
share::schema::is_multivalue_index_aux(create_index_arg.index_type_) ||
|
|
share::schema::is_vec_index(create_index_arg.index_type_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to create aux index, index type invalid", K(ret), K(create_index_arg));
|
|
} else if (OB_FAIL(check_aux_index_schema_exist(tenant_id,
|
|
arg.create_index_arg_,
|
|
schema_guard,
|
|
data_schema,
|
|
schema_already_exist,
|
|
idx_schema))) {
|
|
LOG_WARN("failed to check if schema is generated for aux index table", K(ret), K(arg));
|
|
} else if (schema_already_exist) {
|
|
if (OB_ISNULL(idx_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("idx_schema is nullptr", K(ret));
|
|
} else if (FALSE_IT(index_table_id = idx_schema->get_table_id())) {
|
|
} else if (FALSE_IT(index_status = idx_schema->get_index_status())) {
|
|
} else if (ObIndexStatus::INDEX_STATUS_AVAILABLE == index_status) {
|
|
// 1. index schema exists && index available, fetch index table id
|
|
result.schema_generated_ = true;
|
|
result.aux_table_id_ = index_table_id;
|
|
result.ddl_task_id_ = OB_INVALID_ID; // no need to wait task
|
|
} else { // 2. index schema exists && not available, create ddl task
|
|
result.schema_generated_ = true;
|
|
result.aux_table_id_ = index_table_id;
|
|
if (OB_FAIL(create_aux_index_task_(data_schema,
|
|
idx_schema,
|
|
create_index_arg,
|
|
allocator,
|
|
arg.task_id_/*parent fts*/,
|
|
tenant_data_version,
|
|
trans,
|
|
task_record))) {
|
|
LOG_WARN("failed to create aux index ddl task", K(ret), K(create_index_arg));
|
|
} else if (FALSE_IT(result.ddl_task_id_ = task_record.task_id_)) {
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ObDDLTaskRecordOperator::update_parent_task_message(tenant_id,
|
|
arg.task_id_, *idx_schema, result.aux_table_id_, result.ddl_task_id_, ObDDLUpdateParentTaskIDType::UPDATE_CREATE_INDEX_ID, allocator, trans))) {
|
|
LOG_WARN("fail to update parent task message", K(ret), K(arg.task_id_), K(idx_schema));
|
|
}
|
|
} else { // 3. index scheme not exist, generate schema && create ddl task
|
|
ObTableSchema index_schema;
|
|
if (OB_FAIL(generate_aux_index_schema_(tenant_id,
|
|
schema_guard,
|
|
create_index_arg,
|
|
nonconst_data_schema,
|
|
data_schema,
|
|
gen_columns,
|
|
trans,
|
|
tenant_data_version,
|
|
index_schema))) {
|
|
LOG_WARN("failed to generate aux index schema", K(ret), K(create_index_arg));
|
|
} else if (FALSE_IT(result.schema_generated_ = true)) {
|
|
} else if (FALSE_IT(result.aux_table_id_ = index_schema.get_table_id())) {
|
|
} else if (OB_FAIL(create_aux_index_task_(data_schema,
|
|
&index_schema,
|
|
create_index_arg,
|
|
allocator,
|
|
arg.task_id_/*parent fts*/,
|
|
tenant_data_version,
|
|
trans,
|
|
task_record))) {
|
|
LOG_WARN("failed to create aux index ddl task", K(ret), K(create_index_arg));
|
|
} else if (FALSE_IT(result.ddl_task_id_ = task_record.task_id_)) {
|
|
} else if (OB_FAIL(ObDDLTaskRecordOperator::update_parent_task_message(tenant_id,
|
|
arg.task_id_, index_schema, result.aux_table_id_, result.ddl_task_id_, ObDDLUpdateParentTaskIDType::UPDATE_CREATE_INDEX_ID, allocator, trans))) {
|
|
LOG_WARN("fail to update parent task message", K(ret), K(arg.task_id_), K(index_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_SUCCESS == 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));
|
|
} else if (OB_INVALID_ID == result.ddl_task_id_) { // no need to schedule
|
|
} else {
|
|
DEBUG_SYNC(CREATE_AUX_INDEX_TABLE);
|
|
if (OB_FAIL(GCTX.root_service_->get_ddl_task_scheduler().
|
|
schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(ret), K(task_record));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("finish create aux index", K(ret), K(arg), K(result), "ddl_event_info", ObDDLEventInfo());
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_tables_in_recyclebin(const ObDatabaseSchema &database_schema,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObArray<ObRecycleObject> recycle_objs;
|
|
ObSchemaService *schema_service = nullptr;
|
|
const uint64_t tenant_id = database_schema.get_tenant_id();
|
|
const uint64_t database_id = database_schema.get_database_id();
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is null", K(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", K(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_recycle_objects_of_db(tenant_id,
|
|
database_id,
|
|
trans,
|
|
recycle_objs))) {
|
|
LOG_WARN("fetch recycle objects of db failed", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", KR(ret), K(tenant_id));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < recycle_objs.count(); ++i) {
|
|
const ObRecycleObject &recycle_obj = recycle_objs.at(i);
|
|
const ObSimpleTableSchemaV2* table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_simple_table_schema(recycle_obj.get_tenant_id(),
|
|
recycle_obj.get_table_id(), table_schema))) {
|
|
LOG_WARN("get table schema failed", K(ret), K(recycle_obj));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table is not exist", K(ret), K(recycle_obj));
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(database_schema.get_database_name_str()),
|
|
to_cstring(recycle_obj.get_object_name()));
|
|
} else if (table_schema->is_materialized_view() && OB_FAIL(lock_mview(trans, *table_schema))) {
|
|
LOG_WARN("fail to lock mview", KR(ret), KPC(table_schema));
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < recycle_objs.count(); ++i) {
|
|
const ObRecycleObject &recycle_obj = recycle_objs.at(i);
|
|
const ObSimpleTableSchemaV2* table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_simple_table_schema(recycle_obj.get_tenant_id(),
|
|
recycle_obj.get_table_id(), table_schema))) {
|
|
LOG_WARN("get table schema failed", K(ret), K(recycle_obj));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table is not exist", K(ret), K(recycle_obj));
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(database_schema.get_database_name_str()),
|
|
to_cstring(recycle_obj.get_object_name()));
|
|
} else if (OB_FAIL(lock_table(trans, *table_schema))) {
|
|
LOG_WARN("fail to lock_table", KR(ret), KPC(table_schema));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_index_tablet(const ObTableSchema &index_schema,
|
|
ObMySQLTransaction &trans,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const bool need_check_tablet_cnt,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t tenant_id = index_schema.get_tenant_id();
|
|
SCN frozen_scn;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat error", K(is_inited()), KR(ret));
|
|
} else if (!index_schema.is_index_table() || tenant_data_version <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("arg must be index table", KR(ret), K(tenant_id), K(tenant_data_version), K(index_schema));
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else {
|
|
int64_t start_usec = ObTimeUtility::current_time();
|
|
ObTableCreator table_creator(
|
|
tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(
|
|
tenant_id,
|
|
schema_guard,
|
|
sql_proxy_);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
const uint64_t data_table_id = index_schema.get_data_table_id();
|
|
const ObTableSchema *data_table_schema = NULL;
|
|
const ObTablegroupSchema *data_tablegroup_schema = NULL; // keep NULL if no tablegroup
|
|
if (OB_FAIL(table_creator.init(need_check_tablet_cnt))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, data_table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("data table schema not exists", KR(ret), K(data_table_id));
|
|
} else if (OB_INVALID_ID != data_table_schema->get_tablegroup_id()) {
|
|
if (OB_FAIL(schema_guard.get_tablegroup_schema(
|
|
data_table_schema->get_tenant_id(),
|
|
data_table_schema->get_tablegroup_id(),
|
|
data_tablegroup_schema))) {
|
|
LOG_WARN("get tablegroup_schema failed", KR(ret), KPC(data_table_schema));
|
|
} else if (OB_ISNULL(data_tablegroup_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("data_tablegroup_schema is null", KR(ret), KPC(data_table_schema));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (index_schema.is_index_local_storage()) {
|
|
ObSEArray<const share::schema::ObTableSchema*, 1> schemas;
|
|
ObSEArray<bool, 1> need_create_empty_majors;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, data_table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("data table schema not exists", KR(ret), K(data_table_id));
|
|
} else if (OB_FAIL(schemas.push_back(&index_schema))
|
|
|| OB_FAIL(need_create_empty_majors.push_back(false))) {
|
|
LOG_WARN("failed to push_back", KR(ret), K(index_schema));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.prepare(trans, index_schema, data_tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare ls for index schema tablets", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(
|
|
ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_local_aux_tables_arg(
|
|
schemas,
|
|
data_table_schema,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors /* add_index, need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table tablet failed", KR(ret), K(index_schema));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(new_table_tablet_allocator.prepare(trans, index_schema, data_tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare ls for index schema tablets");
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(
|
|
ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_table_arg(
|
|
index_schema,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
false /*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table tablet failed", KR(ret), K(index_schema));
|
|
}
|
|
}
|
|
|
|
// execute create tablet
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("execute create partition failed", KR(ret));
|
|
}
|
|
|
|
// finishing is always invoked for new table tablet allocator
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) {
|
|
LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret));
|
|
}
|
|
|
|
int64_t cost_usec = ObTimeUtility::current_time() - start_usec;
|
|
LOG_INFO("create table partitions cost: ", K(cost_usec));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_index_on_foreign_key(const ObTableSchema *index_table_schema,
|
|
const common::ObIArray<ObForeignKeyInfo> &foreign_key_infos,
|
|
bool &have_index)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
have_index = false;
|
|
if (foreign_key_infos.count() <= 0) {
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("index table schema is nullptr", K(ret));
|
|
} else {
|
|
const uint64_t index_table_id = index_table_schema->get_table_id();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
if (foreign_key_infos.at(i).ref_cst_id_ == index_table_id) {
|
|
have_index = true;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_table_index(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObArenaAllocator &allocator,
|
|
const uint64_t tenant_data_version,
|
|
obrpc::ObAlterTableRes &res,
|
|
ObIArray<ObDDLTaskRecord> &ddl_tasks)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObIndexBuilder index_builder(*this);
|
|
ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
common::ObArray<const ObForeignKeyInfo *> drop_parent_table_mock_foreign_key_infos_array;
|
|
ObIArray<obrpc::ObDDLRes> &ddl_res_array = res.ddl_res_array_;
|
|
// To many hashset will fill up the stack, construct them on heap instead
|
|
HEAP_VAR(AddIndexNameHashSet, add_index_name_set) {
|
|
HEAP_VAR(DropIndexNameHashSet, drop_index_name_set) {
|
|
HEAP_VAR(RenameIndexNameHashSet, rename_ori_index_name_set) {
|
|
HEAP_VAR(RenameIndexNameHashSet, rename_new_index_name_set) {
|
|
HEAP_VAR(AlterIndexNameHashSet, alter_index_name_set) {
|
|
int64_t index_count = new_table_schema.get_index_tid_count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) {
|
|
ObIndexArg *index_arg = 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);
|
|
uint64_t tenant_data_version = 0;
|
|
const bool is_check_fts_index_conflict = !create_index_arg->is_inner_ && share::schema::is_fts_or_multivalue_index(create_index_arg->index_type_);
|
|
if (is_check_fts_index_conflict && OB_FAIL(check_fts_index_conflict(origin_table_schema.get_tenant_id(), origin_table_schema.get_table_id()))) {
|
|
if (OB_EAGAIN != ret) {
|
|
LOG_WARN("failed to check fts index ", K(ret));
|
|
}
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(create_index_arg->tenant_id_, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), KPC(create_index_arg));
|
|
} else 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));
|
|
}
|
|
if (!new_table_schema.is_partitioned_table()
|
|
&& !new_table_schema.is_auto_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;
|
|
} else if (INDEX_TYPE_SPATIAL_GLOBAL == create_index_arg->index_type_) {
|
|
create_index_arg->index_type_ = INDEX_TYPE_SPATIAL_GLOBAL_LOCAL_STORAGE;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (create_index_arg->index_type_ == INDEX_TYPE_PRIMARY) {
|
|
if (new_table_schema.get_rowkey_column_num() > 0) {
|
|
if (new_table_schema.is_heap_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to add primary key!", K(ret));
|
|
} else {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret));
|
|
}
|
|
}
|
|
continue;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && new_table_schema.mv_major_refresh()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to add index on mv", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "add index on major refresh materialized view is");
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (create_index_arg->index_name_.empty()) {
|
|
if (OB_FAIL(generate_index_name(*create_index_arg,
|
|
new_table_schema,
|
|
add_index_name_set,
|
|
drop_index_name_set,
|
|
schema_guard,
|
|
allocator))) {
|
|
LOG_WARN("generate index name failed", K(ret));
|
|
}
|
|
} else {
|
|
ObString index_name = create_index_arg->index_name_;
|
|
ObString index_table_name;
|
|
bool is_exist = false;
|
|
if (OB_FAIL(check_index_table_exist(origin_table_schema.get_tenant_id(),
|
|
origin_table_schema.get_database_id(),
|
|
origin_table_schema.get_table_id(),
|
|
index_name,
|
|
schema_guard,
|
|
is_exist))) {
|
|
LOG_WARN("failed to check index table", K(ret));
|
|
} else {
|
|
ObIndexNameHashWrapper index_key(index_name);
|
|
if (!is_exist) {
|
|
if (OB_HASH_EXIST == add_index_name_set.exist_refactored(index_key)) {
|
|
//alter table t1 add index c1, add index c1
|
|
is_exist = true;
|
|
} else if (OB_HASH_EXIST == rename_new_index_name_set.exist_refactored(index_key)) {
|
|
//alter table t1 rename index ori_idx to new_idx, add index new_idx // MySQL supports this behavior
|
|
is_exist = true;
|
|
}
|
|
} else {
|
|
if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) {
|
|
//alter table t1 drop index c1, add index c1
|
|
is_exist = false;
|
|
} else if (OB_HASH_EXIST == rename_ori_index_name_set.exist_refactored(index_key)) {
|
|
//alter table t1 rename index ori_idx to new_idx, add index ori_idx // MySQL supports this behavior
|
|
is_exist = false;
|
|
}
|
|
}
|
|
if (is_exist) {
|
|
ret = OB_ERR_KEY_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_KEY_NAME_DUPLICATE, index_name.length(), index_name.ptr());
|
|
LOG_WARN("duplicate index name", K(index_name), K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
HEAP_VARS_3((ObTableSchema, index_schema),
|
|
(obrpc::ObCreateIndexArg, my_arg),
|
|
(obrpc::ObCreateIndexArg, tmp_arg)){
|
|
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_
|
|
|| INDEX_TYPE_SPATIAL_GLOBAL == create_index_arg->index_type_;
|
|
if (global_storage) {
|
|
//FIXME: Cannot build index on tablegroup at present,
|
|
// For a global index, its locality and primary take the value of the primary table
|
|
// The current implementation has the following two problems:
|
|
// 1. If the locality/primary_zone of the primary table is inherited semantics,
|
|
// and the primary table is in the tablegroup, then the locality/primary_zone of
|
|
// the inheritance relationship between the global index table and the primary table
|
|
// is different, and the inherited values may be different;
|
|
// 2. In addition, for the case where the primary_zone is random,
|
|
// currently the leader_coordinator cannot guarantee that the results of
|
|
// the random breakup of the primary table and the global index table primary_zone are consistent.
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(index_schema.assign(create_index_arg->index_schema_))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (FALSE_IT(index_schema.set_tenant_id(origin_table_schema.get_tenant_id()))) {
|
|
}
|
|
}
|
|
bool global_index_without_column_info = (create_index_arg->index_schema_.is_partitioned_table() ||
|
|
create_index_arg->index_schema_.is_auto_partitioned_table()) ?
|
|
false : true;
|
|
bool rowkey_doc_exist = false;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(my_arg.assign(*create_index_arg))) {
|
|
LOG_WARN("fail to assign arg", K(ret));
|
|
} else if (share::schema::is_fts_index(my_arg.index_type_)) {
|
|
const ObTableSchema *rowkey_doc_schema = nullptr;
|
|
if (OB_FAIL(tmp_arg.assign(my_arg))) {
|
|
LOG_WARN("fail to assign arg", K(ret));
|
|
} else if (!tmp_arg.is_valid()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to copy create index arg", K(ret));
|
|
} else if (FALSE_IT(tmp_arg.index_type_ = INDEX_TYPE_ROWKEY_DOC_ID_LOCAL)) {
|
|
} else if (OB_FAIL(ObFtsIndexBuilderUtil::generate_fts_aux_index_name(tmp_arg, &allocator))) {
|
|
LOG_WARN("failed to adjust fts index name", K(ret));
|
|
} else if (OB_FAIL(check_aux_index_schema_exist(origin_table_schema.get_tenant_id(),
|
|
tmp_arg,
|
|
schema_guard,
|
|
&new_table_schema,
|
|
rowkey_doc_exist,
|
|
rowkey_doc_schema))) {
|
|
LOG_WARN("fail to check rowkey doc schema existence", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (share::schema::is_fts_index(my_arg.index_type_) &&
|
|
!rowkey_doc_exist &&
|
|
FALSE_IT(my_arg.index_type_ = INDEX_TYPE_ROWKEY_DOC_ID_LOCAL)) {
|
|
// 1. generate rowkey doc schema if not exist
|
|
// 2. otherwise generate fts index aux schema
|
|
} else if (share::schema::is_fts_index(my_arg.index_type_) &&
|
|
OB_FAIL(ObFtsIndexBuilderUtil::generate_fts_aux_index_name(my_arg, &allocator))) {
|
|
LOG_WARN("failed to adjust fts index name", K(ret));
|
|
} else if (OB_FAIL(ObIndexBuilderUtil::adjust_expr_index_args(
|
|
my_arg, new_table_schema, allocator, gen_columns))) {
|
|
LOG_WARN("adjust fulltext args failed", K(ret));
|
|
} else if (OB_FAIL(index_builder.generate_schema(my_arg,
|
|
new_table_schema,
|
|
global_index_without_column_info,
|
|
true, /*generate_id*/
|
|
index_schema))) {
|
|
LOG_WARN("failed to generate index schema!", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_table_create_index(new_table_schema,
|
|
gen_columns,
|
|
index_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to alter table add index!", K(index_schema), K(ret));
|
|
} else {
|
|
// The index data is stored separately from the main table,
|
|
// the partition needs to be built, and insert ori_schema_version in the outer insert
|
|
if (index_schema.has_tablet()
|
|
&& OB_FAIL(create_index_tablet(index_schema, trans, schema_guard, true/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("fail to create_index_tablet", KR(ret), K(index_schema));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObIndexNameHashWrapper index_key(create_index_arg->index_name_);
|
|
if (OB_FAIL(create_index_arg->index_schema_.assign(index_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(add_index_name_set.set_refactored(index_key))) {
|
|
LOG_WARN("set index name to hash set failed",
|
|
K(create_index_arg->index_name_), K(ret));
|
|
} else {
|
|
++index_count;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else if (ObIndexArg::DROP_INDEX == index_arg->index_action_type_) {
|
|
ObDropIndexArg *drop_index_arg = static_cast<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)) {
|
|
const ObTableSchema *index_table_schema = nullptr;
|
|
bool have_index = false;
|
|
const common::ObIArray<ObForeignKeyInfo> &foreign_key_infos = origin_table_schema.get_foreign_key_infos();
|
|
if (OB_FAIL(get_index_schema_by_name(
|
|
origin_table_schema.get_table_id(),
|
|
origin_table_schema.get_database_id(),
|
|
*drop_index_arg,
|
|
schema_guard,
|
|
index_table_schema))) {
|
|
LOG_WARN("get index schema by name failed", K(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_CANT_DROP_FIELD_OR_KEY;
|
|
LOG_WARN("index table schema should not be null", K(*drop_index_arg), K(ret));
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, drop_index_arg->index_name_.length(), drop_index_arg->index_name_.ptr());
|
|
} else if (OB_FAIL(check_index_on_foreign_key(index_table_schema,
|
|
foreign_key_infos,
|
|
have_index))) {
|
|
LOG_WARN("fail to check index on foreign key", K(ret), K(foreign_key_infos), KPC(index_table_schema));
|
|
} else if (have_index) {
|
|
ret = OB_ERR_ATLER_TABLE_ILLEGAL_FK;
|
|
LOG_WARN("cannot delete index with foreign key dependency", K(ret));
|
|
} else if (!drop_index_arg->is_inner_ && index_table_schema->is_unavailable_index()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to drop a building index", K(ret), K(drop_index_arg->is_inner_), KPC(index_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "dropping a building index is");
|
|
} else if (drop_index_arg->is_add_to_scheduler_) {
|
|
if (OB_FAIL(drop_index_to_scheduler_(trans, schema_guard, alter_table_arg.allocator_, origin_table_schema,
|
|
nullptr /*inc_tablet_ids*/, nullptr /*del_tablet_ids*/, drop_index_arg,
|
|
ddl_operator, res, ddl_tasks))) {
|
|
LOG_WARN("fail to drop index to scheduler", KR(ret), K(drop_index_arg));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ddl_operator.alter_table_drop_index(
|
|
index_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to alter table drop index", K(*drop_index_arg), K(ret));
|
|
} else {
|
|
--index_count;
|
|
}
|
|
}
|
|
}
|
|
} else if (index_arg->index_action_type_ == ObIndexArg::ALTER_INDEX) {
|
|
ObAlterIndexArg *alter_index_arg = static_cast<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_tenant_id(),
|
|
origin_table_schema.get_table_id(),
|
|
origin_table_schema.get_database_id(),
|
|
*alter_index_arg,
|
|
trans))) {
|
|
LOG_WARN("failed to alter table alter index!", K(alter_index_arg), K(ret));
|
|
} else if (OB_HASH_EXIST == alter_index_name_set.exist_refactored(index_key)) {
|
|
ret = OB_ERR_ALTER_TABLE_ALTER_DUPLICATED_INDEX;
|
|
LOG_USER_ERROR(OB_ERR_ALTER_TABLE_ALTER_DUPLICATED_INDEX, index_name.length(), index_name.ptr());
|
|
LOG_WARN("Duplicate alter index operations", K(ret), K(index_name));
|
|
} else if (OB_FAIL(alter_index_name_set.set_refactored(index_key))) {
|
|
LOG_WARN("set index name to hash set failed", K(ret), K(index_name));
|
|
}
|
|
}
|
|
} else if (ObIndexArg::DROP_FOREIGN_KEY == index_arg->index_action_type_) {
|
|
ObDropForeignKeyArg *drop_foreign_key_arg = static_cast<ObDropForeignKeyArg *>(index_arg);
|
|
const ObForeignKeyInfo *parent_table_mock_foreign_key_info = NULL;
|
|
if (OB_FAIL(ddl_operator.alter_table_drop_foreign_key(origin_table_schema, *drop_foreign_key_arg, trans, parent_table_mock_foreign_key_info,
|
|
origin_table_schema.get_in_offline_ddl_white_list()))) {
|
|
LOG_WARN("failed to alter table drop foreign key", K(ret), K(drop_foreign_key_arg));
|
|
} else if (NULL != parent_table_mock_foreign_key_info) {
|
|
if (OB_FAIL(drop_parent_table_mock_foreign_key_infos_array.push_back(parent_table_mock_foreign_key_info))) {
|
|
LOG_WARN("failed to push back to drop_foreign_key_infos_array", K(ret), KPC(parent_table_mock_foreign_key_info));
|
|
}
|
|
}
|
|
} else if (ObIndexArg::ALTER_INDEX_PARALLEL == index_arg->index_action_type_) {
|
|
ObAlterIndexParallelArg *alter_index_parallel_arg =
|
|
static_cast<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_KEY_DOES_NOT_EXISTS;
|
|
const ObString &index_name = alter_index_parallel_arg->index_name_;
|
|
const ObString &data_table_name = origin_table_schema.get_table_name_str();
|
|
LOG_WARN("the index is not exist", K(ret), K(alter_index_parallel_arg));
|
|
LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, index_name.length(), index_name.ptr(),
|
|
data_table_name.length(), data_table_name.ptr());
|
|
} else if (OB_FAIL(ddl_operator.alter_index_table_parallel(
|
|
origin_table_schema.get_tenant_id(),
|
|
origin_table_schema.get_table_id(),
|
|
origin_table_schema.get_database_id(),
|
|
*alter_index_parallel_arg,
|
|
trans))) {
|
|
LOG_WARN("failed to alter index table parallel",
|
|
K(ret), K(origin_table_schema.get_table_name()));
|
|
}
|
|
} else if (ObIndexArg::RENAME_INDEX == index_arg->index_action_type_) {
|
|
ObRenameIndexArg *rename_index_arg = static_cast<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 {
|
|
SMART_VAR(ObTableSchema, new_index_schema) {
|
|
if (OB_FAIL(ddl_operator.alter_table_rename_index(
|
|
origin_table_schema.get_tenant_id(),
|
|
origin_table_schema.get_table_id(),
|
|
origin_table_schema.get_database_id(),
|
|
*rename_index_arg,
|
|
nullptr /* new_index_status */,
|
|
trans,
|
|
new_index_schema))) {
|
|
LOG_WARN("failed to rename index", K(*rename_index_arg), K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(rename_ori_index_name_set.set_refactored(ori_index_key))) {
|
|
const ObString &data_table_name = origin_table_schema.get_table_name_str();
|
|
//To be compatible with Mysql
|
|
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());
|
|
LOG_WARN("set index name to hash set failed", K(ret), K(ori_index_name));
|
|
} else if (OB_FAIL(rename_new_index_name_set.set_refactored(new_index_key))) {
|
|
LOG_WARN("set index name to hash set failed", K(ret), K(new_index_name));
|
|
}
|
|
} else if (ObIndexArg::ALTER_INDEX_TABLESPACE == index_arg->index_action_type_) {
|
|
ObAlterIndexTablespaceArg *alter_index_tablespace_arg =
|
|
static_cast<ObAlterIndexTablespaceArg *>(index_arg);
|
|
bool is_exist = false;
|
|
if (OB_FAIL(check_index_table_exist(origin_table_schema.get_tenant_id(),
|
|
origin_table_schema.get_database_id(),
|
|
origin_table_schema.get_table_id(),
|
|
alter_index_tablespace_arg->index_name_,
|
|
schema_guard,
|
|
is_exist))) {
|
|
LOG_WARN("failed to check index table", K(ret));
|
|
} else if (!is_exist) {
|
|
ret = OB_ERR_KEY_DOES_NOT_EXISTS;
|
|
const ObString &index_name = alter_index_tablespace_arg->index_name_;
|
|
const ObString &data_table_name = origin_table_schema.get_table_name_str();
|
|
LOG_WARN("the index is not exist", K(index_name), K(ret));
|
|
LOG_USER_ERROR(OB_ERR_KEY_DOES_NOT_EXISTS, index_name.length(), index_name.ptr(),
|
|
data_table_name.length(), data_table_name.ptr());
|
|
} else if (OB_FAIL(ddl_operator.alter_index_table_tablespace(origin_table_schema.get_table_id(),
|
|
origin_table_schema.get_database_id(),
|
|
*alter_index_tablespace_arg,
|
|
schema_guard,
|
|
trans))) {
|
|
LOG_WARN("failed to alter index table tablespace",
|
|
K(ret), K(origin_table_schema.get_table_name()));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Unknown index action type!", K_(index_arg->index_action_type), K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !drop_parent_table_mock_foreign_key_infos_array.empty()) {
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
ObArray<const ObMockFKParentTableSchema*> mock_fk_parent_table_schema_ptr_array;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", K(ret));
|
|
} else if (OB_FAIL(prepare_gen_mock_fk_parent_tables_for_drop_fks(
|
|
schema_guard, origin_table_schema.get_tenant_id(),
|
|
drop_parent_table_mock_foreign_key_infos_array,
|
|
mock_fk_parent_table_schema_ptr_array,
|
|
mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("failed to prepare_gen_mock_fk_parent_tables_for_drop_fks", K(ret));
|
|
} else if (OB_FAIL(gen_mock_fk_parent_tables_for_drop_fks(
|
|
schema_guard, mock_fk_parent_table_schema_ptr_array, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("failed to gen_mock_fk_parent_tables_for_drop_fks", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(mock_fk_parent_table_schema_array));
|
|
}
|
|
}
|
|
} // end of alter_index_name_set
|
|
} // end of rename_new_index_name_set
|
|
} // end of rename_ori_index_name_set
|
|
} // end of drop_index_name_set
|
|
} // end of add_index_name_set
|
|
return ret;
|
|
}
|
|
|
|
// MockFKParentTable begin
|
|
|
|
int ObDDLService::gen_mock_fk_parent_table_for_create_fk(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const ObCreateForeignKeyArg &foreign_key_arg,
|
|
const ObMockFKParentTableSchema *tmp_mock_fk_parent_table_ptr,
|
|
ObForeignKeyInfo &foreign_key_info,
|
|
ObMockFKParentTableSchema &mock_fk_parent_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (!foreign_key_arg.is_parent_table_mock_) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("foreign_key_arg is not parent_table_mock", K(ret));
|
|
} else {
|
|
mock_fk_parent_table_schema.reset();
|
|
const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL;
|
|
const ObDatabaseSchema *db_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id,
|
|
foreign_key_arg.parent_database_, db_schema))) {
|
|
LOG_WARN("failed to get parent database schema", K(ret), K(foreign_key_arg));
|
|
} else if (NULL == db_schema) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, foreign_key_arg.parent_database_.length(), foreign_key_arg.parent_database_.ptr());
|
|
} else if (OB_INVALID_ID == db_schema->get_database_id()) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("database id is invalid", K(ret), K(tenant_id), K(db_schema->get_database_id()));
|
|
} else if (db_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("Can't not create outline of db in recyclebin", K(ret), K(*db_schema));
|
|
} else if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name(
|
|
tenant_id,
|
|
db_schema->get_database_id(),
|
|
foreign_key_arg.parent_table_,
|
|
mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("get_mock_fk_parent_table_schema_with_name failed", K(ret), K(tenant_id), K(db_schema->get_database_id()), K(foreign_key_arg.parent_table_));
|
|
} else if (NULL != tmp_mock_fk_parent_table_ptr && NULL != mock_fk_parent_table_ptr) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mock_fk_parent_table_ptr and tmp_mock_fk_parent_table_ptr are both not null", K(ret), KPC(mock_fk_parent_table_ptr), KPC(tmp_mock_fk_parent_table_ptr));
|
|
} else if (NULL != tmp_mock_fk_parent_table_ptr) {
|
|
mock_fk_parent_table_ptr = tmp_mock_fk_parent_table_ptr;
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(mock_fk_parent_table_ptr)) { // mock_fk_parent_table is not exist, need to add a new one
|
|
uint64_t mock_fk_parent_table_id = OB_INVALID_ID;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service must not null", K(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(tenant_id, mock_fk_parent_table_id))) {
|
|
LOG_WARN("fail to fetch_new_table_id", K(ret));
|
|
} else {
|
|
mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_CREATE_TABLE_BY_ADD_FK_IN_CHILD_TBALE);
|
|
mock_fk_parent_table_schema.set_tenant_id(tenant_id);
|
|
mock_fk_parent_table_schema.set_database_id(db_schema->get_database_id());
|
|
mock_fk_parent_table_schema.set_mock_fk_parent_table_id(mock_fk_parent_table_id);
|
|
mock_fk_parent_table_schema.set_mock_fk_parent_table_name(foreign_key_arg.parent_table_);
|
|
uint64_t column_id = OB_INVALID_ID;
|
|
uint64_t max_used_column_id = 0;
|
|
bool is_column_exist = false;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.parent_columns_.count(); ++j) {
|
|
mock_fk_parent_table_schema.get_column_id_by_column_name(foreign_key_arg.parent_columns_.at(j), column_id, is_column_exist);
|
|
if (is_column_exist) {
|
|
ret = OB_ERR_COLUMN_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_COLUMN_DUPLICATE, foreign_key_arg.parent_columns_.at(j).length(), foreign_key_arg.parent_columns_.at(j).ptr());
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema.add_column_info_to_column_array(std::make_pair(++max_used_column_id, foreign_key_arg.parent_columns_.at(j))))) {
|
|
LOG_WARN("fail to add_column_info_to_column_array for mock_fk_parent_table_schema", K(max_used_column_id), K(foreign_key_arg.parent_columns_.at(j)));
|
|
} else if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(max_used_column_id))) {
|
|
LOG_WARN("failed to push parent column id", K(ret), K(max_used_column_id));
|
|
}
|
|
}
|
|
}
|
|
} else { // mock_fk_parent_table is already exist, need to add new columns or update schema version
|
|
bool is_alter_table = (NULL == tmp_mock_fk_parent_table_ptr);
|
|
mock_fk_parent_table_schema.assign(*mock_fk_parent_table_ptr);
|
|
if (is_alter_table) {
|
|
mock_fk_parent_table_schema.reset_column_array();
|
|
}
|
|
uint64_t max_used_column_id = mock_fk_parent_table_ptr->get_column_array().at(mock_fk_parent_table_ptr->get_column_array().count() - 1).first;
|
|
bool is_column_exist = false;
|
|
uint64_t column_id = OB_INVALID_ID;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_arg.parent_columns_.count(); ++j) {
|
|
mock_fk_parent_table_ptr->get_column_id_by_column_name(foreign_key_arg.parent_columns_.at(j), column_id, is_column_exist);
|
|
if (is_column_exist) {
|
|
if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(column_id))) {
|
|
LOG_WARN("failed to push parent column id", K(ret), K(column_id));
|
|
}
|
|
} else if (!is_column_exist) {
|
|
if (OB_FAIL(mock_fk_parent_table_schema.add_column_info_to_column_array(std::make_pair(++max_used_column_id, foreign_key_arg.parent_columns_.at(j))))) {
|
|
LOG_WARN("fail to add_column_info_to_column_array for mock_fk_parent_table_schema", K(max_used_column_id), K(foreign_key_arg.parent_columns_.at(j)));
|
|
} else if (OB_FAIL(foreign_key_info.parent_column_ids_.push_back(max_used_column_id))) {
|
|
LOG_WARN("failed to push parent column id", K(ret), K(max_used_column_id));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (is_alter_table) { // alter table add fk
|
|
if (mock_fk_parent_table_schema.get_column_array().empty()) {
|
|
mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_UPDATE_SCHEMA_VERSION);
|
|
} else {
|
|
mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_ADD_COLUMN);
|
|
}
|
|
} else { // create table add fk
|
|
mock_fk_parent_table_schema.set_operation_type(ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_CREATE_TABLE_BY_ADD_FK_IN_CHILD_TBALE);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::prepare_gen_mock_fk_parent_tables_for_drop_fks(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const int64_t tenant_id,
|
|
const ObIArray<const ObForeignKeyInfo*> &foreign_key_info_array,
|
|
ObIArray<const ObMockFKParentTableSchema*> &mock_fk_parent_table_schema_ptr_array,
|
|
ObIArray<ObMockFKParentTableSchema> &mock_fk_parent_table_schema_array)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
for (int64_t i = 0; i < foreign_key_info_array.count(); ++i) {
|
|
const ObForeignKeyInfo* foreign_key_info = foreign_key_info_array.at(i);
|
|
if (OB_ISNULL(foreign_key_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("foreign_key_info is null", K(ret));
|
|
} else if (!foreign_key_info->is_parent_table_mock_) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("parent_table is not mock", K(ret), KPC(foreign_key_info));
|
|
} else {
|
|
bool is_in_array = false;
|
|
for (int64_t j = 0; OB_SUCC(ret) && !is_in_array && j < mock_fk_parent_table_schema_array.count(); ++j) {
|
|
if (mock_fk_parent_table_schema_array.at(j).get_mock_fk_parent_table_id() == foreign_key_info->parent_table_id_) {
|
|
is_in_array = true;
|
|
if (OB_FAIL(mock_fk_parent_table_schema_array.at(j).add_foreign_key_info(*foreign_key_info))) {
|
|
LOG_WARN("add_foreign_key_info failed", K(ret), K(foreign_key_info));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_in_array) {
|
|
ObMockFKParentTableSchema mock_fk_parent_table_schema;
|
|
const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL;
|
|
if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_id(
|
|
tenant_id, foreign_key_info->parent_table_id_, mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("get_mock_fk_parent_table_schema_with_id failed", K(ret), K(tenant_id), K(foreign_key_info->parent_table_id_));
|
|
} else if (OB_ISNULL(mock_fk_parent_table_ptr)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mock_fk_parent_table_ptr is null", K(ret));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema.assign(*mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("assign mock_fk_parent_table failed", K(ret));
|
|
} else if (FALSE_IT(mock_fk_parent_table_schema.reset_column_array())){
|
|
} else if (FALSE_IT(mock_fk_parent_table_schema.reset_foreign_key_infos())) {
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema.add_foreign_key_info(*foreign_key_info))) {
|
|
LOG_WARN("add_foreign_key_info failed", K(ret), K(foreign_key_info));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) {
|
|
LOG_WARN("push_back mock_fk_parent_table_schema_array failed", K(ret), K(mock_fk_parent_table_schema));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_ptr_array.push_back(mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("push_back mock_fk_parent_table_schema_ptr_array failed", K(ret), KPC(mock_fk_parent_table_ptr));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_mock_fk_parent_tables_for_drop_fks(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObIArray<const ObMockFKParentTableSchema*> &mock_fk_parent_table_schema_ptr_array,
|
|
ObIArray<ObMockFKParentTableSchema> &mock_fk_parent_table_schema_array)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (mock_fk_parent_table_schema_ptr_array.count() != mock_fk_parent_table_schema_array.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("counts of mock_fk_parent_table_schema_ptr_array and mock_fk_parent_table_schema_array are not equal",
|
|
K(ret), K(mock_fk_parent_table_schema_ptr_array.count()), K(mock_fk_parent_table_schema_array.count()));
|
|
}
|
|
if (OB_SUCC(ret) && !mock_fk_parent_table_schema_array.empty()) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < mock_fk_parent_table_schema_array.count(); ++i) {
|
|
ObMockFKParentTableSchema &tmp_mock_fk_parent_table_schema = mock_fk_parent_table_schema_array.at(i);
|
|
const ObMockFKParentTableSchema *orig_mock_fk_parent_table_ptr = mock_fk_parent_table_schema_ptr_array.at(i);
|
|
ObArray<ObForeignKeyInfo> foreign_key_infos_to_be_deleted;
|
|
if (OB_FAIL(foreign_key_infos_to_be_deleted.assign(tmp_mock_fk_parent_table_schema.get_foreign_key_infos()))) {
|
|
LOG_WARN("failed to assign foreign_key_infos_to_be_deleted", K(ret));
|
|
} else {
|
|
tmp_mock_fk_parent_table_schema.reset_foreign_key_infos();
|
|
}
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < orig_mock_fk_parent_table_ptr->get_foreign_key_infos().count(); ++j) {
|
|
bool is_to_be_remained = true;
|
|
for (int64_t k = 0; is_to_be_remained && k < foreign_key_infos_to_be_deleted.count(); ++k) {
|
|
if (orig_mock_fk_parent_table_ptr->get_foreign_key_infos().at(j).foreign_key_id_ == foreign_key_infos_to_be_deleted.at(k).foreign_key_id_) {
|
|
is_to_be_remained = false;
|
|
}
|
|
}
|
|
if (is_to_be_remained) {
|
|
if (OB_FAIL(tmp_mock_fk_parent_table_schema.add_foreign_key_info(
|
|
orig_mock_fk_parent_table_ptr->get_foreign_key_infos().at(j)))) {
|
|
LOG_WARN("failed to add_foreign_key_info",
|
|
K(ret), K(orig_mock_fk_parent_table_ptr->get_foreign_key_infos().at(j)));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(tmp_mock_fk_parent_table_schema.reconstruct_column_array_by_foreign_key_infos(orig_mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("reconstruct_column_array_by_foreign_key_infos failed",
|
|
K(ret), K(tmp_mock_fk_parent_table_schema), KPC(orig_mock_fk_parent_table_ptr));
|
|
} else if (tmp_mock_fk_parent_table_schema.get_column_array().count()
|
|
== orig_mock_fk_parent_table_ptr->get_column_array().count()) {
|
|
// If all columns in mock_fk_parent_table_schema are still referenced by other fk, only update schema version
|
|
tmp_mock_fk_parent_table_schema.set_operation_type(MOCK_FK_PARENT_TABLE_OP_UPDATE_SCHEMA_VERSION);
|
|
} else if (tmp_mock_fk_parent_table_schema.get_column_array().count() == 0) {
|
|
// If all columns in mock_fk_parent_table_schema will be dropped, drop this mock_fk_parent_table
|
|
if (OB_FAIL(tmp_mock_fk_parent_table_schema.set_column_array(orig_mock_fk_parent_table_ptr->get_column_array()))) {
|
|
LOG_WARN("set_column_array failed",
|
|
K(ret), K(tmp_mock_fk_parent_table_schema), KPC(orig_mock_fk_parent_table_ptr));
|
|
} else {
|
|
tmp_mock_fk_parent_table_schema.set_operation_type(MOCK_FK_PARENT_TABLE_OP_DROP_TABLE);
|
|
}
|
|
} else {
|
|
// If part of columns in mock_fk_parent_table_schema will be dropped, drop columns in this mock_fk_parent_table
|
|
tmp_mock_fk_parent_table_schema.set_operation_type(MOCK_FK_PARENT_TABLE_OP_DROP_COLUMN);
|
|
ObMockFKParentTableColumnArray columns_to_be_remained;
|
|
if (OB_FAIL(columns_to_be_remained.assign(tmp_mock_fk_parent_table_schema.get_column_array()))) {
|
|
LOG_WARN("failed to assign columns_to_be_remained", K(ret));
|
|
} else {
|
|
tmp_mock_fk_parent_table_schema.reset_column_array();
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_mock_fk_parent_table_ptr->get_column_array().count(); ++i) {
|
|
bool is_col_to_be_deleted = true;
|
|
for (int64_t j = 0; is_col_to_be_deleted && j < columns_to_be_remained.count(); ++j) {
|
|
if (orig_mock_fk_parent_table_ptr->get_column_array().at(i).first == columns_to_be_remained.at(j).first) {
|
|
is_col_to_be_deleted = false;
|
|
}
|
|
}
|
|
if (is_col_to_be_deleted) {
|
|
if (OB_FAIL(tmp_mock_fk_parent_table_schema.add_column_info_to_column_array(
|
|
std::make_pair(orig_mock_fk_parent_table_ptr->get_column_array().at(i).first,
|
|
orig_mock_fk_parent_table_ptr->get_column_array().at(i).second)))) {
|
|
LOG_WARN("fail to add_column_info_to_column_array", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_mock_fk_parent_table_for_drop_table(
|
|
share::schema::ObSchemaService *schema_service,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const obrpc::ObDropTableArg &drop_table_arg,
|
|
const DropTableIdHashSet &drop_table_set,
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos,
|
|
const ObForeignKeyInfo &violated_foreign_key_info,
|
|
const ObTableSchema *table_schema,
|
|
ObMockFKParentTableSchema &mock_fk_parent_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObTableSchema *child_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(drop_table_arg.tenant_id_, violated_foreign_key_info.child_table_id_, child_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret), K(violated_foreign_key_info));
|
|
} else if (OB_ISNULL(child_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("child_table is null", K(ret), K(violated_foreign_key_info));
|
|
} else if (drop_table_arg.foreign_key_checks_) {
|
|
ret = OB_ERR_TABLE_IS_REFERENCED;
|
|
LOG_USER_ERROR(OB_ERR_TABLE_IS_REFERENCED,
|
|
table_schema->get_table_name_str().length(), table_schema->get_table_name_str().ptr(),
|
|
violated_foreign_key_info.foreign_key_name_.length(), violated_foreign_key_info.foreign_key_name_.ptr(),
|
|
child_table_schema->get_table_name_str().length(), child_table_schema->get_table_name_str().ptr());
|
|
} else if (!drop_table_arg.foreign_key_checks_) {
|
|
ret = OB_SUCCESS;
|
|
uint64_t mock_fk_parent_table_id = OB_INVALID_ID;
|
|
if (drop_table_arg.tables_.count() > 1) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop fk parent table and more tables in one sql");
|
|
LOG_WARN("drop fk parent table and more tables in one sqll", K(ret), KPC(child_table_schema), KPC(table_schema));
|
|
} else if (child_table_schema->get_database_id() != table_schema->get_database_id()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop fk parent table which has child tables in different database");
|
|
LOG_WARN("drop fk parent table which has child tables in different database not supported", K(ret), KPC(child_table_schema), KPC(table_schema));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(table_schema->get_tenant_id(), mock_fk_parent_table_id))) {
|
|
LOG_WARN("fail to fetch_new_table_id", K(ret));
|
|
} else {
|
|
// mock_fk_parent_table_ptr = &mock_fk_parent_table_schema;
|
|
mock_fk_parent_table_schema.set_tenant_id(table_schema->get_tenant_id());
|
|
mock_fk_parent_table_schema.set_database_id(table_schema->get_database_id());
|
|
mock_fk_parent_table_schema.set_mock_fk_parent_table_id(mock_fk_parent_table_id);
|
|
mock_fk_parent_table_schema.set_mock_fk_parent_table_name(table_schema->get_table_name_str());
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); ++i) {
|
|
ObForeignKeyInfo tmp_foreign_key_info = foreign_key_infos.at(i);
|
|
if (OB_HASH_EXIST == drop_table_set.exist_refactored(tmp_foreign_key_info.child_table_id_)) {
|
|
// do-nothing.
|
|
// child table will be dropped in the same drop table SQL. multiple transactions update the schema, but the schema cache is not refreshed.
|
|
} else if (tmp_foreign_key_info.child_table_id_ == tmp_foreign_key_info.parent_table_id_) {
|
|
// do nothing, self referenced fk should be dropped.
|
|
} else if (table_schema->get_table_id() != violated_foreign_key_info.child_table_id_) {
|
|
// The difference of foreign_key_info between orig_parent_table and mock_fk_parent_table is only parent_table_id.
|
|
// parent_column_ids are all the same.
|
|
const int64_t invalid_cst_id = 0;
|
|
tmp_foreign_key_info.set_parent_table_id(mock_fk_parent_table_schema.get_mock_fk_parent_table_id());
|
|
tmp_foreign_key_info.set_is_parent_table_mock(true);
|
|
tmp_foreign_key_info.set_ref_cst_type(CONSTRAINT_TYPE_INVALID);
|
|
tmp_foreign_key_info.set_ref_cst_id(invalid_cst_id);
|
|
if (OB_FAIL(mock_fk_parent_table_schema.add_foreign_key_info(tmp_foreign_key_info))) {
|
|
LOG_WARN("fail to add_foreign_key_info for mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema), K(tmp_foreign_key_info));
|
|
} else {
|
|
// add column info of foreign_key_infos to mock_fk_parent_table_schema info
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_infos.at(i).parent_column_ids_.count(); ++j) {
|
|
const ObColumnSchemaV2 * parent_column_schema = NULL;
|
|
if (OB_ISNULL(parent_column_schema = table_schema->get_column_schema(foreign_key_infos.at(i).parent_column_ids_.at(j)))){
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("got null column schema", KPC(table_schema), K(i), K(foreign_key_infos.at(i)), K(j), K(foreign_key_infos.at(i).parent_column_ids_.at(j)));
|
|
} else {
|
|
bool is_existed = false;
|
|
// check if column info is in mock_fk_parent_table_schema already
|
|
for (int64_t k = 0; !is_existed && k < mock_fk_parent_table_schema.get_column_array().count(); ++k) {
|
|
if (parent_column_schema->get_column_id() == mock_fk_parent_table_schema.get_column_array().at(k).first
|
|
&& 0 == parent_column_schema->get_column_name_str().compare(mock_fk_parent_table_schema.get_column_array().at(k).second)) {
|
|
is_existed = true;
|
|
}
|
|
}
|
|
if (!is_existed) {
|
|
if (OB_FAIL(mock_fk_parent_table_schema.add_column_info_to_column_array(std::make_pair(parent_column_schema->get_column_id(), parent_column_schema->get_column_name_str())))) {
|
|
LOG_WARN("fail to add_column_info_to_column_array for mock_fk_parent_table_schema", KPC(parent_column_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else {
|
|
// This fk is only a self reference fk, do nothing
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_fk_columns_type_for_replacing_mock_fk_parent_table(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &real_parent_table_schema,
|
|
const ObMockFKParentTableSchema *&mock_parent_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
if (OB_ISNULL(mock_parent_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mock_parent_table_schema is not exist", K(ret));
|
|
} else {
|
|
if (OB_ISNULL(mock_parent_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
SQL_RESV_LOG(WARN, "mock_parent_table_schema is null", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < mock_parent_table_schema->get_foreign_key_infos().count(); ++i) {
|
|
const ObTableSchema *child_table_schema = NULL;
|
|
const ObForeignKeyInfo &fk_info = mock_parent_table_schema->get_foreign_key_infos().at(i);
|
|
if (OB_FAIL(schema_guard.get_table_schema(mock_parent_table_schema->get_tenant_id(), fk_info.child_table_id_, child_table_schema))) {
|
|
LOG_WARN("table is not exist", K(ret), K(fk_info.child_table_id_));
|
|
} else if (OB_ISNULL(child_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("parent table schema is null", K(ret));
|
|
} else if (OB_FAIL(child_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("check if oracle compat mode failed", K(ret));
|
|
} else {
|
|
// prepare params for check_foreign_key_columns_type
|
|
ObArray<ObString> child_columns;
|
|
ObArray<ObString> parent_columns;
|
|
bool is_column_exist = false;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < fk_info.child_column_ids_.count(); ++j) {
|
|
ObString child_column_name;
|
|
const ObColumnSchemaV2 *child_col = child_table_schema->get_column_schema(fk_info.child_column_ids_.at(j));
|
|
if (OB_ISNULL(child_col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column is not exist", K(ret), K(fk_info), K(j));
|
|
} else if (OB_FAIL(child_columns.push_back(child_col->get_column_name_str()))) {
|
|
LOG_WARN("fail to push_back to child_columns", K(ret), K(child_col->get_column_name_str()));
|
|
}
|
|
}
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < fk_info.parent_column_ids_.count(); ++j) {
|
|
ObString parent_column_name;
|
|
mock_parent_table_schema->get_column_name_by_column_id(fk_info.parent_column_ids_.at(j), parent_column_name, is_column_exist);
|
|
if (!is_column_exist) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column is not exist", K(ret), K(fk_info), K(j));
|
|
} else if (OB_FAIL(parent_columns.push_back(parent_column_name))) {
|
|
LOG_WARN("fail to push_back to real_parent_table_schema_columns", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ObResolverUtils::check_foreign_key_columns_type(
|
|
!is_oracle_mode/*is_mysql_compat_mode*/,
|
|
*child_table_schema,
|
|
real_parent_table_schema,
|
|
child_columns,
|
|
parent_columns,
|
|
NULL))) {
|
|
ret = OB_ERR_CANNOT_ADD_FOREIGN;
|
|
LOG_WARN("Failed to check_foreign_key_columns_type", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_uk_cst_id_for_replacing_mock_fk_parent_table(
|
|
const ObIArray<const share::schema::ObTableSchema*> &index_table_schemas,
|
|
share::schema::ObForeignKeyInfo &foreign_key_info)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_match = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && !is_match && i < index_table_schemas.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = index_table_schemas.at(i);
|
|
if (index_table_schema->is_unique_index()) {
|
|
const ObColumnSchemaV2 *index_col = NULL;
|
|
const ObIndexInfo &index_info = index_table_schema->get_index_info();
|
|
ObArray<uint64_t> uk_columns;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < index_info.get_size(); ++j) {
|
|
if (OB_ISNULL(index_col = index_table_schema->get_column_schema(index_info.get_column(j)->column_id_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get index column schema failed", K(ret), K(index_info.get_column(j)->column_id_));
|
|
} else if (index_col->is_hidden() || index_col->is_shadow_column()) { // do nothing
|
|
} else if (OB_FAIL(uk_columns.push_back(index_info.get_column(j)->column_id_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("push back index column failed", K(ret), KPC(index_col));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(sql::ObResolverUtils::check_match_columns(foreign_key_info.parent_column_ids_, uk_columns, is_match))) {
|
|
LOG_WARN("Failed to check_match_columns", K(ret));
|
|
} else if (is_match) {
|
|
foreign_key_info.ref_cst_type_ = CONSTRAINT_TYPE_UNIQUE_KEY;
|
|
foreign_key_info.ref_cst_id_ = index_table_schema->get_table_id();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// replace mock fk parent table with real fk parent table
|
|
int ObDDLService::gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t drop_mock_fk_parent_table_id,
|
|
const share::schema::ObTableSchema &real_parent_table,
|
|
const ObIArray<const share::schema::ObTableSchema*> &uk_index_schemas,
|
|
ObMockFKParentTableSchema &mock_fk_parent_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL;
|
|
if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_id(
|
|
real_parent_table.get_tenant_id(),
|
|
drop_mock_fk_parent_table_id,
|
|
mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("check_mock_fk_parent_table_exist_by_id failed", K(ret), K(real_parent_table.get_tenant_id()), K(drop_mock_fk_parent_table_id));
|
|
} else if (OB_ISNULL(mock_fk_parent_table_ptr)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mock_fk_parent_table is not exist", K(ret), K(mock_fk_parent_table_ptr));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema.assign(*mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("mock_fk_parent_table_schema assign failed", K(ret), KPC(mock_fk_parent_table_ptr));
|
|
} else if (FALSE_IT(mock_fk_parent_table_schema.set_operation_type(share::schema::ObMockFKParentTableOperationType::MOCK_FK_PARENT_TABLE_OP_REPLACED_BY_REAL_PREANT_TABLE))) {
|
|
} else if (OB_FAIL(check_fk_columns_type_for_replacing_mock_fk_parent_table(schema_guard, real_parent_table, mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("check_fk_columns_type_for_replacing_mock_fk_parent_table failed", K(ret), K(real_parent_table), KPC(mock_fk_parent_table_ptr));
|
|
} else {
|
|
const ObIArray<ObForeignKeyInfo> &ori_mock_fk_infos_array = mock_fk_parent_table_ptr->get_foreign_key_infos();
|
|
// modify the parent column id of fk, make it fit with real parent table
|
|
// mock_column_id -> column_name -> real_column_id
|
|
bool is_column_exist = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < ori_mock_fk_infos_array.count(); ++i) {
|
|
mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_column_ids_.reuse();
|
|
mock_fk_parent_table_schema.get_foreign_key_infos().at(i).ref_cst_type_ = CONSTRAINT_TYPE_INVALID;
|
|
mock_fk_parent_table_schema.get_foreign_key_infos().at(i).is_parent_table_mock_ = false;
|
|
mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_table_id_ = real_parent_table.get_table_id();
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < ori_mock_fk_infos_array.at(i).parent_column_ids_.count(); ++j) {
|
|
uint64_t mock_parent_table_column_id = ori_mock_fk_infos_array.at(i).parent_column_ids_.at(j);
|
|
ObString column_name;
|
|
mock_fk_parent_table_ptr->get_column_name_by_column_id(mock_parent_table_column_id, column_name, is_column_exist);
|
|
if (!is_column_exist) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column is not exist", K(ret), K(mock_parent_table_column_id), KPC(mock_fk_parent_table_ptr));
|
|
} else {
|
|
const ObColumnSchemaV2 *col_schema = NULL;
|
|
if (OB_ISNULL(col_schema = real_parent_table.get_column_schema(column_name))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get column schema failed", K(ret), K(column_name));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_column_ids_.push_back(col_schema->get_column_id()))) {
|
|
LOG_WARN("push_back to parent_column_ids failed", K(ret), K(col_schema->get_column_id()));
|
|
}
|
|
}
|
|
}
|
|
// check and mofidy ref cst type and ref cst id of fk
|
|
const ObRowkeyInfo &rowkey_info = real_parent_table.get_rowkey_info();
|
|
common::ObArray<uint64_t> pk_column_ids;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_info.get_size(); ++i) {
|
|
uint64_t column_id = 0;
|
|
const ObColumnSchemaV2 *col_schema = NULL;
|
|
if (OB_FAIL(rowkey_info.get_column_id(i, column_id))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get rowkey info", K(ret), K(i), K(rowkey_info));
|
|
} else if (OB_ISNULL(col_schema = real_parent_table.get_column_schema(column_id))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get index column schema failed", K(ret));
|
|
} else if (col_schema->is_hidden() || col_schema->is_shadow_column()) {
|
|
// do nothing
|
|
} else if(OB_FAIL(pk_column_ids.push_back(col_schema->get_column_id()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("push back column_id failed", K(ret), K(col_schema->get_column_id()));
|
|
}
|
|
}
|
|
bool is_match = false;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(sql::ObResolverUtils::check_match_columns(pk_column_ids, mock_fk_parent_table_schema.get_foreign_key_infos().at(i).parent_column_ids_, is_match))) {
|
|
LOG_WARN("check_match_columns failed", K(ret));
|
|
} else if (is_match) {
|
|
mock_fk_parent_table_schema.get_foreign_key_infos().at(i).ref_cst_type_ = CONSTRAINT_TYPE_PRIMARY_KEY;
|
|
} else { // pk is not match, check if uk match
|
|
if (OB_FAIL(get_uk_cst_id_for_replacing_mock_fk_parent_table(
|
|
uk_index_schemas, mock_fk_parent_table_schema.get_foreign_key_infos().at(i)))) {
|
|
LOG_WARN("fail to get_uk_cst_id_for_replacing_mock_fk_parent_table", K(ret));
|
|
} else if (CONSTRAINT_TYPE_INVALID == mock_fk_parent_table_schema.get_foreign_key_infos().at(i).ref_cst_type_) {
|
|
ret = OB_ERR_CANNOT_ADD_FOREIGN;
|
|
LOG_WARN("ref_cst_type is invalid", K(ret), KPC(mock_fk_parent_table_ptr));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// MockFKParentTable end
|
|
|
|
int ObDDLService::get_index_schema_by_name(
|
|
const uint64_t data_table_id,
|
|
const uint64_t database_id,
|
|
const ObDropIndexArg &drop_index_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema *&index_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = drop_index_arg.tenant_id_;
|
|
ObString index_table_name;
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
const ObString &index_name = drop_index_arg.index_name_;
|
|
const bool is_mlog = (obrpc::ObIndexArg::DROP_MLOG == drop_index_arg.index_action_type_);
|
|
|
|
//build index name and get index schema
|
|
if (is_mlog) {
|
|
index_table_name = index_name;
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator,
|
|
data_table_id,
|
|
index_name,
|
|
index_table_name))) {
|
|
LOG_WARN("build_index_table_name failed", K(ret), K(data_table_id), K(index_name));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
const bool is_index = !is_mlog;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
database_id,
|
|
index_table_name,
|
|
is_index,
|
|
index_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(database_id), K(index_table_schema));
|
|
} else if (nullptr == index_table_schema) {
|
|
ret = OB_ERR_CANT_DROP_FIELD_OR_KEY;
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, index_name.length(), index_name.ptr());
|
|
LOG_WARN("get index table schema failed", K(tenant_id),
|
|
K(database_id), K(index_table_name), K(ret));
|
|
} else if (index_table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("index table is in recyclebin", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_valid_index_schema_by_id_for_drop_index_(
|
|
const uint64_t data_table_id,
|
|
const ObDropIndexArg &drop_index_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema *&index_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = drop_index_arg.tenant_id_;
|
|
// drop_index_arg.index_name_ is specified by user. drop_index_arg.index_name_ may be
|
|
// not matched with drop_index_arg.index_table_id_. For example: Drop FTS index need
|
|
// to drop all built-in FTS index tables. One index_name correspond to multiple index
|
|
// tables. So, index_name may be not matched with index_table_id
|
|
const uint64_t table_id = drop_index_arg.index_table_id_;
|
|
const ObString index_name = drop_index_arg.index_name_;
|
|
index_table_schema = nullptr;
|
|
if (OB_UNLIKELY(OB_INVALID_ID == data_table_id || !drop_index_arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", K(ret), K(data_table_id), K(drop_index_arg));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, index_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(table_id), K(index_table_schema));
|
|
} else if (nullptr == index_table_schema) {
|
|
ret = OB_ERR_CANT_DROP_FIELD_OR_KEY;
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, index_name.length(), index_name.ptr());
|
|
LOG_WARN("get index table schema failed", K(tenant_id), K(data_table_id), K(ret));
|
|
} else if (index_table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("index table is in recyclebin", K(ret), K(data_table_id), K(drop_index_arg));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// To avoid ddl hung when drop and add index with same index-name in single stmt,
|
|
// should rename dropping index firstly, and then push it into ddl scheduler queue.
|
|
int ObDDLService::rename_dropping_index_name(
|
|
const uint64_t data_table_id,
|
|
const uint64_t database_id,
|
|
const bool is_inner_and_domain_index,
|
|
const ObDropIndexArg &drop_index_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
common::ObIArray<share::schema::ObTableSchema> &new_index_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObTableSchema *index_table_schema = nullptr;
|
|
int nwrite = 0;
|
|
const int64_t buf_size = number::ObNumber::MAX_PRINTABLE_SIZE;
|
|
char buf[buf_size] = {0};
|
|
ObString index_name = drop_index_arg.index_name_;
|
|
bool need_rename = true;
|
|
// When dropping fts index, one index_name corresponds to multiple index tables. We can not decide
|
|
// right index table by index_name. To handle multiple FTS built-in index tables, inner DDL task
|
|
// will drop these built-in index tables by drop_index_arg.index_table_id_.
|
|
if (!is_inner_and_domain_index) {
|
|
if (OB_FAIL(get_index_schema_by_name(data_table_id, database_id, drop_index_arg,
|
|
schema_guard, index_table_schema))) {
|
|
LOG_WARN("get index schema by name", K(ret), K(data_table_id), K(database_id));
|
|
}
|
|
} else if (OB_FAIL(get_valid_index_schema_by_id_for_drop_index_(data_table_id, drop_index_arg, schema_guard,
|
|
index_table_schema))) {
|
|
LOG_WARN("fail to get valid index schema by id for drop index", K(ret), K(data_table_id), K(drop_index_arg));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
// FTS or multi-value index and inner rpc, if its table schema does not exist, just skip rename and return success.
|
|
need_rename = false;
|
|
} else {
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
ObString cur_index_name;
|
|
if (OB_FAIL(index_table_schema->get_index_name(cur_index_name))) {
|
|
LOG_WARN("build_index_table_name failed", K(ret), K(data_table_id), KPC(index_table_schema));
|
|
} else if (0 != index_name.case_compare(cur_index_name)) {
|
|
// FTS index and inner rpc, it has been renamed, just skip rename and return success.
|
|
need_rename = false;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret) || !need_rename) {
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, index table schema is nullptr", K(ret), KP(index_table_schema), K(data_table_id),
|
|
K(drop_index_arg));
|
|
} else if ((!drop_index_arg.is_inner_ || drop_index_arg.is_parent_task_dropping_fts_index_) && index_table_schema->is_fts_index_aux()) {
|
|
// This task is the parent task of drop fts index, no need to rename.
|
|
if (OB_FAIL(get_dropping_domain_index_invisiable_aux_table_schema(index_table_schema->get_tenant_id(), data_table_id,
|
|
index_table_schema->get_table_id(), true, index_table_schema->get_table_name_str(), schema_guard, ddl_operator,
|
|
trans, new_index_schemas))) {
|
|
LOG_WARN("fail to get dropping fts aux table schema", K(ret), K(data_table_id), K(index_table_schema));
|
|
} else if (OB_FAIL(new_index_schemas.push_back(*index_table_schema))) {
|
|
LOG_WARN("fail to push back index schema", K(ret), KPC(index_table_schema));
|
|
}
|
|
} else if ((!drop_index_arg.is_inner_ || drop_index_arg.is_parent_task_dropping_multivalue_index_) && index_table_schema->is_multivalue_index_aux()) {
|
|
if (OB_FAIL(get_dropping_domain_index_invisiable_aux_table_schema(index_table_schema->get_tenant_id(), data_table_id,
|
|
index_table_schema->get_table_id(), false, index_table_schema->get_table_name_str(), schema_guard, ddl_operator,
|
|
trans, new_index_schemas))) {
|
|
LOG_WARN("fail to get dropping fts aux table schema", K(ret), K(data_table_id), K(index_table_schema));
|
|
} else if (OB_FAIL(new_index_schemas.push_back(*index_table_schema))) {
|
|
LOG_WARN("fail to push back index schema", K(ret), KPC(index_table_schema));
|
|
}
|
|
} else if ((!drop_index_arg.is_inner_ || drop_index_arg.is_vec_inner_drop_) && index_table_schema->is_vec_delta_buffer_type()) {
|
|
// This task is the parent task of drop vec index, no need to rename.
|
|
if (OB_FAIL(get_dropping_vec_index_invisiable_table_schema_(index_table_schema->get_tenant_id(), data_table_id,
|
|
index_table_schema->get_table_id(), drop_index_arg.is_vec_inner_drop_, index_table_schema->get_table_name_str(), schema_guard, ddl_operator,
|
|
trans, new_index_schemas))) {
|
|
LOG_WARN("fail to get dropping vec index table schema", K(ret), K(data_table_id), K(index_table_schema));
|
|
} else if (OB_FAIL(new_index_schemas.push_back(*index_table_schema))) {
|
|
LOG_WARN("fail to push back index schema", K(ret), KPC(index_table_schema));
|
|
}
|
|
} else if ((nwrite = snprintf(buf, buf_size, "%s_%lu",
|
|
"DELETING", ObTimeUtility::current_time())) >= buf_size || nwrite < 0) {
|
|
ret = common::OB_BUF_NOT_ENOUGH;
|
|
LOG_WARN("buf is not large enough", K(ret), K(buf_size));
|
|
} else {
|
|
SMART_VAR(ObTableSchema, new_index_schema) {
|
|
const ObIndexStatus new_index_status = INDEX_STATUS_UNAVAILABLE;
|
|
ObString new_index_name = ObString::make_string(buf);
|
|
obrpc::ObRenameIndexArg rename_index_arg;
|
|
rename_index_arg.tenant_id_ = index_table_schema->get_tenant_id();
|
|
rename_index_arg.origin_index_name_ = index_name;
|
|
rename_index_arg.new_index_name_ = new_index_name;
|
|
if (OB_INVALID_ID != drop_index_arg.index_table_id_) {
|
|
if (OB_FAIL(ddl_operator.alter_table_rename_index_with_origin_index_name(index_table_schema->get_tenant_id(),
|
|
index_table_schema->get_table_id(),
|
|
new_index_name,
|
|
new_index_status,
|
|
trans,
|
|
new_index_schema))) {
|
|
LOG_WARN("fail to alter table rename index", K(ret));
|
|
}
|
|
} else if (OB_FAIL(ddl_operator.alter_table_rename_index(index_table_schema->get_tenant_id(),
|
|
index_table_schema->get_data_table_id(),
|
|
index_table_schema->get_database_id(),
|
|
rename_index_arg,
|
|
&new_index_status,
|
|
trans,
|
|
new_index_schema))) {
|
|
LOG_WARN("rename index failed", K(ret));
|
|
}
|
|
if (FAILEDx(new_index_schemas.push_back(new_index_schema))) {
|
|
LOG_WARN("fail to push back new index schemas", K(ret), K(new_index_schema));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_dropping_domain_index_invisiable_aux_table_schema(
|
|
const uint64_t tenant_id,
|
|
const uint64_t data_table_id,
|
|
const uint64_t index_table_id,
|
|
const bool is_fts_index,
|
|
const ObString &index_name,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
common::ObIArray<share::schema::ObTableSchema> &new_aux_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const share::schema::ObTableSchema *data_table_schema = nullptr;
|
|
ObSEArray<const ObSimpleTableSchemaV2 *, OB_MAX_INDEX_PER_TABLE> indexs;
|
|
if (OB_UNLIKELY(OB_INVALID_ID == data_table_id
|
|
|| OB_INVALID_ID == index_table_id
|
|
|| OB_INVALID_TENANT_ID == tenant_id
|
|
|| index_name.empty())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", K(ret), K(data_table_id), K(index_table_id), K(tenant_id), K(index_name));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, data_table_schema))) {
|
|
LOG_WARN("fail to get index schema with data table id", K(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, data table schema is nullptr", K(ret), KP(data_table_schema));
|
|
} else {
|
|
SMART_VAR(ObTableSchema, new_aux_schema) {
|
|
const ObIArray<share::schema::ObAuxTableMetaInfo> &indexs = data_table_schema->get_simple_index_infos();
|
|
const share::schema::ObTableSchema *doc_word_schema = nullptr;
|
|
const share::schema::ObTableSchema *rowkey_doc_schema = nullptr;
|
|
const share::schema::ObTableSchema *doc_rowkey_schema = nullptr;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < indexs.count(); ++i) {
|
|
const share::schema::ObAuxTableMetaInfo &info = indexs.at(i);
|
|
if (share::schema::is_rowkey_doc_aux(info.index_type_)) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, info.table_id_, rowkey_doc_schema))) {
|
|
LOG_WARN("fail to get rowkey doc table schema", K(ret), K(tenant_id), K(info));
|
|
} else if (OB_ISNULL(rowkey_doc_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpeted error, rowkey doc schema is nullptr", K(ret), K(info));
|
|
} else if (OB_FAIL(new_aux_schemas.push_back(*rowkey_doc_schema))) {
|
|
LOG_WARN("fail to push doc rowkey table schema", K(ret), KPC(rowkey_doc_schema));
|
|
}
|
|
} else if (share::schema::is_doc_rowkey_aux(info.index_type_)) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, info.table_id_, doc_rowkey_schema))) {
|
|
LOG_WARN("fail to get doc rowkey table schema", K(ret), K(tenant_id), K(info));
|
|
} else if (OB_ISNULL(doc_rowkey_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, doc rowkey schema is nullptr", K(ret), K(info));
|
|
} else if (OB_FAIL(new_aux_schemas.push_back(*doc_rowkey_schema))) {
|
|
LOG_WARN("fail to push doc rowkey table schema", K(ret), KPC(doc_rowkey_schema));
|
|
}
|
|
} else if (is_fts_index && share::schema::is_fts_doc_word_aux(info.index_type_)){
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, info.table_id_, doc_word_schema))) {
|
|
LOG_WARN("fail to get doc word table schema", K(ret), K(tenant_id), K(info));
|
|
} else if (OB_ISNULL(doc_word_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, doc word schema is nullptr", K(ret), K(info));
|
|
} else {
|
|
int nwrite = 0;
|
|
const int64_t buf_size = OB_MAX_TABLE_NAME_BUF_LENGTH;
|
|
char buf[buf_size] = {0};
|
|
if (OB_FAIL(databuff_printf(buf, buf_size, "%.*s_fts_doc_word", index_name.length(), index_name.ptr()))) {
|
|
LOG_WARN("fail to printf fts doc word name str", K(ret), K(index_name));
|
|
} else if (0 == doc_word_schema->get_table_name_str().case_compare(buf)) {
|
|
if (OB_FAIL(new_aux_schemas.push_back(*doc_word_schema))) {
|
|
LOG_WARN("fail to push doc word table schema", K(ret), KPC(doc_word_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
STORAGE_FTS_LOG(INFO, "get dropping fts aux table name", K(ret), K(tenant_id), K(data_table_id), K(index_table_id));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_dropping_vec_index_invisiable_table_schema_(
|
|
const uint64_t tenant_id,
|
|
const uint64_t data_table_id,
|
|
const uint64_t index_table_id,
|
|
const bool is_vec_inner_drop,
|
|
const ObString &index_name,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
common::ObIArray<share::schema::ObTableSchema> &new_aux_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const share::schema::ObTableSchema *data_table_schema = nullptr;
|
|
ObSEArray<const ObSimpleTableSchemaV2 *, OB_MAX_INDEX_PER_TABLE> indexs;
|
|
if (OB_UNLIKELY(OB_INVALID_ID == data_table_id
|
|
|| OB_INVALID_ID == index_table_id
|
|
|| OB_INVALID_TENANT_ID == tenant_id
|
|
|| index_name.empty())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", K(ret), K(data_table_id), K(index_table_id), K(tenant_id), K(index_name));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, data_table_schema))) {
|
|
LOG_WARN("fail to get index schema with data table id", K(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, data table schema is nullptr", K(ret), KP(data_table_schema));
|
|
} else {
|
|
SMART_VAR(ObTableSchema, new_aux_schema) {
|
|
const ObIArray<share::schema::ObAuxTableMetaInfo> &indexs = data_table_schema->get_simple_index_infos();
|
|
const share::schema::ObTableSchema *index_id_schema = nullptr;
|
|
const share::schema::ObTableSchema *snapshot_data_schema = nullptr;
|
|
const share::schema::ObTableSchema *rowkey_vid_schema = nullptr;
|
|
const share::schema::ObTableSchema *vid_rowkey_schema = nullptr;
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
bool is_index = true;
|
|
const int64_t database_id = data_table_schema->get_database_id();
|
|
const bool is_hidden_flag = false;
|
|
const bool is_built_in_flag = true;
|
|
bool already_get_index_id_schema = false;
|
|
bool already_get_snapshot_data_table_schema = false;
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < indexs.count(); ++i) {
|
|
const share::schema::ObAuxTableMetaInfo &info = indexs.at(i);
|
|
if (share::schema::is_vec_rowkey_vid_type(info.index_type_)) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, info.table_id_, rowkey_vid_schema))) {
|
|
LOG_WARN("fail to get vec rowkey vid table schema", K(ret), K(tenant_id), K(info));
|
|
} else if (OB_ISNULL(rowkey_vid_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("rowkey_vid_schema is nullptr", K(ret), K(info));
|
|
} else if (OB_FAIL(new_aux_schemas.push_back(*rowkey_vid_schema))) {
|
|
LOG_WARN("fail to push vec rowkey vid table schema", K(ret), KPC(rowkey_vid_schema));
|
|
}
|
|
} else if (share::schema::is_vec_vid_rowkey_type(info.index_type_)) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, info.table_id_, vid_rowkey_schema))) {
|
|
LOG_WARN("fail to get vec vid rowkey table schema", K(ret), K(tenant_id), K(info));
|
|
} else if (OB_ISNULL(vid_rowkey_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("vid_rowkey_schema is nullptr", K(ret), K(info));
|
|
} else if (OB_FAIL(new_aux_schemas.push_back(*vid_rowkey_schema))) {
|
|
LOG_WARN("fail to push vec vid rowkey table schema", K(ret), KPC(vid_rowkey_schema));
|
|
}
|
|
} else if (share::schema::is_vec_index_id_type(info.index_type_)) {
|
|
// 通过索引名获取4号表
|
|
if (already_get_index_id_schema) {
|
|
// 可能存在多个schema,但这里只取包含index_name字串的schema
|
|
} else if (OB_FAIL(ObVecIndexBuilderUtil::get_vec_table_schema_by_name(schema_guard,
|
|
tenant_id,
|
|
database_id,
|
|
index_name,
|
|
info.index_type_,
|
|
&allocator,
|
|
index_id_schema))) {
|
|
LOG_WARN("fail to generate vec index name", K(ret), K(info.index_type_));
|
|
} else if (OB_ISNULL(index_id_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("index_id_schema is nullptr", K(ret), K(index_name));
|
|
} else if (OB_FAIL(new_aux_schemas.push_back(*index_id_schema))) {
|
|
LOG_WARN("fail to push vec table schema", K(ret), K(index_name));
|
|
} else {
|
|
already_get_index_id_schema = true;
|
|
}
|
|
} else if (share::schema::is_vec_index_snapshot_data_type(info.index_type_)) {
|
|
// 通过索引名获取5号表
|
|
if (already_get_snapshot_data_table_schema) {
|
|
// 可能存在多个 index id schema,但这里只取包含index_name字串的schema
|
|
} else if (OB_FAIL(ObVecIndexBuilderUtil::get_vec_table_schema_by_name(schema_guard,
|
|
tenant_id,
|
|
database_id,
|
|
index_name,
|
|
info.index_type_,
|
|
&allocator,
|
|
snapshot_data_schema))) {
|
|
LOG_WARN("fail to generate vec index name", K(ret), K(info.index_type_));
|
|
} else if (OB_ISNULL(snapshot_data_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("snapshot_data_schema is nullptr", K(ret), K(index_name));
|
|
} else if (OB_FAIL(new_aux_schemas.push_back(*snapshot_data_schema))) {
|
|
LOG_WARN("fail to push vec table schema", K(ret), K(index_name));
|
|
} else {
|
|
already_get_snapshot_data_table_schema = true;
|
|
}
|
|
}
|
|
if (OB_TABLE_NOT_EXIST == ret && is_vec_inner_drop) {
|
|
ret = OB_SUCCESS;
|
|
LOG_WARN("table is not exist, maybe index table have been drop already", K(ret));
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("get dropping vec aux table name", K(ret), K(tenant_id), K(data_table_id), K(index_table_id));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_tmp_idx_schemas(
|
|
const ObTableSchema &new_table_schema,
|
|
ObIArray<ObTableSchema> &idx_schemas,
|
|
ObSchemaGetterGuard &schema_guard) {
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
if (OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
RS_LOG(WARN, "get_table_schema failed", K(ret), K(tenant_id),
|
|
"table id", simple_index_infos.at(i).table_id_);
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
RS_LOG(WARN, "table schema should not be null", K(ret));
|
|
} else {
|
|
if (OB_FAIL(idx_schemas.push_back(*index_table_schema))) {
|
|
RS_LOG(WARN, "fail to push back to idx_schemas", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fill_column_collation(
|
|
const ObSQLMode sql_mode,
|
|
const bool is_oracle_mode,
|
|
const ObTableSchema &table_schema,
|
|
common::ObIAllocator &allocator,
|
|
ObColumnSchemaV2 &column_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObObjTypeClass col_tc = column_schema.get_data_type_class();
|
|
ObCollationType collation_type = table_schema.get_collation_type();
|
|
ObCharsetType charset_type = table_schema.get_charset_type();
|
|
const ObCollationType cur_extended_type_info_collation = ObCharset::get_system_collation();
|
|
if (ObStringTC == col_tc) {
|
|
if (OB_FAIL(ObDDLResolver::check_and_fill_column_charset_info(
|
|
column_schema, charset_type, collation_type))) {
|
|
RS_LOG(WARN, "failed to fill column charset info", K(ret));
|
|
} else if (OB_FAIL(ObDDLResolver::check_string_column_length(
|
|
column_schema, is_oracle_mode))) {
|
|
RS_LOG(WARN, "failed to check string column length", K(ret));
|
|
}
|
|
} else if (ObRawTC == col_tc) {
|
|
if (OB_FAIL(ObDDLResolver::check_raw_column_length(column_schema))) {
|
|
RS_LOG(WARN, "failed to check raw column length", K(ret), K(column_schema));
|
|
}
|
|
} else if (ob_is_text_tc(column_schema.get_data_type())) {
|
|
if (OB_FAIL(ObDDLResolver::check_and_fill_column_charset_info(
|
|
column_schema, table_schema.get_charset_type(), table_schema.get_collation_type()))) {
|
|
RS_LOG(WARN, "failed to fill column charset info", K(ret));
|
|
} else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(column_schema,
|
|
table_schema.get_table_id(), true))) {
|
|
RS_LOG(WARN, "failed to check text or blob column length", K(ret));
|
|
}
|
|
} else if (ObEnumSetTC == col_tc) {
|
|
if (OB_FAIL(ObDDLResolver::check_and_fill_column_charset_info(column_schema, charset_type, collation_type))) {
|
|
LOG_WARN("fail to check and fill column charset info", K(ret), K(column_schema));
|
|
} else if (OB_FAIL(ObResolverUtils::check_extended_type_info(
|
|
allocator,
|
|
column_schema.get_extended_type_info(),
|
|
cur_extended_type_info_collation,
|
|
column_schema.get_column_name_str(),
|
|
column_schema.get_data_type(),
|
|
column_schema.get_collation_type(),
|
|
sql_mode))) {
|
|
LOG_WARN("fail to fill extended type info", K(ret), K(column_schema));
|
|
} else if (OB_FAIL(ObDDLResolver::calc_enum_or_set_data_length(column_schema))) {
|
|
LOG_WARN("fail to calc data length", K(ret), K(column_schema));
|
|
}
|
|
} else if (ObGeometryTC == col_tc && !column_schema.is_nullable()) {
|
|
ret = OB_ER_INVALID_USE_OF_NULL;
|
|
LOG_WARN("alter table add geometry column can not has not null constraint", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::resolve_orig_default_value(ObColumnSchemaV2 &alter_column_schema,
|
|
const ObTimeZoneInfoWrap &tz_info_wrap, const common::ObString *nls_formats,
|
|
ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObObj &cur_default_value = alter_column_schema.get_cur_default_value();
|
|
if (alter_column_schema.is_identity_column()) {
|
|
// When column is identity column or generated column, cur_default_value and orig_default_value is different
|
|
// cur_default_value:"SEQUENCE.NEXTVAL"
|
|
// orig_default_value:pure_sequnece_id
|
|
} else if (!cur_default_value.is_null()) {
|
|
if (OB_FAIL(alter_column_schema.set_orig_default_value(cur_default_value))) {
|
|
LOG_WARN("fail to set orig default value for alter table", K(ret), K(cur_default_value));
|
|
}
|
|
} else if (alter_column_schema.is_nullable()) {
|
|
ObObj null_obj;
|
|
null_obj.set_null();
|
|
if (OB_FAIL(alter_column_schema.set_orig_default_value(null_obj))) {
|
|
LOG_WARN("fail to set origin default value", K(ret));
|
|
}
|
|
} else {
|
|
ObObj default_value;
|
|
default_value.set_type(alter_column_schema.get_data_type());
|
|
if (OB_FAIL(default_value.build_not_strict_default_value(alter_column_schema.get_accuracy().get_precision(), alter_column_schema.get_collation_type()))) {
|
|
LOG_WARN("failed to build not strict default value", K(ret));
|
|
} else if (OB_FAIL(alter_column_schema.set_orig_default_value(default_value))) {
|
|
LOG_WARN("failed to set orig default value", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !alter_column_schema.is_identity_column()) {
|
|
ObObj orig_default_value = alter_column_schema.get_orig_default_value();
|
|
if (IS_DEFAULT_NOW_OBJ(orig_default_value) || alter_column_schema.is_default_expr_v2_column()) {
|
|
if (OB_FAIL(ObDDLResolver::calc_default_value(alter_column_schema,
|
|
orig_default_value,
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
allocator))) {
|
|
LOG_WARN("fail to calc default now expr", K(ret));
|
|
} else if (OB_FAIL(alter_column_schema.set_orig_default_value(orig_default_value))) {
|
|
LOG_WARN("fail to set orig default value", K(orig_default_value), K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::resolve_timestamp_column(AlterColumnSchema *alter_column_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObColumnSchemaV2 &new_column_schema,
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap,
|
|
const common::ObString *nls_formats,
|
|
ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
if (OB_ISNULL(alter_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("alter_column_schema is NULL", K(ret));
|
|
} else if (OB_FAIL(new_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 (ObTimestampType != new_column_schema.get_data_type()
|
|
|| new_column_schema.is_generated_column()
|
|
|| false == alter_column_schema->check_timestamp_column_order_
|
|
|| new_column_schema.is_udt_related_column(is_oracle_mode)) {
|
|
//nothing to do
|
|
} else {
|
|
bool is_first_timestamp = false;
|
|
ObTableSchema::const_column_iterator it_begin = new_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = new_table_schema.column_end();
|
|
bool found = false;
|
|
for(; OB_SUCC(ret) && it_begin != it_end && !found; it_begin++) {
|
|
if (OB_ISNULL(it_begin)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("it_begin should not be NULL", K(ret));
|
|
} else if (OB_ISNULL(*it_begin)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin should not be NULL", K(ret));
|
|
} else {
|
|
if (ObTimestampType == (*it_begin)->get_data_type()) {
|
|
if (0 == (*it_begin)->get_column_name_str().case_compare(
|
|
new_column_schema.get_column_name_str())) {
|
|
is_first_timestamp = true;
|
|
}
|
|
found = true;
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_DDL_ALTER_COLUMN == alter_column_schema->alter_type_) {
|
|
//drop default or set default
|
|
//mysql seem like set default now will couse a parser error;
|
|
if (is_first_timestamp && alter_column_schema->is_drop_default_) {
|
|
//new_column_schema is orig_column_schema;
|
|
//if default value is now(), on update current timestamp is false;
|
|
if (!new_column_schema.is_nullable()
|
|
&& !IS_DEFAULT_NOW_OBJ(new_column_schema.get_cur_default_value())) {
|
|
new_column_schema.set_on_update_current_timestamp(true);
|
|
} else {
|
|
//do nothing
|
|
}
|
|
}
|
|
} else {
|
|
bool is_set_default = alter_column_schema->is_set_default_;
|
|
bool is_set_null = alter_column_schema->is_set_nullable_;
|
|
if (is_first_timestamp
|
|
&& !is_set_null
|
|
&& !is_set_default
|
|
&& !new_column_schema.is_on_update_current_timestamp()) {
|
|
new_column_schema.set_nullable(false);
|
|
new_column_schema.get_cur_default_value().set_ext(
|
|
ObActionFlag::OP_DEFAULT_NOW_FLAG);
|
|
new_column_schema.set_on_update_current_timestamp(true);
|
|
} else if (!is_set_null) {
|
|
new_column_schema.set_nullable(false);
|
|
if (!is_set_default) {
|
|
if (alter_column_schema->is_no_zero_date_) {
|
|
ret = OB_INVALID_DEFAULT;
|
|
LOG_USER_ERROR(OB_INVALID_DEFAULT, alter_column_schema->get_column_name_str().length(),
|
|
alter_column_schema->get_column_name_str().ptr());
|
|
} else {
|
|
int64_t zero_date = ObTimeConverter::ZERO_DATETIME;
|
|
ObTimeConverter::round_datetime(alter_column_schema->get_data_scale(), zero_date);
|
|
new_column_schema.get_cur_default_value().set_timestamp(zero_date);
|
|
}
|
|
} else if (new_column_schema.get_cur_default_value().is_null()) {
|
|
ret = OB_INVALID_DEFAULT;
|
|
LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(),
|
|
new_column_schema.get_column_name_str().ptr());
|
|
}
|
|
} else {
|
|
new_column_schema.set_nullable(true);
|
|
if (!is_set_default) {
|
|
new_column_schema.get_cur_default_value().set_null();
|
|
}
|
|
}
|
|
if(OB_SUCC(ret)
|
|
&& OB_DDL_ADD_COLUMN == alter_column_schema->alter_type_) {
|
|
ObObj cur_default_value = new_column_schema.get_cur_default_value();
|
|
if (IS_DEFAULT_NOW_OBJ(cur_default_value) || alter_column_schema->is_default_expr_v2_column()) {
|
|
if (OB_FAIL(ObDDLResolver::calc_default_value(*alter_column_schema,
|
|
cur_default_value,
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
allocator))) {
|
|
LOG_WARN("fail to calc default now expr", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(alter_column_schema->set_orig_default_value(cur_default_value))) {
|
|
OB_LOG(WARN, "fail to set orig default value", K(cur_default_value), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::deal_default_value_padding(ObColumnSchemaV2 &column_schema, ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObString str;
|
|
if (column_schema.get_orig_default_value().is_null()
|
|
|| column_schema.get_data_type() != ObCharType
|
|
|| column_schema.get_collation_type() != CS_TYPE_BINARY) {
|
|
//nothing to do;
|
|
} else if (OB_FAIL(column_schema.get_orig_default_value().get_string(str))) {
|
|
LOG_WARN("fail to get string", K(ret));
|
|
} else {
|
|
int64_t strlen = ObCharset::strlen_char(column_schema.get_collation_type(),
|
|
str.ptr(), str.length());
|
|
if (strlen >= column_schema.get_data_length()) {
|
|
//nothing to do
|
|
//check_default_value_length will check length;
|
|
} else {
|
|
char* ptr = NULL;
|
|
int64_t real_size = str.length() + column_schema.get_data_length() - strlen;
|
|
if (NULL == (ptr = static_cast<char *>(allocator.alloc(real_size)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
} else {
|
|
MEMCPY(ptr, str.ptr(), str.length());
|
|
memset(ptr + str.length(), OB_PADDING_BINARY, column_schema.get_data_length() - strlen);
|
|
ObString new_string(real_size, ptr);
|
|
ObObj new_default_value;
|
|
new_default_value.set_binary(new_string);
|
|
if (OB_FAIL(column_schema.set_orig_default_value(new_default_value))) {
|
|
LOG_WARN("fail to set orig default value", K(ret), K(new_default_value));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::pre_check_orig_column_schema(
|
|
const AlterColumnSchema &alter_column_schema,
|
|
const ObTableSchema &origin_table_schema,
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> &update_column_name_set)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObString &orig_column_name = alter_column_schema.get_origin_column_name();
|
|
const ObColumnSchemaV2 *orig_column_schema = origin_table_schema.get_column_schema(orig_column_name);
|
|
const ObColumnSchemaV2 *column_schema_from_old_table_schema = origin_table_schema.get_column_schema(orig_column_name);
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
bool is_oracle_mode = false;
|
|
if (NULL == column_schema_from_old_table_schema) {
|
|
ret = OB_ERR_BAD_FIELD_ERROR;
|
|
LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(),
|
|
origin_table_schema.get_table_name_str().length(),
|
|
origin_table_schema.get_table_name_str().ptr());
|
|
LOG_WARN("failed to find old column schema", K(ret), K(orig_column_name));
|
|
} else if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) {
|
|
//column that has been modified, can't not modify again
|
|
ret = OB_ERR_BAD_FIELD_ERROR;
|
|
LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR,
|
|
orig_column_name.length(), orig_column_name.ptr(),
|
|
origin_table_schema.get_table_name_str().length(),
|
|
origin_table_schema.get_table_name_str().ptr());
|
|
LOG_WARN("column that has been altered, can't not update again", K(ret));
|
|
} else if (OB_FAIL(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 (OB_FAIL(check_generated_column_modify_authority(*orig_column_schema, alter_column_schema, is_oracle_mode))) {
|
|
LOG_WARN("check generated column modify authority", K(ret), KPC(orig_column_schema), K(alter_column_schema));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_generated_column_modify_authority(
|
|
const ObColumnSchemaV2 &old_column_schema, const AlterColumnSchema &alter_column_schema, bool is_oracle_mode)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (old_column_schema.is_generated_column() && alter_column_schema.is_generated_column()) {
|
|
if ((old_column_schema.is_virtual_generated_column() && alter_column_schema.is_virtual_generated_column())
|
|
|| (old_column_schema.is_stored_generated_column() && alter_column_schema.is_stored_generated_column())) {
|
|
ObString old_def;
|
|
ObString alter_def;
|
|
if (OB_FAIL(old_column_schema.get_cur_default_value().get_string(old_def))) {
|
|
LOG_WARN("get old generated column definition failed", K(ret), K(old_column_schema));
|
|
} else if (OB_FAIL(alter_column_schema.get_cur_default_value().get_string(alter_def))) {
|
|
LOG_WARN("get new generated column definition failed", K(ret), K(alter_column_schema));
|
|
} else if (!ObCharset::case_insensitive_equal(old_def, alter_def)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Modify generated column definition");
|
|
LOG_WARN("generated column schema definition changed", K(ret), K(old_column_schema), K(alter_column_schema));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN;
|
|
LOG_USER_ERROR(OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN, "Changing the STORED status");
|
|
}
|
|
} else if (old_column_schema.is_generated_column() || alter_column_schema.is_generated_column()) {
|
|
if (is_oracle_mode && old_column_schema.is_generated_column()
|
|
&& old_column_schema.get_data_type() != alter_column_schema.get_data_type()) {
|
|
ret = OB_ERR_MODIFY_TYPE_OF_GENCOL;
|
|
LOG_WARN("cannot change the data-type of virtual column without modifying the underlying expression", K(ret), K(old_column_schema), K(alter_column_schema));
|
|
} else if (is_oracle_mode && alter_column_schema.is_generated_column()) {
|
|
ret = OB_ERR_MODIFY_REALCOL_TO_GENCOL;
|
|
LOG_WARN("cannot alter a real column to have an expression", K(ret), K(old_column_schema), K(alter_column_schema));
|
|
} else {
|
|
ret = OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN;
|
|
LOG_USER_ERROR(OB_ERR_UNSUPPORTED_ACTION_ON_GENERATED_COLUMN, "Changing the STORED status");
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_generated_column_schema(
|
|
const AlterColumnSchema &alter_column_schema,
|
|
const ObColumnSchemaV2 &orig_column_schema,
|
|
const ObTableSchema &origin_table_schema,
|
|
const ObTimeZoneInfoWrap &tz_info_wrap,
|
|
const sql::ObLocalSessionVar *local_session_var,
|
|
ObTableSchema &new_table_schema,
|
|
const bool need_update_default_value,
|
|
const bool need_update_session_var,
|
|
ObDDLOperator *ddl_operator,
|
|
common::ObMySQLTransaction *trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema::const_column_iterator col_iter = new_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator col_end = new_table_schema.column_end();
|
|
ObColumnSchemaV2 *column = nullptr;
|
|
bool for_view = false;
|
|
if (OB_ISNULL(tz_info_wrap.get_time_zone_info())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tz info", K(ret));
|
|
}
|
|
for (; OB_SUCC(ret) && col_iter != col_end; col_iter++) {
|
|
if (OB_ISNULL(column = *col_iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*col_iter is NULL", K(ret));
|
|
} else if (column->has_cascaded_column_id(orig_column_schema.get_column_id())) {
|
|
ObColumnSchemaV2 new_generated_column_schema;
|
|
if (OB_FAIL(new_generated_column_schema.assign(*column))) {
|
|
LOG_WARN("failed to copy new gen column", K(ret));
|
|
} else if (need_update_session_var
|
|
&& OB_FAIL(modify_generated_column_local_vars(new_generated_column_schema,
|
|
orig_column_schema.get_column_name_str(),
|
|
orig_column_schema.get_data_type(),
|
|
alter_column_schema,
|
|
new_table_schema,
|
|
local_session_var))) {
|
|
LOG_WARN("modify local session vars failed", K(ret));
|
|
} else if (need_update_default_value
|
|
&& OB_FAIL(modify_generated_column_default_value(new_generated_column_schema,
|
|
const_cast<ObString&>(orig_column_schema.get_column_name_str()),
|
|
alter_column_schema.get_column_name_str(),
|
|
new_table_schema,
|
|
*tz_info_wrap.get_time_zone_info()))) {
|
|
LOG_WARN("modify generated column value failed", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.alter_column(new_generated_column_schema,
|
|
ObTableSchema::CHECK_MODE_ONLINE,
|
|
for_view))) {
|
|
// alter_column must not change column_array_ because we are still looping the column_array_
|
|
LOG_WARN("failed to change column", K(ret));
|
|
} else if (OB_NOT_NULL(ddl_operator) && OB_NOT_NULL(trans)) {
|
|
if (OB_FAIL(ddl_operator->update_single_column(*trans,
|
|
origin_table_schema,
|
|
new_table_schema,
|
|
new_generated_column_schema))) {
|
|
LOG_WARN("generated column failed to alter column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_generated_column_default_value(ObColumnSchemaV2 &generated_column,
|
|
common::ObString &column_name,
|
|
const ObString &new_column_name,
|
|
const ObTableSchema &table_schema,
|
|
const ObTimeZoneInfo &tz_info)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (generated_column.is_generated_column()) {
|
|
ObString col_def;
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
ObRawExprFactory expr_factory(allocator);
|
|
SMART_VARS_3((ObSQLSessionInfo, default_session), (ObExecContext, exec_ctx, allocator),
|
|
(ObPhysicalPlanCtx, phy_plan_ctx, allocator)) {
|
|
LinkExecCtxGuard link_guard(default_session, exec_ctx);
|
|
uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObRawExpr *expr = NULL;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
|
|
if (OB_FAIL(default_session.init(0, 0, &allocator))) {
|
|
LOG_WARN("init empty session failed", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("get tenant_schema failed", K(ret));
|
|
} else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) {
|
|
LOG_WARN("init tenant failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) {
|
|
LOG_WARN("session load system variable failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_default_configs_in_pc())) {
|
|
LOG_WARN("session load default configs failed", K(ret));
|
|
} else if (OB_FAIL(generated_column.get_cur_default_value().get_string(col_def))) {
|
|
LOG_WARN("get cur default value failed", K(ret));
|
|
} else if (FALSE_IT(exec_ctx.set_physical_plan_ctx(&phy_plan_ctx))) {
|
|
} else if (FALSE_IT(exec_ctx.set_my_session(&default_session))) {
|
|
} else if (OB_FAIL(ObRawExprUtils::build_generated_column_expr(NULL,
|
|
col_def,
|
|
expr_factory,
|
|
default_session,
|
|
table_schema,
|
|
expr))) {
|
|
LOG_WARN("build generated column expr failed", K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(table_schema.get_tenant_id(), table_schema.get_table_id(), compat_mode))) {
|
|
LOG_WARN("failed to get table compat mode", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObRawExprModifyColumnName modifyColumnName(new_column_name, column_name, compat_mode);
|
|
if (OB_FAIL(modifyColumnName.modifyColumnName(*expr))) {
|
|
LOG_WARN("modifyColumnName modify column name failed", K(ret));
|
|
} else {
|
|
SMART_VAR(char[OB_MAX_DEFAULT_VALUE_LENGTH], expr_str_buf) {
|
|
MEMSET(expr_str_buf, 0, sizeof(expr_str_buf));
|
|
ObString expr_def;
|
|
int64_t pos = 0;
|
|
ObObj default_value;
|
|
ObRawExprPrinter expr_printer(expr_str_buf, OB_MAX_DEFAULT_VALUE_LENGTH, &pos, &schema_guard, &tz_info);
|
|
if (OB_FAIL(expr_printer.do_print(expr, T_NONE_SCOPE, true))) {
|
|
LOG_WARN("print expr definition failed", K(ret));
|
|
} else if (FALSE_IT(expr_def.assign_ptr(expr_str_buf, static_cast<int32_t>(pos)))) {
|
|
} else if (FALSE_IT(default_value.set_varchar(expr_def))) {
|
|
} else if (OB_FAIL(generated_column.set_cur_default_value(
|
|
default_value,
|
|
generated_column.is_default_expr_v2_column()))) {
|
|
LOG_WARN("set cur default value failed", K(ret));
|
|
} else if (OB_FAIL(generated_column.set_orig_default_value(default_value))) {
|
|
LOG_WARN("set original default value failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
exec_ctx.set_physical_plan_ctx(NULL);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_generated_column_local_vars(ObColumnSchemaV2 &generated_column,
|
|
const common::ObString &column_name,
|
|
const ObObjType origin_type,
|
|
const AlterColumnSchema &new_column_schema,
|
|
const ObTableSchema &table_schema,
|
|
const sql::ObLocalSessionVar *local_session_var) {
|
|
int ret = OB_SUCCESS;
|
|
if (generated_column.is_generated_column()
|
|
&& origin_type != new_column_schema.get_data_type()) {
|
|
ObString col_def;
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
ObRawExprFactory expr_factory(allocator);
|
|
SMART_VARS_3((ObSQLSessionInfo, default_session), (ObExecContext, exec_ctx, allocator),
|
|
(ObPhysicalPlanCtx, phy_plan_ctx, allocator)) {
|
|
LinkExecCtxGuard link_guard(default_session, exec_ctx);
|
|
uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObRawExpr *expr = NULL;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
if (OB_FAIL(default_session.init(0, 0, &allocator))) {
|
|
LOG_WARN("init empty session failed", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("get tenant_schema failed", K(ret));
|
|
} else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) {
|
|
LOG_WARN("init tenant failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) {
|
|
LOG_WARN("session load system variable failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_default_configs_in_pc())) {
|
|
LOG_WARN("session load default configs failed", K(ret));
|
|
} else if (NULL != local_session_var
|
|
&& OB_FAIL(local_session_var->update_session_vars_with_local(default_session))) {
|
|
LOG_WARN("fail to update session vars", K(ret));
|
|
} else if (FALSE_IT(exec_ctx.set_physical_plan_ctx(&phy_plan_ctx))) {
|
|
} else if (FALSE_IT(exec_ctx.set_my_session(&default_session))) {
|
|
} else if (OB_FAIL(generated_column.get_cur_default_value().get_string(col_def))) {
|
|
LOG_WARN("get cur default value failed", K(ret));
|
|
} else if (OB_FAIL(ObRawExprUtils::build_generated_column_expr(NULL,
|
|
col_def,
|
|
expr_factory,
|
|
default_session,
|
|
table_schema,
|
|
expr))) {
|
|
LOG_WARN("build generated column expr failed", K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(table_schema.get_tenant_id(), table_schema.get_table_id(), compat_mode))) {
|
|
LOG_WARN("failed to get table compat mode", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObRawExpr *expr_with_implicit_cast = NULL;
|
|
ObExprResType dst_type;
|
|
dst_type.set_meta(generated_column.get_meta_type());
|
|
dst_type.set_accuracy(generated_column.get_accuracy());
|
|
dst_type.set_collation_level(CS_LEVEL_IMPLICIT);
|
|
ObSQLMode sql_mode = default_session.get_sql_mode();
|
|
if (NULL != local_session_var) {
|
|
sql::ObSessionSysVar *sys_var = NULL;
|
|
if (OB_FAIL(local_session_var->get_local_var(share::SYS_VAR_SQL_MODE, sys_var))) {
|
|
LOG_WARN("fail to get sys var", K(ret));
|
|
} else if (NULL != sys_var) {
|
|
sql_mode = sys_var->val_.get_uint64();
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ObRawExprUtils::erase_operand_implicit_cast(expr, expr))) {
|
|
LOG_WARN("erase implicit cast failed", K(ret));
|
|
} else if (OB_ISNULL(expr)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", K(ret), KP(expr));
|
|
} else if (OB_FAIL(modify_depend_column_type(expr, column_name, new_column_schema, compat_mode))) {
|
|
LOG_WARN("modify column type failed", K(ret));
|
|
} else if (OB_FAIL(expr->formalize_with_local_vars(&default_session,
|
|
local_session_var,
|
|
OB_INVALID_INDEX_INT64))) {
|
|
LOG_WARN("expr formalize failed", K(ret));
|
|
} else if (OB_FAIL(ObRawExprUtils::try_add_cast_expr_above(&expr_factory, &default_session,
|
|
*expr, dst_type, expr_with_implicit_cast))) {
|
|
LOG_WARN("try add cast expr above failed", K(ret));
|
|
} else if (OB_ISNULL(expr_with_implicit_cast)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", K(ret), KP(expr_with_implicit_cast));
|
|
} else if (OB_FAIL(expr_with_implicit_cast->formalize_with_local_vars(&default_session,
|
|
local_session_var,
|
|
OB_INVALID_INDEX_INT64))) {
|
|
LOG_WARN("expr formalize failed", K(ret));
|
|
} else if (OB_FAIL(ObRawExprUtils::extract_local_vars_for_gencol(expr_with_implicit_cast,
|
|
sql_mode,
|
|
generated_column))) {
|
|
LOG_WARN("extract sysvar from expr failed", K(ret));
|
|
}
|
|
}
|
|
exec_ctx.set_physical_plan_ctx(NULL);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_depend_column_type(sql::ObRawExpr *expr,
|
|
const ObString &column_name,
|
|
const AlterColumnSchema &column_schema,
|
|
lib::Worker::CompatMode compat_mode) {
|
|
int ret = OB_SUCCESS;
|
|
if (OB_ISNULL(expr)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", K(ret));
|
|
} else if (expr->has_flag(IS_COLUMN)) {
|
|
ObColumnRefRawExpr *column_expr = static_cast<ObColumnRefRawExpr *>(expr);
|
|
lib::CompatModeGuard compat_guard(compat_mode);
|
|
if (ObColumnNameHashWrapper(column_expr->get_column_name()) == ObColumnNameHashWrapper(column_name)) {
|
|
column_expr->set_data_type(column_schema.get_data_type());
|
|
column_expr->set_lob_column(is_lob_storage(column_schema.get_data_type()));
|
|
if (ob_is_string_type(column_schema.get_data_type())
|
|
|| ob_is_enumset_tc(column_schema.get_data_type())
|
|
|| ob_is_json_tc(column_schema.get_data_type())
|
|
|| ob_is_geometry_tc(column_schema.get_data_type())) {
|
|
column_expr->set_collation_type(column_schema.get_collation_type());
|
|
column_expr->set_collation_level(CS_LEVEL_IMPLICIT);
|
|
} else {
|
|
column_expr->set_collation_type(CS_TYPE_BINARY);
|
|
column_expr->set_collation_level(CS_LEVEL_NUMERIC);
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
column_expr->set_accuracy(column_schema.get_accuracy());
|
|
if (column_schema.is_decimal_int()) {
|
|
ObObjMeta data_meta = column_schema.get_meta_type();
|
|
data_meta.set_scale(column_schema.get_accuracy().get_scale());
|
|
column_expr->set_meta_type(data_meta);
|
|
}
|
|
if (OB_FAIL(column_expr->extract_info())) {
|
|
LOG_WARN("extract column expr info failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && (column_schema.is_enum_or_set() || column_schema.is_collection())) {
|
|
if (OB_FAIL(column_expr->set_enum_set_values(column_schema.get_extended_type_info()))) {
|
|
LOG_WARN("failed to set enum set values", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && column_schema.is_xmltype()) {
|
|
// column_expr->set_udt_id(column_schema.get_sub_data_type());
|
|
// }
|
|
// ToDo : @gehao, need to conver extend type to udt type?
|
|
//if (OB_SUCC(ret) && column_schema.is_sql_xmltype()) {
|
|
column_expr->set_data_type(ObUserDefinedSQLType);
|
|
column_expr->set_subschema_id(ObXMLSqlType);
|
|
// reset accuracy
|
|
}
|
|
}
|
|
} else if (expr->has_flag(CNT_COLUMN)) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) {
|
|
if (OB_FAIL(SMART_CALL(modify_depend_column_type(expr->get_param_expr(i), column_name,
|
|
column_schema, compat_mode)))) {
|
|
LOG_WARN("modify depend column type failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_part_func_expr(const share::schema::ObTableSchema &orig_table_schema,
|
|
const share::schema::ObColumnSchemaV2 &orig_column_schema,
|
|
const share::schema::AlterColumnSchema &alter_column_schema,
|
|
share::schema::ObTableSchema &new_table_schema,
|
|
const common::ObTimeZoneInfo &tz_info,
|
|
common::ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_partition_key = false;
|
|
if (OB_FAIL(orig_table_schema.is_tbl_partition_key(orig_column_schema,
|
|
is_partition_key,
|
|
false /* ignore_presetting_key */))) {
|
|
LOG_WARN("fail to check is tbl partition key", KR(ret), K(orig_table_schema), K(orig_column_schema));
|
|
} else if (is_partition_key) {
|
|
if (share::schema::PARTITION_FUNC_TYPE_KEY_IMPLICIT != orig_table_schema.get_part_option().get_part_func_type()
|
|
&& orig_table_schema.get_part_option().get_part_func_expr_str().empty()) {
|
|
if (orig_table_schema.is_auto_partitioned_table()) {
|
|
// when part_func_expr is empty,
|
|
// auto-partitioned non-partitioned table will choose rowkey for data table and
|
|
// choose index columns for global index as presetting partition key.
|
|
// in this case, it is no need to modify part_func_expr
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("a partitioned table doesn't have part_fun_expr", KR(ret), K(orig_table_schema));
|
|
}
|
|
} else if (OB_FAIL(modify_part_func_expr(orig_column_schema.get_column_name_str(),
|
|
alter_column_schema.get_column_name_str(),
|
|
new_table_schema, tz_info, allocator))) {
|
|
LOG_WARN("modify part func expr failed", KR(ret));
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_part_func_expr(
|
|
const ObString &orig_column_name,
|
|
const ObString &alter_column_name,
|
|
share::schema::ObTableSchema &table_schema,
|
|
const common::ObTimeZoneInfo &tz_info,
|
|
common::ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
if (!table_schema.is_partitioned_table() && !table_schema.is_auto_partitioned_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(table_schema.get_part_level()),
|
|
K(table_schema.get_part_option()));
|
|
} else if (share::schema::PARTITION_FUNC_TYPE_KEY_IMPLICIT != table_schema.get_part_option().get_part_func_type()
|
|
&& table_schema.get_part_option().get_part_func_expr_str().empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid argument", KR(ret), K(table_schema));
|
|
} else if (PARTITION_LEVEL_ZERO == table_schema.get_part_level()
|
|
|| PARTITION_LEVEL_ONE == table_schema.get_part_level()) {
|
|
if (OB_FAIL(modify_func_expr_column_name(
|
|
orig_column_name, alter_column_name, table_schema, tz_info, allocator, false))) {
|
|
LOG_WARN("fail to modify func expr column name",
|
|
K(ret), K(orig_column_name), K(alter_column_name), K(table_schema));
|
|
}
|
|
} else if (PARTITION_LEVEL_TWO == table_schema.get_part_level()) {
|
|
if (OB_FAIL(modify_func_expr_column_name(
|
|
orig_column_name, alter_column_name, table_schema, tz_info, allocator, false))) {
|
|
LOG_WARN("fail to modify func expr column name for partition level one",
|
|
K(ret), K(orig_column_name), K(alter_column_name), K(table_schema));
|
|
} else if (OB_FAIL(modify_func_expr_column_name(
|
|
orig_column_name, alter_column_name, table_schema, tz_info, allocator, true))) {
|
|
LOG_WARN("fail to modify func expr column name for partition level two",
|
|
K(ret), K(orig_column_name), K(alter_column_name), K(table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_func_expr_column_name(
|
|
const ObString &orig_column_name,
|
|
const ObString &alter_column_name,
|
|
ObTableSchema &table_schema,
|
|
const ObTimeZoneInfo &tz_info,
|
|
common::ObIAllocator &allocator,
|
|
bool is_sub_part)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObRawExprFactory expr_factory(allocator);
|
|
uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObRawExpr *expr = NULL;
|
|
ObArray<ObQualifiedName> columns;
|
|
const ObColumnSchemaV2 *col_schema = NULL;
|
|
ObPartitionOption &part_option = table_schema.get_part_option();
|
|
ObPartitionOption &sub_part_option = table_schema.get_sub_part_option();
|
|
ObString orig_part_expr;
|
|
ObArray<ObString> expr_strs;
|
|
|
|
SMART_VAR(ObSQLSessionInfo, default_session) {
|
|
if (!is_sub_part) {
|
|
orig_part_expr = part_option.get_part_func_expr_str();
|
|
} else {
|
|
orig_part_expr = sub_part_option.get_part_func_expr_str();
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (part_option.get_part_func_type()
|
|
== share::schema::PARTITION_FUNC_TYPE_KEY_IMPLICIT) {
|
|
// partition by key(), no part func expr need to change
|
|
// do nothing
|
|
} else if (OB_FAIL(default_session.init(0, 0, &allocator))) {
|
|
LOG_WARN("init empty session failed", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("get tenant_schema failed", K(ret));
|
|
} else if (OB_FAIL(
|
|
default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) {
|
|
LOG_WARN("init tenant failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) {
|
|
LOG_WARN("session load system variable failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_default_configs_in_pc())) {
|
|
LOG_WARN("session load default configs failed", K(ret));
|
|
} else if (orig_part_expr.empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("partition func expr is empty", K(ret));
|
|
} else if (OB_FAIL(split_on(orig_part_expr, ',', expr_strs))) {
|
|
LOG_WARN("fail to split func expr", K(ret), K(orig_part_expr));
|
|
} else {
|
|
char *new_part_func_expr_buf = NULL;
|
|
int64_t outer_pos = 0;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
if (OB_ISNULL(new_part_func_expr_buf =
|
|
static_cast<char *>(allocator.alloc(OB_MAX_SQL_LENGTH)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail to alloc new_part_func_expr", K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(table_schema.get_tenant_id(), table_schema.get_table_id(), compat_mode))) {
|
|
LOG_WARN("failed to get table compat mode", K(ret));
|
|
} else {
|
|
ObRawExprModifyColumnName modifyColumnName(
|
|
alter_column_name, orig_column_name, compat_mode);
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < expr_strs.count(); ++i) {
|
|
expr = NULL;
|
|
columns.reset();
|
|
if (OB_FAIL(ObRawExprUtils::build_generated_column_expr(
|
|
expr_strs.at(i), expr_factory, default_session,
|
|
expr, columns, &table_schema, false /* allow_sequence */, NULL,
|
|
NULL, sql::ObResolverUtils::DISABLE_CHECK, false))) {
|
|
LOG_WARN("build generated column expr failed", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i <columns.count(); i++) {
|
|
const ObQualifiedName &q_name = columns.at(i);
|
|
if (OB_UNLIKELY(!q_name.database_name_.empty()
|
|
|| OB_UNLIKELY(!q_name.tbl_name_.empty()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invaild generated_column column name", K(q_name));
|
|
} else if (OB_ISNULL(col_schema = table_schema.get_column_schema(q_name.col_name_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column schema is null", K(ret), K(q_name.col_name_));
|
|
} else if (OB_FAIL(ObRawExprUtils::init_column_expr(*col_schema, *q_name.ref_expr_))) {
|
|
LOG_WARN("init column expr failed", K(ret));
|
|
} else {
|
|
q_name.ref_expr_->set_ref_id(table_schema.get_table_id(), col_schema->get_column_id());
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(modifyColumnName.modifyColumnName(*expr))) {
|
|
LOG_WARN("modifyColumnName modify column name failed", K(ret));
|
|
} else {
|
|
SMART_VAR(char[OB_MAX_SQL_LENGTH], expr_str_buf) {
|
|
MEMSET(expr_str_buf, 0, sizeof(expr_str_buf));
|
|
int64_t inner_pos = 0;
|
|
ObRawExprPrinter expr_printer(expr_str_buf, OB_MAX_SQL_LENGTH, &inner_pos, &schema_guard, &tz_info);
|
|
if (OB_FAIL(expr_printer.do_print(expr, T_NONE_SCOPE, true))) {
|
|
LOG_WARN("print expr definition failed", K(ret));
|
|
} else if (0 == i
|
|
&& OB_FAIL(databuff_printf(new_part_func_expr_buf,
|
|
OB_MAX_SQL_LENGTH,
|
|
outer_pos,
|
|
"%.*s",
|
|
static_cast<int>(inner_pos),
|
|
expr_str_buf))) {
|
|
LOG_WARN("fail to print expr_str_buf", K(ret), K(i), K(expr_str_buf));
|
|
} else if (0 != i
|
|
&& OB_FAIL(databuff_printf(new_part_func_expr_buf,
|
|
OB_MAX_SQL_LENGTH,
|
|
outer_pos,
|
|
", %.*s",
|
|
static_cast<int>(inner_pos),
|
|
expr_str_buf))) {
|
|
LOG_WARN("fail to print expr_str_buf", K(ret), K(i), K(expr_str_buf));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObString new_part_func_expr_str;
|
|
if (FALSE_IT(new_part_func_expr_str.assign_ptr(
|
|
new_part_func_expr_buf, static_cast<int32_t>(outer_pos)))) {
|
|
} else if (!is_sub_part && OB_FAIL(part_option.set_part_expr(new_part_func_expr_str))) {
|
|
LOG_WARN("set part expr failed", K(ret));
|
|
} else if (is_sub_part && OB_FAIL(sub_part_option.set_part_expr(new_part_func_expr_str))) {
|
|
LOG_WARN("set sub part expr failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_part_func_expr_for_global_index(
|
|
const share::schema::ObColumnSchemaV2 &orig_column,
|
|
const share::schema::ObColumnSchemaV2 &alter_column,
|
|
share::schema::ObTableSchema &new_table_schema,
|
|
const common::ObTimeZoneInfo &tz_info,
|
|
common::ObIAllocator &allocator,
|
|
ObDDLOperator *ddl_operator,
|
|
common::ObMySQLTransaction *trans,
|
|
ObIArray<ObTableSchema> *global_idx_schema_array/*=NULL*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool update_inner_table = nullptr != ddl_operator && nullptr != trans;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is NULL", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
} else {
|
|
const ObTableSchema *index_schema = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, simple_index_infos.at(i).table_id_, index_schema))) {
|
|
LOG_WARN("get table_schema failed", K(ret), K(tenant_id), K(simple_index_infos.at(i).table_id_));
|
|
} else if (OB_ISNULL(index_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("index schema should not be null", K(ret));
|
|
} else if (index_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("index table is in recyclebin", K(ret));
|
|
} else if (index_schema->is_global_index_table()
|
|
&& index_schema->is_partitioned_table()) {
|
|
const ObColumnSchemaV2 *origin_column_schema =
|
|
index_schema->get_column_schema(orig_column.get_column_id());
|
|
if (OB_ISNULL(origin_column_schema)) {
|
|
// skip, this column is not in global index
|
|
} else if (origin_column_schema->is_tbl_part_key_column()) {
|
|
ObTableSchema new_index_schema;
|
|
if (OB_FAIL(new_index_schema.assign(*index_schema))) {
|
|
LOG_WARN("assign index_schema failed", K(ret));
|
|
} else if (OB_FAIL(modify_part_func_expr(origin_column_schema->get_column_name_str(),
|
|
alter_column.get_column_name_str(), new_index_schema, tz_info, allocator))) {
|
|
LOG_WARN("modify part func expr failed", K(ret));
|
|
} else if (update_inner_table && OB_FAIL(ddl_operator->update_partition_option(*trans, new_index_schema))) {
|
|
LOG_WARN("update partition option failed",
|
|
K(ret), K(new_index_schema), K(*index_schema));
|
|
} else if (nullptr != global_idx_schema_array && OB_FAIL(global_idx_schema_array->push_back(new_index_schema))) {
|
|
LOG_WARN("fail to push_back to global_idx_schema_array",
|
|
K(ret), K(new_index_schema));
|
|
}
|
|
}
|
|
} else {
|
|
// skip
|
|
}
|
|
} // end of for
|
|
} // end of else
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_constraint_check_expr(
|
|
const share::schema::ObColumnSchemaV2 &orig_column,
|
|
const share::schema::ObColumnSchemaV2 &alter_column,
|
|
share::schema::ObTableSchema &table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const bool is_oracle_mode,
|
|
const common::ObTimeZoneInfo &tz_info,
|
|
common::ObIAllocator &allocator,
|
|
ObDDLOperator *ddl_operator,
|
|
common::ObMySQLTransaction *trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool update_inner_table = nullptr != ddl_operator && nullptr != trans;
|
|
bool need_modify_check_expr = false;
|
|
ObString new_check_expr_str;
|
|
ObTableSchema::const_constraint_iterator iter = table_schema.constraint_begin();
|
|
|
|
for (; OB_SUCC(ret) && iter != table_schema.constraint_end(); ++iter) {
|
|
need_modify_check_expr = false;
|
|
if ((*iter)->get_check_expr_str().empty()) {
|
|
continue;
|
|
}
|
|
// -- mysql mode, should succeed
|
|
// drop table t1;
|
|
// CREATE TABLE t1 (col1 INT primary key, col2 INT);
|
|
// ALTER TABLE t1 ADD CONSTRAINT my_check3 CHECK (col1+col2> col1+1);
|
|
// ALTER TABLE t1 drop CONSTRAINT my_check3, rename column col2 to col3,rename column col1 to col5;
|
|
//
|
|
// if corresponding constraint is being dropped, columns are allowed to be renamed, and there's no
|
|
// need to rebuild constraint expression.
|
|
bool dropped_cst = false;
|
|
if (!is_oracle_mode
|
|
&& alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::DROP_CONSTRAINT) {
|
|
for (auto cst_iter = alter_table_arg.alter_table_schema_.constraint_begin();
|
|
!dropped_cst && cst_iter != alter_table_arg.alter_table_schema_.constraint_end();
|
|
cst_iter++) {
|
|
if ((*cst_iter)->get_constraint_name_str().case_compare((*iter)->get_constraint_name_str())
|
|
== 0) {
|
|
dropped_cst = true;
|
|
}
|
|
}
|
|
}
|
|
if (dropped_cst) {
|
|
} else if (OB_FAIL(rebuild_constraint_check_expr(orig_column, alter_column, **iter, table_schema,
|
|
tz_info, allocator, new_check_expr_str, need_modify_check_expr))) {
|
|
LOG_WARN("fail to gen constraint check expr", K(ret));
|
|
} else if (need_modify_check_expr) {
|
|
(*iter)->set_check_expr(new_check_expr_str);
|
|
(*iter)->set_is_modify_check_expr(true);
|
|
if (update_inner_table && OB_FAIL(ddl_operator->update_check_constraint_state(*trans,
|
|
table_schema, **iter))) {
|
|
LOG_WARN("update check expr constraint failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// Oracle mode, to refill column id for not null constraint that is referenced to newly-added column.
|
|
// Case: alter table table_name add column_name not null.
|
|
int ObDDLService::refill_columns_id_for_not_null_constraint(
|
|
const share::schema::ObTableSchema &alter_table_schema,
|
|
const share::schema::ObColumnSchemaV2 &alter_column_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema::constraint_iterator target_cst_iter = nullptr;
|
|
if (alter_column_schema.has_not_null_constraint()) {
|
|
ObTableSchema::constraint_iterator cst_iter = alter_table_schema.constraint_begin_for_non_const_iter();
|
|
for (; NULL == target_cst_iter && OB_SUCC(ret) &&
|
|
cst_iter != alter_table_schema.constraint_end_for_non_const_iter();
|
|
cst_iter++) {
|
|
if (OB_ISNULL(*cst_iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected", K(ret));
|
|
} else if (CONSTRAINT_TYPE_NOT_NULL == (*cst_iter)->get_constraint_type()) {
|
|
ObString cst_col_name;
|
|
if (OB_FAIL((*cst_iter)->get_not_null_column_name(cst_col_name))) {
|
|
LOG_WARN("get not null cst column name failed", K(ret));
|
|
} else if (alter_column_schema.get_column_name_str().length() == cst_col_name.length()
|
|
&& 0 == alter_column_schema.get_column_name_str().compare(cst_col_name)) {
|
|
target_cst_iter = cst_iter;
|
|
}
|
|
LOG_DEBUG("alter table add column not null", K((*cst_iter)->get_check_expr_str()), K(cst_col_name),
|
|
K(alter_column_schema.get_column_name_str()));
|
|
}
|
|
}
|
|
ObSEArray<uint64_t, 1> column_id_array;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(target_cst_iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("constraint not found for added not null column", K(ret), K(alter_column_schema));
|
|
} else if (OB_FAIL(column_id_array.push_back(alter_column_schema.get_column_id()))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
} else if (OB_FAIL((*target_cst_iter)->assign_column_ids(column_id_array))) {
|
|
LOG_WARN("assign column ids failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// column_id_array is invalid if newly-added check constraint is referenced to
|
|
// newly-added column, due to invalid column id in resolver phase.
|
|
// Case: alter table table_name add column_name constraint check(column_name < 10) check(column_name > 0).
|
|
int ObDDLService::refill_columns_id_for_check_constraint(
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
const share::schema::ObTableSchema &alter_table_schema,
|
|
const share::schema::ObColumnSchemaV2 &alter_column_schema,
|
|
const bool is_oracle_mode,
|
|
common::ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool need_refill_columns_id = false;
|
|
ObTableSchema::const_constraint_iterator check_cst_iter = nullptr;
|
|
ObTableSchema::constraint_iterator target_cst_iter = nullptr;
|
|
// check whether there are some unexpected case firstly.
|
|
for (check_cst_iter = alter_table_schema.constraint_begin();
|
|
OB_SUCC(ret) && check_cst_iter != alter_table_schema.constraint_end(); check_cst_iter++) {
|
|
if (OB_ISNULL(*check_cst_iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected", K(ret));
|
|
} else if (CONSTRAINT_TYPE_CHECK == (*check_cst_iter)->get_constraint_type()) {
|
|
if (is_oracle_mode) {
|
|
// set error code here to avoid forgetting refill columns id
|
|
// under oracle mode if supported in the future.
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("add column and alter check constraint in single stmt is not supported", K(ret), K(alter_table_schema));
|
|
} else if (1 == (*check_cst_iter)->get_column_cnt()
|
|
&& nullptr == orig_table_schema.get_column_schema(*((*check_cst_iter)->cst_col_begin()))) {
|
|
// Currently, newly-added check constraint is referenced to newly-added column under Mysql mode.
|
|
need_refill_columns_id = true;
|
|
}
|
|
}
|
|
}
|
|
// refill columns id array.
|
|
if (OB_SUCC(ret) && need_refill_columns_id) {
|
|
ObRawExprFactory expr_factory(allocator);
|
|
uint64_t tenant_id = alter_column_schema.get_tenant_id();
|
|
const ObTenantSchema *tenant_schema = nullptr;
|
|
ObSchemaGetterGuard schema_guard;
|
|
SMART_VAR(ObSQLSessionInfo, default_session) {
|
|
if (OB_FAIL(default_session.init(0, 0, &allocator))) {
|
|
LOG_WARN("init empty session failed", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("get tenant_schema failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) {
|
|
LOG_WARN("init tenant failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) {
|
|
LOG_WARN("session load system variable failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_default_configs_in_pc())) {
|
|
LOG_WARN("session load default configs failed", K(ret));
|
|
} else {
|
|
for (target_cst_iter = alter_table_schema.constraint_begin_for_non_const_iter();
|
|
OB_SUCC(ret) && target_cst_iter != alter_table_schema.constraint_end_for_non_const_iter(); target_cst_iter++) {
|
|
if (OB_ISNULL(*target_cst_iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected", K(ret));
|
|
} else if (CONSTRAINT_TYPE_CHECK == (*target_cst_iter)->get_constraint_type()
|
|
&& 1 == (*target_cst_iter)->get_column_cnt()
|
|
&& nullptr == orig_table_schema.get_column_schema(*((*target_cst_iter)->cst_col_begin()))) {
|
|
ObString check_expr = (*target_cst_iter)->get_check_expr_str();
|
|
ObArray<ObQualifiedName> columns;
|
|
ObSEArray<uint64_t, 1> column_id_array;
|
|
ObRawExpr *expr = nullptr;
|
|
const ParseNode *node = nullptr;
|
|
if (check_expr.empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("check expr is empty", K(ret));
|
|
} else if (OB_FAIL(ObRawExprUtils::parse_bool_expr_node_from_str(check_expr,
|
|
expr_factory.get_allocator(), node))) {
|
|
LOG_WARN("parse expr node from string failed", K(ret));
|
|
} else if (OB_ISNULL(node)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("node is null", K(ret));
|
|
} else if (OB_FAIL(ObRawExprUtils::build_check_constraint_expr(expr_factory,
|
|
default_session, *node, expr, columns))) {
|
|
LOG_WARN("build generated column expr failed", K(ret), K(check_expr));
|
|
} else if (1 != columns.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected column count of the new-built check constraint", K(ret), K(check_expr));
|
|
} else if (0 != alter_column_schema.get_column_name_str().case_compare(columns.at(0).col_name_)) {
|
|
// may be newly-added column without check constraint, do nothing.
|
|
} else if (OB_FAIL(column_id_array.push_back(alter_column_schema.get_column_id()))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
} else if (OB_FAIL((*target_cst_iter)->assign_column_ids(column_id_array))) {
|
|
LOG_WARN("assign column ids failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_constraint_check_expr(
|
|
const share::schema::ObColumnSchemaV2 &orig_column,
|
|
const share::schema::ObColumnSchemaV2 &alter_column,
|
|
const share::schema::ObConstraint &cst,
|
|
share::schema::ObTableSchema &table_schema,
|
|
const common::ObTimeZoneInfo &tz_info,
|
|
common::ObIAllocator &allocator,
|
|
ObString &new_check_expr_str,
|
|
bool &need_modify_check_expr)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObRawExprFactory expr_factory(allocator);
|
|
uint64_t tenant_id = orig_column.get_tenant_id();
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObRawExpr *expr = NULL;
|
|
const ParseNode *node = NULL;
|
|
ObArray<ObQualifiedName> columns;
|
|
const ObColumnSchemaV2 *col_schema = NULL;
|
|
ObString orig_check_expr = cst.get_check_expr_str();
|
|
SMART_VAR(ObSQLSessionInfo, default_session) {
|
|
if (OB_FAIL(default_session.init(0, 0, &allocator))) {
|
|
LOG_WARN("init empty session failed", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("get tenant_schema failed", K(ret));
|
|
} else if (OB_FAIL(default_session.init_tenant(tenant_schema->get_tenant_name_str(), tenant_id))) {
|
|
LOG_WARN("init tenant failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_all_sys_vars(schema_guard))) {
|
|
LOG_WARN("session load system variable failed", K(ret));
|
|
} else if (OB_FAIL(default_session.load_default_configs_in_pc())) {
|
|
LOG_WARN("session load default configs failed", K(ret));
|
|
} else if (orig_check_expr.empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("check expr is empty", K(ret));
|
|
} else {
|
|
char *new_check_expr_buf = NULL;
|
|
int64_t outer_pos = 0;
|
|
if (OB_ISNULL(new_check_expr_buf =
|
|
static_cast<char *>(allocator.alloc(OB_MAX_SQL_LENGTH)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail to alloc new_check_expr_buf", K(ret));
|
|
} else {
|
|
char *new_check_expr_buf = NULL;
|
|
int64_t outer_pos = 0;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
if (OB_ISNULL(new_check_expr_buf =
|
|
static_cast<char *>(allocator.alloc(OB_MAX_SQL_LENGTH)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail to alloc new_check_expr_buf", K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(table_schema.get_tenant_id(), table_schema.get_table_id(), compat_mode))) {
|
|
LOG_WARN("failed to get table compat mode", K(ret));
|
|
} else {
|
|
ObRawExprModifyColumnName modifyColumnName(
|
|
alter_column.get_column_name_str(), orig_column.get_column_name_str(), compat_mode);
|
|
if (OB_FAIL(ObRawExprUtils::parse_bool_expr_node_from_str(orig_check_expr,
|
|
expr_factory.get_allocator(), node))) {
|
|
LOG_WARN("parse expr node from string failed", K(ret));
|
|
} else if (OB_ISNULL(node)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("node is null", K(ret));
|
|
} else if (OB_FAIL(ObRawExprUtils::build_check_constraint_expr(expr_factory,
|
|
default_session, *node, expr, columns))) {
|
|
LOG_WARN("build generated column expr failed", K(ret), K(orig_check_expr));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i <columns.count(); i++) {
|
|
const ObQualifiedName &q_name = columns.at(i);
|
|
if (0 == orig_column.get_column_name_str().
|
|
case_compare(q_name.col_name_)) {
|
|
need_modify_check_expr = true;
|
|
}
|
|
if (OB_UNLIKELY(!q_name.database_name_.empty()
|
|
|| OB_UNLIKELY(!q_name.tbl_name_.empty()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid generated_column column name", K(q_name));
|
|
} else if (OB_ISNULL(col_schema = table_schema.get_column_schema(q_name.col_name_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column schema is null", K(ret), K(q_name.col_name_));
|
|
} else if (OB_FAIL(ObRawExprUtils::init_column_expr(*col_schema, *q_name.ref_expr_))) {
|
|
LOG_WARN("init column expr failed", K(ret), K((*col_schema).get_column_name_str()));
|
|
} else {
|
|
q_name.ref_expr_->set_ref_id(table_schema.get_table_id(), col_schema->get_column_id());
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(modifyColumnName.modifyColumnName(*expr))) {
|
|
LOG_WARN("modifyColumnName modify column name failed", K(ret));
|
|
} else {
|
|
SMART_VAR(char[OB_MAX_SQL_LENGTH], expr_str_buf) {
|
|
MEMSET(expr_str_buf, 0, sizeof(expr_str_buf));
|
|
int64_t inner_pos = 0;
|
|
ObRawExprPrinter expr_printer(expr_str_buf, OB_MAX_SQL_LENGTH, &inner_pos, &schema_guard, &tz_info);
|
|
if (OB_FAIL(expr_printer.do_print(expr, T_NONE_SCOPE, true))) {
|
|
LOG_WARN("print expr definition failed", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(new_check_expr_buf,
|
|
OB_MAX_SQL_LENGTH,
|
|
outer_pos,
|
|
"%.*s",
|
|
static_cast<int>(inner_pos),
|
|
expr_str_buf))) {
|
|
LOG_WARN("fail to print expr_str_buf", K(ret), K(expr_str_buf));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
new_check_expr_str.assign_ptr(new_check_expr_buf, static_cast<int32_t>(outer_pos));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_can_alter_column_type(
|
|
const share::schema::ObColumnSchemaV2 &src_column,
|
|
const share::schema::ObColumnSchemaV2 &dst_column,
|
|
const share::schema::ObTableSchema &table_schema,
|
|
const bool is_oracle_mode)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t data_version = 0;
|
|
bool is_change_column_type = false;
|
|
bool is_in_index = false;
|
|
bool has_generated_depend = src_column.has_generated_column_deps();
|
|
if (OB_FAIL(check_is_change_column_type(src_column, dst_column, is_change_column_type))) {
|
|
LOG_WARN("fail to check is change column type", K(ret), K(src_column), K(dst_column));
|
|
} else if (is_change_column_type) {
|
|
if (OB_FAIL(check_column_in_index(src_column.get_column_id(), table_schema, is_in_index))) {
|
|
LOG_WARN("fail to check column is in index table", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(table_schema.get_tenant_id(), data_version))) {
|
|
LOG_WARN("fail to get data version", KR(ret), K(table_schema.get_tenant_id()));
|
|
} else if (is_in_index || has_generated_depend) {
|
|
// is_in_index==true means : src_column is 'the index column' or 'index create by user'.
|
|
const common::ObObjMeta &src_meta = src_column.get_meta_type();
|
|
const common::ObObjMeta &dst_meta = dst_column.get_meta_type();
|
|
if (is_oracle_mode) { // in oracle mode
|
|
if (is_in_index
|
|
&& data_version >= DATA_VERSION_4_3_0_1
|
|
&& common::ObNumberType == src_column.get_meta_type().get_type() && common::ObNumberFloatType == dst_column.get_meta_type().get_type()) {
|
|
// support number -> float in oracle mode in version 4.3
|
|
} else {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("cannot modify column in index table", K(ret), K(src_column), K(dst_column), K(table_schema));
|
|
}
|
|
} else {
|
|
// in mysql mode
|
|
if (((has_generated_depend && !is_in_index) || data_version >= DATA_VERSION_4_2_2_0)
|
|
&& common::is_match_alter_integer_column_online_ddl_rules(src_meta, dst_meta)) {
|
|
// support online ddl with index, generated column depended:
|
|
// smaller integer -> big integer in mysql mode in version 4.2.2
|
|
} else if (((has_generated_depend && !is_in_index) || data_version >= DATA_VERSION_4_3_0_0)
|
|
&& common::is_match_alter_string_column_online_ddl_rules(src_meta, dst_meta, src_column.get_data_length(), dst_column.get_data_length())) {
|
|
// generated column type change of: varchar <-> tinytext; varbinary <-> tinyblob; smaller lob -> larger lob; (4.3.0.0)
|
|
} else {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("cannot modify column in index table", K(ret), K(src_column), K(dst_column), K(table_schema), K(data_version));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_change_column_type(
|
|
const share::schema::ObColumnSchemaV2 &src_column,
|
|
const share::schema::ObColumnSchemaV2 &dst_column,
|
|
bool &is_change_column_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_change_column_type = src_column.get_data_type() != dst_column.get_data_type();
|
|
LOG_INFO("check is change column type", K(is_change_column_type), K(src_column), K(dst_column));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_column_in_index(
|
|
const uint64_t column_id,
|
|
const share::schema::ObTableSchema &table_schema,
|
|
bool &is_in_index)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObArray<ObColDesc> column_ids;
|
|
is_in_index = false;
|
|
if (OB_UNLIKELY(!table_schema.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", K(ret), K(table_schema));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret));
|
|
} else if (OB_FAIL(table_schema.get_rowkey_column_ids(column_ids))) {
|
|
LOG_WARN("fail to get rowkey column ids", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < column_ids.count() && !is_in_index; ++i) {
|
|
if (column_id == column_ids.at(i).col_id_) {
|
|
is_in_index = true;
|
|
LOG_WARN("column in data table rowkey columns", K(column_id));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_in_index) {
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count() && !is_in_index; ++i) {
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id), "table id", simple_index_infos.at(i).table_id_);
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("index table schema must not be NULL", K(ret));
|
|
} else {
|
|
column_ids.reuse();
|
|
if (OB_FAIL(index_table_schema->get_column_ids(column_ids))) {
|
|
LOG_WARN("fail to get column ids", K(ret));
|
|
}
|
|
for (int64_t j = 0; OB_SUCC(ret) && !is_in_index && j < column_ids.count(); ++j) {
|
|
const ObColumnSchemaV2 *column_schema = NULL;
|
|
if (column_id == column_ids.at(j).col_id_) {
|
|
is_in_index = true;
|
|
} else if (column_ids.at(j).col_id_ > common::OB_MIN_SHADOW_COLUMN_ID) {
|
|
// table schema get column_schema by shadow_column_id, the result is nullptr;
|
|
} else if (OB_ISNULL(column_schema = table_schema.get_column_schema(column_ids.at(j).col_id_))) {
|
|
ret = OB_SCHEMA_ERROR;
|
|
LOG_WARN("column schema must not be NULL", K(ret), K(column_ids.at(j)));
|
|
} else if (column_schema->is_generated_column()) {
|
|
ObArray<uint64_t> ref_column_ids;
|
|
if (OB_FAIL(column_schema->get_cascaded_column_ids(ref_column_ids))) {
|
|
LOG_WARN("fail to get cascade column ids", K(ret));
|
|
} else {
|
|
/*
|
|
If the column (namely A) is depended by a generated column, and users build an index on the generated column.
|
|
We consider the column (A) is in index.
|
|
*/
|
|
is_in_index = has_exist_in_array(ref_column_ids, column_id);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fill_new_column_attributes(
|
|
const AlterColumnSchema &alter_column_schema,
|
|
ObColumnSchemaV2 &new_column_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
const uint64_t tenant_id = new_column_schema.get_tenant_id();
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
|
|
} else {
|
|
new_column_schema.set_column_name(alter_column_schema.get_column_name_str());
|
|
new_column_schema.set_charset_type(alter_column_schema.get_charset_type());
|
|
new_column_schema.set_collation_type(alter_column_schema.get_collation_type());
|
|
new_column_schema.set_data_type(alter_column_schema.get_data_type());
|
|
new_column_schema.set_data_length(alter_column_schema.get_data_length());
|
|
new_column_schema.set_data_precision(alter_column_schema.get_data_precision());
|
|
new_column_schema.set_data_scale(alter_column_schema.get_data_scale());
|
|
if (!is_oracle_mode() || alter_column_schema.is_set_default_) {
|
|
new_column_schema.set_cur_default_value(
|
|
alter_column_schema.get_cur_default_value(),
|
|
alter_column_schema.is_default_expr_v2_column());
|
|
}
|
|
new_column_schema.set_zero_fill(alter_column_schema.is_zero_fill());
|
|
new_column_schema.set_is_hidden(alter_column_schema.is_hidden());
|
|
new_column_schema.set_nullable(alter_column_schema.is_nullable());
|
|
new_column_schema.set_autoincrement(alter_column_schema.is_autoincrement());
|
|
new_column_schema.set_column_flags(alter_column_schema.get_column_flags());
|
|
new_column_schema.set_comment(alter_column_schema.get_comment_str());
|
|
new_column_schema.set_schema_version(new_schema_version);
|
|
new_column_schema.set_on_update_current_timestamp(
|
|
alter_column_schema.is_on_update_current_timestamp());
|
|
new_column_schema.set_extended_type_info(alter_column_schema.get_extended_type_info());
|
|
new_column_schema.set_srs_id(alter_column_schema.get_srs_id());
|
|
new_column_schema.set_skip_index_attr(alter_column_schema.get_skip_index_attr().get_packed_value());
|
|
new_column_schema.set_lob_chunk_size(alter_column_schema.get_lob_chunk_size());
|
|
if (OB_FAIL(new_column_schema.get_local_session_var().deep_copy(alter_column_schema.get_local_session_var()))) {
|
|
LOG_WARN("deep copy local session vars failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_modify_column_when_upgrade(
|
|
const share::schema::ObColumnSchemaV2 &new_column,
|
|
const share::schema::ObColumnSchemaV2 &orig_column)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (obrpc::OB_UPGRADE_STAGE_DBUPGRADE != GCTX.get_upgrade_stage()) {
|
|
// do nothing
|
|
} else {
|
|
ObColumnSchemaV2 tmp_column;
|
|
if (OB_FAIL(tmp_column.assign(new_column))) {
|
|
LOG_WARN("assign failed", K(ret), K(new_column));
|
|
} else {
|
|
tmp_column.set_schema_version(orig_column.get_schema_version());
|
|
tmp_column.set_data_length(orig_column.get_data_length());
|
|
tmp_column.set_data_precision(orig_column.get_data_precision());
|
|
tmp_column.set_data_scale(orig_column.get_data_scale());
|
|
if (tmp_column != orig_column) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("can only modify column's length", K(ret), K(new_column), K(orig_column));
|
|
} else if (new_column.get_data_length() < orig_column.get_data_length()
|
|
|| new_column.get_data_precision() < orig_column.get_data_precision()
|
|
|| new_column.get_data_scale() < orig_column.get_data_scale()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("can only increase column's length", K(ret), K(new_column), K(orig_column));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
int ObDDLService::alter_shadow_column_for_index(
|
|
const ObArray<ObTableSchema> &idx_schema_array,
|
|
const AlterColumnSchema *alter_column_schema,
|
|
const ObColumnSchemaV2 &new_column_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_ISNULL(alter_column_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), KP(alter_column_schema));
|
|
} else if (!new_column_schema.is_rowkey_column()) {
|
|
// column is not rowkey column, need not update
|
|
} else {
|
|
const ObColumnSchemaV2 *origin_shadow_column_schema = nullptr;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < idx_schema_array.count(); ++i) {
|
|
const ObTableSchema& idx_table_schema = idx_schema_array.at(i);
|
|
if (idx_table_schema.get_shadow_rowkey_column_num() > 0) {
|
|
const ObColumnSchemaV2 *origin_shadow_column_schema = nullptr;
|
|
if (OB_ISNULL(origin_shadow_column_schema = idx_table_schema.get_column_schema(alter_column_schema->get_column_id() + common::OB_MIN_SHADOW_COLUMN_ID))) {
|
|
ret = OB_ENTRY_NOT_EXIST;
|
|
LOG_WARN("origin_shadow_column not exist", K(ret), KPC(alter_column_schema), K(idx_table_schema));
|
|
} else if (!origin_shadow_column_schema->is_rowkey_column()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("origin_shadow_column_schema is not rowkey column", K(ret), K(idx_table_schema), KPC(origin_shadow_column_schema));
|
|
} else {
|
|
SMART_VAR(ObColumnSchemaV2, new_aux_column_schema) {
|
|
new_aux_column_schema.reset();
|
|
if (OB_FAIL(new_aux_column_schema.assign(*origin_shadow_column_schema))){
|
|
LOG_WARN("fail to assgin new_aux_column_schema", K(ret), KPC(origin_shadow_column_schema));
|
|
} else if (OB_FAIL(fill_new_column_attributes(*alter_column_schema, new_aux_column_schema))) {
|
|
LOG_WARN("failed to fill new column attributes", K(ret), KPC(alter_column_schema), K(new_aux_column_schema));
|
|
} else if (OB_FAIL(ObIndexBuilderUtil::set_shadow_column_info(origin_shadow_column_schema->get_column_name(), origin_shadow_column_schema->get_column_id(), new_aux_column_schema))) {
|
|
LOG_WARN("fail to set shadow_column_info", K(ret), K(new_aux_column_schema), K(origin_shadow_column_schema->get_column_name()));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column(trans,
|
|
idx_table_schema,
|
|
idx_table_schema,
|
|
new_aux_column_schema))) {
|
|
LOG_WARN("schema service update aux column failed", K(ret), K(idx_table_schema), K(new_aux_column_schema));
|
|
} else if (OB_FAIL(ddl_operator.sync_aux_schema_version_for_history(
|
|
trans,
|
|
idx_table_schema))) {
|
|
LOG_WARN("fail to update aux schema version for update column", K(ret), K(idx_table_schema));
|
|
}
|
|
} // end SMART_VAR
|
|
}
|
|
}
|
|
} // end for
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::check_new_column_for_index(
|
|
ObIArray<ObTableSchema> &idx_schemas,
|
|
const ObColumnSchemaV2 &new_column_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int idx_cnt = idx_schemas.count();
|
|
ObTableSchema *index_table_schema = NULL;
|
|
ObColumnSchemaV2 copy_index_column_schema;
|
|
bool for_view = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < idx_cnt; ++i) {
|
|
index_table_schema = &idx_schemas.at(i);
|
|
if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
RS_LOG(WARN, "table schema should not be null", K(ret));
|
|
} else {
|
|
const ObColumnSchemaV2 *origin_idx_column_schema =
|
|
index_table_schema->get_column_schema(new_column_schema.get_column_id());
|
|
if (NULL == origin_idx_column_schema) {
|
|
RS_LOG(INFO, "index table do not contain this column",
|
|
"column_name", new_column_schema.get_column_name_str(),
|
|
"index_table", index_table_schema->get_table_name_str());
|
|
continue;
|
|
} else if (!origin_idx_column_schema->is_rowkey_column()) {
|
|
RS_LOG(INFO, "ingore not rowkey column",
|
|
"column_name", new_column_schema.get_column_name_str(),
|
|
"index_table", index_table_schema->get_table_name_str());
|
|
} else {
|
|
copy_index_column_schema.reset();
|
|
if (OB_FAIL(copy_index_column_schema.assign(new_column_schema))) {
|
|
LOG_WARN("fail to assign column schema", KR(ret), K(new_column_schema));
|
|
} else {
|
|
copy_index_column_schema.set_rowkey_position(origin_idx_column_schema->get_rowkey_position());
|
|
copy_index_column_schema.set_index_position(origin_idx_column_schema->get_index_position());
|
|
copy_index_column_schema.set_tbl_part_key_pos(origin_idx_column_schema->get_tbl_part_key_pos());
|
|
if (OB_FAIL(index_table_schema->alter_column(copy_index_column_schema,
|
|
ObTableSchema::CHECK_MODE_ONLINE,
|
|
for_view))) {
|
|
RS_LOG(WARN, "failed to alter index column schema", K(copy_index_column_schema), K(ret));
|
|
} else if (!index_table_schema->is_valid()) {
|
|
ret = OB_SCHEMA_ERROR;
|
|
RS_LOG(WARN, "idx table schema is invalid!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// wrapper for alter column effects
|
|
// if column is in
|
|
// 1. update index if modified column is in index
|
|
// 2. update materialized view if modified column is in materialized view
|
|
// but 2 is disabled for now
|
|
int ObDDLService::alter_table_update_index_and_view_column(
|
|
const ObTableSchema &new_table_schema,
|
|
const ObColumnSchemaV2 &new_column_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
const ObIArray<ObTableSchema> *global_idx_schema_array/*=NULL*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(alter_table_update_aux_column(new_table_schema, new_column_schema,
|
|
ddl_operator, trans, USER_INDEX, global_idx_schema_array))) {
|
|
LOG_WARN("fail to update index column", K(ret), K(new_table_schema), K(new_column_schema));
|
|
} else if (OB_FAIL(alter_table_update_aux_column(new_table_schema, new_column_schema,
|
|
ddl_operator, trans, AUX_VERTIAL_PARTITION_TABLE))) {
|
|
LOG_WARN("fail to update aux vp column", K(ret), K(new_table_schema), K(new_column_schema));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// aux schema column
|
|
int ObDDLService::alter_table_update_aux_column(
|
|
const ObTableSchema &new_table_schema,
|
|
const ObColumnSchemaV2 &new_column_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
const ObTableType table_type,
|
|
const ObIArray<ObTableSchema> *global_idx_schema_array/*=NULL*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
//update column in aux table
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObColumnSchemaV2 new_aux_column_schema;
|
|
const bool is_index = USER_INDEX == table_type;
|
|
ObSEArray<uint64_t, 16> aux_vp_tid_array;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
const uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
|
|
if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
RS_LOG(WARN, "get schema guard failed", K(ret));
|
|
} else if (is_index && OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
} else if (!is_index && OB_FAIL(new_table_schema.get_aux_vp_tid_array(aux_vp_tid_array))) {
|
|
LOG_WARN("get_aux_tid_array failed", K(ret), K(is_index));
|
|
} else {
|
|
//update all aux table schema
|
|
const ObTableSchema *aux_table_schema = NULL;
|
|
int64_t N = is_index ? simple_index_infos.count() : aux_vp_tid_array.count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
|
|
aux_table_schema = NULL;
|
|
if (is_index && OB_NOT_NULL(global_idx_schema_array) && !global_idx_schema_array->empty()) {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < global_idx_schema_array->count(); ++j) {
|
|
if (simple_index_infos.at(i).table_id_ == global_idx_schema_array->at(j).get_table_id()) {
|
|
aux_table_schema = &(global_idx_schema_array->at(j));
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
uint64_t tid = is_index ? simple_index_infos.at(i).table_id_ : aux_vp_tid_array.at(i);
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(aux_table_schema)
|
|
&& OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, tid, aux_table_schema))) {
|
|
RS_LOG(WARN, "get_table_schema failed", K(ret), K(tenant_id), K(tid));
|
|
} else if (OB_ISNULL(aux_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
RS_LOG(WARN, "aux schema should not be null", K(ret));
|
|
} else if (aux_table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("aux table is in recyclebin", K(ret));
|
|
} else {
|
|
const ObColumnSchemaV2 *origin_column_schema =
|
|
aux_table_schema->get_column_schema(new_column_schema.get_column_id());
|
|
if (NULL != origin_column_schema) {
|
|
// exist such column in aux schema
|
|
if (OB_FAIL(new_aux_column_schema.assign(new_column_schema))) {
|
|
LOG_WARN("fail to assign column", KR(ret), K(new_column_schema));
|
|
} else {
|
|
new_aux_column_schema.set_table_id(aux_table_schema->get_table_id());
|
|
new_aux_column_schema.set_autoincrement(false);
|
|
//save the rowkey postion and aux postion
|
|
if (is_index) {
|
|
new_aux_column_schema.set_rowkey_position(origin_column_schema->get_rowkey_position());
|
|
new_aux_column_schema.set_index_position(origin_column_schema->get_index_position());
|
|
new_aux_column_schema.set_tbl_part_key_pos(origin_column_schema->get_tbl_part_key_pos());
|
|
ObIndexBuilderUtil::del_column_flags_and_default_value(new_aux_column_schema);
|
|
}
|
|
if (!is_index) {
|
|
// VP column of primary table need not update.
|
|
new_aux_column_schema.set_column_flags(AUX_VP_COLUMN_FLAG);
|
|
}
|
|
}
|
|
//will only update some attribute, not include rowkey postion or aux position
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ddl_operator.update_single_column(trans,
|
|
*aux_table_schema,
|
|
*aux_table_schema,
|
|
new_aux_column_schema))) {
|
|
RS_LOG(WARN, "schema service update aux column failed failed",
|
|
"table schema", *aux_table_schema, K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column_group(trans, *aux_table_schema, new_aux_column_schema))) {
|
|
RS_LOG(WARN, "fail to update column group schema name ", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.sync_aux_schema_version_for_history(
|
|
trans,
|
|
*aux_table_schema))) {
|
|
RS_LOG(WARN, "fail to update aux schema version for update column");
|
|
}
|
|
}
|
|
}
|
|
} // end of for
|
|
} // end of else
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_sequence_in_alter_column(const ObTableSchema &table_schema,
|
|
ObColumnSchemaV2 &column_schema,
|
|
ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObSequenceDDLArg &sequence_ddl_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// stmt type check add for modify from identity column to normal column and it's still a identity column
|
|
if (column_schema.is_identity_column() && sequence_ddl_arg.get_stmt_type() != common::OB_INVALID_ID) {
|
|
ObSequenceDDLProxy ddl_operator(*schema_service_);
|
|
ObSequenceSchema sequence_schema = sequence_ddl_arg.sequence_schema();
|
|
char temp_sequence_name[OB_MAX_SEQUENCE_NAME_LENGTH + 1] = { 0 };
|
|
int32_t len = snprintf(temp_sequence_name, sizeof(temp_sequence_name), "%s%lu%c%lu",
|
|
"ISEQ$$_",
|
|
ObSchemaUtils::get_extract_schema_id(column_schema.get_tenant_id(), column_schema.get_table_id()),
|
|
'_',
|
|
column_schema.get_column_id());
|
|
if (OB_UNLIKELY(len < 0)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("create sequence name fail", K(ret), K(column_schema));
|
|
} else {
|
|
ObString sequence_name = ObString::make_string(temp_sequence_name);
|
|
sequence_schema.set_database_id(table_schema.get_database_id());
|
|
sequence_schema.set_sequence_name(sequence_name);
|
|
if (OB_FAIL(ddl_operator.alter_sequence(sequence_schema,
|
|
sequence_ddl_arg.option_bitset_,
|
|
trans,
|
|
schema_guard,
|
|
NULL,
|
|
FROM_TABLE_DDL))) {
|
|
LOG_WARN("alter sequence fail", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::redistribute_column_ids(
|
|
ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
hash::ObHashMap<uint64_t, uint64_t> column_id_map;
|
|
if (OB_FAIL(column_id_map.create(OB_MAX_COLUMN_NUMBER / 2, lib::ObLabel("DDLSrvTmp")))) {
|
|
LOG_WARN("failed to create column id map", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.generate_new_column_id_map(column_id_map))) {
|
|
LOG_WARN("failed to generate new column id map", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.convert_column_ids_for_ddl(column_id_map))) {
|
|
LOG_WARN("failed to convert new table schema column id", K(ret));
|
|
} else {
|
|
// do nothing
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::delete_constraint_update_new_table(
|
|
const AlterTableSchema &alter_table_schema,
|
|
ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// remove constraint from new table schema caused by drop column.
|
|
for (ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin();
|
|
OB_SUCC(ret) && iter != alter_table_schema.constraint_end(); iter ++) {
|
|
if (OB_FAIL(new_table_schema.delete_constraint((*iter)->get_constraint_name_str()))) {
|
|
LOG_WARN("fail to remove constraint from new table schema", K(ret), K(*iter));
|
|
} else {/* do nothing */}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// if all rls policies are column level and all sec columns of each policy will be droppped, then
|
|
// the rls object flag should be removed
|
|
int ObDDLService::update_new_table_rls_flag(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const common::ObIArray<int64_t> &drop_cols_id_arr,
|
|
ObTableSchema &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (!table_schema.has_table_flag(CASCADE_RLS_OBJECT_FLAG) ||
|
|
!table_schema.get_rls_group_ids().empty() ||
|
|
!table_schema.get_rls_context_ids().empty() ||
|
|
0 == drop_cols_id_arr.count()) {
|
|
// do nothing
|
|
} else {
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const ObRlsPolicySchema *policy = NULL;
|
|
const int64_t policy_count = table_schema.get_rls_policy_ids().count();
|
|
bool found = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && !found && i < policy_count; ++i) {
|
|
uint64_t policy_id = table_schema.get_rls_policy_ids().at(i);
|
|
if (OB_FAIL(schema_guard.get_rls_policy_schema_by_id(tenant_id, policy_id, policy))) {
|
|
LOG_WARN("failed to get rls policy schema", K(ret));
|
|
} else if (OB_ISNULL(policy)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get null rls policy schema", KR(ret), K(policy_id), K(table_schema));
|
|
} else if (!policy->is_column_level_policy()) {
|
|
found = true;
|
|
}
|
|
for (int64_t j = 0; OB_SUCC(ret) && !found && j < policy->get_sec_column_count(); ++j) {
|
|
const ObRlsSecColumnSchema* sec_column = policy->get_sec_column_by_idx(j);
|
|
bool found_col = false;
|
|
if (OB_ISNULL(sec_column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rls column is null", KR(ret));
|
|
}
|
|
for (int64_t k = 0; OB_SUCC(ret) && !found_col && k < drop_cols_id_arr.count(); ++k) {
|
|
if (sec_column->get_column_id() == drop_cols_id_arr.at(k)) {
|
|
found_col = true;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !found_col) {
|
|
found = true;
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !found) {
|
|
table_schema.del_table_flag(CASCADE_RLS_OBJECT_FLAG);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_udt_hidden_columns(const ObTableSchema &origin_table_schema, ObTableSchema &new_table_schema,
|
|
const ObColumnSchemaV2 &new_origin_col, int64_t new_schema_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get oracle mode", K(ret));
|
|
} else if (is_oracle_mode && new_origin_col.is_xmltype()) {
|
|
ObSEArray<ObColumnSchemaV2 *, 1> hidden_cols;
|
|
if (OB_FAIL(new_table_schema.get_column_schema_in_same_col_group(new_origin_col.get_column_id(),
|
|
new_origin_col.get_udt_set_id(),
|
|
hidden_cols))) {
|
|
LOG_WARN("failed to get column schema", K(ret));
|
|
} else {
|
|
for (int i = 0; i < hidden_cols.count() && OB_SUCC(ret); i++) {
|
|
ObColumnSchemaV2 *next_col = new_table_schema.get_column_schema_by_prev_next_id(hidden_cols.at(i)->get_next_column_id());
|
|
if (OB_ISNULL(next_col)) {
|
|
// do nothing since local_column is tail column
|
|
} else {
|
|
next_col->set_prev_column_id(hidden_cols.at(i)->get_prev_column_id());
|
|
next_col->set_schema_version(new_schema_version);
|
|
}
|
|
if (OB_FAIL(new_table_schema.delete_column(hidden_cols.at(i)->get_column_name_str()))) {
|
|
LOG_WARN("fail to delete column", K(ret), K(hidden_cols.at(i)->get_column_name_str()));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_column_update_new_table(
|
|
const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
const ObColumnSchemaV2 &orig_column_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = origin_table_schema.get_tenant_id();
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
ObColumnSchemaV2 *new_origin_col = new_table_schema.get_column_schema(orig_column_schema.get_column_name());
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else if (OB_ISNULL(new_origin_col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get column from new table schema", K(ret));
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
|
|
} else {
|
|
ObColumnSchemaV2 *next_col = new_table_schema.get_column_schema_by_prev_next_id(new_origin_col->get_next_column_id());
|
|
if (OB_ISNULL(next_col)) {
|
|
// do nothing since local_column is tail column
|
|
} else {
|
|
next_col->set_prev_column_id(new_origin_col->get_prev_column_id());
|
|
next_col->set_schema_version(new_schema_version);
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(new_table_schema.delete_column(new_origin_col->get_column_name_str()))) {
|
|
LOG_WARN("fail to delete column", K(ret), K(new_origin_col->get_column_name_str()));
|
|
} else if (OB_FAIL(drop_udt_hidden_columns(origin_table_schema, new_table_schema, *new_origin_col, new_schema_version))) {
|
|
LOG_WARN("fail to delete udt hidden column", K(ret), K(new_origin_col->get_column_name_str()));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// update relevant inner table if all of schema_guard, ddl_operator and trans are not null
|
|
int ObDDLService::add_new_column_to_table_schema(
|
|
const ObTableSchema &origin_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap,
|
|
const common::ObString &nls_formats,
|
|
sql::ObLocalSessionVar &local_session_var,
|
|
obrpc::ObSequenceDDLArg &sequence_ddl_arg,
|
|
common::ObIAllocator &allocator,
|
|
ObTableSchema &new_table_schema,
|
|
AlterColumnSchema &alter_column_schema,
|
|
ObIArray<ObString> &gen_col_expr_arr,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
uint64_t &curr_udt_set_id,
|
|
ObDDLOperator *ddl_operator,
|
|
common::ObMySQLTransaction *trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObSQLMode sql_mode = alter_table_schema.get_sql_mode();
|
|
const bool update_inner_table = nullptr != ddl_operator && nullptr != trans;
|
|
bool is_oracle_mode = false;
|
|
bool is_contain_part_key = false;
|
|
LOG_DEBUG("check before alter table column", K(origin_table_schema), K(alter_table_schema), K(new_table_schema));
|
|
if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get oracle mode", K(ret));
|
|
} else if (OB_ISNULL(tz_info_wrap.get_time_zone_info())
|
|
|| OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret));
|
|
}
|
|
// fill column collation
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(fill_column_collation(sql_mode,
|
|
is_oracle_mode,
|
|
new_table_schema,
|
|
allocator,
|
|
alter_column_schema))) {
|
|
LOG_WARN("failed to fill column collation", K(ret));
|
|
} else {
|
|
int64_t max_used_column_id = new_table_schema.get_max_used_column_id();
|
|
const uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
if (is_inner_table(new_table_schema.get_table_id())
|
|
&& (OB_INVALID_ID == alter_column_schema.get_column_id()
|
|
|| alter_column_schema.get_column_id() != max_used_column_id + 1)) {
|
|
// 225 is barrier version, after this adding column in system table need specify column_id
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("inner table should add column at last and specify column_id",
|
|
K(ret), K(alter_column_schema), K(max_used_column_id));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "inner table add column without column_id");
|
|
} else {
|
|
if (alter_column_schema.is_udt_hidden_column()) {
|
|
// udt hidden column
|
|
char col_name[OB_MAX_COLUMN_NAME_LENGTH] = {0};
|
|
alter_column_schema.set_udt_set_id(curr_udt_set_id);
|
|
databuff_printf(col_name, OB_MAX_COLUMN_NAME_LENGTH, "SYS_NC%05lu$",max_used_column_id + 1);
|
|
if (OB_FAIL(alter_column_schema.set_column_name(col_name))) {
|
|
SQL_RESV_LOG(WARN, "failed to set column name", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
alter_column_schema.set_column_id(++max_used_column_id);
|
|
alter_column_schema.set_rowkey_position(0);
|
|
alter_column_schema.set_index_position(0);
|
|
alter_column_schema.set_not_part_key();
|
|
alter_column_schema.set_table_id(new_table_schema.get_table_id());
|
|
alter_column_schema.set_tenant_id(new_table_schema.get_tenant_id());
|
|
if (new_table_schema.is_primary_vp_table()) {
|
|
// The last column add in the primary VP
|
|
alter_column_schema.add_column_flag(PRIMARY_VP_COLUMN_FLAG);
|
|
}
|
|
if (alter_column_schema.is_xmltype()) {
|
|
alter_column_schema.set_udt_set_id(alter_column_schema.get_column_id());
|
|
curr_udt_set_id = alter_column_schema.get_udt_set_id();
|
|
}
|
|
new_table_schema.set_max_used_column_id(max_used_column_id);
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(refill_columns_id_for_check_constraint(origin_table_schema,
|
|
alter_table_schema,
|
|
alter_column_schema,
|
|
is_oracle_mode,
|
|
allocator))) {
|
|
LOG_WARN("fail to refill columns id for check constraint", K(ret));
|
|
} else if (is_oracle_mode
|
|
&& OB_FAIL(refill_columns_id_for_not_null_constraint(alter_table_schema,
|
|
alter_column_schema))) {
|
|
LOG_WARN("fail to refill column id to constraints", K(ret));
|
|
} else if (OB_FAIL(update_prev_id_for_add_column(origin_table_schema,
|
|
new_table_schema, alter_column_schema, ddl_operator, trans))) {
|
|
LOG_WARN("failed to update prev id", K(ret));
|
|
} else if (update_inner_table) {
|
|
if (OB_FAIL(ddl_operator->create_sequence_in_add_column(new_table_schema,
|
|
alter_column_schema, *trans, schema_guard, sequence_ddl_arg))) {
|
|
LOG_WARN("alter table add identity column fail", K(alter_column_schema), K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObSchemaChecker schema_checker;
|
|
if (OB_FAIL(schema_checker.init(schema_guard))) {
|
|
LOG_WARN("failed to init schema guard", K(ret));
|
|
} else if (alter_column_schema.is_udt_related_column(is_oracle_mode)) {
|
|
// udt column/oracle gis not need to do the flowing else ifs:
|
|
// 1. default values is check and calculated in resolver, only check dependency version on RS
|
|
// 2. udt column and it's hidden columns cannot be primary key
|
|
LOG_INFO("alter table add udt related column", K(alter_column_schema));
|
|
} else if (OB_FAIL(ObDDLResolver::check_default_value(
|
|
alter_column_schema.get_cur_default_value(),
|
|
tz_info_wrap, &nls_formats, &local_session_var, allocator,
|
|
new_table_schema,
|
|
alter_column_schema,
|
|
gen_col_expr_arr,
|
|
alter_table_schema.get_sql_mode(),
|
|
false, /* allow_sequence */
|
|
&schema_checker))) {
|
|
LOG_WARN("fail to check default value", K(alter_column_schema), K(ret));
|
|
} else if (OB_FAIL(resolve_orig_default_value(alter_column_schema,
|
|
tz_info_wrap,
|
|
&nls_formats,
|
|
allocator))) {
|
|
LOG_WARN("fail to resolve default value", K(ret));
|
|
} else if (alter_column_schema.is_primary_key_) {
|
|
if (new_table_schema.get_rowkey_column_num() > 0) {
|
|
if (new_table_schema.is_heap_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to add primary key!", K(ret));
|
|
} else {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
const ObColumnSchemaV2 *mem_col = NULL;
|
|
if (OB_FAIL(new_table_schema.add_column(alter_column_schema))) {
|
|
if (OB_ERR_COLUMN_DUPLICATE == ret) {
|
|
const ObString &column_name = alter_column_schema.get_column_name_str();
|
|
LOG_USER_ERROR(OB_ERR_COLUMN_DUPLICATE, column_name.length(), column_name.ptr());
|
|
LOG_WARN("duplicate column name", K(column_name), K(ret));
|
|
}
|
|
LOG_WARN("failed to add new column", K(ret));
|
|
} else if (OB_FAIL(resolve_timestamp_column(&alter_column_schema,
|
|
new_table_schema,
|
|
alter_column_schema,
|
|
tz_info_wrap,
|
|
&nls_formats,
|
|
allocator))) {
|
|
LOG_WARN("fail to resolve timestamp column", K(ret));
|
|
} else if (OB_FAIL(deal_default_value_padding(alter_column_schema, allocator))) {
|
|
LOG_WARN("fail to deal default value padding", K(alter_column_schema), K(ret));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
LOG_WARN("fail to check primary key cover partition column", K(ret));
|
|
} else if (OB_ISNULL(mem_col = new_table_schema.get_column_schema(
|
|
alter_column_schema.get_column_id()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mem_col is NULL", K(ret));
|
|
} else {
|
|
alter_column_schema.set_prev_column_id(mem_col->get_prev_column_id());
|
|
if (update_inner_table) {
|
|
if (OB_FAIL(ddl_operator->insert_single_column(*trans,
|
|
new_table_schema, alter_column_schema))) {
|
|
LOG_WARN("failed to add column", K(ret), K(alter_column_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_column_to_column_group(
|
|
const share::schema::ObTableSchema &origin_table_schema,
|
|
const share::schema::AlterTableSchema &alter_table_schema,
|
|
share::schema::ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t cur_column_group_id = origin_table_schema.get_max_used_column_group_id();
|
|
ObArray<uint64_t> column_ids;
|
|
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 = nullptr;
|
|
|
|
if (!origin_table_schema.is_valid() ) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(origin_table_schema), K(alter_table_schema));
|
|
} else if (!new_table_schema.is_column_store_supported()) {
|
|
/* skip*/
|
|
} else {
|
|
for(; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else if (alter_column_schema->alter_type_ == OB_DDL_ADD_COLUMN) {
|
|
const ObColumnSchemaV2 *column_schema = nullptr;
|
|
if (OB_ISNULL(column_schema = new_table_schema.get_column_schema(alter_column_schema->get_column_name_str()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null column schema", K(ret), KPC(alter_column_schema), K(new_table_schema));
|
|
} else if (column_schema->is_virtual_generated_column()) {
|
|
// skip virtual column
|
|
} else if (OB_FAIL(column_ids.push_back(column_schema->get_column_id()))) {
|
|
LOG_WARN("fali to push back column id", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
/* skip do nothing*/
|
|
} else if (column_ids.count() == 0){
|
|
/* do not add column, skip */
|
|
} else {
|
|
bool is_all_cg_exist = false;
|
|
bool is_each_cg_exist = false;
|
|
if (OB_FAIL(new_table_schema.is_column_group_exist(OB_ALL_COLUMN_GROUP_NAME, is_all_cg_exist))) {
|
|
LOG_WARN("fail to check whether all cg exist", K(ret), K(new_table_schema));
|
|
} else if (OB_FAIL(new_table_schema.is_column_group_exist(OB_EACH_COLUMN_GROUP_NAME, is_each_cg_exist))) {
|
|
LOG_WARN("fail to check whether each cg exist", K(ret), K(new_table_schema));
|
|
}
|
|
|
|
/* update info about each column group*/
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_each_cg_exist) {
|
|
HEAP_VAR(ObTableSchema, tmp_table) {
|
|
if (OB_FAIL(tmp_table.assign(new_table_schema))) {
|
|
LOG_WARN("fail to assign", K(ret), K(new_table_schema), K(tmp_table));
|
|
}
|
|
tmp_table.reset_column_group_info();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < column_ids.count(); i++) {
|
|
ObColumnGroupSchema cg_schema;
|
|
if (OB_FAIL(ObSchemaUtils::build_single_column_group(new_table_schema,
|
|
new_table_schema.get_column_schema(column_ids.at(i)),
|
|
new_table_schema.get_tenant_id(),
|
|
++cur_column_group_id,
|
|
cg_schema))) {
|
|
LOG_WARN("fail to build single column group", K(ret), K(new_table_schema), K(column_ids.at(i)));
|
|
} else if (OB_FAIL(new_table_schema.add_column_group(cg_schema))) {
|
|
LOG_WARN("fail to add new column group schema to table", K(ret), K(cg_schema));
|
|
} else if (OB_FAIL(tmp_table.add_column_group(cg_schema))) {
|
|
LOG_WARN("fail to add new column group schema to tmp_cg", K(ret), K(tmp_table), K(cg_schema));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (tmp_table.get_column_group_count() == 0){
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column_group array should not be empty", K(ret), K(tmp_table));
|
|
} else if (OB_FAIL(ddl_operator.insert_column_groups(trans, tmp_table))) {
|
|
LOG_WARN("fail to insert new table_schema to each column gorup", K(ret), K(tmp_table));
|
|
}
|
|
}
|
|
}
|
|
/* update info about all column group*/
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_all_cg_exist) {
|
|
ObColumnGroupSchema* all_cg = nullptr;
|
|
if (OB_FAIL(new_table_schema.get_column_group_by_name(OB_ALL_COLUMN_GROUP_NAME, all_cg))) {
|
|
LOG_WARN("fail to get all column group", K(ret), K(new_table_schema));
|
|
} else if (OB_ISNULL(all_cg)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column group should not be null", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < column_ids.count(); i++) {
|
|
if (OB_FAIL(all_cg->add_column_id(column_ids.at(i)))) {
|
|
LOG_WARN("fail to add column id", K(ret), K(new_table_schema), K(column_ids.at(i)));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)){
|
|
} else if (column_ids.count() == 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column_ids should not be empty", K(ret), K(column_ids));
|
|
} else if (OB_FAIL(ddl_operator.insert_column_ids_into_column_group(trans, new_table_schema, column_ids, *all_cg))) {
|
|
LOG_WARN("fail to insert column ids into inner table", K(ret), K(new_table_schema),K(column_ids));
|
|
}
|
|
}
|
|
|
|
/* update info about default column group*/
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!is_all_cg_exist && !is_each_cg_exist) {
|
|
ObColumnGroupSchema *default_cg = nullptr;
|
|
if (OB_FAIL(new_table_schema.get_column_group_by_name(OB_DEFAULT_COLUMN_GROUP_NAME, default_cg))) {
|
|
LOG_WARN("fail get default column group", K(ret), K(new_table_schema));
|
|
} else if (OB_ISNULL(default_cg)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column group should not be null", K(ret), K(new_table_schema));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < column_ids.count(); i++) {
|
|
if (OB_FAIL(default_cg->add_column_id(column_ids.at(i)))) {
|
|
LOG_WARN("fail to add column id", K(ret), K(new_table_schema), K(column_ids.at(i)));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)){
|
|
} else if (column_ids.count() == 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column_ids should not be empty", K(ret), K(column_ids));
|
|
} else if (OB_FAIL(ddl_operator.insert_column_ids_into_column_group(trans, new_table_schema, column_ids, *default_cg))) {
|
|
LOG_WARN("fail to insert column ids into inner table", K(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_alter_column_new_table_schema_offline(
|
|
const ObTableSchema &origin_table_schema,
|
|
AlterTableSchema &alter_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
bool &need_redistribute_column_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
const common::ObString *nls_formats = alter_table_arg.nls_formats_;
|
|
common::ObIAllocator &allocator = alter_table_arg.allocator_;
|
|
need_redistribute_column_id = false;
|
|
bool is_contain_part_key = false;
|
|
// drop column related.
|
|
int64_t new_table_cols_cnt = 0;
|
|
ObArray<int64_t> drop_cols_id_arr;
|
|
uint64_t curr_udt_set_id = 0;
|
|
bool is_oracle_mode = false;
|
|
LOG_DEBUG("check before alter table column", K(origin_table_schema), K(alter_table_schema), K(new_table_schema));
|
|
ObSchemaChecker schema_checker;
|
|
if (OB_ISNULL(tz_info_wrap.get_time_zone_info())
|
|
|| OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tz_info_wrap", K(ret), K(tz_info_wrap));
|
|
} else if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
RS_LOG(WARN, "failed to get oracle mode", K(ret));
|
|
} else if (OB_ISNULL(nls_formats)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid nls_formats", K(ret));
|
|
} else if (OB_FAIL(schema_checker.init(schema_guard))) {
|
|
LOG_WARN("init schema checker failed", K(ret));
|
|
} else {
|
|
AlterColumnSchema *alter_column_schema;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> update_column_name_set;
|
|
lib::Worker::CompatMode compat_mode = (is_oracle_mode ?
|
|
lib::Worker::CompatMode::ORACLE : lib::Worker::CompatMode::MYSQL);
|
|
lib::CompatModeGuard tmpCompatModeGuard(compat_mode);
|
|
ObSEArray<ObString, 4> gen_col_expr_arr;
|
|
if (OB_FAIL(update_column_name_set.create(32))) {
|
|
LOG_WARN("failed to create update column name set", K(ret));
|
|
} else if (OB_FAIL(get_all_dropped_column_ids(alter_table_arg,
|
|
origin_table_schema,
|
|
drop_cols_id_arr,
|
|
&new_table_cols_cnt/*final columns count of new table*/))) {
|
|
LOG_WARN("fail to prefetch all drop columns id", K(ret), K(alter_table_arg));
|
|
} else if (drop_cols_id_arr.size() > 0) {
|
|
if (OB_FAIL(check_drop_column_with_drop_foreign_key(alter_table_arg,
|
|
origin_table_schema,
|
|
drop_cols_id_arr))) {
|
|
LOG_WARN("fail to check drop foreign key caused by drop column", K(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(check_drop_column_with_drop_constraint(alter_table_arg,
|
|
schema_guard,
|
|
origin_table_schema,
|
|
drop_cols_id_arr))) {
|
|
LOG_WARN("fail to check drop constraint caused by drop column", K(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(delete_constraint_update_new_table(alter_table_schema,
|
|
new_table_schema))) {
|
|
LOG_WARN("fail to delete constraints from new table", K(ret));
|
|
} else if (OB_FAIL(update_new_table_rls_flag(schema_guard,
|
|
drop_cols_id_arr,
|
|
new_table_schema))) {
|
|
LOG_WARN("fail to update new table flags", K(ret));
|
|
} else {/* do nothing. */}
|
|
}
|
|
bool is_rename_first = false;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(check_rename_first(alter_table_schema, origin_table_schema, is_oracle_mode,
|
|
is_rename_first))) {
|
|
LOG_WARN("check rename first failed", K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && is_rename_first) {
|
|
if (OB_FAIL(pre_rename_mysql_columns_offline(
|
|
origin_table_schema, alter_table_schema, is_oracle_mode, alter_table_arg,
|
|
new_table_schema, schema_checker, schema_guard, update_column_name_set,
|
|
need_redistribute_column_id, is_contain_part_key))) {
|
|
LOG_WARN("pre rename columns failed", K(ret));
|
|
}
|
|
}
|
|
share::schema::ObTableSchema::const_column_iterator iter = origin_table_schema.column_begin();
|
|
share::schema::ObTableSchema::const_column_iterator end = origin_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && iter != end; ++iter) {
|
|
const share::schema::ObColumnSchemaV2 *column = *iter;
|
|
if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid column schema", K(column));
|
|
} else if (column->is_generated_column()) {
|
|
const common::ObObj* ObObjtmp = &column->get_cur_default_value();
|
|
if (OB_FAIL(gen_col_expr_arr.push_back(ObObjtmp->get_string()))) {
|
|
LOG_WARN("fail to push back ObSEArray gen_col_expr_arr", K(ret));
|
|
}
|
|
}
|
|
}
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
// column that has been add, alter, change or modify
|
|
const ObColumnSchemaV2 *orig_column_schema = NULL;
|
|
const ObSchemaOperationType op_type = alter_column_schema->alter_type_;
|
|
switch (op_type) {
|
|
case OB_DDL_DROP_COLUMN: {
|
|
if (OB_FAIL(drop_column_offline(origin_table_schema, new_table_schema,
|
|
schema_guard, orig_column_name,
|
|
new_table_cols_cnt))) {
|
|
LOG_WARN("drop column offline failed", K(ret));
|
|
} else {
|
|
need_redistribute_column_id = true;
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_ADD_COLUMN: {
|
|
if (OB_FAIL(add_new_column_to_table_schema(origin_table_schema,
|
|
alter_table_schema,
|
|
tz_info_wrap,
|
|
*nls_formats,
|
|
alter_table_arg.local_session_var_,
|
|
alter_table_arg.sequence_ddl_arg_,
|
|
alter_table_arg.allocator_,
|
|
new_table_schema,
|
|
*alter_column_schema,
|
|
gen_col_expr_arr,
|
|
schema_guard,
|
|
curr_udt_set_id,
|
|
nullptr,
|
|
nullptr))) {
|
|
LOG_WARN("failed to add new column to table schema", K(ret));
|
|
} else {
|
|
ObSEArray<ObString, 1> new_pk_column;
|
|
if (alter_column_schema->is_primary_key_) {
|
|
if (!new_table_schema.is_heap_table()) {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret));
|
|
} else if (OB_FAIL(new_pk_column.push_back(alter_column_schema->get_column_name_str()))){
|
|
LOG_WARN("failed to push back pk col name", K(ret));
|
|
} else if (OB_FAIL(add_primary_key(new_pk_column, new_table_schema))) {
|
|
LOG_WARN("failed to add pk to table", K(ret), K(new_pk_column), K(new_table_schema));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
LOG_WARN("failed to check primary key cover partition column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else {
|
|
const ObString &column_name = alter_column_schema->get_column_name_str();
|
|
ObColumnNameHashWrapper column_key(column_name);
|
|
if (!is_oracle_mode) {
|
|
// mysql mode
|
|
// do not check updated column name for adding column
|
|
// otherwise following alter stmt will report error:
|
|
//
|
|
// CREATE TABLE t1 (col1 varchar(50), col2 varchar(50),c5 varchar(50),c3 varchar(50) ,c4 varchar(50) primary key,c8 int);
|
|
// ALTER TABLE t1 add c5 int,drop c3,rename column c5 to c3;
|
|
//
|
|
// do nothing
|
|
} else if (OB_HASH_EXIST == update_column_name_set.exist_refactored(column_key)) {
|
|
ret = OB_HASH_EXIST;
|
|
LOG_WARN("duplicate column name", K(ret), K(column_name));
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(column_key))) {
|
|
LOG_WARN("failed to add column name to hash set.", K(column_name), K(ret));
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else {
|
|
need_redistribute_column_id = true;
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_CHANGE_COLUMN:
|
|
case OB_DDL_MODIFY_COLUMN: {
|
|
if (is_rename_first && alter_column_schema->alter_type_ == OB_DDL_CHANGE_COLUMN
|
|
&& is_rename_column(*alter_column_schema)) {
|
|
// do nothing
|
|
} else {
|
|
ObColumnSchemaV2 new_column_schema;
|
|
ObSEArray<ObString, 1> new_pk_column;
|
|
bool is_change_column_order = false;
|
|
orig_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
if (OB_FAIL(prepare_change_modify_column_offline(
|
|
*alter_column_schema, origin_table_schema,
|
|
alter_table_schema, is_oracle_mode,
|
|
alter_table_arg, new_table_schema,
|
|
schema_checker, schema_guard,
|
|
update_column_name_set,
|
|
new_column_schema,
|
|
is_contain_part_key))) {
|
|
LOG_WARN("prepare new column schema failed", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.alter_column(
|
|
new_column_schema, ObTableSchema::CHECK_MODE_OFFLINE, false))) {
|
|
LOG_WARN("alter column failed", K(ret));
|
|
} else if (alter_column_schema->is_primary_key_) {
|
|
if (OB_FAIL(new_pk_column.push_back(alter_column_schema->get_column_name_str()))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!new_pk_column.empty() && OB_FAIL(add_primary_key(new_pk_column, new_table_schema))) {
|
|
LOG_WARN("failed to add pk to table", K(ret), K(new_pk_column), K(new_table_schema));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
RS_LOG(WARN, "fail to check primary key cover partition column", K(ret));
|
|
} else {
|
|
if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) {
|
|
ret = OB_HASH_EXIST;
|
|
RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name));
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) {
|
|
RS_LOG(WARN, "failed to add index_name to hash set.", K(orig_column_name), K(ret));
|
|
} else if (OB_FAIL(check_is_change_column_order(
|
|
new_table_schema, *alter_column_schema, is_change_column_order))) {
|
|
LOG_WARN("failed to check is change column order", K(ret));
|
|
} else if (is_change_column_order) {
|
|
if (alter_column_schema->get_column_id() != orig_column_schema->get_column_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid alter column schema column id", K(ret), K(*alter_column_schema),
|
|
K(*orig_column_schema));
|
|
} else if (OB_FAIL(new_table_schema.reorder_column(
|
|
new_column_schema.get_column_name_str(),
|
|
alter_column_schema->is_first_,
|
|
alter_column_schema->get_prev_column_name(),
|
|
alter_column_schema->get_next_column_name()))) {
|
|
LOG_WARN("failed to reorder column", K(ret));
|
|
} else {
|
|
need_redistribute_column_id = true;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_ALTER_COLUMN: {
|
|
ObSchemaChecker schema_checker;
|
|
orig_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
if (OB_FAIL(schema_checker.init(schema_guard))) {
|
|
LOG_WARN("failed to init schema guard", K(ret));
|
|
} else if (OB_ISNULL(orig_column_schema)) {
|
|
ret = OB_ERR_BAD_FIELD_ERROR;
|
|
LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(),
|
|
origin_table_schema.get_table_name_str().length(), origin_table_schema.get_table_name_str().ptr());
|
|
LOG_WARN("unknown column", KR(ret), K(orig_column_name), K(new_table_schema));
|
|
} else if (OB_FAIL(pre_check_orig_column_schema(*alter_column_schema,
|
|
origin_table_schema,
|
|
update_column_name_set))) {
|
|
RS_LOG(WARN, "failed to pre check orig column schema", K(ret));
|
|
}
|
|
//column that has been modified, can't not modify again
|
|
if (OB_SUCC(ret)) {
|
|
ObColumnSchemaV2 new_column_schema;
|
|
bool for_view = false;
|
|
if (OB_FAIL(new_column_schema.assign(*orig_column_schema))) {
|
|
LOG_WARN("fail to assign column schema", KR(ret));
|
|
} else if (OB_FAIL(resolve_timestamp_column(alter_column_schema,
|
|
new_table_schema,
|
|
new_column_schema,
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
allocator))) {
|
|
RS_LOG(WARN, "fail to resolve timestamp column", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.alter_column(new_column_schema,
|
|
ObTableSchema::CHECK_MODE_OFFLINE,
|
|
for_view))) {
|
|
RS_LOG(WARN, "failed to change column", K(ret));
|
|
} else {
|
|
ObObj default_value;
|
|
if (alter_column_schema->is_drop_default_) {
|
|
default_value.set_null();
|
|
if (OB_FAIL(new_column_schema.set_cur_default_value(default_value, false))) {
|
|
RS_LOG(WARN, "failed to set current default value", K(ret), K(default_value));
|
|
} else {
|
|
new_column_schema.del_column_flag(DEFAULT_EXPR_V2_COLUMN_FLAG);
|
|
}
|
|
} else {
|
|
default_value = alter_column_schema->get_cur_default_value();
|
|
bool is_default_expr_v2 = alter_column_schema->is_default_expr_v2_column();
|
|
if (!default_value.is_null() && ob_is_text_tc(new_column_schema.get_data_type())) {
|
|
ret = OB_INVALID_DEFAULT;
|
|
LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(),
|
|
new_column_schema.get_column_name_str().ptr());
|
|
RS_LOG(WARN, "BLOB, TEXT column can't have a default value!", K(default_value), K(ret));
|
|
} else if (ob_is_json_tc(new_column_schema.get_data_type())
|
|
|| ob_is_geometry_tc(new_column_schema.get_data_type())) {
|
|
// cannot alter json column to any default value
|
|
// text column also cannot be alter to null in mysql
|
|
ret = OB_ERR_BLOB_CANT_HAVE_DEFAULT;
|
|
LOG_USER_ERROR(OB_ERR_BLOB_CANT_HAVE_DEFAULT, new_column_schema.get_column_name_str().length(),
|
|
new_column_schema.get_column_name_str().ptr());
|
|
RS_LOG(WARN, "JSON column can't have a default value!", K(default_value), K(ret));
|
|
} else if (!new_column_schema.is_nullable() && default_value.is_null()) {
|
|
ret = OB_INVALID_DEFAULT;
|
|
LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(),
|
|
new_column_schema.get_column_name_str().ptr());
|
|
RS_LOG(WARN, "not null column with default value null!", K(ret));
|
|
} else if (OB_FAIL(new_column_schema.set_cur_default_value(default_value,
|
|
is_default_expr_v2))) {
|
|
RS_LOG(WARN, "failed to set current default value", K(ret), K(default_value), K(is_default_expr_v2));
|
|
// The check_default_value function not only verifies the
|
|
// default value but also performs type conversion on it.
|
|
// Therefore, the cur_default_value from the column_schema
|
|
// should be passed when calling this function.
|
|
} else if (OB_FAIL(ObDDLResolver::check_default_value(new_column_schema.get_cur_default_value(),
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
&alter_table_arg.local_session_var_,
|
|
allocator,
|
|
new_table_schema,
|
|
new_column_schema,
|
|
gen_col_expr_arr,
|
|
alter_table_schema.get_sql_mode(),
|
|
!alter_column_schema->is_generated_column(), /* allow_sequence */
|
|
&schema_checker))) {
|
|
LOG_WARN("fail to check default value", KPC(alter_column_schema),K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_table_schema.alter_column(new_column_schema,
|
|
ObTableSchema::CHECK_MODE_OFFLINE,
|
|
for_view))) {
|
|
RS_LOG(WARN, "failed to change column", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
RS_LOG(WARN, "failed to check primary key cover partition column", K(ret));
|
|
} else {
|
|
if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) {
|
|
ret = OB_HASH_EXIST;
|
|
RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name));
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) {
|
|
RS_LOG(WARN, "failed to add index_name to hash set.",
|
|
K(orig_column_name), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
RS_LOG(WARN, "invalid offline ddl operator type!", K_(alter_column_schema->alter_type));
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && OB_FAIL(check_has_multi_autoinc(new_table_schema))) {
|
|
LOG_WARN("failed to check table has multi autoinc", K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && need_redistribute_column_id) {
|
|
if (OB_FAIL(redistribute_column_ids(new_table_schema))) {
|
|
LOG_WARN("failed to redistribute column ids", K(ret));
|
|
} else {
|
|
// do nothing
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_table_schema.sort_column_array_by_column_id())) {
|
|
LOG_WARN("failed to sort column", K(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// update relevant inner table if both ddl_operator and trans are not null
|
|
int ObDDLService::update_prev_id_for_add_column(const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
AlterColumnSchema &alter_column_schema,
|
|
ObDDLOperator *ddl_operator,
|
|
common::ObMySQLTransaction *trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = origin_table_schema.get_tenant_id();
|
|
const bool is_first = alter_column_schema.is_first_;
|
|
const bool is_after = (!alter_column_schema.get_prev_column_name().empty());
|
|
const bool is_before = (!alter_column_schema.get_next_column_name().empty());
|
|
const bool is_last = !(is_first || is_after || is_before);
|
|
const bool update_inner_table = nullptr != ddl_operator && nullptr != trans;
|
|
if (is_last) {
|
|
// do nothing
|
|
} else {
|
|
ObString pos_column_name;
|
|
const uint64_t alter_column_id = alter_column_schema.get_column_id();
|
|
if (is_first) {
|
|
// this first means the first of no hidden/shdow column.
|
|
ObColumnIterByPrevNextID iter(new_table_schema);
|
|
const ObColumnSchemaV2 *head_col = NULL;
|
|
const ObColumnSchemaV2 *col = NULL;
|
|
bool is_first = false;
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(col))) {
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("The column is null", K(ret));
|
|
} else if (col->is_shadow_column() || col->is_hidden()) {
|
|
// do nothing
|
|
} else if (!is_first) {
|
|
head_col = col;
|
|
is_first = true;
|
|
}
|
|
}
|
|
if (ret != OB_ITER_END) {
|
|
LOG_WARN("Failed to iterate all table columns. iter quit. ", K(ret));
|
|
} else {
|
|
ret = OB_SUCCESS;
|
|
if (OB_ISNULL(head_col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Failed to get first column", K(ret));
|
|
} else {
|
|
alter_column_schema.set_next_column_name(head_col->get_column_name());
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
pos_column_name = (is_after ? alter_column_schema.get_prev_column_name()
|
|
: alter_column_schema.get_next_column_name());
|
|
ObColumnSchemaV2 *pos_column_schema = new_table_schema.get_column_schema(pos_column_name);
|
|
ObColumnSchemaV2 *update_column_schema = NULL;
|
|
if (OB_ISNULL(pos_column_schema)) {
|
|
ret = OB_ERR_BAD_FIELD_ERROR;
|
|
LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, pos_column_name.length(), pos_column_name.ptr(),
|
|
new_table_schema.get_table_name_str().length(),
|
|
new_table_schema.get_table_name_str().ptr());
|
|
LOG_WARN("pos column is NULL", K(pos_column_name));
|
|
} else {
|
|
if (is_after) {
|
|
// add column after
|
|
alter_column_schema.set_prev_column_id(pos_column_schema->get_column_id());
|
|
update_column_schema = new_table_schema.get_column_schema_by_prev_next_id(pos_column_schema->get_next_column_id());
|
|
if (OB_NOT_NULL(update_column_schema)) {
|
|
update_column_schema->set_prev_column_id(alter_column_id);
|
|
}
|
|
} else {
|
|
// add column before / first
|
|
alter_column_schema.set_prev_column_id(pos_column_schema->get_prev_column_id());
|
|
update_column_schema = pos_column_schema;
|
|
update_column_schema->set_prev_column_id(alter_column_id);
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_ISNULL(update_column_schema)) {
|
|
// alter column is the last column
|
|
} else if (update_inner_table) {
|
|
if (OB_FAIL(ddl_operator->update_single_column(
|
|
*trans,
|
|
origin_table_schema,
|
|
new_table_schema,
|
|
*update_column_schema))) {
|
|
LOG_WARN("Failed to update single column", K(ret), K(update_column_schema->get_column_name_str()));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_table_column(const ObTableSchema &origin_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_data_version,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
ObIArray<ObTableSchema> *global_idx_schema_array/*=NULL*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
const common::ObString *nls_formats = alter_table_arg.nls_formats_;
|
|
common::ObIAllocator &allocator = alter_table_arg.allocator_;
|
|
bool is_oracle_mode = false;
|
|
bool for_view = false;
|
|
LOG_INFO("check before alter table column", K(origin_table_schema),
|
|
K(alter_table_schema), K(new_table_schema));
|
|
ObSchemaChecker schema_checker;
|
|
if (OB_FAIL(schema_checker.init(schema_guard))) {
|
|
LOG_WARN("failed to init schema guard", K(ret));
|
|
} else if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
RS_LOG(WARN, "failed to get oracle mode", K(ret));
|
|
} else if (OB_ISNULL(tz_info_wrap.get_time_zone_info())
|
|
|| OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret));
|
|
} else if (OB_ISNULL(nls_formats)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid nls_formats", K(ret));
|
|
} else {
|
|
AlterColumnSchema *alter_column_schema;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
lib::Worker::CompatMode compat_mode = (is_oracle_mode ?
|
|
lib::Worker::CompatMode::ORACLE : lib::Worker::CompatMode::MYSQL);
|
|
lib::CompatModeGuard tmpCompatModeGuard(compat_mode);
|
|
ObArray<ObTableSchema> idx_schema_array;
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> update_column_name_set;
|
|
ObSEArray<ObString, 4> gen_col_expr_arr;
|
|
uint64_t curr_udt_set_id = 0;
|
|
bool is_origin_table_has_lob_column = false;
|
|
if (OB_FAIL(update_column_name_set.create(32))) {
|
|
LOG_WARN("failed to create update column name set", K(ret));
|
|
} else if (OB_FAIL(generate_tmp_idx_schemas(new_table_schema, idx_schema_array, schema_guard))) {
|
|
LOG_WARN("generate tmp idx schemas failed", K(ret));
|
|
} else {
|
|
share::schema::ObTableSchema::const_column_iterator iter = origin_table_schema.column_begin();
|
|
share::schema::ObTableSchema::const_column_iterator end = origin_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && iter != end; ++iter) {
|
|
const share::schema::ObColumnSchemaV2 *column = *iter;
|
|
if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid column schema", K(column));
|
|
} else if (column->is_generated_column()) {
|
|
const common::ObObj* ObObjtmp = &column->get_cur_default_value();
|
|
if (OB_FAIL(gen_col_expr_arr.push_back(ObObjtmp->get_string()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && is_lob_storage(column->get_data_type())) {
|
|
is_origin_table_has_lob_column = true;
|
|
}
|
|
}
|
|
}
|
|
bool is_rename_first = false;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(check_rename_first(alter_table_schema, origin_table_schema, is_oracle_mode,
|
|
is_rename_first))) {
|
|
LOG_WARN("check rename first failed", K(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && is_rename_first) {
|
|
if (OB_FAIL(pre_rename_mysql_columns_online(
|
|
origin_table_schema, alter_table_schema, is_oracle_mode, new_table_schema,
|
|
alter_table_arg, schema_checker, ddl_operator, update_column_name_set, trans,
|
|
schema_guard, idx_schema_array, global_idx_schema_array))) {
|
|
LOG_WARN("failed to pre alter change columns", K(ret));
|
|
}
|
|
}
|
|
|
|
// Extended type info is resolved in session collation type, then we convert it to
|
|
// system collation in ObDDLResolver::fill_extended_type_info().
|
|
bool is_all_column_exactly_same_type = true;
|
|
const ObCollationType cur_extended_type_info_collation = ObCharset::get_system_collation();
|
|
for(;OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
//cnolumn that has been alter, change or modify
|
|
const ObColumnSchemaV2 *orig_column_schema = NULL;
|
|
ObColumnSchemaV2 new_column_schema;
|
|
switch (alter_column_schema->alter_type_) {
|
|
case OB_DDL_ADD_COLUMN: {
|
|
if (OB_FAIL(add_new_column_to_table_schema(origin_table_schema,
|
|
alter_table_schema,
|
|
tz_info_wrap,
|
|
*nls_formats,
|
|
alter_table_arg.local_session_var_,
|
|
alter_table_arg.sequence_ddl_arg_,
|
|
alter_table_arg.allocator_,
|
|
new_table_schema,
|
|
*alter_column_schema,
|
|
gen_col_expr_arr,
|
|
schema_guard,
|
|
curr_udt_set_id,
|
|
&ddl_operator,
|
|
&trans))) {
|
|
LOG_WARN("failed to add new column to table schema", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_CHANGE_COLUMN: {
|
|
if (is_rename_first && is_rename_column(*alter_column_schema)) { break; }
|
|
orig_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
if (OB_FAIL(prepare_change_modify_column_online(
|
|
*alter_column_schema, origin_table_schema, alter_table_schema,
|
|
is_oracle_mode, alter_table_arg, new_table_schema, schema_checker,
|
|
ddl_operator, trans, schema_guard, global_idx_schema_array,
|
|
update_column_name_set, new_column_schema))) {
|
|
LOG_WARN("prepare alter column failed", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.alter_column(
|
|
new_column_schema, ObTableSchema::CHECK_MODE_ONLINE, for_view))) {
|
|
LOG_WARN("failed to alter column", K(ret));
|
|
} else if (OB_FAIL(check_new_column_for_index(idx_schema_array, new_column_schema))) {
|
|
RS_LOG(WARN, "failed to check new column for index", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
RS_LOG(WARN, "fail to check primary key cover partition column", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column(
|
|
trans, origin_table_schema, new_table_schema, new_column_schema))) {
|
|
RS_LOG(WARN, "failed to alter column", K(alter_column_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column_group(trans, origin_table_schema, new_column_schema))) {
|
|
RS_LOG(WARN, "failed to update column group", K(ret));
|
|
} else if (OB_FAIL(alter_shadow_column_for_index(idx_schema_array, alter_column_schema, new_column_schema, ddl_operator, trans))) {
|
|
RS_LOG(WARN, "failed to alter shadow column for index", K(ret));
|
|
} else if (OB_FAIL(alter_table_update_index_and_view_column(
|
|
new_table_schema,
|
|
new_column_schema,
|
|
ddl_operator,
|
|
trans,
|
|
global_idx_schema_array))) {
|
|
RS_LOG(WARN, "failed to update index column", K(ret));
|
|
} else {
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
if (OB_FAIL(update_column_name_set.exist_refactored(orig_column_key))) {
|
|
if (OB_HASH_EXIST == ret) {
|
|
RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name));
|
|
} else if (OB_HASH_NOT_EXIST == ret) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
RS_LOG(WARN, "exist refactored failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) {
|
|
RS_LOG(WARN, "failed to add index_name to hash set.", K(orig_column_name), K(ret));
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_MODIFY_COLUMN: {
|
|
LOG_DEBUG("check alter column schema", KPC(alter_column_schema));
|
|
orig_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
if (alter_column_schema->is_generated_column()
|
|
&& OB_FAIL(ObDDLResolver::reformat_generated_column_expr(
|
|
alter_column_schema->get_cur_default_value(),
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
orig_column_schema->get_local_session_var(),
|
|
allocator,
|
|
new_table_schema,
|
|
*alter_column_schema,
|
|
alter_table_schema.get_sql_mode(),
|
|
&schema_checker))) {
|
|
LOG_WARN("fail to check default value", KPC(alter_column_schema), K(ret));
|
|
} else if (OB_FAIL(pre_check_orig_column_schema(*alter_column_schema,
|
|
origin_table_schema,
|
|
update_column_name_set))) {
|
|
RS_LOG(WARN, "failed to pre check orig column schema", K(ret));
|
|
} else if (!alter_column_schema->is_generated_column() /* Not support modify to generate columns, so there is no need to check again here */
|
|
&& !alter_column_schema->is_udt_related_column(is_oracle_mode) /* udt default values are checked in resolver */
|
|
&& OB_FAIL(ObDDLResolver::check_default_value(alter_column_schema->get_cur_default_value(),
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
NULL,
|
|
allocator,
|
|
new_table_schema,
|
|
*alter_column_schema,
|
|
gen_col_expr_arr,
|
|
alter_table_schema.get_sql_mode(),
|
|
!alter_column_schema->is_generated_column(), /* allow_sequence */
|
|
&schema_checker))) {
|
|
LOG_WARN("fail to check default value", KPC(alter_column_schema), K(ret));
|
|
} else if (OB_FAIL(alter_sequence_in_alter_column(new_table_schema,
|
|
*alter_column_schema,
|
|
trans,
|
|
schema_guard,
|
|
alter_table_arg.sequence_ddl_arg_))) {
|
|
LOG_WARN("alter table modeify identity column fail", K(alter_column_schema), K(ret));
|
|
} else if (alter_column_schema->is_primary_key_) {
|
|
if (new_table_schema.get_rowkey_column_num() > 0) {
|
|
if (new_table_schema.is_heap_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
RS_LOG(WARN, "not support to add primary key!", K(ret));
|
|
} else {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
RS_LOG(WARN, "multiple primary key defined", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_column_schema->is_autoincrement_) {
|
|
if (alter_column_schema->is_autoincrement()) {
|
|
if (orig_column_schema->get_column_id() != new_table_schema.get_autoinc_column_id()) {
|
|
// not supported now; from non-auto-increment column to auto-increment column
|
|
ret = OB_NOT_SUPPORTED;
|
|
RS_LOG(WARN, "from non-auto-increment column to auto-increment column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(check_can_alter_column_type(*orig_column_schema, *alter_column_schema, origin_table_schema, is_oracle_mode))) {
|
|
LOG_WARN("fail to check can alter column type", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_column_schema.assign(*orig_column_schema))) {
|
|
LOG_WARN("fail to assign column schema", KR(ret));
|
|
} else if (OB_FAIL(fill_new_column_attributes(*alter_column_schema,
|
|
new_column_schema))) {
|
|
RS_LOG(WARN, "fail to fill new column attributes", K(ret));
|
|
} else if (OB_FAIL(check_modify_column_when_upgrade(new_column_schema, *orig_column_schema))) {
|
|
LOG_WARN("fail to check modify column when upgrade",
|
|
K(ret), K(new_column_schema), K(*orig_column_schema));
|
|
} else if (OB_FAIL(resolve_timestamp_column(alter_column_schema,
|
|
new_table_schema,
|
|
new_column_schema,
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
allocator))) {
|
|
RS_LOG(WARN, "fail to resolve timestamp column", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.alter_column(new_column_schema,
|
|
ObTableSchema::CHECK_MODE_ONLINE,
|
|
for_view))) {
|
|
RS_LOG(WARN, "failed to change column", K(ret));
|
|
} else if (OB_FAIL(check_new_column_for_index(
|
|
idx_schema_array,
|
|
new_column_schema))) {
|
|
RS_LOG(WARN, "failed to check new column for index", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
RS_LOG(WARN, "fail to check primary key cover partition column", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column(
|
|
trans,
|
|
origin_table_schema,
|
|
new_table_schema,
|
|
new_column_schema))) {
|
|
RS_LOG(WARN, "failed to alter column", K(alter_column_schema), K(ret));
|
|
} else if (OB_FAIL(alter_shadow_column_for_index(idx_schema_array, alter_column_schema, new_column_schema, ddl_operator, trans))) {
|
|
RS_LOG(WARN, "failed to alter shadow column for index", K(ret));
|
|
} else if (OB_FAIL(alter_table_update_index_and_view_column(new_table_schema,
|
|
new_column_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
RS_LOG(WARN, "failed to update index column", K(ret));
|
|
} else {
|
|
if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) {
|
|
ret = OB_HASH_EXIST;
|
|
RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name));
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) {
|
|
RS_LOG(WARN, "failed to add index_name to hash set.",
|
|
K(orig_column_name), K(ret));
|
|
}
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_ALTER_COLUMN: {
|
|
orig_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
if (OB_FAIL(pre_check_orig_column_schema(*alter_column_schema,
|
|
origin_table_schema,
|
|
update_column_name_set))) {
|
|
RS_LOG(WARN, "failed to pre check orig column schema", K(ret));
|
|
}
|
|
|
|
//column that has been modified, can't not modify again
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_column_schema.assign(*orig_column_schema))) {
|
|
LOG_WARN("fail to assign column schema", KR(ret));
|
|
} else if (OB_FAIL(resolve_timestamp_column(alter_column_schema,
|
|
new_table_schema,
|
|
new_column_schema,
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
allocator))) {
|
|
RS_LOG(WARN, "fail to resolve timestamp column", K(ret));
|
|
} else {
|
|
ObObj default_value;
|
|
if (alter_column_schema->is_drop_default_) {
|
|
default_value.set_null();
|
|
if (OB_FAIL(new_column_schema.set_cur_default_value(default_value, false))) {
|
|
RS_LOG(WARN, "failed to set current default value", K(ret), K(default_value));
|
|
} else {
|
|
new_column_schema.del_column_flag(DEFAULT_EXPR_V2_COLUMN_FLAG);
|
|
}
|
|
} else {
|
|
bool is_default_expr_v2 = alter_column_schema->is_default_expr_v2_column();
|
|
default_value = alter_column_schema->get_cur_default_value();
|
|
if (!default_value.is_null() && ob_is_text_tc(new_column_schema.get_data_type())) {
|
|
ret = OB_INVALID_DEFAULT;
|
|
LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(),
|
|
new_column_schema.get_column_name_str().ptr());
|
|
RS_LOG(WARN, "BLOB, TEXT column can't have a default value!", K(default_value), K(ret));
|
|
} else if (ob_is_json_tc(new_column_schema.get_data_type())
|
|
|| ob_is_geometry_tc(new_column_schema.get_data_type())) {
|
|
// cannot alter json column to any default value
|
|
// text column also cannot be alter to null in mysql
|
|
ret = OB_ERR_BLOB_CANT_HAVE_DEFAULT;
|
|
LOG_USER_ERROR(OB_ERR_BLOB_CANT_HAVE_DEFAULT, new_column_schema.get_column_name_str().length(),
|
|
new_column_schema.get_column_name_str().ptr());
|
|
RS_LOG(WARN, "JSON column can't have a default value!", K(default_value), K(ret));
|
|
} else if (!new_column_schema.is_nullable() && default_value.is_null()) {
|
|
ret = OB_INVALID_DEFAULT;
|
|
LOG_USER_ERROR(OB_INVALID_DEFAULT, new_column_schema.get_column_name_str().length(),
|
|
new_column_schema.get_column_name_str().ptr());
|
|
RS_LOG(WARN, "not null column with default value null!", K(ret));
|
|
} else if (OB_FAIL(new_column_schema.set_cur_default_value(default_value,
|
|
is_default_expr_v2))) {
|
|
RS_LOG(WARN, "failed to set current default value", K(ret), K(default_value), K(is_default_expr_v2));
|
|
// The check_default_value function not only verifies the
|
|
// default value but also performs type conversion on it.
|
|
// Therefore, the cur_default_value from the column_schema
|
|
// should be passed when calling this function.
|
|
} else if (OB_FAIL(ObDDLResolver::check_default_value(new_column_schema.get_cur_default_value(),
|
|
tz_info_wrap,
|
|
nls_formats,
|
|
&alter_table_arg.local_session_var_,
|
|
allocator,
|
|
new_table_schema,
|
|
new_column_schema,
|
|
gen_col_expr_arr,
|
|
alter_table_schema.get_sql_mode(),
|
|
!alter_column_schema->is_generated_column(), /* allow_sequence */
|
|
&schema_checker))) {
|
|
LOG_WARN("fail to check default value", K(new_column_schema),K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_table_schema.alter_column(new_column_schema,
|
|
ObTableSchema::CHECK_MODE_ONLINE,
|
|
for_view))) {
|
|
RS_LOG(WARN, "failed to change column", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
RS_LOG(WARN, "failed to check primary key cover partition column", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column(trans,
|
|
origin_table_schema,
|
|
new_table_schema,
|
|
new_column_schema))) {
|
|
RS_LOG(WARN, "failed to alter column", K(alter_column_schema), K(ret));
|
|
} else if (OB_FAIL(alter_table_update_index_and_view_column(new_table_schema,
|
|
new_column_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
RS_LOG(WARN, "failed to update index column", K(ret));
|
|
} else {
|
|
if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) {
|
|
ret = OB_HASH_EXIST;
|
|
RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name));
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) {
|
|
RS_LOG(WARN, "failed to add index_name to hash set.",
|
|
K(orig_column_name), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_DROP_COLUMN: {
|
|
if (OB_FAIL(drop_column_online(origin_table_schema, new_table_schema,
|
|
orig_column_name, ddl_operator, schema_guard, trans,
|
|
update_column_name_set))) {
|
|
LOG_WARN("online drop column failed", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
RS_LOG(WARN, "unhandled operator type!", K_(alter_column_schema->alter_type));
|
|
break;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && is_all_column_exactly_same_type && OB_NOT_NULL(orig_column_schema)) {
|
|
if (OB_FAIL(ObTableSchema::check_is_exactly_same_type(*orig_column_schema, new_column_schema, is_all_column_exactly_same_type))) {
|
|
RS_LOG(WARN, "check column type exactly same failed", K(ret), KPC(orig_column_schema), K(new_column_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
bool is_add_lob = false;
|
|
if(OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(add_column_to_column_group(origin_table_schema,
|
|
alter_table_schema, new_table_schema, ddl_operator, trans))) {
|
|
LOG_WARN("fail to add_column_to_column_group", K(ret), K(alter_table_schema), K(new_table_schema));
|
|
} else if (OB_FAIL(new_table_schema.check_skip_index_valid())) {
|
|
LOG_WARN("failed to check new table schema skip index", K(ret));
|
|
} else if (!is_origin_table_has_lob_column) {
|
|
if (OB_FAIL(create_aux_lob_table_if_need(
|
|
new_table_schema, schema_guard, ddl_operator, trans,
|
|
false/*need_sync_schema_version*/, is_add_lob))) {
|
|
LOG_WARN("fail to create_aux_lob_table_if_need", K(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_add_lob) {
|
|
const bool require_strict_binary_format = share::ObDDLUtil::use_idempotent_mode(tenant_data_version) && !is_all_column_exactly_same_type;
|
|
if (OB_FAIL(ObDDLLock::lock_for_common_ddl_in_trans(new_table_schema, require_strict_binary_format, trans))) {
|
|
LOG_WARN("failed to lock ddl lock", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_aux_lob_table_if_need(ObTableSchema &data_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
const bool need_sync_schema_version,
|
|
bool &is_add_lob)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_add_lob = false;
|
|
ObArray<ObTableSchema> aux_table_schemas;
|
|
const uint64_t tenant_id = data_table_schema.get_tenant_id();
|
|
SCN frozen_scn;
|
|
uint64_t tenant_data_version = 0;
|
|
ObArray<bool> need_create_empty_majors;
|
|
|
|
if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(build_aux_lob_table_schema_if_need(data_table_schema, aux_table_schemas))) {
|
|
LOG_WARN("fail to build_aux_lob_table_schema_if_need", K(ret), K(data_table_schema));
|
|
} else if (aux_table_schemas.count() == 0) {
|
|
// no need create aux lob table, do nothing
|
|
} else if (is_sys_table(data_table_schema.get_table_id())) {
|
|
// whatever enable_sys_table_ddl is, sys table do not allow create lob aux tables by alter columns
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE];
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),"%s", "system table add or modify column");
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg);
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else {
|
|
ObTableCreator table_creator(
|
|
tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(
|
|
tenant_id,
|
|
schema_guard,
|
|
sql_proxy_);
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
is_add_lob = true;
|
|
if (OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("fail to get last schema version", KR(ret));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_add_lob_in_trans(data_table_schema, trans))) {
|
|
LOG_WARN("failed to add lock online ddl lock", K(ret));
|
|
} else if (OB_FAIL(table_creator.init(true/*need_tablet_cnt_check*/))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
}
|
|
|
|
ObSEArray<const ObTableSchema*, 2> schemas;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < aux_table_schemas.count(); i++) {
|
|
share::schema::ObTableSchema &table_schema = aux_table_schemas.at(i);
|
|
if (OB_FAIL(ddl_operator.create_table(table_schema, trans, NULL,
|
|
need_sync_schema_version && (i == aux_table_schemas.count() - 1)))) {
|
|
LOG_WARN("failed to create table schema", K(ret));
|
|
} else if (OB_FAIL(schemas.push_back(&table_schema))
|
|
|| OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("failed to push_back table schema", K(ret), K(table_schema));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ddl_operator.insert_ori_schema_version(
|
|
trans, tenant_id, table_schema.get_table_id(), last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(table_schema), K(last_schema_version));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
if(OB_FAIL(new_table_tablet_allocator.prepare_like(data_table_schema))) {
|
|
LOG_WARN("fail to prepare like", KR(ret), K(data_table_schema));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_local_aux_tables_arg(
|
|
schemas, &data_table_schema, ls_id_array, tenant_data_version,
|
|
need_create_empty_majors /*online_ddl, need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table partitions failed", KR(ret), K(last_schema_version));
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("fail to execute crate tablet", KR(ret));
|
|
}
|
|
}
|
|
// finishing is always invoked for new table tablet allocator
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) {
|
|
LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret));
|
|
}
|
|
FLOG_INFO("finish create aux lob table for data table", K(ret), K(data_table_schema));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_table_foreign_keys(const share::schema::ObTableSchema &orig_table_schema,
|
|
share::schema::ObTableSchema &inc_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(ddl_operator.add_table_foreign_keys(
|
|
orig_table_schema,
|
|
inc_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to add table foreign_keys", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_not_null_cst_in_column_flag(
|
|
const ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t col_cnt = new_table_schema.get_column_count();
|
|
|
|
common::hash::ObHashSet<uint64_t> drop_col_id;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
/* get column in drop */
|
|
int64_t hash_set_size = alter_table_schema.get_column_count() > 0 ? alter_table_schema.get_column_count() : 1;
|
|
if (OB_FAIL(drop_col_id.create(hash_set_size,
|
|
lib::ObLabel("ChkDropCol"),
|
|
lib::ObLabel("ChkDropCol")))) {
|
|
LOG_WARN("failed to init drop column set", K(ret), K(alter_table_schema));
|
|
}
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin ++) {
|
|
const AlterColumnSchema *alter_col = static_cast<AlterColumnSchema*>(*it_begin);
|
|
if (OB_ISNULL(alter_col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("alter col should not be null", K(ret), K(alter_table_schema));
|
|
} else if (OB_DDL_DROP_COLUMN != alter_col->alter_type_) {
|
|
} else {
|
|
const ObColumnSchemaV2 *orig_col = orig_table_schema.get_column_schema(alter_col->get_origin_column_name());
|
|
if (OB_ISNULL(orig_col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig col should not be null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (OB_FAIL(drop_col_id.set_refactored(orig_col->get_column_id()))) {
|
|
LOG_WARN("failed to set refactor", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
for (ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin(); OB_SUCC(ret) &&
|
|
iter != alter_table_schema.constraint_end(); iter ++) {
|
|
if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) {
|
|
const uint64_t column_id = *((*iter)->cst_col_begin());
|
|
ObColumnSchemaV2 *column = new_table_schema.get_column_schema(column_id);
|
|
if (OB_ISNULL(column)) {
|
|
/* if col in drop, skip */
|
|
if (OB_HASH_EXIST != drop_col_id.exist_refactored(column_id)) {
|
|
if (ret == OB_HASH_NOT_EXIST) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid column", K(ret), K(column_id), K(alter_table_schema));
|
|
} else {
|
|
LOG_WARN("failed to get check col in drop col list", K(ret), K(column_id));
|
|
}
|
|
}
|
|
} else if (column->has_not_null_constraint()) {
|
|
column->drop_not_null_cst();
|
|
if (OB_FAIL(ddl_operator.update_single_column(
|
|
trans, orig_table_schema, new_table_schema, *column))) {
|
|
LOG_WARN("failed to update single column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_not_null_cst_in_column_flag(
|
|
const ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
for (ObTableSchema::const_constraint_iterator iter = alter_table_schema.constraint_begin(); OB_SUCC(ret) &&
|
|
iter != alter_table_schema.constraint_end(); iter ++) {
|
|
if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) {
|
|
const uint64_t column_id = *((*iter)->cst_col_begin());
|
|
ObColumnSchemaV2 *column = new_table_schema.get_column_schema(column_id);
|
|
if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid column", K(ret));
|
|
} else {
|
|
int64_t column_flags = column->get_column_flags();
|
|
column->add_or_del_column_flag(NOT_NULL_ENABLE_FLAG, (*iter)->get_enable_flag());
|
|
column->add_or_del_column_flag(NOT_NULL_RELY_FLAG, (*iter)->get_rely_flag());
|
|
column->add_or_del_column_flag(NOT_NULL_VALIDATE_FLAG, (*iter)->is_validated());
|
|
if (column_flags != column->get_column_flags()
|
|
&& OB_FAIL(ddl_operator.update_single_column(
|
|
trans, orig_table_schema, new_table_schema, *column))) {
|
|
LOG_WARN("failed to update single column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_table_auto_increment(
|
|
const ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObAutoincrementService &auto_inc_service = ObAutoincrementService::get_instance();
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
const uint64_t table_id = orig_table_schema.get_table_id();
|
|
const uint64_t column_id = orig_table_schema.get_autoinc_column_id();
|
|
const bool is_order_mode = orig_table_schema.is_order_auto_increment_mode();
|
|
const int64_t truncate_version = orig_table_schema.get_truncate_version();
|
|
uint64_t current_auto_increment = 0;
|
|
// Step 1: Determine whether the auto-increment value needs to be increased or decreased.
|
|
// If increased, only the table schema needs to be updated. It reduces the complexity of the
|
|
// alter_table operation.
|
|
const bool is_reduced_autoinc =
|
|
(alter_table_schema.get_auto_increment() < orig_table_schema.get_auto_increment());
|
|
if (!is_reduced_autoinc && OB_FAIL(auto_inc_service.get_sequence_value(tenant_id,
|
|
table_id, column_id, is_order_mode, truncate_version, current_auto_increment))) {
|
|
LOG_WARN("fail to get sequence value", K(ret));
|
|
} else if (is_reduced_autoinc ||
|
|
(alter_table_schema.get_auto_increment() < current_auto_increment)) {
|
|
// Step 2: Query the maximum value of the auto-increment column and use it as a base value.
|
|
const ObDatabaseSchema *db_schema = nullptr;
|
|
const ObColumnSchemaV2 *column_schema = nullptr;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, orig_table_schema.get_database_id(),
|
|
db_schema))) {
|
|
LOG_WARN("failed to get database schema", K(ret), K(orig_table_schema.get_data_table_id()));
|
|
} else if (OB_ISNULL(db_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, database schema must not be nullptr", K(ret));
|
|
} else if (OB_ISNULL(column_schema = orig_table_schema.get_column_schema(column_id))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get alter column schema", K(ret), K(column_id));
|
|
} else if (OB_FAIL(lock_table(trans, orig_table_schema))) {
|
|
LOG_WARN("failed to lock table for alter auto_increment", K(ret));
|
|
} else {
|
|
uint64_t current_max_value = 0;
|
|
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
|
ObTimeoutCtx timeout_ctx;
|
|
ObSqlString sql;
|
|
sqlclient::ObMySQLResult *result = NULL;
|
|
common::ObCommonSqlProxy *user_sql_proxy = GCTX.ddl_sql_proxy_;
|
|
ObSessionParam session_param;
|
|
int64_t sql_mode = alter_table_arg.sql_mode_;
|
|
session_param.sql_mode_ = reinterpret_cast<int64_t *>(&sql_mode);
|
|
session_param.ddl_info_.set_is_ddl(true);
|
|
// if data_table_id != dest_table_id, meaning this is happening in ddl double write
|
|
session_param.ddl_info_.set_source_table_hidden(orig_table_schema.is_user_hidden_table());
|
|
ObObj obj;
|
|
const int64_t DDL_INNER_SQL_EXECUTE_TIMEOUT = ObDDLUtil::calc_inner_sql_execute_timeout();
|
|
const bool is_unsigned_type = ob_is_unsigned_type(column_schema->get_data_type());
|
|
if (OB_FAIL(timeout_ctx.set_trx_timeout_us(DDL_INNER_SQL_EXECUTE_TIMEOUT))) {
|
|
LOG_WARN("set trx timeout failed", K(ret));
|
|
} else if (OB_FAIL(timeout_ctx.set_timeout(DDL_INNER_SQL_EXECUTE_TIMEOUT))) {
|
|
LOG_WARN("set timeout failed", K(ret));
|
|
} else if (OB_FAIL(sql.assign_fmt("SELECT /*+no_rewrite*/ CAST(MAX(`%s`) AS %s) AS MAX_VALUE FROM `%s`.`%s`",
|
|
column_schema->get_column_name(),
|
|
is_unsigned_type ? "UNSIGNED" : "SIGNED",
|
|
db_schema->get_database_name(),
|
|
orig_table_schema.get_table_name()))) {
|
|
LOG_WARN("failed to assign fmt", K(ret), K(column_schema->get_column_name_str()),
|
|
K(db_schema->get_database_name_str()));
|
|
} else if (OB_FAIL(user_sql_proxy->read(res, tenant_id, sql.ptr(), &session_param))) {
|
|
LOG_WARN("fail to read", KR(ret), K(sql));
|
|
} else if (OB_ISNULL(result = res.get_result())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get result failed", K(ret));
|
|
} else if (OB_FAIL(result->next())) {
|
|
if (OB_ITER_END == ret) {
|
|
// empty table
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("fail to get next", KR(ret));
|
|
}
|
|
} else if (OB_FAIL(result->get_obj("MAX_VALUE", obj))) {
|
|
LOG_WARN("fail to get result obj", K(ret));
|
|
} else if (is_unsigned_type) {
|
|
current_max_value = obj.get_uint64();
|
|
} else {
|
|
current_max_value = MAX(0, obj.get_int());
|
|
}
|
|
}
|
|
|
|
// Step 3: Clear the auto increment cache and reset the inner table of the auto-increment
|
|
// column. all new auto_inc request will be based on the auto_increment value in the table
|
|
// schema.
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(auto_inc_service.clear_autoinc_cache_all(tenant_id, table_id,
|
|
column_id, is_order_mode,
|
|
false /* ignore_rpc_errors */))) {
|
|
LOG_WARN("fail to clear autoinc cache all", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.reinit_autoinc_row(new_table_schema, trans))) {
|
|
LOG_WARN("fail to reinit autoinc row", K(ret));
|
|
} else {
|
|
// Step 4: Update the table schema with the maximum values of auto_increment by setting and
|
|
// max_value plus 1.
|
|
const uint64_t next_value =
|
|
(current_max_value == UINT64_MAX) ? UINT64_MAX : (current_max_value + 1);
|
|
new_table_schema.set_auto_increment(MAX(next_value, new_table_schema.get_auto_increment()));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_table_constraints(const ObAlterTableArg::AlterConstraintType op_type,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &inc_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_can_alter_table_constraints(op_type, schema_guard, orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to check can alter constraints", K(ret), K(op_type), K(inc_table_schema));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (obrpc::ObAlterTableArg::ADD_CONSTRAINT == op_type) {
|
|
if (OB_FAIL(ddl_operator.add_table_constraints(inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to add table constraints", K(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == op_type) {
|
|
if (OB_FAIL(drop_not_null_cst_in_column_flag(orig_table_schema, inc_table_schema, new_table_schema, ddl_operator, trans))) {
|
|
LOG_WARN("failed to drop not null cst in column flag", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_table_constraints(orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to drop table constraints", K(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE == op_type) {
|
|
if (OB_FAIL(alter_not_null_cst_in_column_flag(orig_table_schema, inc_table_schema,
|
|
new_table_schema, ddl_operator, trans))) {
|
|
LOG_WARN("failed to drop not null cst in column flag", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.modify_check_constraints_state(
|
|
orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to drop table constraints", K(ret));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected op type", K(op_type), K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_can_alter_table_constraints(
|
|
const ObAlterTableArg::AlterConstraintType op_type,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
|
|
if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check if tenant mode is oracle mode", K(ret));
|
|
} else {
|
|
for (ObTableSchema::const_constraint_iterator iter = inc_table_schema.constraint_begin(); OB_SUCC(ret) &&
|
|
iter != inc_table_schema.constraint_end(); iter ++) {
|
|
const ObString &cst_name = (*iter)->get_constraint_name_str();
|
|
if (ObAlterTableArg::AlterConstraintType::ADD_CONSTRAINT == op_type) {
|
|
bool is_check_constraint_name_exist = true;
|
|
if (OB_FAIL(check_constraint_name_is_exist(schema_guard,
|
|
orig_table_schema,
|
|
cst_name,
|
|
false,
|
|
is_check_constraint_name_exist))) {
|
|
LOG_WARN("fail to check check constraint name is exist or not", K(ret), K(cst_name));
|
|
} else if (is_check_constraint_name_exist) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_WARN("check constraint name is duplicate", K(ret), K(cst_name));
|
|
if (!is_oracle_mode) {
|
|
LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst_name.length(), cst_name.ptr());
|
|
}
|
|
}
|
|
}
|
|
for (ObTableSchema::const_constraint_iterator iter_r = iter + 1; OB_SUCC(ret) &&
|
|
iter_r != inc_table_schema.constraint_end(); iter_r ++) {
|
|
if (is_oracle_mode && (*iter_r)->get_constraint_name_str() == cst_name) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_WARN("duplicate constraint name", K(ret), K(cst_name));
|
|
} else if (!is_oracle_mode && 0 == cst_name.case_compare((*iter_r)->get_constraint_name_str())) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst_name.length(), cst_name.ptr());
|
|
LOG_WARN("duplicate constraint name", K(ret), K(cst_name));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObConstraint* const* res = nullptr;
|
|
if (is_oracle_mode) {
|
|
res = std::find_if(orig_table_schema.constraint_begin(), orig_table_schema.constraint_end(), [&cst_name](const ObConstraint *cst)
|
|
{ return cst_name == cst->get_constraint_name_str(); });
|
|
} else {
|
|
res = std::find_if(orig_table_schema.constraint_begin(), orig_table_schema.constraint_end(), [&cst_name](const ObConstraint* cst)
|
|
{ return 0 == cst_name.case_compare(cst->get_constraint_name_str()); });
|
|
}
|
|
if (orig_table_schema.constraint_end() == res) {
|
|
if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == op_type
|
|
|| obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE == op_type) {
|
|
ret = OB_ERR_CONTRAINT_NOT_FOUND;
|
|
LOG_WARN("constraint not exist", K(ret), K(cst_name));
|
|
}
|
|
} else {
|
|
if (obrpc::ObAlterTableArg::ADD_CONSTRAINT == op_type) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
if (!is_oracle_mode) {
|
|
LOG_USER_ERROR(OB_ERR_CONSTRAINT_NAME_DUPLICATE, cst_name.length(), cst_name.ptr());
|
|
}
|
|
LOG_WARN("constraint duplicate", K(ret), K(cst_name));
|
|
} else if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == op_type
|
|
|| obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE == op_type) {
|
|
// drop or modify state
|
|
if ((*iter)->get_constraint_id() != OB_INVALID_ID) {
|
|
if ((*iter)->get_constraint_id() != (*res)->get_constraint_id()) {
|
|
ret = OB_ERR_CONTRAINT_NOT_FOUND;
|
|
LOG_WARN("constraint not exist", K(ret), K((*iter)->get_constraint_id()), K((*res)->get_constraint_id()));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (CONSTRAINT_TYPE_PRIMARY_KEY == (*res)->get_constraint_type()) {
|
|
if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == op_type) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected err", K(ret), KPC(*res), K(inc_table_schema));
|
|
} else {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("alter pk constraint state not supported now", K(ret), K((*res)->get_constraint_type()));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter state of primary key constraint");
|
|
}
|
|
} else {
|
|
const_cast<ObConstraint *>(*iter)->set_constraint_id((*res)->get_constraint_id());
|
|
const_cast<ObConstraint *>(*iter)->set_constraint_type((*res)->get_constraint_type());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_partition_by_subpart_name(const ObTableSchema &orig_table_schema,
|
|
const ObSubPartition &subpart_name,
|
|
const ObPartition *&part,
|
|
const ObSubPartition *&subpart)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (part != NULL) {
|
|
ObSubPartIterator sub_iter(orig_table_schema, *part, CHECK_PARTITION_MODE_NORMAL);
|
|
while (OB_SUCC(ret) && OB_SUCC(sub_iter.next(subpart))) {
|
|
if (OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get null subpartition", K(ret));
|
|
} else if (ObCharset::case_insensitive_equal(subpart_name.get_part_name(),
|
|
subpart->get_part_name())) {
|
|
break;
|
|
}
|
|
}
|
|
} else {
|
|
ObPartIterator iter(orig_table_schema, CHECK_PARTITION_MODE_NORMAL);
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
|
|
if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part is null", KR(ret));
|
|
} else {
|
|
ObSubPartIterator sub_iter(orig_table_schema, *part, CHECK_PARTITION_MODE_NORMAL);
|
|
subpart = NULL;
|
|
while (OB_SUCC(ret) && OB_SUCC(sub_iter.next(subpart))) {
|
|
if (OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get null subpartition", K(ret));
|
|
} else if (ObCharset::case_insensitive_equal(subpart_name.get_part_name(),
|
|
subpart->get_part_name())) {
|
|
break;
|
|
}
|
|
}
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_inc_table_schema_for_add_part(
|
|
const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// TODO: Add partition in interval partitioned table should reorganize part_idx, which will be supported in the later version.
|
|
int64_t max_part_idx = OB_INVALID_INDEX;
|
|
if (orig_table_schema.is_interval_part()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("add interval part will support later", KR(ret));
|
|
} else if (OB_FAIL(orig_table_schema.get_max_part_idx(max_part_idx, orig_table_schema.is_external_table()))) {
|
|
LOG_WARN("fail to get max part idx", KR(ret), K(orig_table_schema));
|
|
} else {
|
|
const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) {
|
|
ObPartition *inc_part = inc_part_array[i];
|
|
if (OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part is null", KR(ret), K(i));
|
|
} else {
|
|
// part_id will be generated by generate_object_id_for_partition_schema()
|
|
inc_part->set_part_idx(max_part_idx + i + 1);
|
|
if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level()) {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) {
|
|
ObSubPartition *subpart = inc_part->get_subpart_array()[j];
|
|
if (OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("subpart is null", KR(ret), K(j));
|
|
} else if (subpart->get_sub_part_idx() >= 0
|
|
&& subpart->get_sub_part_idx() != j) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("subpart_idx should be invalid", KR(ret), KPC(subpart), K(j));
|
|
} else {
|
|
// subpart_id will be generated by generate_object_id_for_partition_schema()
|
|
// For some situations(not hash like), subpart_idx from ddl_resolver is invalid.
|
|
subpart->set_sub_part_idx(j);
|
|
}
|
|
} // end for iterate inc subpart
|
|
}
|
|
}
|
|
} // end for iterate inc part
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_inc_table_schema_for_add_subpart(
|
|
const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) {
|
|
ObPartition *inc_part = inc_part_array[i];
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
ObPartIterator iter(orig_table_schema, check_partition_mode);
|
|
const ObPartition *part = NULL;
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
|
|
if (OB_ISNULL(part) || OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KP(part), KP(inc_part), KR(ret));
|
|
} else if (ObCharset::case_insensitive_equal(inc_part->get_part_name(),
|
|
part->get_part_name())) {
|
|
inc_part->set_part_id(part->get_part_id());
|
|
int64_t max_subpart_idx = OB_INVALID_INDEX;
|
|
if (OB_FAIL(part->get_max_sub_part_idx(max_subpart_idx))) {
|
|
LOG_WARN("fail to get max sub part idx", KR(ret), KPC(part));
|
|
} else {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) {
|
|
ObSubPartition *subpart = inc_part->get_subpart_array()[j];
|
|
if (OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("subpart is null", KR(ret), K(j));
|
|
} else {
|
|
// subpart_id will be generated by generate_object_id_for_partition_schema()
|
|
subpart->set_sub_part_idx(max_subpart_idx + j + 1);
|
|
}
|
|
} // end for iterate inc subpart
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
LOG_WARN("part should exists", KR(ret));
|
|
} else if (OB_FAIL(ret)) {
|
|
LOG_WARN("iter part failed", KR(ret));
|
|
}
|
|
} // end for iterate inc part
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_inc_table_schema_for_rename_part_(
|
|
const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
ObPartition *inc_part = nullptr;
|
|
const int64_t inc_part_num = inc_table_schema.get_partition_num();
|
|
if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part array is null", KR(ret), K(inc_table_schema));
|
|
} else if (OB_UNLIKELY(1 != inc_part_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rename multiple partitions at a time not support", KR(ret),K(inc_part_num));
|
|
} else if (OB_ISNULL(inc_part = inc_part_array[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part is null", KR(ret), KP(inc_part_array), K(inc_table_schema));
|
|
} else {
|
|
const ObPartition *part = nullptr;
|
|
const ObString &part_name = inc_part->get_part_name();
|
|
const ObString &new_part_name = inc_table_schema.get_new_part_name();
|
|
if (OB_FAIL(orig_table_schema.get_partition_by_name(part_name, part))) {
|
|
LOG_WARN("should check schema first or should not in this step", KR(ret), K(part_name), K(orig_table_schema));
|
|
} else if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get part failed", KR(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(inc_part->assign(*part))) {
|
|
LOG_WARN("failed to assign partition", KR(ret), KPC(part), KPC(inc_part));
|
|
} else if (OB_FAIL(inc_part->set_part_name(new_part_name))) {
|
|
LOG_WARN("failed to set inc part name", KR(ret), KPC(part), K(new_part_name));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_inc_table_schema_for_rename_subpart_(
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
share::schema::AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
ObPartition *inc_part = nullptr;
|
|
const int64_t inc_part_num = inc_table_schema.get_partition_num();
|
|
if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part array is null", KR(ret), K(inc_table_schema));
|
|
} else if (OB_UNLIKELY(1 != inc_part_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rename multiple subpartitions at a time not support", KR(ret), K(inc_part_num));
|
|
} else if (OB_ISNULL(inc_part = inc_part_array[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_part is null", KR(ret), KP(inc_part_array), K(inc_table_schema));
|
|
} else {
|
|
const ObPartition *part = nullptr;
|
|
const ObSubPartition *subpart = nullptr;
|
|
ObSubPartition **inc_subpart_array = inc_part->get_subpart_array();
|
|
ObSubPartition *inc_subpart = nullptr;
|
|
ObSubPartition inc_subpart_content;
|
|
const int64_t inc_subpart_num = inc_part->get_subpartition_num();
|
|
const ObString &new_part_name = inc_table_schema.get_new_part_name();
|
|
if (OB_ISNULL(inc_subpart_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc subpart array is null", KR(ret), K(inc_table_schema));
|
|
} else if (OB_UNLIKELY(1 != inc_subpart_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rename multiple subpartitions at a time not support", KR(ret), K(inc_subpart_num));
|
|
} else if (OB_ISNULL(inc_subpart = inc_subpart_array[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc subpart is null", KR(ret), KP(inc_subpart_array));
|
|
} else if (OB_FAIL(orig_table_schema.get_subpartition_by_name(inc_subpart->get_part_name(), part, subpart))) {
|
|
LOG_WARN("should check schema first or should not in this step", KR(ret), KP(part), KP(subpart));
|
|
} else if (OB_ISNULL(part) || OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get null part/subpart", KR(ret), KP(part), KP(subpart));
|
|
} else if (OB_FAIL(inc_subpart_content.assign(*subpart))) {
|
|
LOG_WARN("failed to assign subpartition", KR(ret), KPC(part), K(inc_subpart_content));
|
|
} else if (OB_FAIL(inc_subpart_content.set_part_name(new_part_name))) {
|
|
LOG_WARN("set inc subpart name faild", KR(ret), K(inc_subpart_content), K(new_part_name));
|
|
} else if (OB_FAIL(inc_part->ObBasePartition::assign(*part))) {
|
|
LOG_WARN("failed to assign inc part", KR(ret), KPC(inc_part), KPC(part));
|
|
} else if (OB_FAIL(inc_part->add_partition(inc_subpart_content))) {
|
|
LOG_WARN("inc_part fail to add inc_subpart", KR(ret), KPC(inc_part), K(inc_subpart_content));
|
|
} else {
|
|
//not used in rename subpartition, just for integrity
|
|
inc_part->set_sub_part_num(part->get_sub_part_num());
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_inc_table_schema_for_drop_part(
|
|
const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) {
|
|
ObPartition *inc_part = inc_part_array[i];
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
ObPartIterator iter(orig_table_schema, check_partition_mode);
|
|
const ObPartition *part = NULL;
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
|
|
if (OB_ISNULL(part) || OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KP(part), KP(inc_part), KR(ret));
|
|
} else if (ObCharset::case_insensitive_equal(inc_part->get_part_name(),
|
|
part->get_part_name())) {
|
|
if (OB_FAIL(inc_part->assign(*part))) {
|
|
LOG_WARN("failed to assign partition", KR(ret), K(part), K(inc_part));
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
LOG_WARN("part should exists", KR(ret));
|
|
} else if (OB_SUCC(ret) && orig_table_schema.is_interval_part()
|
|
&& inc_part->get_high_bound_val()
|
|
== orig_table_schema.get_transition_point()) {
|
|
// deleting the partition whose high level is transition point not support yet.
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("delete the max range part of interval part table", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "delete the max range part of interval part table");
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::gen_inc_table_schema_for_drop_subpart(const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartition *inc_part = NULL;
|
|
if (1 != inc_table_schema.get_partition_num()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("drop subparts not in a part", KR(ret));
|
|
} else if (OB_ISNULL(inc_part = inc_table_schema.get_part_array()[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_part is null", K(ret));
|
|
} else {
|
|
const ObPartition *part = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part->get_subpartition_num(); i++) {
|
|
ObSubPartition *inc_subpart = inc_part->get_subpart_array()[i];
|
|
if (OB_ISNULL(inc_subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_subpart is null", K(ret));
|
|
} else if (i == 0) {
|
|
const ObSubPartition *subpart = NULL;
|
|
if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, part, subpart))) {
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_PARTITION_NOT_EXIST);
|
|
LOG_WARN("part should exists", K(ret));
|
|
} else {
|
|
LOG_WARN("fail to get partition by subpart name", K(ret));
|
|
}
|
|
} else if (OB_ISNULL(part) || OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get null part/subpart", K(ret));
|
|
} else if (part->get_sub_part_num() <= inc_part->get_subpartition_num()) {
|
|
ret = OB_ERR_DROP_LAST_PARTITION;
|
|
LOG_WARN("cannot drop all subpartitions", K(part),
|
|
"partitions current", part->get_sub_part_num(),
|
|
"partitions to be dropped", inc_part->get_subpartition_num(),
|
|
K(ret));
|
|
LOG_USER_ERROR(OB_ERR_DROP_LAST_PARTITION);
|
|
} else if (OB_FAIL(inc_subpart->assign(*subpart))) {
|
|
LOG_WARN("failed to assign partition", K(ret), K(part), K(inc_part));
|
|
} else {
|
|
inc_part->set_part_id(part->get_part_id());
|
|
}
|
|
} else if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part is null", KR(ret));
|
|
} else {
|
|
const ObSubPartition *subpart = NULL;
|
|
if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, part, subpart))) {
|
|
if (OB_ITER_END == ret) {
|
|
part = NULL;
|
|
if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, part, subpart))) {
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_PARTITION_NOT_EXIST);
|
|
LOG_WARN("part should exists", K(ret));
|
|
} else {
|
|
LOG_WARN("fail to get partition by subpart name", K(ret));
|
|
}
|
|
} else if (OB_ISNULL(part) || OB_ISNULL(subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get null part/subpart", K(ret));
|
|
} else {
|
|
ret = OB_INVALID_PARTITION;
|
|
LOG_USER_ERROR(OB_INVALID_PARTITION);
|
|
LOG_WARN("subparts for deleted in different part", K(ret));
|
|
}
|
|
} else {
|
|
LOG_WARN("fail to get partition by subpart name", K(ret));
|
|
}
|
|
} else if (OB_FAIL(inc_subpart->assign(*subpart))) {
|
|
LOG_WARN("failed to assign partition", K(ret), K(part), K(inc_part));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_inc_table_schema_for_trun_part(
|
|
const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema,
|
|
AlterTableSchema &del_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
const int64_t del_part_num = del_table_schema.get_part_option().get_part_num();
|
|
ObPartition **del_part_array = del_table_schema.get_part_array();
|
|
if (OB_ISNULL(inc_part_array) || OB_ISNULL(del_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("partition array is null", KR(ret), K(inc_part_array), K(del_part_array), K(inc_table_schema));
|
|
} else if (inc_part_num != del_part_num) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc is not equal to del", KR(ret), K(del_table_schema), K(inc_table_schema));
|
|
}
|
|
for (int64_t i = 0; i < inc_part_num && OB_SUCC(ret); ++i) {
|
|
ObPartition *inc_part = inc_part_array[i];
|
|
ObPartition *del_part = del_part_array[i];
|
|
if (OB_ISNULL(inc_part) || OB_ISNULL(del_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("partition is null", KR(ret), K(i), K(inc_part_num));
|
|
} else {
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
ObPartIterator iter(orig_table_schema, check_partition_mode);
|
|
const ObPartition *part = NULL;
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
|
|
if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(part), KR(ret));
|
|
} else if (ObCharset::case_insensitive_equal(inc_part->get_part_name(),
|
|
part->get_part_name())) {
|
|
if (OB_FAIL(inc_part->assign(*part))) {
|
|
LOG_WARN("failed to assign partition", KR(ret), K(part), K(inc_part));
|
|
} else if (OB_FAIL(del_part->assign(*part))) {
|
|
LOG_WARN("failed to assign partition", KR(ret), K(part), K(del_part));
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
LOG_WARN("part should exists", KR(ret), KPC(inc_part));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// sql puts the subpartitions of multiple partitions that may exist
|
|
// into one partition and transfers it
|
|
// This function generates the necessary partitions
|
|
// and puts all the subpartitions in the right position
|
|
int ObDDLService::gen_inc_table_schema_for_trun_subpart(
|
|
const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema,
|
|
AlterTableSchema &del_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartition *inc_part_p = NULL;
|
|
ObPartition inc_part_set;
|
|
if (OB_ISNULL(inc_table_schema.get_part_array())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_array is null", KR(ret));
|
|
} else if (OB_ISNULL(inc_part_p = inc_table_schema.get_part_array()[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_part_p is null", KR(ret));
|
|
} else if (OB_FAIL(inc_part_set.assign(*inc_part_p))) {
|
|
LOG_WARN("failed to assign partition", KR(ret), K(inc_part_set), KPC(inc_part_p));
|
|
} else {
|
|
inc_table_schema.reset_partition_array();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_set.get_subpartition_num(); i++) {
|
|
const ObPartition *inc_part = NULL;
|
|
const ObSubPartition *orig_subpart = NULL;
|
|
const ObPartition *orig_part = NULL;
|
|
ObSubPartition *inc_subpart = inc_part_set.get_subpart_array()[i];
|
|
if (OB_ISNULL(inc_subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_subpart is null", KR(ret));
|
|
} else if (OB_FAIL(get_partition_by_subpart_name(orig_table_schema, *inc_subpart, orig_part, orig_subpart))) {
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_PARTITION_NOT_EXIST);
|
|
LOG_WARN("part should exists", KR(ret), KPC(inc_subpart));
|
|
} else {
|
|
LOG_WARN("fail to get partition by subpart name", KR(ret));
|
|
}
|
|
} else if (OB_ISNULL(orig_part) || OB_ISNULL(orig_subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get null part/subpart", KR(ret));
|
|
} else if (0 != i && OB_FAIL(get_part_by_part_id(
|
|
inc_table_schema, orig_part->get_part_id(), inc_part))) {
|
|
LOG_WARN("fail to get partition", KR(ret), K(orig_part->get_part_id()));
|
|
} else if (OB_ISNULL(inc_part)) {
|
|
ObPartition new_part;
|
|
new_part.set_part_id(orig_part->get_part_id());
|
|
if (OB_FAIL(new_part.add_partition(*orig_subpart))){
|
|
LOG_WARN("failed to add partition", K(orig_subpart), KR(ret));
|
|
} else if (OB_FAIL(inc_table_schema.add_partition(new_part))) {
|
|
LOG_WARN("failed to add partition", KR(ret));
|
|
}
|
|
} else if (OB_FAIL(const_cast<ObPartition*>(inc_part)->add_partition(*orig_subpart))) {
|
|
LOG_WARN("failed to add partition", K(orig_subpart), KR(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObPartition **part_array = inc_table_schema.get_part_array();
|
|
const int64_t part_num = inc_table_schema.get_partition_num();
|
|
inc_table_schema.set_part_num(part_num);
|
|
if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_array is NULL", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < part_num; i++) {
|
|
if (OB_ISNULL(part_array[i])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_array[i] is NULL", K(ret), K(i));
|
|
} else {
|
|
part_array[i]->set_sub_part_num(part_array[i]->get_subpartition_num());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (FAILEDx(del_table_schema.assign_partition_schema(inc_table_schema))) {
|
|
LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
const char* ObDDLService::ddl_type_str(const ObDDLType ddl_type)
|
|
{
|
|
const char *str = "";
|
|
if (DDL_CREATE_INDEX == ddl_type) {
|
|
str = "create index";
|
|
} else if (DDL_CREATE_PARTITIONED_LOCAL_INDEX == ddl_type) {
|
|
str = "create partitioned local index";
|
|
} else if (DDL_MODIFY_COLUMN == ddl_type) {
|
|
str = "alter column";
|
|
} else if (DDL_CHECK_CONSTRAINT == ddl_type) {
|
|
str = "add or modify check constraint";
|
|
} else if (DDL_FOREIGN_KEY_CONSTRAINT == ddl_type) {
|
|
str = "alter foreign key constraint";
|
|
} else if (DDL_ADD_PRIMARY_KEY == ddl_type) {
|
|
str = "add primary key";
|
|
} else if (DDL_DROP_PRIMARY_KEY == ddl_type) {
|
|
str = "drop primary key";
|
|
} else if (DDL_ALTER_PRIMARY_KEY == ddl_type) {
|
|
str = "alter primary key";
|
|
} else if (DDL_ALTER_PARTITION_BY == ddl_type) {
|
|
str = "alter partition by";
|
|
} else if (DDL_DROP_COLUMN == ddl_type) {
|
|
str = "drop column";
|
|
} else if (DDL_ADD_NOT_NULL_COLUMN == ddl_type) {
|
|
str = "add not null column";
|
|
} else if (DDL_ADD_COLUMN_OFFLINE == ddl_type) {
|
|
str = "add column offline";
|
|
} else if (DDL_ADD_COLUMN_ONLINE == ddl_type) {
|
|
str = "add column online";
|
|
} else if (DDL_COLUMN_REDEFINITION == ddl_type) {
|
|
str = "column redefinition";
|
|
} else if (DDL_TABLE_REDEFINITION == ddl_type) {
|
|
str = "table redefinition";
|
|
} else if (DDL_DIRECT_LOAD == ddl_type) {
|
|
str = "direct load";
|
|
} else if (DDL_DIRECT_LOAD_INSERT == ddl_type) {
|
|
str = "direct load insert";
|
|
} else if (DDL_MODIFY_AUTO_INCREMENT == ddl_type) {
|
|
str = "modify auto_increment";
|
|
} else if (DDL_CONVERT_TO_CHARACTER == ddl_type) {
|
|
str = "convert to character";
|
|
} else if (DDL_CHANGE_COLUMN_NAME == ddl_type) {
|
|
str = "change column name";
|
|
} else if (DDL_TABLE_RESTORE == ddl_type) {
|
|
str = "recover restore table ddl";
|
|
} else if (DDL_MVIEW_COMPLETE_REFRESH == ddl_type) {
|
|
str = "mview complete refresh";
|
|
} else if (DDL_CREATE_MVIEW == ddl_type) {
|
|
str = "create mview";
|
|
} else if (DDL_CREATE_MLOG == ddl_type) {
|
|
str = "create materialized view log";
|
|
} else if (DDL_DROP_MLOG == ddl_type) {
|
|
str = "drop materialized view log";
|
|
} else if (DDL_AUTO_SPLIT_BY_RANGE == ddl_type) {
|
|
str = "auto split by range";
|
|
} else if (DDL_AUTO_SPLIT_NON_RANGE == ddl_type) {
|
|
str = "auto split by non range";
|
|
} else if (DDL_MANUAL_SPLIT_BY_RANGE == ddl_type) {
|
|
str = "manual split by range";
|
|
} else if (DDL_MANUAL_SPLIT_NON_RANGE == ddl_type) {
|
|
str = "manual split non range";
|
|
} else if (DDL_PARTITION_SPLIT_RECOVERY_TABLE_REDEFINITION == ddl_type) {
|
|
str = "partition split recovery table redefinition";
|
|
}
|
|
|
|
return str;
|
|
}
|
|
|
|
int ObDDLService::get_part_by_part_id(
|
|
const ObPartitionSchema &partition_schema,
|
|
const int64_t part_id,
|
|
const ObPartition *&part)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
part = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < partition_schema.get_partition_num(); i++) {
|
|
const ObPartition *it = partition_schema.get_part_array()[i];
|
|
if (OB_NOT_NULL(it) && it->get_part_id() == part_id) {
|
|
part = it;
|
|
break;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::old_update_global_index_schema_(const obrpc::ObAlterTableArg &arg,
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &index_table_schema,
|
|
ObTableSchema &new_index_table_schema,
|
|
const int64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_UNLIKELY(!index_table_schema.is_valid()) || OB_UNLIKELY(!orig_table_schema.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(index_table_schema), K(orig_table_schema));
|
|
} else if (!arg.is_update_global_indexes_) {
|
|
if (OB_FAIL(ddl_operator.update_index_status(
|
|
index_table_schema.get_tenant_id(),
|
|
index_table_schema.get_data_table_id(),
|
|
index_table_schema.get_table_id(),
|
|
INDEX_STATUS_UNUSABLE,
|
|
orig_table_schema.get_in_offline_ddl_white_list(),
|
|
trans,
|
|
nullptr /* ddl_stmt_str */))) {
|
|
LOG_WARN("update_index_status failed", KR(ret), K(index_table_schema.get_data_table_id()));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(drop_directly_and_create_index_schema_(schema_guard, orig_table_schema, new_index_table_schema,
|
|
nullptr /* ddl_stmt */, &trans, tenant_data_version))) {
|
|
LOG_WARN("ddl_service_ rebuild_index failed", KR(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::make_index_unusable_(common::ObIAllocator &allocator,
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &index_table_schema,
|
|
ObTableSchema &new_index_table_schema,
|
|
ObIArray<ObDDLTaskRecord> &ddl_tasks,
|
|
ObIArray<obrpc::ObDDLRes> &ddl_res_array)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_UNLIKELY(!index_table_schema.is_valid()) || OB_UNLIKELY(!orig_table_schema.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(index_table_schema), K(orig_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.update_index_status(
|
|
index_table_schema.get_tenant_id(),
|
|
index_table_schema.get_data_table_id(),
|
|
index_table_schema.get_table_id(),
|
|
INDEX_STATUS_UNAVAILABLE,
|
|
orig_table_schema.get_in_offline_ddl_white_list(),
|
|
trans,
|
|
nullptr /* ddl_stmt_str */))) {
|
|
LOG_WARN("update_index_status failed", KR(ret), K(index_table_schema.get_data_table_id()));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
bool has_index_task = false;
|
|
obrpc::ObDropIndexArg *drop_index_arg = nullptr;
|
|
typedef common::ObSEArray<share::schema::ObTableSchema, 4> TableSchemaArray;
|
|
SMART_VAR(TableSchemaArray, drop_index_schemas) {
|
|
if (OB_FAIL(ObDDLTaskRecordOperator::check_has_index_or_mlog_task(
|
|
trans, index_table_schema, orig_table_schema.get_tenant_id(), orig_table_schema.get_table_id(),
|
|
has_index_task))) {
|
|
LOG_WARN("fail to check has index task", KR(ret), K(orig_table_schema.get_tenant_id()),
|
|
K(orig_table_schema.get_table_id()), K(index_table_schema.get_table_id()));
|
|
} else if (has_index_task) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to drop a building or dropping index", KR(ret));
|
|
} else if (OB_FAIL(drop_index_schemas.push_back(new_index_table_schema))) {
|
|
LOG_WARN("fail to push back new index table schema", KR(ret));
|
|
} else if (OB_FAIL(prepare_drop_index_arg_(allocator, new_index_table_schema, true/*only_set_status*/,
|
|
true/*is_add_to_scheduler*/, true/*is_inner*/, drop_index_arg))) {
|
|
LOG_WARN("fail to prepare drop index arg", KR(ret));
|
|
} else if (OB_ISNULL(drop_index_arg)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("drop index arg is nullptr", KR(ret));
|
|
} else if (OB_FAIL(submit_drop_index_task_and_fill_ddl_result_(allocator, trans, drop_index_schemas, orig_table_schema,
|
|
drop_index_arg, nullptr /*inc_data_tablet_ids*/, nullptr /*del_data_tablet_ids*/, has_index_task,
|
|
ddl_tasks, ddl_res_array))) {
|
|
LOG_WARN("fail to submit drop index task", KR(ret));
|
|
}
|
|
} // end smart var
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::prepare_drop_index_arg_(common::ObIAllocator &allocator,
|
|
ObTableSchema &index_table_schema,
|
|
const bool only_set_status,
|
|
const bool is_add_to_scheduler,
|
|
const bool is_inner,
|
|
obrpc::ObDropIndexArg *&drop_index_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
drop_index_arg = nullptr;
|
|
void *tmp_ptr = nullptr;
|
|
if (OB_UNLIKELY(!index_table_schema.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("index table schema's origin index name is nullptr", KR(ret), K(index_table_schema));
|
|
} else if (OB_FAIL(index_table_schema.generate_origin_index_name())) {
|
|
LOG_WARN("fail to generate origin index name", KR(ret));
|
|
} else if (nullptr == (tmp_ptr = (ObDropIndexArg*)allocator.alloc(sizeof(ObDropIndexArg)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
SHARE_LOG(ERROR, "failed to alloc memory!", KR(ret));
|
|
} else {
|
|
drop_index_arg = new (tmp_ptr)ObDropIndexArg();
|
|
drop_index_arg->index_action_type_ = ObIndexArg::DROP_INDEX;
|
|
drop_index_arg->is_add_to_scheduler_ = is_add_to_scheduler;
|
|
drop_index_arg->is_inner_ = is_inner;
|
|
drop_index_arg->tenant_id_ = index_table_schema.get_tenant_id();
|
|
if (OB_FAIL(ob_write_string(allocator, index_table_schema.get_origin_index_name_str(),drop_index_arg->index_name_))) {
|
|
LOG_WARN("fail to write index name", KR(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::submit_drop_index_task_and_fill_ddl_result_(common::ObIAllocator &allocator,
|
|
ObMySQLTransaction &trans,
|
|
const common::ObIArray<share::schema::ObTableSchema> &drop_index_schemas,
|
|
const ObTableSchema &orig_table_schema,
|
|
const obrpc::ObDropIndexArg *drop_index_arg,
|
|
const common::ObIArray<common::ObTabletID> *inc_data_tablet_ids,
|
|
const common::ObIArray<common::ObTabletID> *del_data_tablet_ids,
|
|
bool &has_index_task,
|
|
ObIArray<ObDDLTaskRecord> &ddl_tasks,
|
|
ObIArray<obrpc::ObDDLRes> &ddl_res_array)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObIndexBuilder index_builder(*this);
|
|
ObDDLRes ddl_res;
|
|
ObDDLTaskRecord task_record;
|
|
if (0 >= drop_index_schemas.count() || OB_ISNULL(drop_index_arg)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("drop index shemas count should larger than 0 or drop_index_arg is null",
|
|
KR(ret), K(drop_index_schemas.count()), KP(drop_index_arg));
|
|
} else if (OB_FAIL(index_builder.submit_drop_index_task(trans, orig_table_schema, drop_index_schemas,
|
|
*drop_index_arg, inc_data_tablet_ids, del_data_tablet_ids,
|
|
allocator, has_index_task, task_record))) {
|
|
LOG_WARN("failed to submit drop index task", KR(ret));
|
|
} else {
|
|
const ObTableSchema &drop_index_schema = drop_index_schemas.at(0);
|
|
ddl_res.task_id_ = task_record.task_id_;
|
|
ddl_res.tenant_id_ = drop_index_schema.get_tenant_id();
|
|
ddl_res.schema_id_ = drop_index_schema.get_table_id();
|
|
if (OB_FAIL(ddl_tasks.push_back(task_record))) {
|
|
LOG_WARN("push back ddl task failed", KR(ret));
|
|
} else if (OB_FAIL(ddl_res_array.push_back(ddl_res))) {
|
|
LOG_WARN("push back ddl res array failed", KR(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
int ObDDLService::drop_and_create_index_schema_(obrpc::ObAlterTableArg &arg,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &index_table_schema,
|
|
const int64_t tenant_data_version,
|
|
ObTableSchema &new_index_table_schema,
|
|
ObIArray<ObDDLTaskRecord> &ddl_tasks,
|
|
ObIArray<obrpc::ObDDLRes> &ddl_res_array)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else if (OB_UNLIKELY(!orig_table_schema.is_valid() || !index_table_schema.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(orig_table_schema), K(index_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.update_index_status(
|
|
index_table_schema.get_tenant_id(),
|
|
index_table_schema.get_data_table_id(),
|
|
index_table_schema.get_table_id(),
|
|
INDEX_STATUS_UNAVAILABLE,
|
|
orig_table_schema.get_in_offline_ddl_white_list(),
|
|
trans,
|
|
nullptr /* ddl_stmt_str */))) {
|
|
LOG_WARN("update_index_status failed", KR(ret), K(index_table_schema.get_data_table_id()));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
obrpc::ObDropIndexArg *drop_index_arg = nullptr;
|
|
bool has_index_task = false;
|
|
bool is_inner_and_domain_index = index_table_schema.is_fts_or_multivalue_index();
|
|
ObSArray<obrpc::ObIndexArg *> &index_arg_list = arg.index_arg_list_;
|
|
if (OB_FAIL(ObDDLTaskRecordOperator::check_has_index_or_mlog_task(
|
|
trans, index_table_schema, orig_table_schema.get_tenant_id(), orig_table_schema.get_table_id(), has_index_task))) {
|
|
LOG_WARN("fail to check has index task", KR(ret), K(orig_table_schema.get_tenant_id()),
|
|
K(orig_table_schema.get_table_id()), K(index_table_schema.get_table_id()));
|
|
} else if (has_index_task) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to drop a building or dropping index", KR(ret));
|
|
} else if (OB_FAIL(prepare_drop_index_arg_(arg.allocator_, new_index_table_schema, true/*only_set_status*/,
|
|
true/*is_add_to_scheduler*/, true/*is_inner*/, drop_index_arg))) {
|
|
LOG_WARN("fail to prepare drop index arg", KR(ret));
|
|
} else if (OB_ISNULL(drop_index_arg)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("drop index arg is nullptr", KR(ret));
|
|
} else if (OB_FAIL(index_arg_list.push_back(drop_index_arg))) {
|
|
LOG_WARN("push back to index_arg_list failed", KR(ret), KP(drop_index_arg));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t new_table_id = OB_INVALID_ID;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", KR(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(
|
|
new_index_table_schema.get_tenant_id(), new_table_id))) {
|
|
LOG_WARN("fail to fetch new table id", KR(ret));
|
|
} else {
|
|
new_index_table_schema.set_table_id(new_table_id);
|
|
new_index_table_schema.set_index_status(INDEX_STATUS_UNAVAILABLE);
|
|
if (OB_FAIL(generate_tablet_id(new_index_table_schema))) {
|
|
LOG_WARN("failed to generate tablet id", KR(ret));
|
|
} else if (OB_FAIL(create_index_or_mlog_table_in_trans(new_index_table_schema,
|
|
nullptr /* ddl_stmt */, &trans, schema_guard,
|
|
false/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("create_index_in_trans failed", KR(ret), K(new_index_table_schema));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::prepare_create_index_arg_(common::ObIAllocator &allocator,
|
|
const ObTableSchema &new_index_table_schema,
|
|
const obrpc::ObIndexArg::IndexActionType index_action_type,
|
|
obrpc::ObCreateIndexArg *&create_index_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
void *tmp_ptr = nullptr;
|
|
create_index_arg = nullptr;
|
|
if (nullptr == (tmp_ptr = (ObCreateIndexArg*)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_ = index_action_type;
|
|
create_index_arg->index_type_ = new_index_table_schema.get_index_type();
|
|
if (OB_FAIL(create_index_arg->index_schema_.assign(new_index_table_schema))) {
|
|
LOG_WARN("fail to assign index schema", KR(ret), K(new_index_table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_global_index(ObAlterTableArg &arg,
|
|
const uint64_t tenant_id,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
const uint64_t tenant_data_version,
|
|
obrpc::ObAlterTableRes &res,
|
|
ObIArray<ObDDLTaskRecord> &ddl_tasks)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
ObSchemaGetterGuard schema_guard;
|
|
bool is_oracle_mode = false;
|
|
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 if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode))) {
|
|
LOG_WARN("fail to get compat mode", KR(ret), K(tenant_id));
|
|
} else {
|
|
SMART_VAR(ObTableSchema, new_index_table_schema) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = nullptr;
|
|
new_index_table_schema.reset();
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id),
|
|
"table id", simple_index_infos.at(i).table_id_, K(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (index_table_schema->is_unavailable_index()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("drop/truncate partition before finishing index build not support", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop/truncate partition before finishing index build ");
|
|
} else if (!index_table_schema->can_read_index()) {
|
|
// If the index is not available, the partition operation will not do any intervention
|
|
} else if (!index_table_schema->is_global_index_table()) {
|
|
// do nothing
|
|
} else if (OB_FAIL(new_index_table_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign schema", KR(ret));
|
|
} else {
|
|
if (tenant_data_version < DATA_VERSION_4_3_3_0
|
|
|| is_oracle_mode) {
|
|
if (OB_FAIL(old_update_global_index_schema_(arg, trans, ddl_operator, schema_guard,
|
|
orig_table_schema, *index_table_schema,
|
|
new_index_table_schema,tenant_data_version))) {
|
|
LOG_WARN("fail to update global index schema", KR(ret), K(arg));
|
|
}
|
|
} else {
|
|
ObIArray<obrpc::ObDDLRes> &ddl_res_array = res.ddl_res_array_;
|
|
if (arg.is_update_global_indexes_) {
|
|
if(OB_FAIL(drop_and_create_index_schema_(arg, schema_guard, trans, ddl_operator, orig_table_schema,
|
|
*index_table_schema, tenant_data_version, new_index_table_schema,
|
|
ddl_tasks, ddl_res_array))) {
|
|
LOG_WARN("fail to drop and create index schema", KR(ret), K(arg));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(make_index_unusable_(arg.allocator_, trans, ddl_operator, orig_table_schema, *index_table_schema,
|
|
new_index_table_schema, ddl_tasks, ddl_res_array))) {
|
|
LOG_WARN("fail to make index unusable", KR(ret), K(arg));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)
|
|
|| !arg.is_update_global_indexes_) {
|
|
// do nothing
|
|
} else {
|
|
ObSArray<obrpc::ObIndexArg *> &index_arg_list = arg.index_arg_list_;
|
|
obrpc::ObCreateIndexArg *create_index_arg = nullptr;
|
|
if (OB_FAIL(prepare_create_index_arg_(arg.allocator_, new_index_table_schema,
|
|
ObIndexArg::REBUILD_INDEX, create_index_arg))) {
|
|
LOG_WARN("fail to prepare create index arg", KR(ret));
|
|
} 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));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}// end smart var
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fill_interval_info_for_set_interval(const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
AlterTableSchema &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartition **part_array = orig_table_schema.get_part_array();
|
|
int64_t part_num = orig_table_schema.get_partition_num();
|
|
if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to part_array is null", K(orig_table_schema), K(inc_table_schema), KR(ret));
|
|
} else if (part_num < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part num is less 1", K(orig_table_schema), K(inc_table_schema), KR(ret));
|
|
} else if (OB_ISNULL(part_array[part_num - 1])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("the last part is null", K(orig_table_schema), K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(new_table_schema.set_transition_point(part_array[part_num - 1]->get_high_bound_val()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to set_transition_point", K(orig_table_schema), K(inc_table_schema),KR(ret));
|
|
} else if (OB_FAIL(new_table_schema.set_interval_range(inc_table_schema.get_interval_range()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to new_table_schema.set_interval_range", K(orig_table_schema), K(inc_table_schema),KR(ret));
|
|
} else {
|
|
new_table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_INTERVAL);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fill_interval_info_for_offline(const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (new_table_schema.is_interval_part()) {
|
|
ObPartition **part_array = orig_table_schema.get_part_array();
|
|
int64_t part_num = orig_table_schema.get_partition_num();
|
|
if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to part_array is null", K(orig_table_schema), KR(ret));
|
|
} else if (part_num < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part num is less 1", K(orig_table_schema), KR(ret));
|
|
} else if (OB_ISNULL(part_array[part_num - 1])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("the last part is null", K(orig_table_schema), KR(ret));
|
|
} else if (OB_FAIL(new_table_schema.set_transition_point(part_array[part_num - 1]->get_high_bound_val()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to set_transition_point", K(orig_table_schema), KR(ret));
|
|
} else if (OB_FAIL(new_table_schema.set_interval_range(orig_table_schema.get_interval_range()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to new_table_schema.set_interval_range", K(orig_table_schema), KR(ret));
|
|
} else {
|
|
new_table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_INTERVAL);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
//For truncate part/subpart, inc_table_schema and del_table_schema should be different in the later process.
|
|
//For other situations, del_table_schema is useless and equal to inc_table_schema.
|
|
//For split partition, upd_table_schema records the partition information which need to be changed.
|
|
//For other situations, upd_table_schema is useless and equal to new_table_schema.
|
|
int ObDDLService::generate_tables_array(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObIArray<const ObTableSchema*> &orig_table_schemas,
|
|
ObIArray<ObTableSchema*> &new_table_schemas,
|
|
ObIArray<AlterTableSchema*> &inc_table_schemas,
|
|
ObIArray<AlterTableSchema*> &del_table_schemas,
|
|
common::ObIArray<ObTableSchema*> &upd_table_schemas,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
AlterTableSchema &inc_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObArenaAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObAlterTableArg::AlterPartitionType op_type = alter_table_arg.alter_part_type_;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObSEArray<uint64_t, 20> aux_table_ids;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
AlterTableSchema tmp_inc_table_schema;
|
|
bool modify_sub_part_template_flags = false;
|
|
if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level()
|
|
&& orig_table_schema.sub_part_template_def_valid()) {
|
|
// sub_part_template_def_valid() is only used for schema printer.
|
|
// To simplify relate logic, we consider that add partition/subpartition or add subpartition
|
|
// make cause partitions different.
|
|
if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == op_type
|
|
|| obrpc::ObAlterTableArg::DROP_SUB_PARTITION == op_type
|
|
|| (obrpc::ObAlterTableArg::ADD_PARTITION == op_type && !alter_table_schema.sub_part_template_def_valid())) {
|
|
modify_sub_part_template_flags = true;
|
|
new_table_schema.unset_sub_part_template_def_valid();
|
|
}
|
|
}
|
|
const ObString new_part_name = inc_table_schema.get_new_part_name();
|
|
if ((!orig_table_schema.has_tablet() && !orig_table_schema.is_external_table())
|
|
|| orig_table_schema.is_index_local_storage()
|
|
|| orig_table_schema.is_aux_lob_table()
|
|
|| orig_table_schema.is_mlog_table()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table_schema must be data table or global indexes", KR(ret), K(orig_table_schema));
|
|
} else if (orig_table_schema.is_external_table() && (obrpc::ObAlterTableArg::ADD_PARTITION != op_type
|
|
&& obrpc::ObAlterTableArg::DROP_PARTITION != op_type)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("external should only support add and drop partition ddl", K(ret));
|
|
} else if (OB_FAIL(orig_table_schemas.push_back(&orig_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(new_table_schemas.push_back(&new_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(new_table_schema));
|
|
} else if (OB_FAIL(tmp_inc_table_schema.assign(new_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(new_table_schema));
|
|
} else if (OB_FAIL(tmp_inc_table_schema.assign_partition_schema(inc_table_schema))) {
|
|
LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(tmp_inc_table_schema.set_transition_point(inc_table_schema.get_transition_point()))) {
|
|
LOG_WARN("fail to set transition point", K(ret));
|
|
} else if (OB_FAIL(tmp_inc_table_schema.set_interval_range(inc_table_schema.get_interval_range()))) {
|
|
LOG_WARN("fail to set interval range", K(ret));
|
|
} else if (OB_FAIL(inc_table_schema.assign(tmp_inc_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(new_table_schema));
|
|
} else if (OB_FAIL(inc_table_schema.set_new_part_name(new_part_name))) {
|
|
LOG_WARN("fail to set new part name", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(inc_table_schemas.push_back(&inc_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get_simple_index_infos failed", KR(ret));
|
|
}
|
|
if (obrpc::ObAlterTableArg::RENAME_PARTITION == op_type
|
|
||obrpc::ObAlterTableArg::RENAME_SUB_PARTITION == op_type) {
|
|
//do nothing
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
if (OB_FAIL(aux_table_ids.push_back(simple_index_infos.at(i).table_id_))) {
|
|
LOG_WARN("fail to push back index table id", KR(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t mtid = orig_table_schema.get_aux_lob_meta_tid();
|
|
uint64_t ptid = orig_table_schema.get_aux_lob_piece_tid();
|
|
if (!((mtid != OB_INVALID_ID && ptid != OB_INVALID_ID) || (mtid == OB_INVALID_ID && ptid == OB_INVALID_ID))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Expect meta tid and piece tid both valid or both invalid", KR(ret), K(mtid), K(ptid));
|
|
} else if (OB_INVALID_ID != mtid &&
|
|
OB_FAIL(aux_table_ids.push_back(mtid))) {
|
|
LOG_WARN("fail to push back lob meta tid", KR(ret), K(mtid));
|
|
} else if (OB_INVALID_ID != ptid &&
|
|
OB_FAIL(aux_table_ids.push_back(ptid))) {
|
|
LOG_WARN("fail to push back lob piece tid", KR(ret), K(ptid));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t mlog_tid = orig_table_schema.get_mlog_tid();
|
|
if ((OB_INVALID_ID != mlog_tid)
|
|
&& OB_FAIL(aux_table_ids.push_back(mlog_tid))) {
|
|
LOG_WARN("failed to push back materialized view log tid", KR(ret), K(mlog_tid));
|
|
}
|
|
}
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < aux_table_ids.count(); ++i) {
|
|
const ObTableSchema *orig_aux_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
aux_table_ids.at(i), orig_aux_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), K(aux_table_ids.at(i)), KR(ret));
|
|
} else if (OB_ISNULL(orig_aux_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", KR(ret));
|
|
} else if (orig_aux_table_schema->is_index_local_storage()
|
|
|| orig_aux_table_schema->is_aux_lob_table()
|
|
|| orig_aux_table_schema->is_mlog_table()) {
|
|
ObTableSchema *new_aux_table_schema = NULL;
|
|
AlterTableSchema *inc_aux_table_schema = NULL;
|
|
void *new_schema_ptr = allocator.alloc(sizeof(ObTableSchema));
|
|
void *inc_schema_ptr = allocator.alloc(sizeof(AlterTableSchema));
|
|
if (OB_ISNULL(new_schema_ptr) || OB_ISNULL(inc_schema_ptr)) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail alloc memory", KR(ret), KP(new_schema_ptr), KP(inc_schema_ptr));
|
|
} else {
|
|
new_aux_table_schema = new (new_schema_ptr)ObTableSchema(&allocator);
|
|
inc_aux_table_schema = new (inc_schema_ptr)AlterTableSchema(&allocator);
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(new_aux_table_schema->assign(*orig_aux_table_schema))) {
|
|
LOG_WARN("fail to assign schema", KR(ret), KPC(orig_aux_table_schema));
|
|
} else if (modify_sub_part_template_flags
|
|
&& FALSE_IT(new_aux_table_schema->unset_sub_part_template_def_valid())) {
|
|
} else if (OB_FAIL(inc_aux_table_schema->assign(*new_aux_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), KPC(new_aux_table_schema));
|
|
} else if (OB_FAIL(inc_aux_table_schema->assign_partition_schema_without_auto_part_attr(inc_table_schema))) {
|
|
LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(inc_aux_table_schema->set_transition_point(inc_table_schema.get_transition_point()))) {
|
|
LOG_WARN("fail to set transition point", K(ret));
|
|
} else if (OB_FAIL(inc_aux_table_schema->set_interval_range(inc_table_schema.get_interval_range()))) {
|
|
LOG_WARN("fail to set interval range", K(ret));
|
|
} else if (OB_FAIL(new_table_schemas.push_back(new_aux_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(new_aux_table_schema));
|
|
} else if (OB_FAIL(orig_table_schemas.push_back(orig_aux_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(orig_aux_table_schema));
|
|
} else if (OB_FAIL(inc_table_schemas.push_back(inc_aux_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), KPC(inc_aux_table_schema));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (obrpc::ObAlterTableArg::REORGANIZE_PARTITION == op_type ||
|
|
obrpc::ObAlterTableArg::SPLIT_PARTITION == op_type ||
|
|
obrpc::ObAlterTableArg::AUTO_SPLIT_PARTITION == op_type) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_table_schemas.count(); i++) {
|
|
ObTableSchema *upd_table_schema = NULL;
|
|
void *upd_schema_ptr = allocator.alloc(sizeof(ObTableSchema));
|
|
if (OB_ISNULL(upd_schema_ptr)) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail alloc memory", KR(ret));
|
|
} else if (FALSE_IT(upd_table_schema = new (upd_schema_ptr)ObTableSchema(&allocator))) {
|
|
} else if (OB_FAIL(upd_table_schema->assign(*new_table_schemas.at(i)))) {
|
|
LOG_WARN("fail to assign partition schema", K(new_table_schemas), KR(ret));
|
|
} else if (FALSE_IT(upd_table_schema->reset_partition_array())) {
|
|
} else if (FALSE_IT(upd_table_schema->reset_hidden_partition_array())) {
|
|
} else if (OB_FAIL(upd_table_schemas.push_back(upd_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), KPC(upd_table_schema));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(generate_split_info_for_schemas_(op_type,
|
|
orig_table_schemas, inc_table_schemas,
|
|
new_table_schemas, upd_table_schemas))) {
|
|
LOG_WARN("fail to generate split info for schema", KR(ret));
|
|
}
|
|
} else if (OB_FAIL(upd_table_schemas.assign(new_table_schemas))) {
|
|
LOG_WARN("failed to assign new_table_schemas", KR(ret), K(new_table_schemas));
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == op_type
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == op_type) {
|
|
// truncate part/subpart may cause del_table_schema and inc_table_schema
|
|
for (int i = 0; OB_SUCC(ret) && i < inc_table_schemas.count(); i++) {
|
|
AlterTableSchema *del_table_schema = NULL;
|
|
void *del_schema_ptr = allocator.alloc(sizeof(AlterTableSchema));
|
|
if (OB_ISNULL(del_schema_ptr)) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail alloc memory", KR(ret));
|
|
} else if (FALSE_IT(del_table_schema = new (del_schema_ptr)AlterTableSchema(&allocator))) {
|
|
} else if (OB_FAIL(del_table_schema->assign(*inc_table_schemas.at(i)))) {
|
|
LOG_WARN("fail to assign partition schema", K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(del_table_schemas.push_back(del_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), KPC(del_table_schema));
|
|
}
|
|
}
|
|
}
|
|
// for other situations, del_table_schemas is useless and equal to inc_table_schemas
|
|
else if (OB_FAIL(del_table_schemas.assign(inc_table_schemas))) {
|
|
LOG_WARN("failed to assign inc_table_schemas", KR(ret), K(inc_table_schemas));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::reset_interval_info_for_interval_to_range(ObTableSchema &new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObRowkey null_row_key;
|
|
null_row_key.reset();
|
|
if (OB_FAIL(new_table_schema.set_transition_point(null_row_key))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to set_transition_point", K(new_table_schema), KR(ret));
|
|
} else if (OB_FAIL(new_table_schema.set_interval_range(null_row_key))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to new_table_schema.set_interval_range", K(new_table_schema), KR(ret));
|
|
} else {
|
|
new_table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_RANGE_COLUMNS);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::split_global_index_partitions(obrpc::ObAlterTableArg &arg, obrpc::ObAlterTableRes &res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
const ObTableSchema *orig_index_schema = NULL;
|
|
int64_t refreshed_schema_version = 0;
|
|
uint64_t tenant_data_version = 0;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(arg.session_id_);
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", KR(ret), K(arg));
|
|
} else if (OB_UNLIKELY(obrpc::ObAlterTableArg::AlterPartitionType::AUTO_SPLIT_PARTITION != arg.alter_part_type_)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid alter partition type", KR(ret), K(arg.alter_part_type_), K(arg));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", 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 with version in inner table", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
alter_table_schema.get_origin_database_name(),
|
|
alter_table_schema.get_origin_table_name(),
|
|
true /*is_index*/,
|
|
orig_index_schema))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), K(alter_table_schema));
|
|
} else if (OB_ISNULL(orig_index_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("NULL ptr", KR(ret), K(alter_table_schema));
|
|
} else if (OB_UNLIKELY(!orig_index_schema->is_global_index_table() ||
|
|
is_sys_index_table(orig_index_schema->get_table_id()))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid index type", KR(ret), KPC(orig_index_schema));
|
|
} else if (OB_FAIL(fill_part_name(*orig_index_schema, arg.alter_table_schema_))) {
|
|
LOG_WARN("failed to fill part name", KR(ret), K(orig_index_schema), K(arg));
|
|
} else if (OB_FAIL(check_split_global_index_partition_(schema_guard, arg, *orig_index_schema))) {
|
|
LOG_WARN("fail to check split global idnex partition", KR(ret), KPC(orig_index_schema), K(arg));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObArray<const ObTableSchema*> orig_index_schemas;
|
|
ObArray<ObTableSchema*> new_index_schemas;
|
|
ObArray<AlterTableSchema*> inc_index_schemas;
|
|
ObArray<const ObTableSchema *> inc_index_schema_ptrs;
|
|
ObArray<AlterTableSchema*> del_index_schemas;
|
|
ObArray<ObTableSchema*> upd_index_schemas;
|
|
ObArenaAllocator allocator("ModifyPart");
|
|
obrpc::ObAlterTableRes res;
|
|
|
|
HEAP_VAR(ObArray<ObDDLTaskRecord>, ddl_tasks) {
|
|
HEAP_VAR(ObTableSchema, new_table_schema) {
|
|
if (OB_FAIL(new_table_schema.assign(*orig_index_schema))) {
|
|
LOG_WARN("fail to assign schema", KR(ret));
|
|
} else if (OB_FAIL(generate_tables_array(arg,
|
|
orig_index_schemas,
|
|
new_index_schemas,
|
|
inc_index_schemas,
|
|
del_index_schemas,
|
|
upd_index_schemas,
|
|
*orig_index_schema,
|
|
new_table_schema,
|
|
alter_table_schema,
|
|
schema_guard,
|
|
allocator))) {
|
|
LOG_WARN("generate tables array failed", KR(ret), K(arg));
|
|
} else if (OB_FAIL(alter_tables_partitions(arg,
|
|
orig_index_schemas,
|
|
new_index_schemas,
|
|
inc_index_schemas,
|
|
del_index_schemas,
|
|
upd_index_schemas,
|
|
ddl_operator,
|
|
schema_guard,
|
|
trans))) {
|
|
LOG_WARN("alter table partitions failed", KR(ret), K(arg));
|
|
} else if (OB_FAIL(ddl_operator.inc_table_schema_version(trans, tenant_id,
|
|
orig_index_schema->get_data_table_id()))) {
|
|
LOG_WARN("fail to inc data table schema version", KR(ret), KPC(orig_index_schema));
|
|
} else if (OB_UNLIKELY(inc_index_schemas.count() != 1) ||
|
|
OB_UNLIKELY(new_index_schemas.count() != 1) ||
|
|
OB_ISNULL(inc_index_schemas.at(0)) ||
|
|
OB_ISNULL(new_index_schemas.at(0))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid schema arr", KR(ret), K(inc_index_schemas.count()), KP(inc_index_schemas.at(0)),
|
|
K(new_index_schemas.count()), KP(new_index_schemas.at(0)));
|
|
} else if (FALSE_IT(inc_index_schemas.at(0)->set_schema_version(
|
|
new_index_schemas.at(0)->get_schema_version()))) {
|
|
} else if (OB_FAIL(inc_index_schema_ptrs.push_back(inc_index_schemas.at(0)))) {
|
|
LOG_WARN("failed to push back", K(ret));
|
|
} else {
|
|
const ObDDLType ddl_type = ObDDLType::DDL_AUTO_SPLIT_BY_RANGE;
|
|
ObSplitPartitionHelper split_partition_helper(trans, schema_guard, arg.allocator_, tenant_id,
|
|
tenant_data_version, ddl_type, new_index_schemas, upd_index_schemas, inc_index_schema_ptrs, arg.parallelism_);
|
|
ObDDLTaskRecord task_record;
|
|
if (OB_FAIL(split_partition_helper.execute(task_record))) {
|
|
LOG_WARN("failed to execute split partition helper", KR(ret));
|
|
} else if (OB_FAIL(ddl_tasks.push_back(task_record))) {
|
|
LOG_WARN("failed to push back", KR(ret));
|
|
} else {
|
|
res.ddl_type_ = ddl_type;
|
|
res.task_id_ = task_record.task_id_;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_UNLIKELY(ddl_tasks.count() != 1)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid task number", KR(ret), K(arg), K(ddl_tasks));
|
|
}
|
|
|
|
const bool is_commit = OB_SUCC(ret);
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(is_commit))) {
|
|
LOG_WARN("trans end failed", K(is_commit), K(temp_ret));
|
|
ret = is_commit ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", KR(ret));
|
|
} else if (OB_TMP_FAIL(GCTX.root_service_->get_ddl_scheduler().schedule_ddl_task(ddl_tasks.at(0)))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(ddl_tasks.at(0)));
|
|
}
|
|
}
|
|
} // end HEAP_VAR(ObTableSchema, new_table_schema)
|
|
} // end HEAP_VAR(ObArray<ObDDLTaskRecord>, ddl_tasks)
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_tables_partitions(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObIArray<const ObTableSchema*> &orig_table_schemas,
|
|
ObIArray<ObTableSchema*> &new_table_schemas,
|
|
ObIArray<AlterTableSchema*> &inc_table_schemas,
|
|
ObIArray<AlterTableSchema*> &del_table_schemas,
|
|
ObIArray<ObTableSchema*> &upd_table_schemas,
|
|
ObDDLOperator &ddl_operator,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (orig_table_schemas.count() != new_table_schemas.count()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("orig_table_schemas.count() new_table_schemas.count() is not equal", KR(ret),
|
|
K(orig_table_schemas), K(new_table_schemas));
|
|
} else if (orig_table_schemas.count() != inc_table_schemas.count()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("orig_table_schemas.count() inc_table_schemas.count() is not equal", KR(ret),
|
|
K(orig_table_schemas), K(inc_table_schemas));
|
|
} else if (orig_table_schemas.count() != del_table_schemas.count()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("orig_table_schemas.count() inc_table_schemas.count() is not equal", KR(ret),
|
|
K(orig_table_schemas), K(del_table_schemas));
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_table_schemas.count(); ++i) {
|
|
// todo fill AlterTableSchema for splitting partition with inc_table_schema
|
|
if (OB_ISNULL(new_table_schemas.at(i)) || OB_ISNULL(inc_table_schemas.at(i))
|
|
|| OB_ISNULL(orig_table_schemas.at(i)) || OB_ISNULL(del_table_schemas.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema ptr is NULL", KR(ret), K(new_table_schemas.at(i)),
|
|
K(inc_table_schemas.at(i)), K(orig_table_schemas.at(i)), K(del_table_schemas.at(i)));
|
|
|
|
} else if (OB_FAIL(alter_table_partitions(alter_table_arg,
|
|
*orig_table_schemas.at(i),
|
|
*inc_table_schemas.at(i),
|
|
*del_table_schemas.at(i),
|
|
*upd_table_schemas.at(i),
|
|
*new_table_schemas.at(i),
|
|
ddl_operator,
|
|
schema_guard,
|
|
trans,
|
|
*orig_table_schemas.at(0)))) {
|
|
LOG_WARN("alter table partitions failed", KR(ret), K(i), KPC(new_table_schemas.at(i)), KPC(inc_table_schemas.at(i)));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
bool ObDDLService::is_add_and_drop_partition(const obrpc::ObAlterTableArg::AlterPartitionType &op_type)
|
|
{
|
|
return obrpc::ObAlterTableArg::ADD_PARTITION == op_type
|
|
|| obrpc::ObAlterTableArg::ADD_SUB_PARTITION == op_type
|
|
|| obrpc::ObAlterTableArg::DROP_PARTITION == op_type
|
|
|| obrpc::ObAlterTableArg::DROP_SUB_PARTITION == op_type
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_PARTITION == op_type
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == op_type;
|
|
}
|
|
|
|
int ObDDLService::alter_table_partitions(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
AlterTableSchema &inc_table_schema,
|
|
AlterTableSchema &del_table_schema,
|
|
ObTableSchema &upd_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
const ObTableSchema &orig_data_table_schema)
|
|
{
|
|
DEBUG_SYNC(BEFORE_ALTER_TABLE_PARTITION);
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
const int64_t table_id = orig_table_schema.get_table_id();
|
|
const obrpc::ObAlterTableArg::AlterPartitionType &op_type = alter_table_arg.alter_part_type_;
|
|
int64_t schema_version = OB_INVALID_VERSION;
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("valid schema service", K(ret));
|
|
} else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == op_type) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("split table partitions is not supported", KR(ret), K(orig_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "split table partitions is");
|
|
} else if (alter_table_arg.is_split_partition()) {
|
|
if (OB_FAIL(ddl_operator.split_table_partitions(orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
upd_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to add table partitions", KR(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::ADD_PARTITION == op_type) {
|
|
if (OB_FAIL(ObDDLLock::lock_for_add_partition_in_trans(orig_table_schema, trans))) {
|
|
LOG_WARN("failed to lock for add drop partition", K(ret));
|
|
} else if (OB_FAIL(gen_inc_table_schema_for_add_part(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to gen inc table schema for add part",
|
|
KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(inc_table_schema))) {
|
|
LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(fix_local_idx_part_name_for_add_part_(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to fix local idx part name for add part", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.add_table_partitions(orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to add table partitions", KR(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == op_type) {
|
|
if (OB_FAIL(ObDDLLock::lock_for_add_partition_in_trans(orig_table_schema, trans))) {
|
|
LOG_WARN("failed to lock for add drop partition", K(ret));
|
|
} else if (OB_FAIL(fix_local_idx_part_name_(orig_data_table_schema, orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to fix local idx part name", KR(ret), K(orig_data_table_schema), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(gen_inc_table_schema_for_add_subpart(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to gen inc table schema for add subpart",
|
|
KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema, true))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(inc_table_schema))) {
|
|
LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(fix_local_idx_part_name_for_add_subpart_(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to fix local idx part name for add subpart", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} 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::RENAME_PARTITION == op_type) {
|
|
if (OB_FAIL(gen_inc_table_schema_for_rename_part_(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to gen inc table schema for rename part",
|
|
KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(lock_partitions(trans, inc_table_schema))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.rename_table_partitions(orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to rename table partitions", KR(ret), K(orig_table_schema), K(inc_table_schema), K(new_table_schema));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::RENAME_SUB_PARTITION == op_type) {
|
|
if (OB_FAIL(gen_inc_table_schema_for_rename_subpart_(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to gen inc table schema for rename subpart",
|
|
KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(lock_partitions(trans, inc_table_schema))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.rename_table_subpartitions(orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to rename table subpartitions", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::DROP_PARTITION == op_type) {
|
|
if (OB_FAIL(fix_local_idx_part_name_(orig_data_table_schema, orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fix local idx part name", KR(ret), K(orig_data_table_schema), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(gen_inc_table_schema_for_drop_part(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to gen inc table schema for drop part",
|
|
KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(lock_partitions(trans, inc_table_schema))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.drop_table_partitions(orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to drop table partitions", KR(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::DROP_SUB_PARTITION == op_type) {
|
|
if (inc_table_schema.get_partition_num() != 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("drop subparts not in a part", KR(ret));
|
|
} else if (OB_FAIL(fix_local_idx_subpart_name_(orig_data_table_schema, orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to fix local idx subpart name", KR(ret), K(orig_data_table_schema), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(gen_inc_table_schema_for_drop_subpart(orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to gen inc table for drop subpart",
|
|
KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(lock_partitions(trans, inc_table_schema))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.drop_table_subpartitions(orig_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to drop table partitions", KR(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == op_type) {
|
|
if (OB_FAIL(fix_local_idx_part_name_(orig_data_table_schema, orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fix local idx part name", KR(ret), K(orig_data_table_schema), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(gen_inc_table_schema_for_trun_part(
|
|
orig_table_schema, inc_table_schema, del_table_schema))) {
|
|
LOG_WARN("fail to generate inc table schema", KR(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(lock_partitions(trans, del_table_schema))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(del_table_schema));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(inc_table_schema))) {
|
|
LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.truncate_table_partitions(orig_table_schema,
|
|
inc_table_schema,
|
|
del_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to truncate partitions", KR(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == op_type) {
|
|
if (OB_FAIL(fix_local_idx_subpart_name_(orig_data_table_schema, orig_table_schema, inc_table_schema))) {
|
|
LOG_WARN("fail to fix local idx subpart name", KR(ret), K(orig_data_table_schema), K(orig_table_schema), K(inc_table_schema));
|
|
} else if (OB_FAIL(gen_inc_table_schema_for_trun_subpart(
|
|
orig_table_schema, inc_table_schema, del_table_schema))) {
|
|
LOG_WARN("fail to generate inc table schema", KR(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(lock_partitions(trans, del_table_schema))) {
|
|
LOG_WARN("failed to get tablet ids", KR(ret), K(orig_table_schema), K(del_table_schema));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema, true))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(inc_table_schema))) {
|
|
LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.truncate_table_subpartitions(orig_table_schema,
|
|
inc_table_schema,
|
|
del_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to drop table partitions", KR(ret));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::SET_INTERVAL == op_type) {
|
|
// interval part table: modify interval
|
|
// range part table: switch interval part table
|
|
if (OB_FAIL(fill_interval_info_for_set_interval(orig_table_schema,
|
|
new_table_schema,
|
|
inc_table_schema))) {
|
|
LOG_WARN("failed to fill interval info for set interval", KR(ret),
|
|
K(orig_table_schema), K(inc_table_schema));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::INTERVAL_TO_RANGE == op_type) {
|
|
if (OB_FAIL(reset_interval_info_for_interval_to_range(new_table_schema))) {
|
|
LOG_WARN("failed to reset interval info for interval to range", KR(ret),
|
|
K(orig_table_schema), K(inc_table_schema));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_add_and_drop_partition(op_type) && !alter_table_arg.is_split_partition()) {
|
|
if (OB_FAIL(check_alter_partition_with_tablegroup(&orig_table_schema, new_table_schema, schema_guard))) {
|
|
LOG_WARN("fail to check alter partition with tablegroup", KR(ret), K(orig_table_schema), K(new_table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_enable_sys_table_ddl(const ObTableSchema &table_schema,
|
|
const ObSchemaOperationType operation_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (is_inner_table(table_schema.get_table_id())) {
|
|
const bool enable_sys_table_ddl = common::ObServerConfig::get_instance().enable_sys_table_ddl;
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE];
|
|
if (!enable_sys_table_ddl) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
switch(operation_type) {
|
|
case OB_DDL_CREATE_TABLE: {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s", "create system table");
|
|
break;
|
|
}
|
|
case OB_DDL_ALTER_TABLE: {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s", "alter system table");
|
|
break;
|
|
}
|
|
case OB_DDL_TABLE_RENAME: {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s", "rename system table");
|
|
break;
|
|
}
|
|
case OB_DDL_DROP_TABLE: {
|
|
if (table_schema.is_view_table()) {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s", "drop system view");
|
|
} else {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s", "drop system table");
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_TRUNCATE_TABLE_CREATE: {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s", "truncate system table");
|
|
break;
|
|
}
|
|
default : {
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s", "ddl on system table");
|
|
}
|
|
}
|
|
} else if (OB_DDL_DROP_TABLE == operation_type && table_schema.has_partition()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s", "drop partitioned system table");
|
|
} else if (OB_DDL_DROP_COLUMN == operation_type && is_sys_table(table_schema.get_table_id())) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"%s", "system table drop column");
|
|
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// FIXME: this function should move to observer
|
|
int ObDDLService::alter_table_sess_active_time_in_trans(obrpc::ObAlterTableArg &alter_table_arg,
|
|
obrpc::ObAlterTableRes &res,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
alter_table_schema.set_sess_active_time(ObTimeUtility::current_time());
|
|
ObArray<uint64_t> tenant_ids;
|
|
ObDDLType &ddl_type = res.ddl_type_;
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
} else if (OB_FAIL(alter_table_schema.alter_option_bitset_.add_member(obrpc::ObAlterTableArg::SESSION_ACTIVE_TIME))) {
|
|
LOG_WARN("failed to add member SESSION_ACTIVE_TIME for alter table schema", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_ids(tenant_ids))) {
|
|
LOG_WARN("fail to get schema guard", K(ret));
|
|
} else {
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObSEArray<const ObSimpleTableSchemaV2 *, 512> table_schemas;
|
|
bool found = false;
|
|
FOREACH_CNT_X(tenant_id, tenant_ids, OB_SUCC(ret) && !found) {
|
|
if (OB_FAIL(schema_service_->get_tenant_schema_guard(*tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get tenant schema guard", KR(ret), "tenant_id", *tenant_id);
|
|
} else if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(*tenant_id, table_schemas))) {
|
|
LOG_WARN("fail to get table schema", K(ret), "tenant_id", *tenant_id);
|
|
} else {
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); i++) {
|
|
const ObSimpleTableSchemaV2 *table_schema = table_schemas.at(i);
|
|
if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("got invalid schema", K(ret), K(i));
|
|
} else if (table_schema->is_tmp_table() && alter_table_arg.session_id_ == table_schema->get_session_id()) {
|
|
// session_id must belong to one tenant
|
|
found = true;
|
|
database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(*tenant_id, table_schema->get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get database schema", K(ret), "tenant_id", *tenant_id);
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database schema is null", K(ret));
|
|
} else if (database_schema->is_in_recyclebin() || table_schema->is_in_recyclebin()) {
|
|
LOG_INFO("skip table schema in recyclebin", K(*table_schema));
|
|
} else {
|
|
uint64_t tenant_data_version = 0;
|
|
const uint64_t tenant_id = table_schema->get_tenant_id();
|
|
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(tenant_id);
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_is_offline_ddl(alter_table_arg, tenant_data_version, res.ddl_type_, res.ddl_need_retry_at_executor_))) {
|
|
LOG_WARN("failed to to check is offline ddl", K(ret));
|
|
} else {
|
|
// offline ddl cannot appear at the same time with other ddl types
|
|
if (is_long_running_ddl(res.ddl_type_)) {
|
|
if (OB_FAIL(do_offline_ddl_in_trans(alter_table_arg, tenant_data_version, res))) {
|
|
LOG_WARN("failed to do offline ddl in trans", K(ret), K(alter_table_arg));;
|
|
}
|
|
} else {
|
|
if (OB_FAIL(alter_table_in_trans(alter_table_arg, res, tenant_data_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));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
}
|
|
} // end FOREACH_CNT_X
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_tables_attribute(ObIArray<ObTableSchema*> &new_table_schemas,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
const ObSchemaOperationType operation_type,
|
|
const ObString &ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t schema_count = new_table_schemas.count();
|
|
if (schema_count < 1) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("local index should exist", K(new_table_schemas), KR(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < schema_count; ++i) {
|
|
if (OB_ISNULL(new_table_schemas.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema ptr is NULL", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_table_attribute(*new_table_schemas.at(i),
|
|
trans,
|
|
operation_type,
|
|
0 == i ? &ddl_stmt_str : NULL))) {
|
|
LOG_WARN("failed to update data table schema version and max used column is!", KR(ret), KPC(new_table_schemas.at(i)));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_need_add_progressive_round(
|
|
const uint64_t tenant_data_version,
|
|
const ObTableSchema &table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
bool &need_add_progressive_round)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
need_add_progressive_round = true;
|
|
bool is_column_store_schema = false;
|
|
bool has_all_column_group = false;
|
|
if (tenant_data_version < DATA_VERSION_4_3_4_0) {
|
|
// do nothing
|
|
} else if (OB_FAIL(table_schema.get_is_column_store(is_column_store_schema))) {
|
|
LOG_WARN("failed to get is column store", KR(ret));
|
|
} else if (!is_column_store_schema) {
|
|
} else if (OB_FAIL(table_schema.has_all_column_group(has_all_column_group))) {
|
|
LOG_WARN("failed to check has all column group", KR(ret));
|
|
} else if (!has_all_column_group) {
|
|
// for pure column store table
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
need_add_progressive_round = false;
|
|
for (;OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else if (OB_DDL_ADD_COLUMN != alter_column_schema->alter_type_) {
|
|
need_add_progressive_round = true;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
//fix me :Check whether the newly added index column covers the partition column --by rongxuan.lc
|
|
// It can be repaired after the featrue that add index in alter_table statement
|
|
int ObDDLService::alter_table_in_trans(obrpc::ObAlterTableArg &alter_table_arg,
|
|
obrpc::ObAlterTableRes &res,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLType &ddl_type = res.ddl_type_;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
HEAP_VAR(ObArray<ObDDLTaskRecord>, ddl_tasks) {
|
|
HEAP_VAR(ObTableSchema, new_table_schema) {
|
|
HEAP_VAR(ObAlterTableArg, const_alter_table_arg) {
|
|
HEAP_VAR(ObMockFKParentTableSchema, mock_fk_parent_table_schema) {
|
|
HEAP_VAR(ObSchemaGetterGuard, schema_guard) {
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
int64_t refreshed_schema_version = 0;
|
|
ObArenaAllocator allocator;
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
bool need_modify_notnull_validate = false;
|
|
bool is_add_not_null_col = false;
|
|
ObString empty_stmt;
|
|
const bool need_deep_copy_arg = alter_table_arg.alter_constraint_type_ == ObAlterTableArg::ADD_CONSTRAINT
|
|
|| alter_table_arg.alter_constraint_type_ == ObAlterTableArg::ALTER_CONSTRAINT_STATE
|
|
|| alter_table_arg.foreign_key_arg_list_.count() > 0;
|
|
// because the sql optimizer is using validate flag to generate optimized plan,
|
|
// if add or modify a foregin key to validate state, it should take effect after checking whether the data is satisfied with the foreigin key
|
|
if (need_deep_copy_arg && OB_FAIL(ObConstraintTask::deep_copy_table_arg(allocator,
|
|
alter_table_arg,
|
|
const_alter_table_arg))) {
|
|
LOG_WARN("deep copy table arg failed", K(ret));
|
|
} else if (alter_table_arg.foreign_key_checks_ && 1 == alter_table_arg.foreign_key_arg_list_.count()
|
|
&& alter_table_arg.foreign_key_arg_list_.at(0).need_validate_data_) {
|
|
if ((!alter_table_arg.foreign_key_arg_list_.at(0).is_modify_fk_state_
|
|
&& alter_table_arg.foreign_key_arg_list_.at(0).validate_flag_)
|
|
|| (alter_table_arg.foreign_key_arg_list_.at(0).is_modify_validate_flag_
|
|
&& alter_table_arg.foreign_key_arg_list_.at(0).validate_flag_)) {
|
|
alter_table_arg.ddl_stmt_str_ = empty_stmt;
|
|
alter_table_arg.foreign_key_arg_list_.at(0).validate_flag_ = CST_FK_NO_VALIDATE;
|
|
}
|
|
} else if (OB_FAIL(need_modify_not_null_constraint_validate(
|
|
alter_table_arg, is_add_not_null_col, need_modify_notnull_validate))) {
|
|
LOG_WARN("check need modify not null constraint validate failed", K(ret));
|
|
} else if (need_modify_notnull_validate) {
|
|
alter_table_arg.ddl_stmt_str_ = empty_stmt;
|
|
ObConstraint *cst = *alter_table_arg.alter_table_schema_.constraint_begin_for_non_const_iter();
|
|
cst->set_validate_flag(CST_FK_NO_VALIDATE);
|
|
cst->set_is_modify_validate_flag(false);
|
|
const uint64_t col_id = *(cst->cst_col_begin());
|
|
ObColumnSchemaV2 *col_schema = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.alter_table_schema_.get_column_count(); i++) {
|
|
if (alter_table_arg.alter_table_schema_.get_column_schema_by_idx(i)->get_column_id() == col_id) {
|
|
col_schema = alter_table_arg.alter_table_schema_.get_column_schema_by_idx(i);
|
|
}
|
|
}
|
|
if (OB_ISNULL(col_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column schema not found", K(ret), K(alter_table_arg));
|
|
} else {
|
|
col_schema->del_column_flag(NOT_NULL_VALIDATE_FLAG);
|
|
}
|
|
} else if (is_add_not_null_col) {
|
|
for (int64_t i = 0; i < alter_table_arg.alter_table_schema_.get_column_count() && OB_SUCC(ret); i++) {
|
|
ObColumnSchemaV2 *col_schema = NULL;
|
|
if (OB_ISNULL(col_schema = alter_table_arg.alter_table_schema_.get_column_schema_by_idx(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column schema is null", K(ret));
|
|
} else if (OB_DDL_ADD_COLUMN == static_cast<AlterColumnSchema *>(col_schema)->alter_type_) {
|
|
col_schema->set_is_hidden(true);
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_and_check_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema))) {
|
|
LOG_WARN("fail to get and check table schema", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id));
|
|
} else if (OB_UNLIKELY(NULL == tenant_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tenant schema is null", K(ret), KP(tenant_schema), K(tenant_id));
|
|
} else if (orig_table_schema->is_materialized_view()) {
|
|
const ObTableSchema *mv_orig_table_schema = nullptr;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
orig_table_schema->get_data_table_id(), mv_orig_table_schema))) {
|
|
LOG_WARN("failed to get mv container table schema", KR(ret));
|
|
} else if (OB_ISNULL(mv_orig_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mv container table schema is null", KR(ret));
|
|
} else {
|
|
orig_table_schema = mv_orig_table_schema;
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
bool need_update_index_table = false;
|
|
AlterLocalityOp alter_locality_op = ALTER_LOCALITY_OP_INVALID;
|
|
bool need_progressive_merge = false;
|
|
if (alter_table_arg.is_alter_columns_) {
|
|
if (OB_FAIL(check_need_add_progressive_round(
|
|
tenant_data_version,
|
|
*orig_table_schema,
|
|
alter_table_arg.alter_table_schema_,
|
|
need_progressive_merge))) {
|
|
LOG_WARN("failed to check need progressive round", KR(ret));
|
|
}
|
|
} else if (alter_table_arg.is_alter_options_
|
|
&& alter_table_arg.need_progressive_merge()) {
|
|
if (alter_table_arg.alter_table_schema_.alter_option_bitset_.
|
|
has_member(ObAlterTableArg::ENCRYPTION) &&
|
|
alter_table_arg.alter_table_schema_.is_equal_encryption(*orig_table_schema)) {
|
|
// If the values before and after changing the encryption algorithm in the table are the same,
|
|
// the merge is not marked
|
|
} else {
|
|
need_progressive_merge = true;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && 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)) {
|
|
// check foreign key info and check constraint
|
|
// and put foreign_key_info into table schema
|
|
if (OB_FAIL(deal_with_cst_for_alter_table(schema_guard, tenant_id, orig_table_schema, alter_table_arg, mock_fk_parent_table_schema))) {
|
|
LOG_WARN("deal with cst for alter table failed", K(ret));
|
|
} else if (MOCK_FK_PARENT_TABLE_OP_INVALID != mock_fk_parent_table_schema.get_operation_type()
|
|
&& OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) {
|
|
// scence : alter table add fk referenced a parent table which is not exist
|
|
LOG_WARN("push back to mock_fk_parent_table_schema_array failed", K(ret), K(mock_fk_parent_table_schema));
|
|
} else if (OB_FAIL(set_new_table_options(alter_table_arg,
|
|
alter_table_schema, *tenant_schema, new_table_schema,
|
|
*orig_table_schema, schema_guard,
|
|
need_update_index_table, alter_locality_op))) {
|
|
LOG_WARN("failed to set new table options", K(ret), K(new_table_schema),
|
|
K(*orig_table_schema), K(ret));
|
|
} else {
|
|
new_table_schema.set_table_flags(alter_table_schema.get_table_flags());
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(orig_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("check if oracle compat mode failed", K(ret));
|
|
} else if (is_oracle_mode && OB_FAIL(check_table_udt_id_is_exist(schema_guard, alter_table_schema, tenant_id))) {
|
|
LOG_WARN("check udt id failed", KR(ret), K(alter_table_schema));
|
|
}
|
|
}
|
|
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool is_rename_and_need_table_lock =
|
|
alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::TABLE_NAME) && // is rename table;
|
|
orig_table_schema->is_user_table() && // only user table
|
|
((MOCK_DATA_VERSION_4_2_3_0 <= tenant_data_version && tenant_data_version < DATA_VERSION_4_3_0_0) /* ([4.2.3, 4.3.0)) */
|
|
|| DATA_VERSION_4_3_2_0 <= tenant_data_version /* [4.3.2, ~) */ ); // need table lock and rw defense;
|
|
if (OB_FAIL(ret)) {
|
|
//do nothing
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
// All alter table behaviors will cause the status to change, which is not as fine as oracle
|
|
} else if (need_modify_dep_obj_status(alter_table_arg)
|
|
&& OB_FAIL(ObDependencyInfo::modify_dep_obj_status(trans, tenant_id,
|
|
orig_table_schema->get_table_id(),
|
|
ddl_operator, *schema_service_))) {
|
|
LOG_WARN("failed to modify obj status", K(ret));
|
|
} else if (is_rename_and_need_table_lock &&
|
|
OB_FAIL(lock_table(trans, *orig_table_schema))) {
|
|
LOG_WARN("failed to get the table_lock of origin table schema for rename op", K(ret), KPC(orig_table_schema));
|
|
} 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(alter_table_column(*orig_table_schema,
|
|
alter_table_schema,
|
|
new_table_schema,
|
|
alter_table_arg,
|
|
schema_guard,
|
|
tenant_data_version,
|
|
ddl_operator,
|
|
trans,
|
|
&global_idx_schema_array))) {
|
|
LOG_WARN("failed to alter table column!", K(*orig_table_schema), K(new_table_schema), K(ret));
|
|
}
|
|
}
|
|
//table options
|
|
// if there is no auto-increment column, ignore table option auto_increment
|
|
if (0 == new_table_schema.get_autoinc_column_id()) {
|
|
new_table_schema.set_auto_increment(1);
|
|
}
|
|
ObSArray<std::pair<uint64_t, int64_t>> idx_schema_versions;
|
|
if (OB_FAIL(ret)) {
|
|
// error occur
|
|
} else if (OB_FAIL(ddl_operator.alter_table_options(
|
|
schema_guard,
|
|
new_table_schema,
|
|
*orig_table_schema,
|
|
(need_update_index_table || is_rename_and_need_table_lock),
|
|
trans,
|
|
&global_idx_schema_array,
|
|
&idx_schema_versions))) {
|
|
ObString origin_table_name = alter_table_schema.get_origin_table_name();
|
|
LOG_WARN("failed to alter table options,", K(origin_table_name), K(new_table_schema), K(ret));
|
|
} else if (is_rename_and_need_table_lock &&
|
|
OB_FAIL(build_rw_defense_for_table_(
|
|
tenant_data_version,
|
|
new_table_schema,
|
|
new_table_schema.get_schema_version(),
|
|
idx_schema_versions,
|
|
trans))) {
|
|
LOG_WARN("failed to add rw defense for table", K(ret), K(new_table_schema));
|
|
}
|
|
if (OB_SUCC(ret) && !alter_table_schema.alter_option_bitset_.is_empty()) {
|
|
const bool require_strict_binary_format = share::ObDDLUtil::use_idempotent_mode(tenant_data_version) && alter_table_arg.need_progressive_merge();
|
|
if (OB_FAIL(ObDDLLock::lock_for_common_ddl_in_trans(*orig_table_schema, require_strict_binary_format, trans))) {
|
|
LOG_WARN("failed to lock ddl", K(ret));
|
|
}
|
|
}
|
|
// table foreign key
|
|
if (OB_SUCC(ret)
|
|
&& !alter_table_arg.alter_table_schema_.get_foreign_key_infos().empty()) {
|
|
if (OB_FAIL(alter_table_foreign_keys(
|
|
*orig_table_schema,
|
|
alter_table_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("alter table foreign keys failed", K(ret));
|
|
}
|
|
}
|
|
|
|
if (alter_table_arg.has_rename_action()) {
|
|
OZ (rebuild_trigger_package(schema_guard, new_table_schema, ddl_operator, trans),
|
|
new_table_schema.get_table_name_str());
|
|
}
|
|
|
|
//table indexs
|
|
if (OB_SUCC(ret) && alter_table_arg.is_alter_indexs_) {
|
|
if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) {
|
|
LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id), K(orig_table_schema->get_table_id()));
|
|
} else if (OB_FAIL(alter_table_index(alter_table_arg,
|
|
*orig_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version,
|
|
res,
|
|
ddl_tasks))) {
|
|
LOG_WARN("failed to alter table index!", K(ret));
|
|
}
|
|
}
|
|
// table constraints
|
|
if (OB_SUCC(ret) && alter_table_arg.alter_constraint_type_ !=
|
|
obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION) {
|
|
if (OB_FAIL(alter_table_constraints(
|
|
alter_table_arg.alter_constraint_type_,
|
|
schema_guard,
|
|
*orig_table_schema,
|
|
alter_table_schema,
|
|
new_table_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("alter table constraints failed", K(ret));
|
|
}
|
|
}
|
|
// alter auto table attr
|
|
if (OB_SUCC(ret) && alter_table_arg.alter_auto_partition_attr_) {
|
|
ObAlterAutoPartAttrOp alter_auto_part(*this);
|
|
if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) {
|
|
LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id), K(orig_table_schema->get_table_id()));
|
|
} else if (OB_FAIL(alter_auto_part.alter_table_auto_part_attr_if_need(
|
|
alter_table_arg,
|
|
ddl_type,
|
|
schema_guard,
|
|
new_table_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("fail to alter table part attr.", K(ret));
|
|
}
|
|
}
|
|
//table partitions
|
|
// the first element is data_table_schema
|
|
// the others element are local_index_schemas
|
|
ObArray<const ObTableSchema*> orig_table_schemas;
|
|
ObArray<ObTableSchema*> new_table_schemas;
|
|
// 1. truncate part/subpart: del_table_schemas will be deleted and inc_table_schemas will be added
|
|
// 2. others: inc_table_schemas and del_table_schemas is pointed same partitions
|
|
// 3. split partition: upd_table_schemas record the partition information which need to be changed.
|
|
// 4. other situations, upd_table_schemas are useless and point to same partition with new_table_schemas.
|
|
ObArray<AlterTableSchema*> inc_table_schemas;
|
|
ObArray<AlterTableSchema*> del_table_schemas;
|
|
ObArray<ObTableSchema*> upd_table_schemas;
|
|
ObArenaAllocator allocator("ModifyPart");
|
|
//table partitions
|
|
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))) {
|
|
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,
|
|
trans,
|
|
tenant_data_version,
|
|
res,
|
|
ddl_tasks))) {
|
|
LOG_WARN("update_global_index failed", K(ret));
|
|
} else if (OB_FAIL(generate_tables_array(alter_table_arg,
|
|
orig_table_schemas,
|
|
new_table_schemas,
|
|
inc_table_schemas,
|
|
del_table_schemas,
|
|
upd_table_schemas,
|
|
*orig_table_schema,
|
|
new_table_schema,
|
|
alter_table_schema,
|
|
schema_guard,
|
|
allocator))) {
|
|
LOG_WARN("failed to generate tables array", KR(ret));
|
|
} else if (OB_FAIL(alter_tables_partitions(alter_table_arg,
|
|
orig_table_schemas,
|
|
new_table_schemas,
|
|
inc_table_schemas,
|
|
del_table_schemas,
|
|
upd_table_schemas,
|
|
ddl_operator,
|
|
schema_guard,
|
|
trans))) {
|
|
LOG_WARN("alter table partitions failed", K(ret));
|
|
} else if (orig_table_schemas.count() != new_table_schemas.count()
|
|
|| inc_table_schemas.count() != orig_table_schemas.count()
|
|
|| del_table_schemas.count() != orig_table_schemas.count()
|
|
|| inc_table_schemas.count() <= 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("array count is unexpected" , K(orig_table_schemas), K(new_table_schemas),
|
|
K(inc_table_schemas), K(del_table_schemas), KR(ret));
|
|
} else if (alter_table_arg.task_id_ > 0 && OB_FAIL(ObDDLRetryTask::update_task_status_wait_child_task_finish(trans, tenant_id, alter_table_arg.task_id_))) {
|
|
LOG_WARN("update ddl task status failed", K(ret));
|
|
} else {
|
|
if (alter_table_schema.is_external_table()) {
|
|
if (alter_table_arg.alter_part_type_ == ObAlterTableArg::ADD_PARTITION
|
|
|| alter_table_arg.alter_part_type_ == ObAlterTableArg::DROP_PARTITION) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_partition_num(); i++) {
|
|
ObAlterTableResArg arg;
|
|
CK (OB_NOT_NULL(alter_table_schema.get_part_array()) &&
|
|
OB_NOT_NULL(alter_table_schema.get_part_array()[i]));
|
|
if (OB_SUCC(ret)) {
|
|
arg.schema_id_ = alter_table_schema.get_table_id();
|
|
arg.schema_type_ = ObSchemaType::TABLE_SCHEMA;
|
|
arg.schema_version_ = alter_table_schema.get_part_array()[i]->get_schema_version();
|
|
arg.part_object_id_ = alter_table_schema.get_part_array()[i]->get_part_id();
|
|
OZ (res.res_arg_array_.push_back(arg));
|
|
}
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("alter external table op type is not valid", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// alter table auto increment value
|
|
if (OB_SUCC(ret) &&
|
|
alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::AUTO_INCREMENT) &&
|
|
0 != new_table_schema.get_autoinc_column_id()) {
|
|
if (OB_FAIL(alter_table_auto_increment(*orig_table_schema,
|
|
alter_table_schema,
|
|
alter_table_arg,
|
|
schema_guard,
|
|
new_table_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("fail to alter table auto increment value", 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 (alter_table_arg.is_split_partition()) {
|
|
operation_type = OB_DDL_SPLIT_PARTITION;
|
|
} else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_TRUNCATE_PARTITION;
|
|
} else if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_ADD_SUB_PARTITION;
|
|
} else if (obrpc::ObAlterTableArg::DROP_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_DROP_SUB_PARTITION;
|
|
} else if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_TRUNCATE_SUB_PARTITION;
|
|
} else if (obrpc::ObAlterTableArg::RENAME_PARTITION == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_RENAME_PARTITION;
|
|
} else if (obrpc::ObAlterTableArg::RENAME_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_RENAME_SUB_PARTITION;
|
|
} else if (obrpc::ObAlterTableArg::SET_INTERVAL == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_SET_INTERVAL;
|
|
} else if (obrpc::ObAlterTableArg::INTERVAL_TO_RANGE == alter_table_arg.alter_part_type_) {
|
|
operation_type = OB_DDL_INTERVAL_TO_RANGE;
|
|
}
|
|
|
|
if (!alter_table_arg.is_alter_partitions_) {
|
|
if (alter_table_arg.alter_auto_partition_attr_) {
|
|
// do nothing
|
|
} else if (OB_FAIL(ddl_operator.update_table_attribute(
|
|
new_table_schema,
|
|
trans,
|
|
operation_type,
|
|
&alter_table_arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to update tablets attribute", K(ret), K(new_table_schema));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(update_tables_attribute(
|
|
new_table_schemas,
|
|
ddl_operator,
|
|
trans,
|
|
operation_type,
|
|
alter_table_arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to update tablets attribute", K(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::SESSION_ID) &&
|
|
0 == new_table_schema.get_session_id() && !new_table_schema.is_tmp_table() &&
|
|
OB_FAIL(ddl_operator.delete_temp_table_info(trans, new_table_schema))) {
|
|
LOG_WARN("failed to delete temp table info", K(ret));
|
|
}
|
|
}
|
|
// scence : alter table rename to a mock fk parent table existed, will replace mock fk parent table with real table
|
|
if (OB_SUCC(ret) && alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::TABLE_NAME)) {
|
|
const ObMockFKParentTableSchema *ori_mock_fk_parent_table_ptr = NULL;
|
|
if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name(
|
|
orig_table_schema->get_tenant_id(),
|
|
new_table_schema.get_database_id(),
|
|
new_table_schema.get_table_name_str(),
|
|
ori_mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("get_mock_fk_parent_table_schema_with_name failed", K(ret),
|
|
K(orig_table_schema->get_tenant_id()), K(new_table_schema.get_database_id()), K(new_table_schema.get_table_name_str()));
|
|
} else if (OB_NOT_NULL(ori_mock_fk_parent_table_ptr)) {
|
|
ObArray<ObAuxTableMetaInfo> simple_index_infos;
|
|
ObArray<const share::schema::ObTableSchema*> unique_index_schemas;
|
|
if (OB_FAIL(orig_table_schema->get_simple_index_infos(simple_index_infos))) {
|
|
SERVER_LOG(WARN, "get simple_index_infos without delay_deleted_tid failed", K(ret));
|
|
} else {
|
|
const ObTableSchema *index_schema = NULL;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < simple_index_infos.count(); ++j) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(alter_table_schema.get_tenant_id(), simple_index_infos.at(j).table_id_, index_schema))) {
|
|
LOG_WARN("fail to get to_table_schema schema", K(ret));
|
|
} else if (OB_ISNULL(index_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("index_schema is null", K(ret));
|
|
} else if (index_schema->is_unique_index() && OB_FAIL(unique_index_schemas.push_back(index_schema))) {
|
|
LOG_WARN("fail to push_back index_schema to unique_index_schemas", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// alter table rename to mock fk parent table name with other actions is not supported
|
|
// so we can use orig_table_schema to check and gen mock_fk_parent_table_for_replacing
|
|
if (OB_FAIL(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table(
|
|
schema_guard, ori_mock_fk_parent_table_ptr->get_mock_fk_parent_table_id(),
|
|
*orig_table_schema, unique_index_schemas, mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table", K(ret));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (FAILEDx(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("failed to deal_with_mock_fk_parent_tables", K(ret), K(mock_fk_parent_table_schema_array.count()));
|
|
}
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("fail to get last schema version", KR(ret));
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.index_arg_list_.size(); ++i) {
|
|
const ObIndexArg *index_arg = alter_table_arg.index_arg_list_.at(i);
|
|
if (OB_ISNULL(index_arg)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("index arg should not be null", K(ret));
|
|
} else if (index_arg->index_action_type_ == ObIndexArg::ADD_INDEX) {
|
|
const ObCreateIndexArg *create_index_arg = static_cast<const ObCreateIndexArg *>(index_arg);
|
|
const ObTableSchema &index_schema = create_index_arg->index_schema_;
|
|
if (OB_FAIL(ddl_operator.insert_ori_schema_version(
|
|
trans, tenant_id, index_schema.get_table_id(), last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!", K(ret), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
const bool need_create_empty_major_sstable = !alter_table_arg.is_split_partition();
|
|
common::ObArray<bool> need_create_empty_majors;
|
|
common::ObArray<const ObTableSchema*> inc_table_schema_ptrs;
|
|
common::ObArray<const ObTableSchema*> del_table_schema_ptrs;
|
|
for (int i = 0; i < inc_table_schemas.count() && OB_SUCC(ret); i++) {
|
|
const ObTableSchema *tmp_table_schema = inc_table_schemas.at(i);
|
|
if (OB_ISNULL(tmp_table_schema) || OB_ISNULL(new_table_schemas.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schemas is NULL", KR(ret), K(i), K(tmp_table_schema));
|
|
} else if (FALSE_IT(inc_table_schemas.at(i)->set_schema_version(new_table_schemas.at(i)->get_schema_version()))) {
|
|
} else if (OB_FAIL(inc_table_schema_ptrs.push_back(tmp_table_schema))
|
|
|| OB_FAIL(need_create_empty_majors.push_back(need_create_empty_major_sstable))) {
|
|
LOG_WARN("fail to push back", KR(ret), KPC(tmp_table_schema));
|
|
}
|
|
}
|
|
for (int i = 0; i < del_table_schemas.count() && OB_SUCC(ret); i++) {
|
|
const ObTableSchema *tmp_table_schema = del_table_schemas.at(i);
|
|
if (OB_ISNULL(tmp_table_schema) || OB_ISNULL(new_table_schemas.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schemas is NULL", KR(ret), K(i), K(tmp_table_schema));
|
|
} else if (FALSE_IT(del_table_schemas.at(i)->set_schema_version(new_table_schemas.at(i)->get_schema_version()))) {
|
|
} else if (OB_FAIL(del_table_schema_ptrs.push_back(tmp_table_schema))) {
|
|
LOG_WARN("fail to push back", KR(ret), KPC(tmp_table_schema));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (obrpc::ObAlterTableArg::DROP_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::DROP_SUB_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObTabletDrop tablet_drop(tenant_id, trans, new_schema_version);
|
|
if (alter_table_arg.alter_table_schema_.is_external_table()) {
|
|
} else if (OB_FAIL(tablet_drop.init())) {
|
|
LOG_WARN("fail to init tablet drop", KR(ret), K(del_table_schema_ptrs));
|
|
} else if (OB_FAIL(tablet_drop.add_drop_tablets_of_table_arg(del_table_schema_ptrs))) {
|
|
LOG_WARN("failed to add drop tablets", KR(ret), K(del_table_schema_ptrs));
|
|
} else if (OB_FAIL(tablet_drop.execute())) {
|
|
LOG_WARN("failed to execute", KR(ret), K(del_table_schema_ptrs));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (obrpc::ObAlterTableArg::ADD_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
SCN frozen_scn;
|
|
const bool need_check_tablet_cnt = obrpc::ObAlterTableArg::ADD_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_table_arg.alter_part_type_;
|
|
if (alter_table_arg.alter_table_schema_.is_external_table()) {
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObTableCreator table_creator(tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
const ObTableSchema *tmp_table_schema = inc_table_schema_ptrs.at(0);
|
|
const ObTablegroupSchema *tablegroup_schema = NULL; // keep NULL if no tablegroup
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(tenant_id, schema_guard, sql_proxy_);
|
|
if (OB_ISNULL(tmp_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema is null", KR(ret), K(inc_table_schema_ptrs));
|
|
} else if (OB_FAIL(table_creator.init(need_check_tablet_cnt))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
} else if (OB_INVALID_ID != tmp_table_schema->get_tablegroup_id()) {
|
|
if (OB_FAIL(schema_guard.get_tablegroup_schema(
|
|
tmp_table_schema->get_tenant_id(),
|
|
tmp_table_schema->get_tablegroup_id(),
|
|
tablegroup_schema))) {
|
|
LOG_WARN("get tablegroup_schema failed", KR(ret), KPC(tmp_table_schema));
|
|
} else if (OB_ISNULL(tablegroup_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tablegroup_schema is null", KR(ret), KPC(tmp_table_schema));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(new_table_tablet_allocator.prepare(trans, *tmp_table_schema, tablegroup_schema, true))) {
|
|
LOG_WARN("failed to prepare tablet allocator", KR(ret), KPC(tmp_table_schema));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg(
|
|
inc_table_schema_ptrs,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors/*need_create_empty_majors*/))) {
|
|
LOG_WARN("create table partitions failed", KR(ret), K(alter_table_schema),
|
|
K(inc_table_schema_ptrs));
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("execute create partition failed", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && alter_table_arg.is_split_partition()) {
|
|
// TODO(zhenhan.gzh): modify ddl_type based on split type(auto/manual) and part_type
|
|
const ObDDLType ddl_type = alter_table_arg.is_auto_split_partition() ? ObDDLType::DDL_AUTO_SPLIT_BY_RANGE : ObDDLType::DDL_MANUAL_SPLIT_BY_RANGE;
|
|
ObSplitPartitionHelper split_partition_helper(trans, schema_guard, alter_table_arg.allocator_, tenant_id,
|
|
tenant_data_version, ddl_type, new_table_schemas, upd_table_schemas, inc_table_schema_ptrs, alter_table_arg.parallelism_);
|
|
ObDDLTaskRecord task_record;
|
|
if (OB_FAIL(split_partition_helper.execute(task_record))) {
|
|
LOG_WARN("failed to execute split partition helper", KR(ret));
|
|
} else if (OB_FAIL(ddl_tasks.push_back(task_record))) {
|
|
LOG_WARN("failed to push back", KR(ret));
|
|
} else {
|
|
res.ddl_type_ = ddl_type;
|
|
res.task_id_ = task_record.task_id_;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_table_arg.alter_part_type_) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("split partition is not supported", KR(ret), K(alter_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "split partition is");
|
|
}
|
|
|
|
if (OB_SUCC(ret) &&
|
|
alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::INCREMENT_MODE) &&
|
|
0 != orig_table_schema->get_autoinc_column_id()) {
|
|
ObAutoincrementService &autoinc_service = ObAutoincrementService::get_instance();
|
|
if (OB_FAIL(autoinc_service.clear_autoinc_cache_all(tenant_id,
|
|
orig_table_schema->get_table_id(),
|
|
orig_table_schema->get_autoinc_column_id(),
|
|
orig_table_schema->is_order_auto_increment_mode()))) {
|
|
LOG_WARN("fail to clear autoinc cache", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (DDL_CREATE_INDEX == ddl_type || DDL_CREATE_PARTITIONED_LOCAL_INDEX == ddl_type || DDL_NORMAL_TYPE == ddl_type) {
|
|
ObIndexBuilder index_builder(*this);
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
int tmp_ret = OB_SUCCESS;
|
|
const int64_t new_table_id = new_table_schema.get_table_id();
|
|
ObArray<ObTabletID> inc_tablet_ids;
|
|
ObArray<ObTabletID> del_tablet_ids;
|
|
if (index_arg_list.count() > 0) {
|
|
if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
if (OB_FAIL(get_tablets_with_table_id_(inc_table_schemas, new_table_id, inc_tablet_ids))) {
|
|
LOG_WARN("fail to get tablets with table id", KR(ret), K(inc_table_schemas), K(new_table_id));
|
|
}
|
|
}
|
|
if (obrpc::ObAlterTableArg::DROP_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::DROP_SUB_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_table_arg.alter_part_type_
|
|
|| obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == alter_table_arg.alter_part_type_) {
|
|
if (FAILEDx(get_tablets_with_table_id_(del_table_schemas, new_table_id, del_tablet_ids))) {
|
|
LOG_WARN("fail to get tablets with table id", KR(ret), K(inc_table_schemas), K(new_table_id));
|
|
}
|
|
}
|
|
|
|
}
|
|
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));
|
|
ObDDLTaskRecord task_record;
|
|
if (OB_ISNULL(index_arg)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("index arg should not be null", KR(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_PRIMARY == create_index_arg->index_type_ ||
|
|
is_vec_index(create_index_arg->index_type_)) {
|
|
} else if (OB_FAIL(index_builder.submit_build_index_task(trans,
|
|
*create_index_arg,
|
|
orig_table_schema,
|
|
&inc_tablet_ids,
|
|
&del_tablet_ids,
|
|
&index_schema,
|
|
alter_table_arg.parallelism_,
|
|
const_alter_table_arg.consumer_group_id_,
|
|
tenant_data_version,
|
|
alter_table_arg.allocator_,
|
|
task_record))) {
|
|
LOG_WARN("fail to submit build index task", KR(ret), "type", create_index_arg->index_type_);
|
|
} else if (OB_FAIL(ddl_tasks.push_back(task_record))) {
|
|
LOG_WARN("fail to push ddl task", KR(ret), K(task_record));
|
|
} else {
|
|
res.task_id_ = task_record.task_id_;
|
|
ObDDLRes ddl_res;
|
|
ddl_res.tenant_id_ = tenant_id;
|
|
ddl_res.schema_id_ = create_index_arg->index_schema_.get_schema_version();
|
|
ddl_res.task_id_ = task_record.task_id_;
|
|
obrpc::ObAlterTableResArg arg(TABLE_SCHEMA,
|
|
create_index_arg->index_schema_.get_table_id(),
|
|
create_index_arg->index_schema_.get_schema_version());
|
|
if (OB_FAIL(res.res_arg_array_.push_back(arg))) {
|
|
LOG_WARN("push back to res_arg_array failed", KR(ret), K(arg));
|
|
} else if (OB_FAIL(res.ddl_res_array_.push_back(ddl_res))) {
|
|
LOG_WARN("failed to push back ddl res array", KR(ret));
|
|
}
|
|
}
|
|
// TODO @wenyu alter table drop index submit drop index task in alter_table_index() now.
|
|
// should be unified here
|
|
} else if (ObIndexArg::DROP_INDEX == index_arg->index_action_type_ && !alter_table_arg.is_alter_indexs_) {
|
|
ObDropIndexArg *drop_index_arg = static_cast<ObDropIndexArg *>(index_arg);
|
|
if (OB_ISNULL(drop_index_arg)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("drop index arg is null", KR(ret));
|
|
} else if (OB_FAIL(drop_index_to_scheduler_(trans, schema_guard, alter_table_arg.allocator_ , *orig_table_schema,
|
|
&inc_tablet_ids, &del_tablet_ids, drop_index_arg,
|
|
ddl_operator, res, ddl_tasks))) {
|
|
LOG_WARN("fail to drop index to scheduler", KR(ret), KPC(drop_index_arg));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::ADD_CONSTRAINT
|
|
|| alter_table_arg.alter_constraint_type_ == obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE) {
|
|
ObTableSchema::const_constraint_iterator iter = alter_table_arg.alter_table_schema_.constraint_begin();
|
|
/*
|
|
* observer need return while create index, create constraint or modify constraint.
|
|
* in the resolver phase, it ensures that no other actions will happen at the same time while alter table.
|
|
* check constraint need return constriant_id_ and schema_version_. other constraint return schema_version_.
|
|
* the schema version is data table after finish alter table.
|
|
*/
|
|
const bool need_check = need_check_constraint_validity(const_alter_table_arg);
|
|
res.constriant_id_ = (*iter)->get_constraint_id();
|
|
res.schema_version_ = new_table_schema.get_schema_version();
|
|
if (need_check) {
|
|
bool need_modify_notnull_validate = false;
|
|
bool is_add_not_null_col = false;
|
|
if (OB_FAIL(need_modify_not_null_constraint_validate(
|
|
const_alter_table_arg, is_add_not_null_col, need_modify_notnull_validate))) {
|
|
} else {
|
|
ObDDLTaskRecord task_record;
|
|
ObTableLockOwnerID owner_id;
|
|
ddl_type = is_add_not_null_col ?
|
|
ObDDLType::DDL_ADD_NOT_NULL_COLUMN : ObDDLType::DDL_CHECK_CONSTRAINT;
|
|
ObCreateDDLTaskParam param(new_table_schema.get_tenant_id(),
|
|
ddl_type,
|
|
&new_table_schema,
|
|
nullptr,
|
|
(*iter)->get_constraint_id(),
|
|
new_table_schema.get_schema_version(),
|
|
0/*parallelsim*/,
|
|
const_alter_table_arg.consumer_group_id_,
|
|
&alter_table_arg.allocator_,
|
|
&const_alter_table_arg);
|
|
if (OB_FAIL(GCTX.root_service_->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit constraint task failed", K(ret));
|
|
} else if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE,
|
|
task_record.task_id_))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(task_record.task_id_));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_common_ddl(new_table_schema,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to lock online ddl lock", K(ret));
|
|
} else if (OB_FAIL(ddl_tasks.push_back(task_record))) {
|
|
LOG_WARN("fail to push ddl task", K(ret), K(task_record));
|
|
} else {
|
|
res.task_id_ = task_record.task_id_;
|
|
}
|
|
}
|
|
}
|
|
} else if (const_alter_table_arg.foreign_key_arg_list_.count() > 0) {
|
|
if (1 != const_alter_table_arg.foreign_key_arg_list_.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, can not add more than one foreign key at the same time", K(ret));
|
|
} else {
|
|
const ObCreateForeignKeyArg &foreign_key_arg = const_alter_table_arg.foreign_key_arg_list_.at(0);
|
|
int64_t fk_id = OB_INVALID_ID;
|
|
res.schema_version_ = new_table_schema.get_schema_version();
|
|
if (const_alter_table_arg.foreign_key_checks_
|
|
&& foreign_key_arg.need_validate_data_
|
|
&& ((!foreign_key_arg.is_modify_fk_state_
|
|
&& foreign_key_arg.validate_flag_)
|
|
|| (foreign_key_arg.is_modify_validate_flag_
|
|
&& foreign_key_arg.validate_flag_))) {
|
|
const ObIArray<ObForeignKeyInfo> &fk_infos = alter_table_schema.get_foreign_key_infos();
|
|
const int64_t fk_cnt = fk_infos.count();
|
|
const ObTableSchema *parent_table_schema = nullptr;
|
|
ObDDLTaskRecord task_record;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < fk_infos.count(); ++i) {
|
|
const ObForeignKeyInfo fk_info = fk_infos.at(i);
|
|
if (0 == foreign_key_arg.foreign_key_name_.compare(fk_info.foreign_key_name_)) {
|
|
fk_id = fk_info.foreign_key_id_;
|
|
if (fk_info.parent_table_id_ != fk_info.child_table_id_) {
|
|
// add fk parent table obj info for ddl task record
|
|
const_alter_table_arg.based_schema_object_infos_.reset();
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, fk_info.parent_table_id_, parent_table_schema))) {
|
|
LOG_WARN("failed to get fk parent table schema", K(ret));
|
|
} else if (OB_ISNULL(parent_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("parent table not exist", K(ret), K(fk_info));
|
|
} else if (OB_FAIL(const_alter_table_arg.based_schema_object_infos_.push_back(ObBasedSchemaObjectInfo(
|
|
parent_table_schema->get_table_id(), TABLE_SCHEMA, parent_table_schema->get_schema_version())))) {
|
|
LOG_WARN("failed to add fk info", K(ret));
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_INVALID_ID == fk_id) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("can not find foreign key", K(ret));
|
|
} else {
|
|
ObTableLockOwnerID owner_id;
|
|
ObCreateDDLTaskParam param(new_table_schema.get_tenant_id(),
|
|
ObDDLType::DDL_FOREIGN_KEY_CONSTRAINT,
|
|
&new_table_schema,
|
|
nullptr,
|
|
fk_id,
|
|
new_table_schema.get_schema_version(),
|
|
0/*parallelism*/,
|
|
const_alter_table_arg.consumer_group_id_,
|
|
&alter_table_arg.allocator_,
|
|
&const_alter_table_arg);
|
|
if (OB_FAIL(GCTX.root_service_->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit constraint task", K(ret));
|
|
} else if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE,
|
|
task_record.task_id_))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(task_record.task_id_));
|
|
} else if (nullptr != parent_table_schema && OB_FAIL(ObDDLLock::lock_for_common_ddl(*parent_table_schema,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to lock online ddl lock", K(ret));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_common_ddl(new_table_schema,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to lock online ddl lock", K(ret));
|
|
} else if (OB_FAIL(ddl_tasks.push_back(task_record))) {
|
|
LOG_WARN("fail to push ddl task", K(ret), K(task_record));
|
|
} else {
|
|
res.task_id_ = task_record.task_id_;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
const bool is_commit = OB_SUCC(ret);
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(is_commit))) {
|
|
LOG_WARN("trans end failed", K(is_commit), K(temp_ret));
|
|
ret = is_commit ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
DEBUG_SYNC(AFTER_CREATE_SPLIT_TASK);
|
|
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCCESS == tmp_ret && i < ddl_tasks.count(); i++) {
|
|
ObDDLTaskRecord &task_record = ddl_tasks.at(i);
|
|
if (OB_TMP_FAIL(GCTX.root_service_->get_ddl_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(delete_auto_increment_attribute(orig_table_schema, new_table_schema, alter_table_schema))) {
|
|
LOG_WARN("fail to delete auto-incr attribute", K(ret), KPC(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
DEBUG_SYNC(AFTER_CREATE_SPLIT_TASK);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_column_group(const obrpc::ObAlterTableArg &alter_table_arg, ObDDLType &ddl_type) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_DDL_ADD_COLUMN_GROUP == alter_table_arg.alter_table_schema_.alter_type_ ||
|
|
OB_DDL_DROP_COLUMN_GROUP == alter_table_arg.alter_table_schema_.alter_type_) {
|
|
ddl_type = ObDDLType::DDL_ALTER_COLUMN_GROUP;
|
|
if (alter_table_arg.alter_table_schema_.get_column_group_count() <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument, alter table arg don't have any column group when alter column group",
|
|
K(ret), K(alter_table_arg.alter_table_schema_));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::check_is_offline_ddl(ObAlterTableArg &alter_table_arg,
|
|
const uint64_t tenant_data_version,
|
|
ObDDLType &ddl_type,
|
|
bool &ddl_need_retry_at_executor)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ddl_type = ObDDLType::DDL_INVALID;
|
|
bool is_oracle_mode = false;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_UNLIKELY(tenant_data_version <= 0)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", K(ret), K(tenant_data_version));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret));
|
|
} else {
|
|
char err_msg[number::ObNumber::MAX_PRINTABLE_SIZE] = {0};
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
if (OB_FAIL(get_and_check_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema))) {
|
|
LOG_WARN("fail to get and check table schema", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", KR(ret), KPC(orig_table_schema));
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.is_alter_columns_
|
|
&& OB_FAIL(check_alter_table_column(alter_table_arg,
|
|
*orig_table_schema,
|
|
schema_guard,
|
|
is_oracle_mode,
|
|
tenant_data_version,
|
|
ddl_type,
|
|
ddl_need_retry_at_executor))) {
|
|
LOG_WARN("fail to check alter table column", K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.is_alter_indexs_
|
|
&& OB_FAIL(check_alter_table_index(alter_table_arg, ddl_type))) {
|
|
LOG_WARN("fail to check alter table index", K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.is_alter_partitions_
|
|
&& OB_FAIL(check_alter_table_partition(alter_table_arg,
|
|
*orig_table_schema,
|
|
is_oracle_mode,
|
|
ddl_type))) {
|
|
LOG_WARN("fail to check alter table partition", K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.alter_auto_partition_attr_) {
|
|
ObAlterAutoPartAttrOp alter_auto_table(*this);
|
|
if (OB_FAIL(alter_auto_table.check_alter_table_partition_attr(alter_table_arg,
|
|
*orig_table_schema,
|
|
is_oracle_mode,
|
|
ddl_type))) {
|
|
LOG_WARN("fail to check alter table partition", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(check_alter_column_group(alter_table_arg, ddl_type))) {
|
|
LOG_WARN("fail to check alter column gorup", K(ret), K(alter_table_arg.alter_table_schema_), K(ddl_type));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && alter_table_arg.alter_constraint_type_!= obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION
|
|
&& OB_FAIL(check_alter_table_constraint(alter_table_arg, *orig_table_schema, ddl_type))) {
|
|
LOG_WARN("fail to check alter table constraint", K(ret), K(alter_table_arg), K(ddl_type));
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.is_convert_to_character_
|
|
&& OB_FAIL(check_convert_to_character(alter_table_arg, *orig_table_schema, ddl_type))) {
|
|
LOG_WARN("fail to check convert to character", K(ret));
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.foreign_key_arg_list_.count() > 0 && ddl_type == ObDDLType::DDL_INVALID) {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
}
|
|
|
|
bool is_dec_lob_inrow_threshold = false;
|
|
if (OB_SUCC(ret) && OB_FAIL(check_alter_lob_inrow_threshold(
|
|
alter_table_arg, alter_table_schema, orig_table_schema,
|
|
is_dec_lob_inrow_threshold, ddl_type))) {
|
|
LOG_WARN("fail to check alter lob_inrow_threshold", K(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret)
|
|
&& is_long_running_ddl(ddl_type)
|
|
&& ! is_dec_lob_inrow_threshold
|
|
&& (alter_table_arg.is_alter_options_
|
|
|| !alter_table_arg.alter_table_schema_.get_foreign_key_infos().empty())) {
|
|
if (alter_table_arg.is_alter_options_) {// alter options
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and alter options in single statment", ddl_type_str(ddl_type));
|
|
} else if (!alter_table_arg.alter_table_schema_.get_foreign_key_infos().empty()) {// alter foreign key
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s and alter foreign key in single statment", ddl_type_str(ddl_type));
|
|
}
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
}
|
|
if (OB_SUCC(ret) && is_long_running_ddl(ddl_type)) {
|
|
if (orig_table_schema->is_primary_vp_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s with primary vp table",
|
|
ddl_type_str(ddl_type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, err_msg);
|
|
}
|
|
// offline ddl is allowed on table with trigger(enable/disable).
|
|
}
|
|
if (OB_SUCC(ret) && (is_long_running_ddl(ddl_type)
|
|
|| alter_table_arg.is_alter_indexs_
|
|
|| alter_table_arg.alter_constraint_type_!= obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION)) {
|
|
// For add fk operation, fk occurs in orig_table_schema after publish schema,
|
|
// so the routine here can not refuse the add fk operation if related schema executing offline ddl.
|
|
// Don't worry, add_table_foreign_keys in alter_table_in_trans will refuse it.
|
|
if (OB_FAIL(check_fk_related_table_ddl(*orig_table_schema, ddl_type))) {
|
|
LOG_WARN("check whether the foreign key related table is executing ddl failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && is_double_table_long_running_ddl(ddl_type)) {
|
|
bool has_index_operation = false;
|
|
bool has_fts_or_multivalue_or_vec_index = false;
|
|
bool is_adding_constraint = false;
|
|
bool is_column_store = false;
|
|
uint64_t table_id = alter_table_arg.alter_table_schema_.get_table_id();
|
|
if (orig_table_schema->required_by_mview_refresh()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("double table long running ddl on table required by materialized view refresh is "
|
|
"not supported",
|
|
KR(ret));
|
|
LOG_USER_ERROR(
|
|
OB_NOT_SUPPORTED,
|
|
"double table long running ddl on table required by materialized view refresh is");
|
|
} else if (orig_table_schema->is_mlog_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("double table long running ddl on materialized view log is not supported", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "double table long running ddl on materialized view log is");
|
|
} else if (OB_FAIL(check_has_index_operation(schema_guard,
|
|
tenant_id,
|
|
table_id,
|
|
has_index_operation))) {
|
|
LOG_WARN("check has index operation failed", K(ret));
|
|
} else if (OB_FAIL(check_alter_domain_column_allowed(alter_table_arg, orig_table_schema))) {
|
|
LOG_WARN("failed to check domain operate column constraint", K(ret));
|
|
} else if (OB_FAIL(check_has_domain_index(schema_guard,
|
|
tenant_id,
|
|
table_id,
|
|
has_fts_or_multivalue_or_vec_index))) {
|
|
LOG_WARN("check has fts index failed", K(ret));
|
|
} else if (OB_FAIL(check_is_adding_constraint(tenant_id, table_id, is_adding_constraint))) {
|
|
LOG_WARN("failed to call check_is_adding_constraint", K(ret));
|
|
} else if (has_index_operation) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "The DDL cannot be run concurrently with creating index.");
|
|
} else if (has_fts_or_multivalue_or_vec_index) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Run this DDL operation on table with fulltext/multivalue/vector index.");
|
|
} else if (is_adding_constraint) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "The DDL cannot be run concurrently with adding constraint.");
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_domain_column_allowed(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema *orig_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
AlterColumnSchema *alter_column_schema = NULL;
|
|
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
const ObColumnSchemaV2 *orig_column_schema = orig_table_schema->get_column_schema(orig_column_name);
|
|
const ObSchemaOperationType op_type = alter_column_schema->alter_type_;
|
|
if (op_type == OB_DDL_DROP_COLUMN &&
|
|
OB_NOT_NULL(orig_column_schema) &&
|
|
orig_column_schema->is_multivalue_generated_array_column()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "user direct multivalue index generated budy column");
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_has_domain_index(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const uint64_t data_table_id,
|
|
bool &domain_index_exist)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
domain_index_exist = false;
|
|
ObRootService *root_service = GCTX.root_service_;
|
|
const ObTableSchema *table_schema = nullptr;
|
|
if (OB_ISNULL(root_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_WARN("error sys, root service must not be nullptr", K(ret));
|
|
} else if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, table_schema))) {
|
|
LOG_WARN("get table schema failed", K(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, table schema must not be nullptr", K(ret), K(data_table_id));
|
|
} else {
|
|
const common::ObIArray<ObAuxTableMetaInfo> &index_infos = table_schema->get_simple_index_infos();
|
|
if (index_infos.count() > 0) {
|
|
// if there is indexes in new tables, if so, the indexes is already rebuilt in new table
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_infos.count(); ++i) {
|
|
if (share::schema::is_doc_rowkey_aux(index_infos.at(i).index_type_) ||
|
|
share::schema::is_vec_vid_rowkey_type(index_infos.at(i).index_type_) ||
|
|
share::schema::is_vec_rowkey_vid_type(index_infos.at(i).index_type_)) {
|
|
domain_index_exist = true;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_is_oracle_mode_add_column_not_null_ddl(const obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
bool &is_oracle_mode_add_column_not_null_ddl,
|
|
bool &is_default_value_null)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_oracle_mode_add_column_not_null_ddl = false;
|
|
is_default_value_null = false;
|
|
bool is_oracle_mode = false;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const ObTableSchema *table_schema = nullptr;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(get_and_check_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
table_schema))) {
|
|
LOG_WARN("fail to get and check table schema", K(ret));
|
|
} else if (OB_FAIL(table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", KR(ret), KPC(table_schema));
|
|
} else if (is_oracle_mode
|
|
&& alter_table_arg.is_alter_columns_
|
|
&& !alter_table_arg.is_alter_indexs_
|
|
&& !alter_table_arg.is_alter_partitions_
|
|
&& !alter_table_arg.is_convert_to_character_
|
|
&& obrpc::ObAlterTableArg::ADD_CONSTRAINT == alter_table_arg.alter_constraint_type_) {
|
|
is_oracle_mode_add_column_not_null_ddl = true;
|
|
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 = nullptr;
|
|
for(; OB_SUCC(ret) && is_oracle_mode_add_column_not_null_ddl && 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_ADD_COLUMN == alter_column_schema->alter_type_
|
|
&& alter_column_schema->get_orig_default_value().is_null()
|
|
&& !alter_column_schema->is_identity_column()
|
|
&& !alter_column_schema->is_xmltype()
|
|
&& !alter_column_schema->is_first_
|
|
&& !alter_column_schema->get_next_column_name()
|
|
&& !alter_column_schema->get_prev_column_name()
|
|
&& !is_lob_storage(alter_column_schema->get_data_type())) {
|
|
// MDS can not register more than 1 buffer ctx in the same trans(RW defensive and create lob),
|
|
// so that add not null lob will not hit the fast path.
|
|
} else {
|
|
is_oracle_mode_add_column_not_null_ddl = false;
|
|
}
|
|
// Check is not null column default value null
|
|
if (!is_default_value_null && alter_column_schema->has_not_null_constraint()) {
|
|
const ObObj &cur_default_value = alter_column_schema->get_cur_default_value();
|
|
is_default_value_null = (cur_default_value.is_null_oracle() ||
|
|
(cur_default_value.is_character_type()
|
|
&& (0 == cur_default_value.get_string().case_compare(N_NULL)
|
|
|| 0 == cur_default_value.get_string().case_compare("''"))));
|
|
}
|
|
}
|
|
}
|
|
} else {
|
|
is_oracle_mode_add_column_not_null_ddl = false;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check whether there is index operation, including add index and drop index.
|
|
int ObDDLService::check_has_index_operation(ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const uint64_t table_id,
|
|
bool &has_index_operation)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
has_index_operation = false;
|
|
// 1. get table schema
|
|
const ObTableSchema *orig_table = nullptr;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> index_infos;
|
|
|
|
if (OB_UNLIKELY(tenant_id == OB_INVALID_ID || table_id == OB_INVALID_ID)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, orig_table))) {
|
|
LOG_WARN("get table schema failed", K(ret));
|
|
} else if (OB_ISNULL(orig_table)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid table id", "table_id", table_id);
|
|
} else if (OB_FAIL(orig_table->get_simple_index_infos(index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_infos.count(); i++) {
|
|
// 2. get all index schemas
|
|
const ObTableSchema *index_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
index_infos.at(i).table_id_,
|
|
index_schema))) {
|
|
LOG_WARN("get index schema failed", K(ret), K(index_infos.at(i).table_id_));
|
|
} else if (OB_ISNULL(index_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid index table id", "index_table_id", index_infos.at(i).table_id_);
|
|
} else if (index_schema->is_unavailable_index()) {
|
|
// 3. check if index is still constructing
|
|
has_index_operation = true;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check if is adding check constraint, foreign key, not null constraint
|
|
int ObDDLService::check_is_adding_constraint(const uint64_t tenant_id, const uint64_t table_id, bool &is_building)
|
|
{
|
|
ObArenaAllocator allocator(lib::ObLabel("DdlTasRecord"));
|
|
return ObDDLTaskRecordOperator::check_is_adding_constraint(sql_proxy_, allocator, tenant_id, table_id, is_building);
|
|
}
|
|
|
|
// check whether the foreign key related table is executing specifing long-running ddl.
|
|
// And ddl should be refused if the foreign key related table is executing above ddl.
|
|
int ObDDLService::check_fk_related_table_ddl(
|
|
const share::schema::ObTableSchema &data_table_schema,
|
|
const share::ObDDLType &ddl_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = data_table_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || share::ObDDLType::DDL_INVALID == ddl_type)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(tenant_id), K(ddl_type));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else {
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = data_table_schema.get_foreign_key_infos();
|
|
const ObCheckExistedDDLMode check_mode = is_double_table_long_running_ddl(ddl_type) ?
|
|
ObCheckExistedDDLMode::ALL_LONG_RUNNING_DDL : ObCheckExistedDDLMode::DOUBLE_TABLE_RUNNING_DDL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i);
|
|
const uint64_t related_table_id = data_table_schema.get_table_id() == foreign_key_info.parent_table_id_
|
|
? foreign_key_info.child_table_id_
|
|
: foreign_key_info.parent_table_id_;
|
|
bool has_long_running_ddl = false;
|
|
const ObTableSchema *related_schema = nullptr;
|
|
if (foreign_key_info.is_parent_table_mock_
|
|
|| data_table_schema.get_table_id() == related_table_id) {
|
|
// mock table and self reference foreign key table, no need to check.
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, related_table_id, related_schema))) {
|
|
LOG_WARN("get schema failed", K(ret), K(tenant_id), K(related_table_id));
|
|
} else if (OB_ISNULL(related_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, related schema is nullptr", K(ret), K(related_table_id), K(foreign_key_info));
|
|
} else if (!related_schema->check_can_do_ddl()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "execute ddl while foreign key related table is executing long running ddl");
|
|
} else if (OB_FAIL(ObDDLTaskRecordOperator::check_has_long_running_ddl(sql_proxy_,
|
|
tenant_id,
|
|
related_table_id,
|
|
check_mode,
|
|
has_long_running_ddl))) {
|
|
LOG_WARN("check has long running ddl failed", K(ret), K(tenant_id), K(related_table_id));
|
|
} else if (has_long_running_ddl) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("foreign key related table is executing offline ddl", K(ret), K(check_mode), K(tenant_id),
|
|
"table_id", data_table_schema.get_table_id(), K(related_table_id));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "execute ddl while there are some long running ddl on foreign key related table");
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_can_bind_tablets(const share::ObDDLType ddl_type,
|
|
bool &bind_tablets)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
switch (ddl_type) {
|
|
case DDL_ADD_COLUMN_OFFLINE:
|
|
case DDL_DROP_COLUMN:
|
|
case DDL_COLUMN_REDEFINITION: {
|
|
bind_tablets = true;
|
|
break;
|
|
}
|
|
default: {
|
|
bind_tablets = false;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check if the ddl contains primary key operation.
|
|
int ObDDLService::check_ddl_with_primary_key_operation(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
bool &with_primary_key_operation)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
with_primary_key_operation = false;
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
for (int64_t i = 0; OB_SUCC(ret) && !with_primary_key_operation && i < index_arg_list.size(); ++i) {
|
|
ObIndexArg *index_arg = const_cast<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 {
|
|
const ObIndexArg::IndexActionType type = index_arg->index_action_type_;
|
|
with_primary_key_operation = ObIndexArg::DROP_PRIMARY_KEY == type
|
|
|| ObIndexArg::ADD_PRIMARY_KEY == type
|
|
|| ObIndexArg::ALTER_PRIMARY_KEY == type;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::do_offline_ddl_in_trans(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const uint64_t tenant_data_version,
|
|
obrpc::ObAlterTableRes &res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool bind_tablets = false;
|
|
ObSchemaGetterGuard schema_guard;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
const ObDDLType ddl_type = res.ddl_type_;
|
|
ObRootService *root_service = GCTX.root_service_;
|
|
bool need_redistribute_column_id = false;
|
|
if (OB_UNLIKELY(DDL_INVALID == ddl_type || tenant_data_version <= 0)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("unexpected ddl type", K(ret), K(ddl_type), K(alter_table_arg), K(tenant_data_version));
|
|
} else if (OB_ISNULL(root_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, root service must not be nullptr", K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_can_bind_tablets(ddl_type, bind_tablets))) {
|
|
LOG_WARN("failed to check can bind tablets", K(ret), K(ddl_type));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObTableSchema new_table_schema;
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
if (OB_FAIL(get_and_check_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema))) {
|
|
LOG_WARN("fail to get and check table schema", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(ObSchemaUtils::mock_default_cg(orig_table_schema->get_tenant_id(), new_table_schema))) {
|
|
LOG_WARN("fail to mock default cg", K(ret), K(orig_table_schema), K(new_table_schema));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLTaskRecord task_record;
|
|
int64_t task_id = 0;
|
|
int64_t refreshed_schema_version = 0;
|
|
bool with_primary_key_operation = false;
|
|
ObTableLockOwnerID owner_id;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObDDLTask::fetch_new_task_id(root_service->get_sql_proxy(), tenant_id, task_id))) {
|
|
LOG_WARN("fetch new task id failed", K(ret));
|
|
} else if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE,
|
|
task_id))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(task_id));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_offline_ddl(*orig_table_schema,
|
|
nullptr,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to lock ddl lock", K(ret));
|
|
}
|
|
// TODO yiren, refactor it, create user hidden table after alter index/column/part/cst...
|
|
if (OB_FAIL(ret)) {
|
|
} else if (need_modify_dep_obj_status(alter_table_arg)
|
|
&& OB_FAIL(ObDependencyInfo::modify_dep_obj_status(trans, tenant_id, orig_table_schema->get_table_id(),
|
|
ddl_operator, *schema_service_))) {
|
|
LOG_WARN("failed to modify obj status", K(ret));
|
|
} else if (OB_FAIL(check_ddl_with_primary_key_operation(alter_table_arg,
|
|
with_primary_key_operation))) {
|
|
LOG_WARN("check ddl with primary key operation failed", K(ret));
|
|
} else if (with_primary_key_operation) {
|
|
if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) {
|
|
LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id), KPC(orig_table_schema));
|
|
} else if (OB_FAIL(alter_table_primary_key(alter_table_arg,
|
|
*orig_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version))) {
|
|
LOG_WARN("failed to alter table primary key", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && ObDDLType::DDL_TABLE_REDEFINITION == ddl_type) {
|
|
bool need_modify_notnull_validate = false;
|
|
bool is_add_not_null_col = false;
|
|
if (OB_FAIL(need_modify_not_null_constraint_validate(
|
|
alter_table_arg, is_add_not_null_col, need_modify_notnull_validate))) {
|
|
LOG_WARN("check need modify not null constraint validate failed", K(ret));
|
|
} else if (need_modify_notnull_validate) {
|
|
ObConstraint *cst = *alter_table_schema.constraint_begin_for_non_const_iter();
|
|
const uint64_t col_id = *(cst->cst_col_begin());
|
|
ObColumnSchemaV2 *col_schema = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_column_count(); i++) {
|
|
if (alter_table_schema.get_column_schema_by_idx(i)->get_column_id() == col_id) {
|
|
col_schema = alter_table_schema.get_column_schema_by_idx(i);
|
|
}
|
|
}
|
|
if (OB_ISNULL(col_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column schema not found", K(ret), K(alter_table_arg));
|
|
} else {
|
|
col_schema->del_column_flag(NOT_NULL_VALIDATE_FLAG);
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.is_alter_columns_) {
|
|
if (ObDDLType::DDL_MODIFY_AUTO_INCREMENT == ddl_type) {
|
|
new_table_schema.set_in_offline_ddl_white_list(true);
|
|
new_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL);
|
|
} else if (OB_FAIL(check_can_alter_column(tenant_id,
|
|
alter_table_schema,
|
|
*orig_table_schema))) {
|
|
LOG_WARN("fail to can alter column", K(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(gen_alter_column_new_table_schema_offline(
|
|
*orig_table_schema,
|
|
alter_table_schema,
|
|
new_table_schema,
|
|
alter_table_arg,
|
|
schema_guard,
|
|
need_redistribute_column_id))) {
|
|
LOG_WARN("failed to alter table column!", K(*orig_table_schema), K(new_table_schema), K(ret));
|
|
} else if (OB_FAIL(adjust_cg_for_offline(new_table_schema))) {
|
|
LOG_WARN("fail to adjust cg after alter column", K(ret));
|
|
} else if (OB_FAIL(create_user_hidden_table(*orig_table_schema,
|
|
new_table_schema,
|
|
&alter_table_arg.sequence_ddl_arg_,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version))) {
|
|
LOG_WARN("fail to create user hidden table", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.is_alter_partitions_) {
|
|
if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) {
|
|
LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id),
|
|
K(orig_table_schema->get_table_id()));
|
|
} else if (OB_FAIL(alter_table_partition_by(alter_table_arg,
|
|
*orig_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
tenant_data_version))) {
|
|
LOG_WARN("failed to alter table partition by", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.alter_auto_partition_attr_) {
|
|
const bool bind_tablets = false;
|
|
ObAlterAutoPartAttrOp alter_auto_part(*this);
|
|
// In the alter table partition by range(xxx) size(xxx) (partitions...) case,
|
|
// it is necessary to modify attributes related to automatic partitioning here.
|
|
if (OB_FAIL(check_restore_point_allow(tenant_id, *orig_table_schema))) {
|
|
LOG_WARN("check restore point allow failed,", K(ret), K(tenant_id),
|
|
K(orig_table_schema->get_table_id()));
|
|
} else if (OB_FAIL(alter_auto_part.alter_table_partition_attr(
|
|
alter_table_arg, *orig_table_schema, new_table_schema))) {
|
|
LOG_WARN("failed to alter table partition by", K(ret));
|
|
} else if (OB_FAIL(create_user_hidden_table(*orig_table_schema,
|
|
new_table_schema,
|
|
&alter_table_arg.sequence_ddl_arg_,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version))) {
|
|
LOG_WARN("fail to create user_hidden table", K(ret));
|
|
}
|
|
}
|
|
// table constraints
|
|
if (OB_SUCC(ret) && alter_table_arg.alter_constraint_type_ !=
|
|
obrpc::ObAlterTableArg::CONSTRAINT_NO_OPERATION) {
|
|
if (ObDDLType::DDL_TABLE_REDEFINITION == ddl_type
|
|
|| ObDDLType::DDL_MODIFY_COLUMN == ddl_type) {
|
|
if (OB_FAIL(alter_table_constraints(
|
|
alter_table_arg.alter_constraint_type_,
|
|
schema_guard,
|
|
*orig_table_schema,
|
|
alter_table_schema,
|
|
new_table_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("alter table constraints failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.is_convert_to_character_) {
|
|
if (OB_FAIL(convert_to_character(alter_table_arg,
|
|
*orig_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
tenant_data_version))) {
|
|
LOG_WARN("failed to convert to character", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && ddl_type == ObDDLType::DDL_ALTER_COLUMN_GROUP) {
|
|
if (OB_FAIL(alter_column_group(alter_table_arg,
|
|
*orig_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("failed to alter table column group", K(ret));
|
|
} else if (OB_FAIL(create_user_hidden_table(*orig_table_schema,
|
|
new_table_schema,
|
|
&alter_table_arg.sequence_ddl_arg_,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version))) {
|
|
LOG_WARN("fail to create user_hidden table", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && ObDDLType::DDL_TABLE_REDEFINITION == ddl_type
|
|
&& is_dec_table_lob_inrow_threshold(alter_table_arg, alter_table_schema, orig_table_schema)) {
|
|
new_table_schema.set_lob_inrow_threshold(alter_table_schema.get_lob_inrow_threshold());
|
|
OZ (create_user_hidden_table(*orig_table_schema,
|
|
new_table_schema,
|
|
&alter_table_arg.sequence_ddl_arg_,
|
|
false/*bind_tablets*/,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
tenant_data_version));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && need_redistribute_column_id) {
|
|
if (OB_FAIL(redistribute_column_ids(new_table_schema))) {
|
|
LOG_WARN("failed to redistribute column ids", K(ret));
|
|
} else {
|
|
// do nothing
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_table_schema.sort_column_array_by_column_id())) {
|
|
LOG_WARN("failed to sort column", K(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema,
|
|
trans,
|
|
operation_type))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
} else {
|
|
alter_table_arg.alter_table_schema_.set_schema_version(new_table_schema.get_schema_version());
|
|
if (ObDDLType::DDL_MODIFY_AUTO_INCREMENT != ddl_type) {
|
|
alter_table_arg.alter_table_schema_.set_association_table_id(new_table_schema.get_table_id());
|
|
}
|
|
}
|
|
}
|
|
// submit async build index task
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_double_table_long_running_ddl(ddl_type)) {
|
|
bool has_conflict_ddl = false;
|
|
ObCreateDDLTaskParam param(tenant_id,
|
|
ddl_type,
|
|
orig_table_schema,
|
|
&new_table_schema,
|
|
0/*object_id*/,
|
|
new_table_schema.get_schema_version(),
|
|
alter_table_arg.parallelism_,
|
|
alter_table_arg.consumer_group_id_,
|
|
&alter_table_arg.allocator_,
|
|
&alter_table_arg,
|
|
0/*parent_task_id*/,
|
|
task_id,
|
|
res.ddl_need_retry_at_executor_);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (orig_table_schema->is_external_table()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
char err_msg[OB_MAX_ERROR_MSG_LEN] = {0};
|
|
(void)snprintf(err_msg, sizeof(err_msg), "%s on external table is", ddl_type_str(ddl_type));
|
|
LOG_WARN("double table long running ddl on temporary table is disallowed", K(ret), K(ddl_type));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg);
|
|
} else if (OB_FAIL(ObDDLTaskRecordOperator::check_has_conflict_ddl(sql_proxy_, tenant_id, orig_table_schema->get_table_id(), 0, ddl_type, has_conflict_ddl))) {
|
|
LOG_WARN("failed to check ddl conflict", K(ret));
|
|
} else if (has_conflict_ddl) {
|
|
ret = OB_EAGAIN;
|
|
LOG_WARN("failed to alter table that has conflict ddl", K(ret), K(orig_table_schema->get_table_id()));
|
|
} else if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit ddl task failed", K(ret));
|
|
} else {
|
|
res.task_id_ = task_record.task_id_;
|
|
res.ddl_need_retry_at_executor_ = task_record.ddl_need_retry_at_executor_;
|
|
}
|
|
} else if (is_simple_table_long_running_ddl(ddl_type)) {
|
|
ObCreateDDLTaskParam param(tenant_id,
|
|
ddl_type,
|
|
&new_table_schema,
|
|
nullptr,
|
|
0/*object_id*/,
|
|
new_table_schema.get_schema_version(),
|
|
alter_table_arg.parallelism_,
|
|
alter_table_arg.consumer_group_id_,
|
|
&alter_table_arg.allocator_,
|
|
&alter_table_arg,
|
|
0/*parent_task_id*/,
|
|
task_id);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit ddl task failed", K(ret));
|
|
} else {
|
|
res.task_id_ = task_record.task_id_;
|
|
res.ddl_need_retry_at_executor_ = task_record.ddl_need_retry_at_executor_;
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to identify long running ddl type", K(ret), K(ddl_type));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN_RET(temp_ret, "trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
DEBUG_SYNC(RENAME_TABLE_BEFORE_PUBLISH_SCHEMA);
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
} else if (OB_TMP_FAIL(root_service->get_ddl_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_not_null_column_to_table_schema(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const uint64_t tenant_data_version,
|
|
const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = origin_table_schema.get_tenant_id();
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
ObSEArray<ObString, 4> gen_col_expr_arr;
|
|
share::schema::ObTableSchema::const_column_iterator iter = origin_table_schema.column_begin();
|
|
share::schema::ObTableSchema::const_column_iterator end = origin_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && iter != end; ++iter) {
|
|
const share::schema::ObColumnSchemaV2 *column = *iter;
|
|
if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid column schema", K(column));
|
|
} else if (column->is_generated_column()) {
|
|
const common::ObObj* ObObjtmp = &column->get_cur_default_value();
|
|
if (OB_FAIL(gen_col_expr_arr.push_back(ObObjtmp->get_string()))) {
|
|
LOG_WARN("fail to push back ObSEArray gen_col_expr_arr", K(ret));
|
|
}
|
|
}
|
|
}
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
const common::ObString *nls_formats = alter_table_arg.nls_formats_;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(tz_info_wrap.get_time_zone_info())
|
|
|| OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret));
|
|
} else if (OB_ISNULL(nls_formats)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid nls_formats", K(ret));
|
|
} else {
|
|
uint64_t curr_udt_set_id = 0;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else if (OB_FAIL(add_new_column_to_table_schema(origin_table_schema,
|
|
alter_table_schema,
|
|
tz_info_wrap,
|
|
*nls_formats,
|
|
alter_table_arg.local_session_var_,
|
|
alter_table_arg.sequence_ddl_arg_,
|
|
alter_table_arg.allocator_,
|
|
new_table_schema,
|
|
*alter_column_schema,
|
|
gen_col_expr_arr,
|
|
schema_guard,
|
|
curr_udt_set_id,
|
|
&ddl_operator,
|
|
&trans))) {
|
|
LOG_WARN("failed to add new column to table schema", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(check_has_multi_autoinc(new_table_schema))) {
|
|
LOG_WARN("failed to check table has multi autoinc", K(ret));
|
|
} else if (OB_FAIL(alter_table_constraints(
|
|
obrpc::ObAlterTableArg::ADD_CONSTRAINT,
|
|
schema_guard,
|
|
origin_table_schema,
|
|
alter_table_schema,
|
|
new_table_schema,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("alter table constraints failed", K(ret));
|
|
} else if (OB_FAIL(add_column_to_column_group(origin_table_schema,
|
|
alter_table_schema, new_table_schema, ddl_operator, trans))) {
|
|
LOG_WARN("fail to add_column_to_column_group", K(ret), K(alter_table_schema), K(new_table_schema));
|
|
} else if (OB_FAIL(new_table_schema.check_skip_index_valid())) {
|
|
LOG_WARN("failed to check new table schema skip index", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema, trans,
|
|
OB_DDL_ALTER_TABLE, &alter_table_arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
} else {
|
|
ObArray<ObTabletID> new_tablet_ids;
|
|
if (OB_FAIL(new_table_schema.get_tablet_ids(new_tablet_ids))) {
|
|
LOG_WARN("failed to get tablet ids", K(ret));
|
|
} else if (OB_FAIL(build_single_table_rw_defensive_(tenant_id, tenant_data_version, new_tablet_ids, new_table_schema.get_schema_version(), trans))) {
|
|
LOG_WARN("failed to build rw defensive", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_not_null_column_default_null_to_table_schema(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const uint64_t tenant_data_version,
|
|
const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_table_empty = false;
|
|
bool is_oracle_mode = false;
|
|
const uint64_t tenant_id = origin_table_schema.get_tenant_id();
|
|
if (OB_FAIL(origin_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret), K(origin_table_schema));
|
|
} else if (OB_UNLIKELY(!is_oracle_mode)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("add column not null in mysql mode is online ddl, not offline ddl", K(ret), K(is_oracle_mode));
|
|
} else if (OB_FAIL(lock_table(trans, origin_table_schema))) {
|
|
LOG_WARN("failed to lock ddl lock", K(ret));
|
|
} else if (OB_FAIL(ObDDLUtil::check_table_empty_in_oracle_mode(tenant_id,
|
|
origin_table_schema.get_table_id(),
|
|
schema_guard,
|
|
is_table_empty))) {
|
|
LOG_WARN("failed to check table empty in oracle mode", K(ret));
|
|
} else if (!is_table_empty) {
|
|
ret = OB_ERR_TABLE_ADD_NOT_NULL_COLUMN_NOT_EMPTY;
|
|
LOG_WARN("table add not null column to is not empty", K(ret), K(tenant_id), K(origin_table_schema.get_table_id()));
|
|
} else if (OB_FAIL(add_not_null_column_to_table_schema(alter_table_arg,
|
|
tenant_data_version,
|
|
origin_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("failed to add new column to table schema");
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::do_oracle_add_column_not_null_in_trans(obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_data_version,
|
|
const bool is_default_value_null)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
SMART_VAR(ObTableSchema, new_table_schema) {
|
|
const ObTableSchema *origin_table_schema = nullptr;
|
|
bool is_oracle_mode = false;
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
const common::ObString *nls_formats = alter_table_arg.nls_formats_;
|
|
if (OB_FAIL(get_and_check_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
origin_table_schema))) {
|
|
LOG_WARN("fail to get and check table schema", K(ret));
|
|
} else if (OB_FAIL(origin_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret), K(origin_table_schema));
|
|
} else if (OB_UNLIKELY(!is_oracle_mode)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("add column not null in mysql mode is online ddl, not offline ddl", K(ret), K(is_oracle_mode));
|
|
} else if (OB_ISNULL(tz_info_wrap.get_time_zone_info())
|
|
|| OB_ISNULL(tz_info_wrap.get_time_zone_info()->get_tz_info_map())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret));
|
|
} else if (OB_ISNULL(nls_formats)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid nls_formats", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.assign(*origin_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (is_default_value_null) {
|
|
if (OB_FAIL(add_not_null_column_default_null_to_table_schema(
|
|
alter_table_arg,
|
|
tenant_data_version,
|
|
*origin_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("failed to add default value null column to table schema", K(ret));
|
|
}
|
|
} else if (OB_FAIL(add_not_null_column_to_table_schema(alter_table_arg,
|
|
tenant_data_version,
|
|
*origin_table_schema,
|
|
new_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("failed to add column to table schema", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// add lob.
|
|
if (!origin_table_schema->has_lob_column() && new_table_schema.has_lob_column()) {
|
|
// ATTENTION.
|
|
// MDS can not register more than 1 buffer ctx in the same trans(RW defensive and create lob),
|
|
// so that add not null lob will not hit the fast path.
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("Not supported to fast add not null lob column in this path", K(ret), K(alter_table_arg));
|
|
}
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN_RET(temp_ret, "trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_hidden_table(
|
|
const obrpc::ObCreateHiddenTableArg &create_hidden_table_arg,
|
|
obrpc::ObCreateHiddenTableRes &res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
const uint64_t tenant_id = create_hidden_table_arg.get_tenant_id();
|
|
const int64_t table_id = create_hidden_table_arg.get_table_id();
|
|
const uint64_t dest_tenant_id = tenant_id;
|
|
ObRootService *root_service = GCTX.root_service_;
|
|
bool bind_tablets = true;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
const ObDatabaseSchema *orig_database_schema = nullptr;
|
|
common::ObArenaAllocator allocator_for_redef(lib::ObLabel("StartRedefTable"));
|
|
if (OB_UNLIKELY(!create_hidden_table_arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("create_hidden_table_arg is invalid", K(ret), K(create_hidden_table_arg));
|
|
} else if (OB_ISNULL(root_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, root service must not be nullptr", K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, orig_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret));
|
|
} else if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("orig table schema is nullptr", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, orig_table_schema->get_database_id(), orig_database_schema))) {
|
|
LOG_WARN("fail to get orig database schema", K(ret));
|
|
} else if (OB_ISNULL(orig_database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_database_schema is nullptr", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else {
|
|
HEAP_VAR(ObTableSchema, new_table_schema) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
common::ObArenaAllocator allocator;
|
|
ObDDLTaskRecord task_record;
|
|
ObTableLockOwnerID owner_id;
|
|
int64_t task_id = 0;
|
|
int64_t refreshed_schema_version = 0;
|
|
new_table_schema.set_tenant_id(dest_tenant_id);
|
|
new_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL);
|
|
new_table_schema.set_table_referenced_by_mv(ObTableReferencedByMVFlag::IS_NOT_REFERENCED_BY_MV);
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObDDLTask::fetch_new_task_id(root_service->get_sql_proxy(), tenant_id, task_id))) {
|
|
LOG_WARN("fetch new task id failed", K(ret));
|
|
} else if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE, task_id))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(task_id));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_offline_ddl(*orig_table_schema,
|
|
nullptr,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to lock ddl lock", K(ret));
|
|
} else if (OB_UNLIKELY(orig_table_schema->is_offline_ddl_table())) {
|
|
ret = OB_SCHEMA_EAGAIN;
|
|
LOG_WARN("table in offline ddl or direct load, retry", K(ret), K(orig_table_schema->get_table_id()), K(orig_table_schema->get_table_mode()));
|
|
} else if (OB_FAIL(create_user_hidden_table(
|
|
*orig_table_schema,
|
|
new_table_schema,
|
|
nullptr,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
allocator,
|
|
tenant_data_version))) {
|
|
LOG_WARN("fail to create hidden table", K(ret));
|
|
} else {
|
|
LOG_INFO("create hidden table success!", K(table_id), K(new_table_schema));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
HEAP_VAR(obrpc::ObAlterTableArg, alter_table_arg) {
|
|
ObPrepareAlterTableArgParam param;
|
|
if (OB_FAIL(param.init(create_hidden_table_arg.get_consumer_group_id(),
|
|
create_hidden_table_arg.get_session_id(),
|
|
create_hidden_table_arg.get_sql_mode(),
|
|
create_hidden_table_arg.get_ddl_stmt_str(),
|
|
orig_table_schema->get_table_name_str(),
|
|
orig_database_schema->get_database_name_str(),
|
|
orig_database_schema->get_database_name_str(),
|
|
create_hidden_table_arg.get_tz_info(),
|
|
create_hidden_table_arg.get_tz_info_wrap(),
|
|
create_hidden_table_arg.get_nls_formats()))) {
|
|
LOG_WARN("param init failed", K(ret));
|
|
} else if (OB_FAIL(root_service->get_ddl_scheduler().prepare_alter_table_arg(param, &new_table_schema, alter_table_arg))) {
|
|
LOG_WARN("prepare alter table arg fail", K(ret));
|
|
} else if (!create_hidden_table_arg.get_tablet_ids().empty()){
|
|
const ObIArray<ObTabletID> &tablet_ids = create_hidden_table_arg.get_tablet_ids();
|
|
alter_table_arg.alter_table_schema_.reset_partition_array();
|
|
for (int64_t i = 0; OB_SUCC(ret) && (i < tablet_ids.count()); ++i) {
|
|
ObPartition part;
|
|
if (OB_FALSE_IT(part.set_tablet_id(tablet_ids.at(i)))) {
|
|
} else if (OB_FAIL(alter_table_arg.alter_table_schema_.add_partition(part))) {
|
|
LOG_WARN("failed to add partition", KR(ret), K(part));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
alter_table_arg.is_direct_load_partition_ = true;
|
|
}
|
|
}
|
|
|
|
LOG_DEBUG("alter table arg preparation complete!", K(ret), K(alter_table_arg));
|
|
if (OB_SUCC(ret)) {
|
|
ObCreateDDLTaskParam param(tenant_id,
|
|
create_hidden_table_arg.get_ddl_type(),
|
|
orig_table_schema,
|
|
&new_table_schema,
|
|
table_id,
|
|
orig_table_schema->get_schema_version(),
|
|
create_hidden_table_arg.get_parallelism(),
|
|
create_hidden_table_arg.get_consumer_group_id(),
|
|
&allocator_for_redef,
|
|
&alter_table_arg,
|
|
0,
|
|
task_id);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit ddl task failed", K(ret));
|
|
} else {
|
|
res.tenant_id_ = tenant_id;
|
|
res.table_id_ = table_id;
|
|
res.dest_tenant_id_ = dest_tenant_id;
|
|
res.dest_table_id_ = task_record.target_object_id_;
|
|
res.schema_version_ = task_record.schema_version_;
|
|
res.trace_id_ = task_record.trace_id_;
|
|
res.task_id_ = task_record.task_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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
} else if (OB_TMP_FAIL(root_service->get_ddl_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record));
|
|
} else {
|
|
LOG_INFO("schedule ddl task success", K(task_record));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::mview_complete_refresh(
|
|
const obrpc::ObMViewCompleteRefreshArg &arg,
|
|
obrpc::ObMViewCompleteRefreshRes &res,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
ObRootService *root_service = GCTX.root_service_;
|
|
int64_t refreshed_schema_version = 0;
|
|
common::ObArenaAllocator allocator("MVRef");
|
|
ObDDLTaskRecord task_record;
|
|
uint64_t tenant_data_version = 0;
|
|
if (OB_UNLIKELY(!arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid args", KR(ret), K(arg));
|
|
} else if (OB_ISNULL(root_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_WARN("error unexpected, root service must not be nullptr", KR(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(mview_complete_refresh_in_trans(arg, res, trans, allocator, schema_guard, tenant_data_version, task_record))) {
|
|
LOG_WARN("failed to do mview complete refresh in trans", KR(ret), K(arg));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, KR(temp_ret));
|
|
ret = COVER_SUCC(temp_ret);
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", KR(ret));
|
|
} else if (OB_TMP_FAIL(root_service->get_ddl_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", KR(tmp_ret), K(task_record));
|
|
} else {
|
|
LOG_INFO("schedule ddl task success", K(task_record));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::mview_complete_refresh_in_trans(
|
|
const obrpc::ObMViewCompleteRefreshArg &arg,
|
|
obrpc::ObMViewCompleteRefreshRes &res,
|
|
ObDDLSQLTransaction &trans,
|
|
common::ObIAllocator &allocator,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_data_version,
|
|
ObDDLTaskRecord &task_record)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const int64_t mview_table_id = arg.table_id_;
|
|
ObRootService *root_service = GCTX.root_service_;
|
|
const ObTableSchema *mview_table_schema = nullptr;
|
|
const ObDatabaseSchema *database_schema = nullptr;
|
|
const ObTableSchema *container_table_schema = nullptr;
|
|
int64_t refreshed_schema_version = 0;
|
|
int64_t task_id = 0;
|
|
if (OB_UNLIKELY(!arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid args", KR(ret), K(arg));
|
|
} else if (OB_ISNULL(root_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_WARN("error unexpected, root service must not be nullptr", KR(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, mview_table_id, mview_table_schema))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), K(mview_table_id));
|
|
} else if (OB_ISNULL(mview_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("mview table schema is nullptr", KR(ret), K(tenant_id), K(mview_table_id));
|
|
} else if (OB_UNLIKELY(!mview_table_schema->is_materialized_view())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected not materialized view", KR(ret), KPC(mview_table_schema));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, mview_table_schema->get_database_id(), database_schema))) {
|
|
LOG_WARN("fail to get database schema", KR(ret), K(tenant_id), "database_id", mview_table_schema->get_database_id());
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database schema is nullptr", KR(ret), K(tenant_id), "database_id", mview_table_schema->get_database_id());
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, mview_table_schema->get_data_table_id(), container_table_schema))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), "container_table_id", mview_table_schema->get_data_table_id());
|
|
} else if (OB_ISNULL(container_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("container table schema is nullptr", KR(ret), K(tenant_id), "container_table_id", mview_table_schema->get_data_table_id());
|
|
} else {
|
|
HEAP_VAR(ObTableSchema, new_container_table_schema) {
|
|
if (OB_FAIL(new_container_table_schema.assign(*container_table_schema))) {
|
|
LOG_WARN("fail to assign schema", KR(ret));
|
|
} else {
|
|
const bool bind_tablets = true;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
new_container_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL);
|
|
ObTableLockOwnerID owner_id;
|
|
if (OB_FAIL(ObDDLTask::fetch_new_task_id(root_service->get_sql_proxy(), tenant_id, task_id))) {
|
|
LOG_WARN("fetch new task id failed", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE,
|
|
task_id))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(task_id));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_offline_ddl(*container_table_schema,
|
|
nullptr,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to lock ddl lock", KR(ret));
|
|
} else if (OB_FAIL(create_user_hidden_table(*container_table_schema,
|
|
new_container_table_schema,
|
|
nullptr/*sequence_ddl_arg*/,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
allocator,
|
|
tenant_data_version))) {
|
|
LOG_WARN("fail to create hidden table", KR(ret));
|
|
} else {
|
|
LOG_INFO("create hidden table success!", K(mview_table_id), "container_table_id", container_table_schema->get_table_id(),
|
|
K(new_container_table_schema));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
HEAP_VAR(obrpc::ObAlterTableArg, alter_table_arg) {
|
|
ObString empty_ddl_stmt_str;
|
|
ObPrepareAlterTableArgParam prepare_param;
|
|
ObArray<ObDependencyInfo> dependency_infos;
|
|
if (OB_FAIL(prepare_param.init(arg.consumer_group_id_,
|
|
arg.session_id_,
|
|
arg.sql_mode_,
|
|
empty_ddl_stmt_str,
|
|
container_table_schema->get_table_name_str(),
|
|
database_schema->get_database_name_str(),
|
|
database_schema->get_database_name_str(),
|
|
arg.tz_info_,
|
|
arg.tz_info_wrap_,
|
|
arg.nls_formats_))) {
|
|
LOG_WARN("prepare param init failed", KR(ret));
|
|
} else if (OB_FAIL(root_service->get_ddl_scheduler().prepare_alter_table_arg(prepare_param, &new_container_table_schema, alter_table_arg))) {
|
|
LOG_WARN("prepare alter table arg fail", KR(ret));
|
|
} else if (OB_FAIL(alter_table_arg.based_schema_object_infos_.assign(arg.based_schema_object_infos_))) {
|
|
LOG_WARN("fail to assign based schema object infos", KR(ret));
|
|
} else {
|
|
alter_table_arg.mview_refresh_info_.is_mview_complete_refresh_ = true;
|
|
alter_table_arg.mview_refresh_info_.mview_table_id_ = mview_table_id;
|
|
alter_table_arg.mview_refresh_info_.last_refresh_scn_ = arg.last_refresh_scn_;
|
|
alter_table_arg.mview_refresh_info_.start_time_ = ObTimeUtil::current_time();
|
|
LOG_DEBUG("alter table arg preparation complete!", K(alter_table_arg));
|
|
ObCreateDDLTaskParam param(tenant_id,
|
|
DDL_MVIEW_COMPLETE_REFRESH,
|
|
container_table_schema,
|
|
&new_container_table_schema,
|
|
container_table_schema->get_table_id(),
|
|
refreshed_schema_version,
|
|
arg.parallelism_,
|
|
arg.consumer_group_id_,
|
|
&allocator,
|
|
&alter_table_arg,
|
|
arg.parent_task_id_,
|
|
task_id);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit ddl task failed", KR(ret));
|
|
} else {
|
|
res.trace_id_ = task_record.trace_id_;
|
|
res.task_id_ = task_record.task_id_;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::recover_restore_table_ddl_task(
|
|
const obrpc::ObRecoverRestoreTableDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLTaskRecord task_record;
|
|
common::ObArenaAllocator allocator(lib::ObLabel("CreateDDLParam"));
|
|
ObRootService *root_service = GCTX.root_service_;
|
|
if (OB_UNLIKELY(!arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(arg));
|
|
} else if (OB_ISNULL(root_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root_service is nullptr", K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
// For the restore table operation, the role of the source tenant is backup, there will be no write operation on it,
|
|
// thus we need no lock on it.
|
|
// Same as the offline ddl, we will create a restore dest table, a hidden one with table mode `hidden_offline_ddl`.
|
|
// Different from the offline ddl, we will not change any attribute of the source table.
|
|
uint64_t tenant_data_version = 0;
|
|
int64_t refreshed_dst_tenant_version = 0;
|
|
const uint64_t session_id = arg.target_schema_.get_session_id();
|
|
ObSchemaGetterGuard hold_buf_src_tenant_schema_guard;
|
|
ObSchemaGetterGuard hold_buf_dst_tenant_schema_guard;
|
|
ObSchemaGetterGuard *src_tenant_schema_guard = nullptr;
|
|
ObSchemaGetterGuard *dst_tenant_schema_guard = nullptr;
|
|
hold_buf_src_tenant_schema_guard.set_session_id(session_id);
|
|
hold_buf_dst_tenant_schema_guard.set_session_id(session_id);
|
|
const ObTableSchema *src_table_schema = nullptr;
|
|
const ObDatabaseSchema *src_db_schema = nullptr;
|
|
const ObDatabaseSchema *dst_db_schema = nullptr;
|
|
const uint64_t src_tenant_id = arg.src_tenant_id_;
|
|
const uint64_t dst_tenant_id = arg.target_schema_.get_tenant_id();
|
|
ObDDLSQLTransaction dst_tenant_trans(schema_service_); // for dst tenant only.
|
|
bool has_fts_index = false;
|
|
bool has_mv_index = false;
|
|
HEAP_VARS_2((ObTableSchema, dst_table_schema),
|
|
(obrpc::ObAlterTableArg, alter_table_arg)) {
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(
|
|
src_tenant_id, dst_tenant_id,
|
|
hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard,
|
|
src_tenant_schema_guard, dst_tenant_schema_guard))) {
|
|
LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(src_tenant_id), K(dst_tenant_id), K(arg));
|
|
} else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(src_tenant_id, arg.src_table_id_, src_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(session_id), K(arg));
|
|
} else if (OB_ISNULL(src_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("orig table schema is nullptr", K(ret), K(session_id), K(arg));
|
|
} else if (OB_FAIL(src_table_schema->check_has_fts_index(*src_tenant_schema_guard, has_fts_index))) {
|
|
LOG_WARN("failed to check if data table has fulltext index", K(ret));
|
|
} else if (has_fts_index) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("failed to import table when table has fulltext index", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "import table with fulltext index");
|
|
} else if (OB_FAIL(src_table_schema->check_has_multivalue_index(*src_tenant_schema_guard, has_mv_index))) {
|
|
LOG_WARN("failed to check if data table has multivalue index", K(ret));
|
|
} else if (has_mv_index) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("failed to import table when table has fulltext index", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "import table with multivalue index");
|
|
} else if (OB_FAIL(src_tenant_schema_guard->get_database_schema(src_tenant_id, src_table_schema->get_database_id(), src_db_schema))) {
|
|
LOG_WARN("fail to get orig database schema", K(ret));
|
|
} else if (OB_ISNULL(src_db_schema)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("unknown database", K(ret), K(src_tenant_id), "db_id", src_table_schema->get_database_id());
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->get_schema_version(dst_tenant_id, refreshed_dst_tenant_version))) {
|
|
LOG_WARN("failed to get tenant schema version", K(ret), K(dst_tenant_id));
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->get_database_schema(dst_tenant_id, arg.target_schema_.get_database_id(), dst_db_schema))) {
|
|
LOG_WARN("fail to get orig database schema", K(ret), K(arg));
|
|
} else if (OB_ISNULL(dst_db_schema)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("unknown database", K(ret), K(dst_tenant_id), K(session_id), "db_id", arg.target_schema_.get_database_id());
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(dst_tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(dst_tenant_id));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
bool is_dest_table_column_store = false;
|
|
ObString index_name("");
|
|
if (OB_FAIL(dst_tenant_trans.start(sql_proxy_, dst_tenant_id, refreshed_dst_tenant_version))) {
|
|
LOG_WARN("start transaction failed", K(ret), K(dst_tenant_id), K(refreshed_dst_tenant_version));
|
|
} else if (OB_FAIL(dst_table_schema.assign(arg.target_schema_))) {
|
|
LOG_WARN("assign failed", K(ret), K(session_id), K(arg));
|
|
} else if (OB_FAIL(dst_table_schema.get_is_column_store(is_dest_table_column_store))) {
|
|
LOG_WARN("judge if dest table is column store failed", K(ret), K(arg));
|
|
} else if (is_dest_table_column_store) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not supported to retore table with column store", K(ret), K(arg));
|
|
} else if (OB_FAIL(create_user_hidden_table(*src_table_schema, dst_table_schema, nullptr/*sequence_ddl_arg*/,
|
|
false/*bind_tablets*/, *src_tenant_schema_guard, *dst_tenant_schema_guard, ddl_operator,
|
|
dst_tenant_trans, allocator, tenant_data_version, index_name, true /*ignore_cs_replica*/))) {
|
|
LOG_WARN("create user hidden table failed", K(ret), K(arg), K(tenant_data_version));
|
|
} else {
|
|
ObPrepareAlterTableArgParam param;
|
|
if (OB_FAIL(param.init(arg.consumer_group_id_, session_id, 0/*sql_mode, unused*/, arg.ddl_stmt_str_,
|
|
src_table_schema->get_table_name_str(), src_db_schema->get_database_name_str(),
|
|
dst_db_schema->get_database_name_str(), arg.tz_info_, arg.tz_info_wrap_, arg.nls_formats_))) {
|
|
LOG_WARN("fail to prepare alter table arg param", K(ret), K(arg));
|
|
} else if (OB_FAIL(root_service->get_ddl_scheduler().prepare_alter_table_arg(param, &dst_table_schema, alter_table_arg))) {
|
|
LOG_WARN("prepare alter table arg failed", K(ret), K(param));
|
|
} else {
|
|
alter_table_arg.alter_table_schema_.set_schema_version(dst_table_schema.get_schema_version());
|
|
alter_table_arg.alter_table_schema_.set_table_name(arg.target_schema_.get_table_name_str());
|
|
alter_table_arg.parallelism_ = arg.parallelism_;
|
|
alter_table_arg.consumer_group_id_ = arg.consumer_group_id_;
|
|
ObCreateDDLTaskParam param(dst_table_schema.get_tenant_id(),
|
|
ObDDLType::DDL_TABLE_RESTORE,
|
|
src_table_schema,
|
|
&dst_table_schema,
|
|
src_table_schema->get_table_id()/*object_id*/,
|
|
src_table_schema->get_schema_version(),
|
|
MAX(1, arg.parallelism_),
|
|
arg.consumer_group_id_,
|
|
&allocator,
|
|
&alter_table_arg,
|
|
0,
|
|
arg.ddl_task_id_);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, dst_tenant_trans, task_record))) {
|
|
LOG_WARN("submit ddl task failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (dst_tenant_trans.is_started()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(dst_tenant_trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(tmp_ret));
|
|
ret = (OB_SUCC(ret)) ? tmp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(dst_table_schema.get_tenant_id()))) {
|
|
LOG_WARN("publish_schema failed", K(ret), K(dst_table_schema));
|
|
} else if (OB_TMP_FAIL(root_service->get_ddl_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_and_check_table_schema(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const ObTableSchema *&orig_table_schema,
|
|
bool is_offline_ddl/*false*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
const ObString &origin_database_name = alter_table_schema.get_origin_database_name();
|
|
const ObString &origin_table_name = alter_table_schema.get_origin_table_name();
|
|
bool is_alter_comment = false;
|
|
if (origin_database_name.empty() || origin_table_name.empty()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("database name or table name is null", K(alter_table_schema),
|
|
K(origin_database_name), K(origin_table_name), K(ret));
|
|
} else {
|
|
bool is_index = false;
|
|
bool is_db_in_recyclebin = false;
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
origin_database_name,
|
|
origin_table_name,
|
|
is_index,
|
|
orig_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id),
|
|
K(origin_database_name), K(origin_table_name));
|
|
} else if (NULL == orig_table_schema) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(origin_database_name),
|
|
to_cstring(origin_table_name));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(
|
|
tenant_id,
|
|
orig_table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), K(*orig_table_schema));
|
|
} else if (!alter_table_arg.is_inner_ && is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not alter table in recyclebin", K(ret), K(alter_table_arg));
|
|
} else if (!alter_table_arg.skip_sys_table_check_
|
|
&& OB_FAIL(check_enable_sys_table_ddl(*orig_table_schema, OB_DDL_ALTER_TABLE))) {
|
|
LOG_WARN("ddl is not allowed on system table", K(ret));
|
|
} else if (!alter_table_arg.is_inner_
|
|
&& (orig_table_schema->is_in_recyclebin() || is_db_in_recyclebin)) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not alter table in recyclebin",
|
|
K(ret), K(alter_table_arg), K(is_db_in_recyclebin));
|
|
} else if (orig_table_schema->is_materialized_view()) {
|
|
bool allow_alter_mview = false;
|
|
if (alter_table_arg.is_alter_indexs_) {
|
|
bool is_alter_pk = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && (i < alter_table_arg.index_arg_list_.count()); ++i) {
|
|
const ObIndexArg *index_arg = alter_table_arg.index_arg_list_.at(i);
|
|
if (OB_ISNULL(index_arg)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("index arg is null", KR(ret));
|
|
} else if ((ObIndexArg::ADD_PRIMARY_KEY == index_arg->index_action_type_)
|
|
|| (ObIndexArg::DROP_PRIMARY_KEY == index_arg->index_action_type_)
|
|
|| (ObIndexArg::ALTER_PRIMARY_KEY == index_arg->index_action_type_)) {
|
|
is_alter_pk = true;
|
|
break;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_alter_pk) {
|
|
allow_alter_mview = true;
|
|
}
|
|
} else {
|
|
// only allow for alter tablegroup
|
|
if (!alter_table_arg.is_alter_indexs_ && !alter_table_arg.is_alter_columns_ && !alter_table_arg.is_alter_partitions_ &&
|
|
!alter_table_arg.is_update_global_indexes_ && !alter_table_arg.is_convert_to_character_ && alter_table_arg.is_alter_options_) {
|
|
if (1 == alter_table_arg.alter_table_schema_.alter_option_bitset_.num_members() &&
|
|
alter_table_arg.alter_table_schema_.alter_option_bitset_.has_member(ObAlterTableArg::TABLEGROUP_NAME)) {
|
|
allow_alter_mview = true;
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !allow_alter_mview) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("alter materialized view is not supported", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter materialized view is");
|
|
}
|
|
} else if (OB_FAIL(alter_table_arg.is_alter_comment(is_alter_comment))) {
|
|
LOG_WARN("failed to get is alter comment", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(orig_table_schema->get_tenant_id(), compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(orig_table_schema->get_tenant_id()));
|
|
} else if (!orig_table_schema->is_user_table()
|
|
&& !orig_table_schema->is_sys_table()
|
|
&& !(orig_table_schema->is_view_table() && is_alter_comment
|
|
&& ObSQLUtils::is_data_version_ge_422_or_431(compat_version))
|
|
&& !orig_table_schema->is_tmp_table()
|
|
&& !orig_table_schema->is_external_table()) {
|
|
ret = OB_ERR_WRONG_OBJECT;
|
|
LOG_USER_ERROR(OB_ERR_WRONG_OBJECT,
|
|
to_cstring(origin_database_name), to_cstring(origin_table_name), "BASE TABLE");
|
|
// NOTE: if_offline_ddl is true only in the execution phase of offline ddl, skip check_can_do_ddl check
|
|
} else if (!is_offline_ddl && !orig_table_schema->check_can_do_ddl()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("offline ddl is being executed, other ddl operations are not allowed", K(ret),
|
|
K(is_offline_ddl), K(*orig_table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/*
|
|
* FIXME:
|
|
* Before minor merge sstable sparse format support, in order to relieve the dependence on gc_snapshot_timestamp,
|
|
* first implement a temporary solution to avoid the operation of modifying columns in the process of
|
|
* merging the standalone cluster.
|
|
* 2.2.x,
|
|
* to primary cluster, Since RS determines the major version first,
|
|
* and then determine the schema_version used for the merge by ddl, it can be judged
|
|
* whether the merge has started according to whether the __all_zone version numbers are consistent
|
|
* to standalone, It is impossible to make the DDL of the primary cluster and the merge of standalone library
|
|
* to be mutually exclusive, so this solution is only to reduce the probability of bad cases.
|
|
* When the merge checksum is inconsistent in the standalone cluster, it can be corrected by drop replica
|
|
* >= 3.3, allow drop column when doing major freeze
|
|
*/
|
|
int ObDDLService::check_can_alter_column(
|
|
const int64_t tenant_id,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const ObTableSchema &orig_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t table_id = orig_table_schema.get_table_id();
|
|
DEBUG_SYNC(BEFORE_ALTER_TABLE_COLUMN);
|
|
if (THIS_WORKER.is_timeout_ts_valid()
|
|
&& THIS_WORKER.is_timeout()) {
|
|
ret = OB_TIMEOUT;
|
|
LOG_WARN("already timeout", KR(ret));
|
|
} else if (OB_FAIL(check_restore_point_allow(tenant_id, orig_table_schema))) {
|
|
LOG_WARN("restore point check fail, cannot alter column", K(ret), K(tenant_id), K(table_id));
|
|
} else {
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
AlterColumnSchema *alter_column_schema = NULL;
|
|
bool need_drop_column = false;
|
|
for(; OB_SUCC(ret) && !need_drop_column && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(*it_begin)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else {
|
|
alter_column_schema = static_cast<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));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// If there is a restore point on the table, it is not allowed to create indexes, add or delete columns.
|
|
int ObDDLService::check_restore_point_allow(const int64_t tenant_id, const ObTableSchema &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_exist = false;
|
|
ObArray<ObTabletID> tablet_ids;
|
|
if (!is_inner_table(table_schema.get_table_id()) && OB_FAIL(get_snapshot_mgr().check_restore_point(
|
|
get_sql_proxy(), tenant_id, table_schema.get_table_id(), is_exist))) {
|
|
LOG_WARN("failed to check restore point", K(ret), K(tenant_id));
|
|
}
|
|
if (OB_SUCC(ret) && is_exist) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("restore point exist, cannot alter ", K(ret), K(tenant_id), K(table_schema.get_table_id()));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "restore point exist, create index/alter");
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
// This code will be used for partition operations of table and tablegroup
|
|
// 1. for table, parameter is_drop_truncate_and_alter_index parameter avoids the drop/truncate partition
|
|
// of table with global index and the index create in the same alter table statement
|
|
// 2. for tablegroup, avoid drop/truncate partition in tablegroup with global index
|
|
// (After the tablegroup supports drop/truncate partitions of table with global indexes,
|
|
// the behavior will be unified.)
|
|
int ObDDLService::check_index_valid_for_alter_partition(
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const bool is_drop_truncate_and_alter_index,
|
|
const bool is_split)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObSEArray<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(
|
|
tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed",
|
|
"table id", simple_index_infos.at(i).table_id_, KR(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", KR(ret));
|
|
} else if (index_table_schema->is_global_index_table()) {
|
|
if (is_drop_truncate_and_alter_index) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("alter index and drop/truncate tables with global index not support", KR(ret),
|
|
"index_tid", simple_index_infos.at(i).table_id_, K(index_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter index and drop/truncate tables with global index");
|
|
}
|
|
} else if (index_table_schema->is_global_local_index_table()) {
|
|
if (is_split) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can not convert a non-partitioned table with non-partitioned global index to a partitioned table", KR(ret),
|
|
"index_tid", simple_index_infos.at(i).table_id_, K(index_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "converting a non-partitioned table with non-partitioned global index to a partitioned table");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_set_interval(const share::schema::ObTableSchema &orig_table_schema,
|
|
const obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartitionFuncType part_func_type = orig_table_schema.get_part_option().get_part_func_type();
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
ObPartition **part_array = orig_table_schema.get_part_array();
|
|
int64_t part_num = orig_table_schema.get_partition_num();
|
|
if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level()
|
|
&& !orig_table_schema.has_sub_part_template_def()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("interval part of composited-partitioned table not support", K(ret), K(orig_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval part of composited-partitioned table");
|
|
} else if (1 != orig_table_schema.get_partition_key_column_num()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("more than one partition key not support", K(ret), K(orig_table_schema));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "more than one partition key");
|
|
} else if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to part_array is null", K(orig_table_schema), K(alter_table_schema), KR(ret));
|
|
} else if (PARTITION_FUNC_TYPE_INTERVAL == part_func_type) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("change interval range in inteval table not support yet", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "change interval range in inteval table");
|
|
} else if (PARTITION_FUNC_TYPE_INTERVAL != part_func_type
|
|
&& PARTITION_FUNC_TYPE_RANGE_COLUMNS != part_func_type) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("type is unexpected when set interval", K(orig_table_schema), K(alter_table_schema), KR(ret));
|
|
} else if (part_num < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part num is less 1", K(orig_table_schema), K(alter_table_schema), KR(ret));
|
|
} else if (OB_ISNULL(part_array[part_num - 1])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("the last part is null", K(orig_table_schema), K(alter_table_schema), KR(ret));
|
|
} else if (OB_FAIL(ObPartitionUtils::check_interval_partition_table(part_array[part_num - 1]->get_high_bound_val(),
|
|
alter_table_schema.get_interval_range()))) {
|
|
LOG_WARN("fail to check_interval_partition_table", KR(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
// Check various conditions of partition management
|
|
// after 2.0 do not allow separate additions, deletions, and splitting of tables in tablegroup.
|
|
int ObDDLService::check_alter_partitions(const ObTableSchema &orig_table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const obrpc::ObAlterTableArg::AlterPartitionType alter_part_type = alter_table_arg.alter_part_type_;
|
|
const uint64_t tablegroup_id = orig_table_schema.get_tablegroup_id();
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
bool is_drop_or_truncate = false;
|
|
bool is_split = false;
|
|
bool is_oracle_mode = false;
|
|
bool has_local_index = false;
|
|
uint64_t compat_version = 0;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (compat_version < DATA_VERSION_4_2_0_0 && OB_INVALID_ID != tablegroup_id) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can not handle table in tablegroup when observer is upgrading", K(ret), K(tenant_id));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Can Not Handle Table In Tablegroup When Observer Is Upgrading");
|
|
} else if (GCONF.in_upgrade_mode()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("in upgrade, can not do partition maintenance", K(ret));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "partition maintenance during upgrade");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (orig_table_schema.is_materialized_view()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("alter partition of materialized view is not supported", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter partition of materialized view is");
|
|
} else if (orig_table_schema.is_interval_part()) {
|
|
if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level()
|
|
&& !orig_table_schema.has_sub_part_template_def()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("interval part of composited-partitioned table not support", K(ret), K(orig_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "interval part of composited-partitioned table");
|
|
}
|
|
} else if (OB_FAIL(orig_table_schema.check_has_local_index(schema_guard, has_local_index))) {
|
|
LOG_WARN("fail to check_has_local_index", K(ret), K(has_local_index));
|
|
} else if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_part_type
|
|
&& obrpc::ObAlterTableArg::REPARTITION_TABLE == alter_part_type
|
|
&& obrpc::ObAlterTableArg::PARTITIONED_PARTITION == alter_part_type) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("split partition in 4.0 not allowed", K(ret), K(tablegroup_id));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "split partition in 4.0");
|
|
}
|
|
bool has_fts_or_multivalue_or_vec_index = false;
|
|
const int64_t table_id = orig_table_schema.get_table_id();
|
|
if (OB_FAIL(ret) ||
|
|
alter_part_type == obrpc::ObAlterTableArg::DROP_PARTITION ||
|
|
alter_part_type == obrpc::ObAlterTableArg::TRUNCATE_PARTITION ) {
|
|
} else if (OB_FAIL(check_has_domain_index(schema_guard,
|
|
tenant_id,
|
|
table_id,
|
|
has_fts_or_multivalue_or_vec_index))) {
|
|
LOG_WARN("failed to check if have fts index", K(ret), K(table_id));
|
|
} else if (has_fts_or_multivalue_or_vec_index) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("alter partition operation on table with fulltext/multivalue/vector index not supported", K(ret), K(orig_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter partition operation on table with fulltext/multivalue/vector index");
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (obrpc::ObAlterTableArg::TRUNCATE_PARTITION == alter_part_type) {
|
|
bool is_truncate = true;
|
|
if (OB_FAIL(check_alter_drop_partitions(orig_table_schema, alter_table_arg, is_truncate))) {
|
|
LOG_WARN("failed to check truncate partitions", 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 (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 (alter_table_arg.is_split_partition()) {
|
|
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));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::RENAME_PARTITION == alter_part_type) {
|
|
if (OB_FAIL(check_alter_rename_partitions_(orig_table_schema, alter_table_arg))) {
|
|
LOG_WARN("failed to check rename partition", KR(ret), K(orig_table_schema), K(alter_table_arg));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::RENAME_SUB_PARTITION == alter_part_type) {
|
|
if (OB_FAIL(check_alter_rename_subpartitions_(orig_table_schema, alter_table_arg))) {
|
|
LOG_WARN("failed to check rename subpartition", KR(ret), K(orig_table_schema), K(alter_table_arg));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::ADD_PARTITION == alter_part_type) {
|
|
if (OB_FAIL(check_alter_add_partitions(orig_table_schema, alter_table_arg))) {
|
|
LOG_WARN("failed to check add paritions", K(ret), K(orig_table_schema), K(alter_table_arg));
|
|
}
|
|
} else if (obrpc::ObAlterTableArg::ADD_SUB_PARTITION == alter_part_type) {
|
|
if (OB_FAIL(check_alter_add_subpartitions(orig_table_schema, alter_table_arg))) {
|
|
LOG_WARN("failed to check add paritions", K(ret), K(orig_table_schema), K(alter_table_arg));
|
|
}
|
|
} else if (is_oracle_mode && obrpc::ObAlterTableArg::SET_INTERVAL == alter_part_type) {
|
|
if (OB_FAIL(check_alter_set_interval(orig_table_schema, alter_table_arg))) {
|
|
LOG_WARN("failed to check set interval", K(ret), K(orig_table_schema), K(alter_table_arg));
|
|
}
|
|
} else if (is_oracle_mode && obrpc::ObAlterTableArg::INTERVAL_TO_RANGE == alter_part_type) {
|
|
if (PARTITION_FUNC_TYPE_INTERVAL != orig_table_schema.get_part_option().get_part_func_type()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("type is unexpected when interval to range", K(orig_table_schema), K(alter_table_arg), KR(ret));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected alter partition type", K(ret), K(alter_part_type));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(check_index_valid_for_alter_partition(orig_table_schema,
|
|
schema_guard,
|
|
is_drop_or_truncate && alter_table_arg.index_arg_list_.size() != 0,
|
|
is_split))) {
|
|
LOG_WARN("failed to check index valid", K(ret), K(is_split), K(is_drop_or_truncate), K(orig_table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_table_pk(const share::schema::ObTableSchema &orig_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// Check whether table is the parent table of the foreign key constraint
|
|
// check whether it is partition table
|
|
if (orig_table_schema.is_parent_table()) {
|
|
// check whether it is self-referential
|
|
if (orig_table_schema.is_child_table()) {
|
|
const ObIArray<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;
|
|
}
|
|
|
|
// rename partition
|
|
//1. is partition table
|
|
//2. ensure that origin partition name exist and new partition name not exist
|
|
//3. currently this partition is not splitting
|
|
|
|
int ObDDLService::check_alter_rename_partitions_(const share::schema::ObTableSchema &orig_table_schema,
|
|
const obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
|
|
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();
|
|
ObPartition *inc_part = nullptr;
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
const ObPartition *part = nullptr;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(orig_table_schema.get_tenant_id(), tenant_data_version))) {
|
|
LOG_WARN("get data version failed", KR(ret), K(orig_table_schema.get_tenant_id()));
|
|
} else if (tenant_data_version < DATA_VERSION_4_2_1_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("cluster version and feature mismatch", KR(ret));
|
|
} else if (!orig_table_schema.is_user_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("unsupport behavior on not user table", KR(ret), K(orig_table_schema));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
|
|
LOG_USER_ERROR(OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED);
|
|
LOG_WARN("unsupport management on non_partition table", KR(ret));
|
|
} else if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_array is null", KR(ret), KP(part_array));
|
|
} else if (OB_UNLIKELY(1 != part_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rename multi part at a time not support", KR(ret), K(part_num));
|
|
} else if (OB_ISNULL(inc_part = part_array[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part is null", KR(ret), KP(part_array), K(alter_table_schema));
|
|
} else {
|
|
const ObString &origin_partition_name = inc_part->get_part_name();
|
|
const ObString &new_partition_name = alter_table_schema.get_new_part_name();
|
|
if (OB_UNLIKELY(ObCharset::case_insensitive_equal(origin_partition_name, new_partition_name))) {
|
|
ret = OB_ERR_RENAME_PARTITION_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_RENAME_PARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
|
|
LOG_WARN("origin part name equal to new part name", KR(ret), K(origin_partition_name), K(new_partition_name));
|
|
} else if (OB_FAIL(orig_table_schema.check_partition_duplicate_with_name(new_partition_name))) {
|
|
if (OB_DUPLICATE_OBJECT_NAME_EXIST == ret) {
|
|
ret = OB_ERR_RENAME_PARTITION_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_RENAME_PARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
|
|
LOG_WARN("new part name duplicate with existed partition", KR(ret), K(new_partition_name));
|
|
} else {
|
|
LOG_WARN("check new part name duplicate failed", KR(ret), K(new_partition_name));
|
|
}
|
|
} else if (OB_FAIL(orig_table_schema.get_partition_by_name(origin_partition_name, part))) {
|
|
LOG_WARN("get part by name failed", KR(ret), K(origin_partition_name), K(orig_table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// rename subpartition
|
|
//1. is subpartition table
|
|
//2. ensure that origin subpartition name exist and new subpartition name not exist
|
|
//3. currently this partition is not splitting
|
|
|
|
int ObDDLService::check_alter_rename_subpartitions_(const share::schema::ObTableSchema &orig_table_schema,
|
|
const obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
|
|
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();
|
|
ObPartition *inc_part = nullptr;
|
|
const ObPartition *part = nullptr;
|
|
const ObSubPartition *subpart = nullptr;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(orig_table_schema.get_tenant_id(), tenant_data_version))) {
|
|
LOG_WARN("get data version failed", KR(ret), K(orig_table_schema.get_tenant_id()));
|
|
} else if (tenant_data_version < DATA_VERSION_4_2_1_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("cluster version and feature mismatch", KR(ret));
|
|
} else if (!orig_table_schema.is_user_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("unsupport behavior on not user table", KR(ret), K(orig_table_schema));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
|
|
LOG_USER_ERROR(OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED);
|
|
LOG_WARN("unsupport management on not partition table", KR(ret));
|
|
} else if (PARTITION_LEVEL_ONE == part_level) {
|
|
ret = OB_ERR_NOT_COMPOSITE_PARTITION;
|
|
LOG_USER_ERROR(OB_ERR_NOT_COMPOSITE_PARTITION);
|
|
LOG_WARN("unsupport management on not composite partition table", KR(ret));
|
|
} else if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_array is null", KR(ret), K(alter_table_schema));
|
|
} else if (OB_UNLIKELY(1 != part_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rename subpart on multi part at a time not support", KR(ret), K(part_num));
|
|
} else if (OB_ISNULL(inc_part = part_array[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part is null", KR(ret), KP(part_array));
|
|
} else {
|
|
ObSubPartition **subpart_array = inc_part->get_subpart_array();
|
|
ObSubPartition *inc_subpart = nullptr;
|
|
const int64_t subpart_num = inc_part->get_subpartition_num();
|
|
if (OB_ISNULL(subpart_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("subpart_array is null", KR(ret));
|
|
} else if (OB_UNLIKELY(1 != subpart_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("rename multi subpart at a time not support", KR(ret), K(part_num));
|
|
} else if (OB_ISNULL(inc_subpart = subpart_array[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_subpart is null", KR(ret), KP(subpart_array));
|
|
} else {
|
|
const ObString &origin_partition_name = inc_subpart->get_part_name();
|
|
const ObString &new_partition_name = alter_table_schema.get_new_part_name();
|
|
if (OB_UNLIKELY(ObCharset::case_insensitive_equal(origin_partition_name, new_partition_name))) {
|
|
ret = OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
|
|
LOG_WARN("origin subpart name equal to new subpart name", KR(ret), K(origin_partition_name), K(new_partition_name));
|
|
} else if (OB_FAIL(orig_table_schema.check_partition_duplicate_with_name(new_partition_name))) {
|
|
if (OB_DUPLICATE_OBJECT_NAME_EXIST == ret) {
|
|
ret = OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
|
|
LOG_WARN("new subpart name duplicate with existed partition", KR(ret), K(new_partition_name));
|
|
} else {
|
|
LOG_WARN("check new subpart name duplicate failed", KR(ret), K(new_partition_name));
|
|
}
|
|
} else if (OB_FAIL(orig_table_schema.get_subpartition_by_name(origin_partition_name, part, subpart))) {
|
|
LOG_WARN("get subpart by name failed", KR(ret), K(origin_partition_name), K(orig_table_schema));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
// drop or truncate partition
|
|
//1. is partition table
|
|
//2. Cannot drop all partitions, but truncate does not have this restriction
|
|
//3. Ensure that all operating partitions exist1
|
|
//4. Currently this partition is not splitting
|
|
//5. The partition type can only be list or range
|
|
int ObDDLService::check_alter_drop_partitions(const share::schema::ObTableSchema &orig_table_schema,
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const bool is_truncate)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const int64_t part_num = alter_table_schema.get_part_option().get_part_num();
|
|
ObPartition **part_array = alter_table_schema.get_part_array();
|
|
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
|
|
const ObPartitionOption &part_option = orig_table_schema.get_part_option();
|
|
if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part_array is null", K(ret), K(part_array));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
|
|
LOG_WARN("unsupport management on non-partition table", K(ret));
|
|
} else if (!part_option.is_range_part() && !part_option.is_list_part()) {
|
|
ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION;
|
|
LOG_WARN("drop partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(check_table_pk(orig_table_schema))) {
|
|
LOG_WARN("cannot drop/truncate partition with foreign keys", K(ret), K(alter_table_arg));
|
|
} else if (is_truncate) {
|
|
} else if (alter_table_schema.get_part_option().get_part_num() >=
|
|
orig_table_schema.get_part_option().get_part_num()) {
|
|
ret = OB_ERR_DROP_LAST_PARTITION;
|
|
LOG_WARN("cannot drop all partitions",
|
|
"partitions current", orig_table_schema.get_part_option().get_part_num(),
|
|
"partitions to be dropped", alter_table_schema.get_part_option().get_part_num(),
|
|
K(ret));
|
|
LOG_USER_ERROR(OB_ERR_DROP_LAST_PARTITION);
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < part_num; ++i) {
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
ObPartIterator iter(orig_table_schema, check_partition_mode);
|
|
const ObPartition *part = NULL;
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
|
|
if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(part), K(ret));
|
|
} else if (ObCharset::case_insensitive_equal(part->get_part_name(),
|
|
part_array[i]->get_part_name())) {
|
|
break;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
if (OB_ITER_END != ret) {
|
|
LOG_WARN("iter failed", K(ret));
|
|
} else {
|
|
ret = OB_ERR_DROP_PARTITION_NON_EXISTENT;
|
|
LOG_WARN("partition to be dropped not exist", K(ret), "partition name", part_array[i]->get_part_name());
|
|
LOG_USER_ERROR(OB_ERR_DROP_PARTITION_NON_EXISTENT);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_drop_subpartitions(const share::schema::ObTableSchema &orig_table_schema,
|
|
const obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
|
|
const ObPartitionOption &subpart_option = orig_table_schema.get_sub_part_option();
|
|
if (PARTITION_LEVEL_ZERO == part_level || PARTITION_LEVEL_ONE == part_level) {
|
|
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
|
|
LOG_WARN("unsupport management on non-partition table", K(ret));
|
|
} else if (!subpart_option.is_range_part() && !subpart_option.is_list_part()) {
|
|
ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION;
|
|
LOG_WARN("drop partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(check_table_pk(orig_table_schema))) {
|
|
LOG_WARN("cannot drop/truncate partition with foreign keys", K(ret), K(alter_table_arg));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// filter out the partition which is same to orig_table_schema in alter_table_arg
|
|
int ObDDLService::filter_out_duplicate_interval_part(const share::schema::ObTableSchema &orig_table_schema,
|
|
share::schema::ObTableSchema &alter_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t j = 0;
|
|
int64_t inc_num = 0;
|
|
const ObRowkey *rowkey_orig= NULL;
|
|
const int64_t inc_part_num = alter_table_schema.get_part_option().get_part_num();
|
|
const int64_t orig_part_num = orig_table_schema.get_part_option().get_part_num();
|
|
ObPartition **inc_part_array = alter_table_schema.get_part_array();
|
|
ObPartition **orig_part_array = orig_table_schema.get_part_array();
|
|
if (!orig_table_schema.is_interval_part()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_table_schema is not interval part", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (OB_ISNULL(orig_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (orig_table_schema.get_interval_range() != alter_table_schema.get_interval_range()
|
|
|| orig_table_schema.get_transition_point() != alter_table_schema.get_transition_point()) {
|
|
ret = OB_ERR_INTERVAL_PARTITION_ERROR;
|
|
LOG_WARN("interval_range or transition_point is changed", KR(ret), K(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) {
|
|
ObString empty_str;
|
|
const ObRowkey *rowkey_cur = NULL;
|
|
if (OB_ISNULL(inc_part_array[i])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
// interval part name is generated in rs, so reset.
|
|
else if (OB_FAIL(inc_part_array[i]->set_part_name(empty_str))) {
|
|
LOG_WARN("fail to set_part_name", KR(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (NULL == (rowkey_cur = &inc_part_array[i]->get_high_bound_val())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
while (OB_SUCC(ret) && j < orig_part_num) {
|
|
if (OB_ISNULL(orig_part_array[j])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (NULL == (rowkey_orig = &orig_part_array[j]->get_high_bound_val())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (*rowkey_orig < *rowkey_cur) {
|
|
j++;
|
|
} else {
|
|
break;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (*rowkey_orig != *rowkey_cur) {
|
|
if (inc_num != i) {
|
|
inc_part_array[inc_num] = inc_part_array[i];
|
|
}
|
|
inc_num++;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (0 == inc_num) {
|
|
LOG_INFO("all interval part for add is exist", K(alter_table_schema), K(orig_table_schema));
|
|
ret = OB_ERR_INTERVAL_PARTITION_EXIST;
|
|
} else if (inc_num != inc_part_num) {
|
|
alter_table_schema.set_part_num(inc_num);
|
|
alter_table_schema.set_partition_num(inc_num);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// add partition
|
|
//1. ensure it is partition table
|
|
//2. The number of new partitions cannot exceed the limit of the number of partitions
|
|
//3. The partition name of the newly added partition cannot conflict with the previous partition
|
|
//4. Partition type, currently only supports list or range partition
|
|
//5. The value of the newly added partition is consistent with the previous one
|
|
int ObDDLService::check_alter_add_partitions(const share::schema::ObTableSchema &orig_table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const int64_t inc_part_num = alter_table_schema.get_part_option().get_part_num();
|
|
const int64_t orig_part_num = orig_table_schema.get_part_option().get_part_num();
|
|
ObPartition **inc_part_array = alter_table_schema.get_part_array();
|
|
ObPartition **orig_part_array = orig_table_schema.get_part_array();
|
|
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
|
|
const ObPartitionOption &part_option = orig_table_schema.get_part_option();
|
|
bool is_oracle_mode = false;
|
|
if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (OB_ISNULL(orig_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_part_array is null", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
|
|
LOG_WARN("unsupport management on non-partition table", K(ret));
|
|
} else if (!part_option.is_range_part() && !part_option.is_list_part()) {
|
|
ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION;
|
|
LOG_WARN("add partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg));
|
|
} else if ((is_oracle_mode && OB_MAX_PARTITION_NUM_ORACLE < orig_table_schema.get_all_part_num() + inc_part_num)
|
|
|| (!is_oracle_mode
|
|
&& ObResolverUtils::get_mysql_max_partition_num(orig_table_schema.get_tenant_id())
|
|
< orig_table_schema.get_all_part_num() + inc_part_num)) {
|
|
ret = OB_TOO_MANY_PARTITIONS_ERROR;
|
|
LOG_WARN("too many partitions", K(ret),
|
|
"partition cnt current", orig_table_schema.get_all_part_num(),
|
|
"partition cnt to be added", inc_part_num);
|
|
}
|
|
if (!orig_table_schema.is_interval_part()) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) {
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
ObPartIterator iter(orig_table_schema, check_partition_mode);
|
|
const ObPartition *part = NULL;
|
|
ObPartition *inc_part = inc_part_array[i];
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
|
|
if (OB_ISNULL(part) || OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(part), K(ret));
|
|
} else if (ObCharset::case_insensitive_equal(part->get_part_name(),
|
|
inc_part->get_part_name())) {
|
|
ret = OB_ERR_SAME_NAME_PARTITION;
|
|
LOG_WARN("duplicate partition name", K(ret), K(inc_part_array[i]->get_part_name()));
|
|
LOG_USER_ERROR(OB_ERR_SAME_NAME_PARTITION, inc_part_array[i]->get_part_name().length(),
|
|
inc_part_array[i]->get_part_name().ptr());
|
|
} else if (PARTITION_LEVEL_TWO == orig_table_schema.get_part_level()) {
|
|
if (0 == inc_part->get_subpartition_num() || OB_ISNULL(inc_part->get_subpart_array())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("subpart_array is NULL", K(part), K(ret), K(i));
|
|
} else {
|
|
for (int j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) {
|
|
ObSubPartition *subpart = NULL;
|
|
ObSubPartition *inc_subpart = inc_part->get_subpart_array()[j];
|
|
int64_t k = 0, subpart_num = part->get_subpartition_num();
|
|
for (k = 0; OB_SUCC(ret) && k < subpart_num; k++) {
|
|
subpart = part->get_subpart_array()[k];
|
|
if (OB_ISNULL(subpart) || OB_ISNULL(inc_subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(part), K(inc_subpart), K(ret));
|
|
} else if (ObCharset::case_insensitive_equal(subpart->get_part_name(),
|
|
inc_subpart->get_part_name())) {
|
|
ret = OB_ERR_SAME_NAME_SUBPARTITION;
|
|
LOG_WARN("duplicate subpartition name", K(ret), K(subpart->get_part_name()));
|
|
LOG_USER_ERROR(OB_ERR_SAME_NAME_SUBPARTITION, subpart->get_part_name().length(),
|
|
subpart->get_part_name().ptr());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}// end for
|
|
}
|
|
|
|
// check the part of inc part is increased
|
|
if (OB_FAIL(ret)) {
|
|
} else if (orig_table_schema.is_range_part()) {
|
|
const ObRowkey *rowkey_last = NULL;
|
|
if (orig_table_schema.is_interval_part()) {
|
|
rowkey_last = &orig_table_schema.get_transition_point();
|
|
} else if (OB_ISNULL(orig_part_array[orig_part_num - 1])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", K(ret), K(orig_part_array), K(alter_table_schema));
|
|
} else {
|
|
rowkey_last = &orig_part_array[orig_part_num - 1]->get_high_bound_val();
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(rowkey_last)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", K(ret), K(orig_part_array), K(alter_table_schema));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) {
|
|
const ObRowkey *rowkey_cur = &inc_part_array[i]->get_high_bound_val();
|
|
|
|
if (*rowkey_cur <= *rowkey_last) {
|
|
ret = OB_ERR_ADD_PART_BOUN_NOT_INC;
|
|
LOG_WARN("range values should increasing", K(ret), K(rowkey_cur), K(rowkey_last));
|
|
LOG_USER_ERROR(OB_ERR_ADD_PART_BOUN_NOT_INC);
|
|
} else {
|
|
rowkey_last = rowkey_cur;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && orig_table_schema.is_interval_part()
|
|
&& filter_out_duplicate_interval_part(orig_table_schema, alter_table_schema)) {
|
|
LOG_WARN("fail to filter out duplicate interval part", KR(ret), K(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
} else if (orig_table_schema.is_list_part()) {
|
|
if (OB_FAIL(check_add_list_partition(orig_table_schema, alter_table_schema))) {
|
|
LOG_WARN("failed to check add list partition", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_split_partitions(const share::schema::ObTableSchema &orig_table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObPartitionLevel target_part_level = alter_table_arg.is_auto_split_partition() ?
|
|
orig_table_schema.get_target_part_level_for_auto_partitioned_table() :
|
|
orig_table_schema.get_part_level();
|
|
|
|
if (alter_table_arg.is_auto_split_partition() && !orig_table_schema.is_auto_partitioned_table()) {
|
|
// the table might be disabled auto-partition after trigger auto splitting partition
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("attempt to auto split partition for non-auto-partitioned table", KR(ret),
|
|
K(alter_table_arg),
|
|
K(orig_table_schema));
|
|
} else if (OB_FAIL(orig_table_schema.check_enable_split_partition(alter_table_arg.is_auto_split_partition()))) {
|
|
LOG_WARN("fail to check enable split partition", KR(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(check_split_partition_val_(orig_table_schema, alter_table_schema,
|
|
target_part_level, alter_table_arg.alter_part_type_))) {
|
|
LOG_WARN("fail to check split partition val", KR(ret), K(orig_table_schema), K(alter_table_arg));
|
|
} else if (OB_FAIL(check_split_partition_name_(orig_table_schema, alter_table_schema, target_part_level))) {
|
|
LOG_WARN("fail to check split partition name", KR(ret), K(orig_table_schema), K(alter_table_arg));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_split_partition_val_(const share::schema::ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const ObPartitionLevel target_part_level,
|
|
const obrpc::ObAlterTableArg::AlterPartitionType type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObPartitionLevel ori_part_level = orig_table_schema.get_part_level();
|
|
ObPartition **split_part_array = alter_table_schema.get_part_array();
|
|
const int64_t split_part_num = alter_table_schema.get_partition_num();
|
|
|
|
if (OB_ISNULL(split_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("split_part_array is null", KR(ret), K(orig_table_schema), K(alter_table_schema));
|
|
} else if (PARTITION_LEVEL_ONE == target_part_level) {
|
|
if (PARTITION_LEVEL_ZERO == ori_part_level) {
|
|
// only auto partitioning could run here, user could not manual split non-partitioned table
|
|
if (obrpc::ObAlterTableArg::AUTO_SPLIT_PARTITION != type) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("invalid operate", KR(ret), K(orig_table_schema), K(alter_table_schema), K(type));
|
|
} else if (OB_FAIL(check_split_partitions_from_same_source_(split_part_array, split_part_num,
|
|
0, split_part_num,
|
|
orig_table_schema,
|
|
target_part_level,
|
|
type))) {
|
|
LOG_WARN("fail to check split partitions from same source partition", KR(ret),
|
|
K(target_part_level),
|
|
K(orig_table_schema),
|
|
K(alter_table_schema));
|
|
}
|
|
} else if (PARTITION_LEVEL_ONE == ori_part_level) {
|
|
// in future, we need to support to split multiple partitions, which means the split_part_array
|
|
// might include various set of split partitions from different source partitions.
|
|
// thus, we need to:
|
|
// 1. divide split partitions into different sets based on split_source_tablet_id
|
|
// 2. check whether each set is valid.
|
|
// moreover, we assume that the split partitions from same source is continued in the split_part_array.
|
|
common::hash::ObHashSet<uint64_t> traversed_source_ids;
|
|
|
|
if (OB_ISNULL(split_part_array[0])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret));
|
|
} else if (OB_FAIL(traversed_source_ids.create(hash::cal_next_prime(split_part_num)))) {
|
|
LOG_WARN("fail to create hashset", KR(ret));
|
|
} else if (OB_FAIL(traversed_source_ids.set_refactored(split_part_array[0]->get_split_source_tablet_id().id(), 0 /*flag*/))) {
|
|
LOG_WARN("fail to set tablet_id", KR(ret), KPC(split_part_array[0]));
|
|
} else {
|
|
int64_t start_idx = 0; // the first idx of a set of same source splitting partitions
|
|
bool traverse_same_source_over = false; // traverse a set of same source splitting partitions over
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < split_part_num; ++i) {
|
|
ObPartition *split_part = split_part_array[i];
|
|
if (!split_part->get_split_source_tablet_id().is_valid()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid source_tablet_id", KR(ret), KPC(split_part));
|
|
} else if (i == split_part_num - 1) {
|
|
traverse_same_source_over = true;
|
|
} else if (OB_ISNULL(split_part_array[i+1])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret), K(alter_table_schema));
|
|
} else if (split_part_array[i+1]->get_split_source_tablet_id() != split_part->get_split_source_tablet_id()) {
|
|
traverse_same_source_over = true;
|
|
// TODO:
|
|
// we don't support to split multiple source partitions from same table in the meanwhile now.
|
|
// when we support the online ddl lock, we need to delete the following error code
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to split multiple source partitions", KR(ret), K(alter_table_schema));
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (traverse_same_source_over) {
|
|
if (OB_FAIL(check_split_partitions_from_same_source_(split_part_array, split_part_num,
|
|
start_idx, i+1,
|
|
orig_table_schema,
|
|
target_part_level,
|
|
type))) {
|
|
LOG_WARN("fail to check split partitions from same source partition", KR(ret),
|
|
K(target_part_level),
|
|
K(orig_table_schema),
|
|
K(alter_table_schema));
|
|
} else if (i != split_part_num - 1) {
|
|
ObTabletID curr_source_tablet_id = split_part_array[i+1]->get_split_source_tablet_id();
|
|
start_idx = i + 1;
|
|
traverse_same_source_over = false;
|
|
if (OB_FAIL(traversed_source_ids.set_refactored(curr_source_tablet_id.id(), 0 /*flag*/))) {
|
|
// it means the split parts from same source part are not continuous in array
|
|
if (ret == OB_HASH_EXIST) {
|
|
LOG_WARN("split part array should be sorted by source_tablet_id", KR(ret), K(curr_source_tablet_id.id()),
|
|
K(alter_table_schema));
|
|
} else {
|
|
LOG_WARN("fail to set tablet_id", KR(ret), K(curr_source_tablet_id));
|
|
}
|
|
}
|
|
}
|
|
} // end if (traverse_same_source_over)
|
|
} // end for
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(ori_part_level), K(target_part_level),
|
|
K(alter_table_schema), K(orig_table_schema));
|
|
}
|
|
} else if (PARTITION_LEVEL_TWO == target_part_level) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to split subpartition", KR(ret), K(alter_table_schema), K(orig_table_schema));
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(ori_part_level),
|
|
K(alter_table_schema), K(orig_table_schema));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// 1. the number of a set of split partition must be between [2, SPLIT_NUM_LIMIT]
|
|
// 2. source_tablet_id must exist in origin table schema
|
|
// 3. split partition array should be sorted by high_bound_val
|
|
// 4. if the table is range partition, the high_bound_val range of split partitions should be equal to that of source partition
|
|
int ObDDLService::check_split_partitions_from_same_source_(ObPartition **split_part_array,
|
|
const int64_t part_array_size,
|
|
const int64_t start, const int64_t end,
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
const ObPartitionLevel target_part_level,
|
|
const obrpc::ObAlterTableArg::AlterPartitionType type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObPartitionLevel ori_part_level = orig_table_schema.get_part_level();
|
|
const ObPartitionOption &part_option = orig_table_schema.get_part_option();
|
|
ObPartition **orig_part_array = orig_table_schema.get_part_array();
|
|
const int64_t orig_part_num = orig_table_schema.get_partition_num();
|
|
const int64_t split_part_num = end - start;
|
|
static const int64_t MANUAL_SPLIT_NUM_LIMIT = INT64_MAX; // not limit the number of manual splitting partition
|
|
static const int64_t AUTO_SPLIT_NUM_LIMIT = 2;
|
|
const int64_t SPLIT_NUM_LIMIT = obrpc::ObAlterTableArg::AUTO_SPLIT_PARTITION == type ?
|
|
AUTO_SPLIT_NUM_LIMIT : MANUAL_SPLIT_NUM_LIMIT;
|
|
|
|
if (end > part_array_size || end <= start || start < 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid interval", KR(ret), K(start), K(end), K(part_array_size));
|
|
} else if (split_part_num < 2) {
|
|
ret = OB_ERR_SPLIT_INTO_ONE_PARTITION;
|
|
LOG_WARN("split partition number is invalid", KR(ret), K(split_part_num));
|
|
} else if (split_part_num > SPLIT_NUM_LIMIT) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("invalid splitting partition number", KR(ret), K(split_part_num), K(SPLIT_NUM_LIMIT), K(type));
|
|
char buffer[number::ObNumber::MAX_PRINTABLE_SIZE] = {0};
|
|
(void)snprintf(buffer, sizeof(buffer), "%s splitting partition number more than %ld is",
|
|
obrpc::ObAlterTableArg::AUTO_SPLIT_PARTITION == type ? "auto" : "manual",
|
|
SPLIT_NUM_LIMIT);
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, buffer);
|
|
} else if (orig_table_schema.is_partitioned_table()
|
|
&& !orig_table_schema.is_valid_split_part_type()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("only support to split range part", KR(ret), K(orig_table_schema));
|
|
} else {
|
|
// check sorting by value
|
|
for (int64_t i = start; OB_SUCC(ret) && i < end; ++i) {
|
|
ObPartition *split_part = split_part_array[i];
|
|
if (OB_ISNULL(split_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret));
|
|
} else if (i > 0 && !ObBasePartition::range_like_func_less_than(split_part_array[i-1], split_part)) {
|
|
ret = OB_ERR_RANGE_NOT_INCREASING_ERROR;
|
|
LOG_WARN("range values should increasing", KR(ret), K(split_part_array[i-1]->get_high_bound_val()),
|
|
K(split_part_array[i]->get_high_bound_val()));
|
|
}
|
|
}
|
|
}
|
|
|
|
// check existence of source_tablet_id and value validity of split partitions
|
|
if (OB_FAIL(ret)) {
|
|
} else if (PARTITION_LEVEL_ONE == target_part_level) {
|
|
if (PARTITION_LEVEL_ZERO == ori_part_level) {
|
|
for (int64_t i = start; OB_SUCC(ret) && i < end; ++i) {
|
|
const ObPartition *split_part = split_part_array[i];
|
|
if (split_part->get_split_source_tablet_id() != orig_table_schema.get_tablet_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("source_tablet_id is invalid", KR(ret), K(split_part->get_split_source_tablet_id()),
|
|
K(orig_table_schema.get_table_id()));
|
|
}
|
|
}
|
|
} else if (PARTITION_LEVEL_ONE == ori_part_level) {
|
|
const ObTabletID source_tablet_id = split_part_array[start]->get_split_source_tablet_id();
|
|
const ObPartition *part = NULL;
|
|
bool find = false;
|
|
|
|
for (int64_t j = 0; !find && OB_SUCC(ret) && j < orig_part_num; ++j) {
|
|
part = orig_part_array[j];
|
|
if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret));
|
|
} else if (part->get_tablet_id() == source_tablet_id) { // check existence
|
|
find = true;
|
|
// check boundary
|
|
const ObRowkey& min_val = split_part_array[start]->get_high_bound_val();
|
|
const ObRowkey& max_val = split_part_array[end - 1]->get_high_bound_val();
|
|
if (max_val != part->get_high_bound_val()) {
|
|
ret = OB_ERR_RANGE_NOT_INCREASING_ERROR;
|
|
LOG_WARN("the value of last partition is not equal the origin range", KR(ret), K(max_val), KPC(part));
|
|
} else if (j != 0 && min_val <= orig_part_array[j - 1]->get_high_bound_val()) {
|
|
ret = OB_ERR_RANGE_NOT_INCREASING_ERROR;
|
|
LOG_WARN("the value of split partition conflicts with existent partition", KR(ret), K(min_val),
|
|
KPC(orig_part_array[j - 1]));
|
|
}
|
|
}
|
|
} // end for
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_UNLIKELY(!find)) {
|
|
ret = OB_UNKNOWN_PARTITION;
|
|
LOG_WARN("source_tablet_id not exists in origin table", KR(ret), K(source_tablet_id), K(orig_table_schema));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(target_part_level), K(ori_part_level));
|
|
}
|
|
} else if (PARTITION_LEVEL_TWO == target_part_level) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not supported the part level", KR(ret), K(target_part_level), K(ori_part_level));
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(target_part_level), K(ori_part_level));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// the name of split patitions should be different with other partitions' name
|
|
// except that of its source partition
|
|
int ObDDLService::check_split_partition_name_(const share::schema::ObTableSchema &orig_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const ObPartitionLevel target_part_level)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObPartitionLevel ori_part_level = orig_table_schema.get_part_level();
|
|
ObPartition **split_part_array = alter_table_schema.get_part_array();
|
|
const int64_t split_part_num = alter_table_schema.get_partition_num();
|
|
|
|
if (OB_ISNULL(split_part_array)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(alter_table_schema));
|
|
} else if (split_part_num < 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(split_part_num));
|
|
} else {
|
|
ObArenaAllocator allocator;
|
|
ObPartition** sorted_part_array;
|
|
if (OB_ISNULL(sorted_part_array = static_cast<ObPartition**>(allocator.alloc(
|
|
sizeof(ObPartition*) * (split_part_num))))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail to alloc", KR(ret), K(split_part_num));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < split_part_num; ++i) {
|
|
sorted_part_array[i] = split_part_array[i];
|
|
}
|
|
ob_sort(sorted_part_array, sorted_part_array + split_part_num, ObPartitionNameCmp(CS_TYPE_UTF8MB4_GENERAL_CI));
|
|
}
|
|
|
|
ObString last_split_part_name;
|
|
const ObPartition *ori_part = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < split_part_num; ++i) {
|
|
ObPartition *split_part = sorted_part_array[i];
|
|
if (OB_ISNULL(split_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret));
|
|
} else if (split_part->is_empty_partition_name()) {
|
|
// will be generated part name in fill_part_name() of ObDDLService
|
|
} else if (!last_split_part_name.empty()
|
|
&& ObCharset::case_insensitive_equal(split_part->get_part_name(),
|
|
last_split_part_name)) {
|
|
ret = OB_ERR_SAME_NAME_PARTITION;
|
|
LOG_WARN("duplicate partition name in split part array", KR(ret), "part name", split_part->get_part_name());
|
|
LOG_USER_ERROR(OB_ERR_SAME_NAME_PARTITION, split_part->get_part_name().length(),
|
|
split_part->get_part_name().ptr());
|
|
} else if (PARTITION_LEVEL_ONE == target_part_level) {
|
|
if (PARTITION_LEVEL_ZERO == ori_part_level) {
|
|
} else if (PARTITION_LEVEL_ONE != ori_part_level) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(ori_part_level), K(target_part_level));
|
|
} else if (OB_FAIL(orig_table_schema.get_partition_by_name(split_part->get_part_name(),
|
|
ori_part))) {
|
|
if (OB_UNLIKELY(ret != OB_UNKNOWN_PARTITION)) {
|
|
LOG_WARN("fail to get partition by name", KR(ret), KPC(split_part));
|
|
} else {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
} else if (OB_ISNULL(ori_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", KR(ret));
|
|
} else if (ori_part->get_tablet_id() != split_part->get_split_source_tablet_id()) {
|
|
ret = OB_ERR_SAME_NAME_PARTITION;
|
|
LOG_WARN("duplicate partition name with orign part array", KR(ret),
|
|
"part name", split_part->get_part_name());
|
|
LOG_USER_ERROR(OB_ERR_SAME_NAME_PARTITION, split_part->get_part_name().length(),
|
|
split_part->get_part_name().ptr());
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
last_split_part_name = split_part->get_part_name();
|
|
}
|
|
} else if (PARTITION_LEVEL_TWO == target_part_level) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to split subpartition", KR(ret), K(ori_part_level), K(target_part_level),
|
|
K(orig_table_schema), K(alter_table_schema));
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(ori_part_level), K(target_part_level),
|
|
K(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
} // end for
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_split_global_index_partition_(ObSchemaGetterGuard &schema_guard,
|
|
obrpc::ObAlterTableArg &arg,
|
|
const share::schema::ObTableSchema &orig_index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_db_in_recyclebin = false;
|
|
uint64_t tenant_id = orig_index_schema.get_tenant_id();
|
|
|
|
if (OB_FAIL(schema_guard.check_database_in_recyclebin(
|
|
tenant_id,
|
|
orig_index_schema.get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", KR(ret), K(tenant_id), K(orig_index_schema));
|
|
} else if (OB_UNLIKELY(is_db_in_recyclebin)) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("database is in recyclebin", KR(ret), K(orig_index_schema));
|
|
} else if (OB_UNLIKELY(orig_index_schema.is_in_recyclebin())) {
|
|
// when drop data table, the related index of it will be moved to recyclebin;
|
|
// when drop index, the index will be removed directly
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("index table is in recyclebin", KR(ret), K(orig_index_schema));
|
|
} else if (!orig_index_schema.check_can_do_ddl()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("offline ddl is being executed, other ddl operations are not allowed", KR(ret), K(orig_index_schema));
|
|
} else if (OB_FAIL(check_restore_point_allow(tenant_id, orig_index_schema))) {
|
|
LOG_WARN("check restore point allow failed,", KR(ret), K(tenant_id), K(orig_index_schema.get_table_id()));
|
|
} else if (OB_FAIL(check_alter_partitions(orig_index_schema, arg))) {
|
|
LOG_WARN("check alter partitions failed", KR(ret), K(orig_index_schema), K(arg));
|
|
}
|
|
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(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
|
|
LOG_WARN("unsupport management on non-partition table", K(ret));
|
|
} else if (!subpart_option.is_range_part() && !subpart_option.is_list_part()) {
|
|
ret = OB_ERR_ONLY_ON_RANGE_LIST_PARTITION;
|
|
LOG_WARN("add partition can only be used on RANGE/LIST partitions", K(ret), K(alter_table_arg));
|
|
} else if ((is_oracle_mode && OB_MAX_PARTITION_NUM_ORACLE < orig_table_schema.get_all_part_num() + part_num)
|
|
|| (!is_oracle_mode
|
|
&& ObResolverUtils::get_mysql_max_partition_num(orig_table_schema.get_tenant_id())
|
|
< orig_table_schema.get_all_part_num() + part_num)) {
|
|
ret = OB_TOO_MANY_PARTITIONS_ERROR;
|
|
LOG_WARN("too many partitions", K(ret),
|
|
"partition cnt current", orig_table_schema.get_all_part_num(),
|
|
"partition cnt to be added", part_num);
|
|
}
|
|
|
|
// To add a subpartition separately, verify whether the partition exists
|
|
// and whether the subpartition does not exist
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_partition_num(); ++i) {
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
ObPartIterator iter(orig_table_schema, check_partition_mode);
|
|
const ObPartition *part = NULL;
|
|
const ObPartition *inc_part = part_array[i];
|
|
while (OB_SUCC(ret) && OB_SUCC(iter.next(part))) {
|
|
if (OB_ISNULL(part) || OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(part), K(ret));
|
|
} else if (ObCharset::case_insensitive_equal(part->get_part_name(),
|
|
inc_part->get_part_name())) {
|
|
if (0 == inc_part->get_subpartition_num()
|
|
|| OB_ISNULL(inc_part->get_subpart_array())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("nontemplate subpart_array is NULL", K(part), K(ret), K(i));
|
|
} else if (OB_FAIL(orig_parts.push_back(part))) {
|
|
LOG_WARN("fail to push back orig parts", KR(ret), KPC(part));
|
|
} else {
|
|
for (int j = 0; OB_SUCC(ret) && j < inc_part->get_subpartition_num(); j++) {
|
|
ObSubPartition *subpart = NULL;
|
|
ObSubPartition *inc_subpart = inc_part->get_subpart_array()[j];
|
|
int64_t k = 0, subpart_num = part->get_subpartition_num();
|
|
for (k = 0; OB_SUCC(ret) && k < subpart_num; k++) {
|
|
subpart = part->get_subpart_array()[k];
|
|
if (OB_ISNULL(subpart) || OB_ISNULL(inc_subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(part), K(inc_subpart), K(ret));
|
|
} else if (ObCharset::case_insensitive_equal(subpart->get_part_name(),
|
|
inc_subpart->get_part_name())) {
|
|
ret = OB_ERR_SAME_NAME_SUBPARTITION;
|
|
LOG_WARN("duplicate subpartition name", K(ret), K(subpart->get_part_name()));
|
|
LOG_USER_ERROR(OB_ERR_SAME_NAME_SUBPARTITION, subpart->get_part_name().length(),
|
|
subpart->get_part_name().ptr());
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && k >= subpart_num) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_PARTITION_NOT_EXIST;
|
|
LOG_WARN("duplicate partition name", K(ret), K(inc_part->get_part_name()));
|
|
LOG_USER_ERROR(OB_PARTITION_NOT_EXIST);
|
|
}
|
|
}// end for
|
|
|
|
// check whether the value of list and range is valid
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_schema.get_partition_num(); ++i) {
|
|
ObPartition *inc_part = part_array[i];
|
|
int64_t inc_subpart_num = inc_part->get_subpartition_num();
|
|
ObSubPartition ** inc_subpart_array = inc_part->get_subpart_array();
|
|
|
|
if (OB_ISNULL(inc_part) || OB_ISNULL(inc_subpart_array) || 0 == inc_subpart_num) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(inc_part), K(inc_subpart_num), K(ret));
|
|
} else if (alter_table_schema.get_partition_num() != orig_parts.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_parts count not equal inc__part count", K(orig_parts.count()), K(inc_subpart_num), K(ret));
|
|
} else if (orig_table_schema.is_range_subpart()) {
|
|
const int64_t orig_subpart_num = orig_parts.at(i)->get_subpartition_num();
|
|
ObSubPartition **orig_subpart_array = orig_parts.at(i)->get_subpart_array();
|
|
if (OB_ISNULL(orig_subpart_array) || OB_ISNULL(orig_subpart_array[orig_subpart_num - 1])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(ret), K(orig_subpart_array), K(orig_subpart_array[orig_subpart_num - 1]));
|
|
} else {
|
|
const ObRowkey *rowkey_last =
|
|
&(orig_subpart_array[orig_subpart_num - 1]->get_high_bound_val());
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < inc_subpart_num; ++j) {
|
|
const ObRowkey *rowkey_cur =
|
|
&inc_subpart_array[j]->get_high_bound_val();
|
|
if (*rowkey_cur <= *rowkey_last) {
|
|
ret = OB_ERR_ADD_PART_BOUN_NOT_INC;
|
|
LOG_WARN("range values should increasing", K(ret), K(rowkey_cur), K(rowkey_last));
|
|
} else {
|
|
rowkey_last = rowkey_cur;
|
|
}
|
|
}
|
|
}
|
|
} else if (orig_table_schema.is_list_subpart()) {
|
|
if (OB_FAIL(check_add_list_subpartition(*orig_parts.at(i), *inc_part))) {
|
|
LOG_WARN("failed to check add list partition", K(ret), K(orig_table_schema), K(alter_table_schema));
|
|
}
|
|
}
|
|
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// Check whether the newly added partition has a duplicate value with the old partition
|
|
int ObDDLService::check_add_list_partition(const share::schema::ObPartitionSchema &orig_part,
|
|
const share::schema::ObPartitionSchema &new_part,
|
|
const int64_t split_part_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t part_num = new_part.get_partition_num();
|
|
ObPartition **part_array = new_part.get_part_array();
|
|
int64_t orig_part_num = orig_part.get_partition_num();
|
|
ObPartition **orig_part_array = orig_part.get_part_array();
|
|
common::ObRowkey row_key;
|
|
if (OB_ISNULL(part_array) || OB_ISNULL(orig_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", K(ret), KP(part_array), KP(orig_part_array));
|
|
} else {
|
|
common::hash::ObHashSet<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 (!orig_part.is_external_table() && 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;
|
|
}
|
|
int ObDDLService::alter_table(obrpc::ObAlterTableArg &alter_table_arg,
|
|
obrpc::ObAlterTableRes &res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t start_usec = 0;
|
|
int64_t end_usec = 0;
|
|
int64_t cost_usec = 0;
|
|
start_usec = ObTimeUtility::current_time();
|
|
bool is_alter_sess_active_time = false;
|
|
bool is_alter_duplicate_scope = false;
|
|
bool is_alter_comment = false;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
int64_t &task_id = res.task_id_;
|
|
ObDDLType &ddl_type = res.ddl_type_;
|
|
bool &ddl_need_retry_at_executor = res.ddl_need_retry_at_executor_;
|
|
ddl_type = DDL_INVALID;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
}
|
|
|
|
ObSimpleTableSchemaV2 orig_table;
|
|
if (OB_SUCC(ret)) {
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
is_alter_sess_active_time = alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::SESSION_ACTIVE_TIME);
|
|
is_alter_duplicate_scope = alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::DUPLICATE_SCOPE);
|
|
is_alter_comment = alter_table_schema.alter_option_bitset_.has_member(obrpc::ObAlterTableArg::COMMENT);
|
|
LOG_DEBUG("debug view comment", K(is_alter_comment), K(alter_table_schema));
|
|
ObTZMapWrap tz_map_wrap;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_alter_duplicate_scope) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("alter table duplicate scope not supported", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter table duplicate scope");
|
|
} else if (OB_FAIL(OTTZ_MGR.get_tenant_tz(tenant_id, tz_map_wrap))) {
|
|
LOG_WARN("get tenant timezone map failed", K(ret), K(tenant_id));
|
|
} else if (FALSE_IT(alter_table_arg.set_tz_info_map(tz_map_wrap.get_tz_map()))) {
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id,
|
|
schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_parallel_ddl_conflict(schema_guard, alter_table_arg))) {
|
|
LOG_WARN("check parallel ddl conflict failed", KR(ret));
|
|
} 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", KR(ret), KP(orig_table_schema));
|
|
} else if (OB_FAIL(orig_table.assign(*orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
}
|
|
}
|
|
}
|
|
// check schema
|
|
if (OB_SUCC(ret)
|
|
&& alter_table_arg.is_alter_partitions_
|
|
&& obrpc::ObAlterTableArg::REPARTITION_TABLE != alter_table_arg.alter_part_type_) {
|
|
if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema is null", KR(ret), K(alter_table_arg));
|
|
} else {
|
|
if (obrpc::ObAlterTableArg::DROP_PARTITION != alter_table_arg.alter_part_type_) {
|
|
if ((alter_table_arg.alter_table_schema_.is_range_part()
|
|
&& !alter_table_arg.alter_table_schema_.is_interval_part())
|
|
|| alter_table_arg.alter_table_schema_.is_list_part()) {
|
|
if (OB_FAIL(fill_part_name(*orig_table_schema, alter_table_arg.alter_table_schema_))) {
|
|
LOG_WARN("failed to fill part name", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(check_alter_partitions(*orig_table_schema, alter_table_arg))) {
|
|
LOG_WARN("check alter partitions failed", K(ret), K(orig_table_schema), K(alter_table_arg));
|
|
}
|
|
}
|
|
}
|
|
|
|
//do alter table in transaction
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t data_version = 0;
|
|
bool is_oracle_mode_add_column_not_null_ddl = false;
|
|
bool is_default_value_null = false;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
|
|
LOG_WARN("failed to get tenant data version", K(ret), K(tenant_id), K(data_version));
|
|
} else if (is_alter_sess_active_time) {
|
|
if (OB_FAIL(alter_table_sess_active_time_in_trans(alter_table_arg,
|
|
res,
|
|
data_version))) {
|
|
LOG_WARN("alter_table_in_trans failed", K(ret));
|
|
} else {
|
|
LOG_INFO("refresh session active time of temp tables succeed!", K(ret));
|
|
}
|
|
} else if (OB_FAIL(check_is_offline_ddl(alter_table_arg, data_version, ddl_type, ddl_need_retry_at_executor))) {
|
|
LOG_WARN("failed to check is offline ddl", K(ret));
|
|
} else if (((MOCK_DATA_VERSION_4_2_1_3 <= data_version && DATA_VERSION_4_3_0_0 > data_version)
|
|
|| (DATA_VERSION_4_3_2_0 <= data_version)) // [4213, 430) & [4320, )
|
|
&& OB_FAIL(check_is_oracle_mode_add_column_not_null_ddl(alter_table_arg,
|
|
schema_guard,
|
|
is_oracle_mode_add_column_not_null_ddl,
|
|
is_default_value_null))) {
|
|
LOG_WARN("falied to check is oracle add column not null", K(ret));
|
|
} else if (is_oracle_mode_add_column_not_null_ddl) {
|
|
if (OB_FAIL(do_oracle_add_column_not_null_in_trans(alter_table_arg,
|
|
schema_guard,
|
|
data_version,
|
|
is_default_value_null))) {
|
|
LOG_WARN("add column not null failed", K(ret));
|
|
} else {
|
|
ddl_type = ObDDLType::DDL_NORMAL_TYPE;
|
|
}
|
|
} else {
|
|
// offline ddl cannot appear at the same time with other ddl types
|
|
if (is_long_running_ddl(ddl_type)) {
|
|
if (OB_FAIL(do_offline_ddl_in_trans(alter_table_arg, data_version, res))) {
|
|
LOG_WARN("failed to do offline ddl in trans", K(ret), K(alter_table_arg), K(ddl_type));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(alter_table_in_trans(alter_table_arg, res, data_version))) {
|
|
LOG_WARN("alter_table_in_trans failed", 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(ret), K(cost_usec), K(ddl_type), "ddl_stmt", alter_table_arg.ddl_stmt_str_);
|
|
}
|
|
}
|
|
|
|
// just for debug
|
|
if (OB_SUCC(ret) && false == is_alter_sess_active_time) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
const uint64_t table_id = (OB_INVALID_ID == orig_table.get_table_id()) ?
|
|
alter_table_arg.alter_table_schema_.get_table_id() :
|
|
orig_table.get_table_id();
|
|
const ObTableSchema *new_table_schema = NULL;
|
|
HEAP_VAR(ObSchemaGetterGuard, new_schema_guard) {
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, new_schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(new_schema_guard.get_table_schema(tenant_id, table_id, new_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(tmp_ret), K(tenant_id), K(table_id));
|
|
} else if (NULL == new_table_schema) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(tmp_ret));
|
|
}
|
|
}
|
|
// check const_column_iterator and ObColumnIterByPrevNextID
|
|
if (OB_SUCC(ret)) {
|
|
ObTableSchema::const_column_iterator iter = new_table_schema->column_begin();
|
|
ObColumnIterByPrevNextID iter2(*new_table_schema);
|
|
const ObColumnSchemaV2 *col = NULL;
|
|
while (OB_SUCC(ret) && OB_SUCC(iter2.next(col))) {
|
|
if (new_table_schema->column_end() == iter) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Inconsistent iterators: const_column_iterator is less than ObColumnIterByPrevNextID", K(ret));
|
|
} else {
|
|
++iter;
|
|
}
|
|
}
|
|
if (ret != OB_ITER_END) {
|
|
LOG_WARN("Failed to iterate all table columns. iter quit. ", K(ret));
|
|
} else if (iter != new_table_schema->column_end()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Inconsistent iterators: const_column_iterator is bigger than ObColumnIterByPrevNextID", K(ret));
|
|
} else {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rename_table(const obrpc::ObRenameTableArg &rename_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
const uint64_t tenant_id = rename_table_arg.tenant_id_;
|
|
bool is_oracle_mode = false;
|
|
bool sequence_exist = false;
|
|
RenameOracleObjectType rename_oracle_obj_type = RENAME_TYPE_INVALID;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::MYSQL;
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_guard or schema service is null",
|
|
K(schema_service), K(ret));
|
|
} else if (OB_INVALID_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is invalid", K(tenant_id), K(ret));
|
|
} else if (is_virtual_tenant_id(tenant_id) || OB_SYS_TENANT_ID == tenant_id) {
|
|
compat_mode = lib::Worker::CompatMode::MYSQL;
|
|
is_oracle_mode = false;
|
|
} else if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) {
|
|
LOG_WARN("failed to get compat mode", K(ret), K(tenant_id));
|
|
} else {
|
|
if (lib::Worker::CompatMode::ORACLE == compat_mode) {
|
|
is_oracle_mode = true;
|
|
} else {
|
|
is_oracle_mode = false;
|
|
}
|
|
if (is_oracle_mode && rename_table_arg.rename_table_items_.size() > 1) {
|
|
ret = OB_ERR_ALTER_TABLE_RENAME_WITH_OPTION;
|
|
LOG_WARN("alter table rename can't be combined with other operations in oracle mode",
|
|
K(ret), K(rename_table_arg));
|
|
}
|
|
}
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
uint64_t tenant_data_version = OB_INVALID_VERSION;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("fail to get data version", K(ret), K(tenant_id), K(tenant_data_version));
|
|
} else {
|
|
//todo use array to replace hashmap and hashset @hualong
|
|
//record table already be renamed in the schema mgr
|
|
common::hash::ObHashSet<ObTableItem> delete_table_set;
|
|
//record new table name set
|
|
//table_item -> table_id
|
|
common::hash::ObHashMap<ObTableItem, uint64_t> new_table_map;
|
|
ObArray<std::pair<uint64_t, share::schema::ObObjectType>> all_dep_objs;
|
|
const int64_t rename_items_count = rename_table_arg.rename_table_items_.size();
|
|
|
|
if (OB_FAIL(delete_table_set.create(rename_items_count))) {
|
|
LOG_WARN("failed to add create ObHashSet", KR(ret));
|
|
} else if (OB_FAIL(new_table_map.create(rename_items_count, ObModIds::OB_HASH_BUCKET_RENAME_TABLE_MAP))) {
|
|
LOG_WARN("failed to add create ObHashMap", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < rename_table_arg.rename_table_items_.size(); ++i) {
|
|
const ObRenameTableItem &rename_item = rename_table_arg.rename_table_items_.at(i);
|
|
const ObTableSchema *table_schema = nullptr;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
rename_item.origin_db_name_,
|
|
rename_item.origin_table_name_,
|
|
false,
|
|
table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret));
|
|
} else if (nullptr == table_schema) {
|
|
// skip
|
|
} else if (table_schema->is_materialized_view()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("rename materialized view is not supported",
|
|
KR(ret), K(table_schema->get_table_name()));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "rename materialized view is");
|
|
} else if (table_schema->is_mlog_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("rename materialized view log is not supported",
|
|
KR(ret), K(table_schema->get_table_name()));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "rename materialized view log is");
|
|
} else if (table_schema->required_by_mview_refresh()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("rename table required by materialized view refresh is not supported",
|
|
KR(ret), K(table_schema->get_table_name()));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "rename table with materialized view log is");
|
|
} else if (OB_FAIL(ObDependencyInfo::collect_all_dep_objs(
|
|
tenant_id, table_schema->get_table_id(), trans, all_dep_objs))) {
|
|
LOG_WARN("failed to collect dep info", 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_
|
|
|| ObString(OB_PUBLIC_SCHEMA_NAME) == to_table_item.database_name_) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("rename table to __recyclebin database is not allowd", K(to_table_item));
|
|
} else if (OB_FAIL(schema_guard.check_oracle_object_exist(tenant_id,
|
|
database_schema->get_database_id(), to_table_item.table_name_, OB_MAX_SCHEMA,
|
|
INVALID_ROUTINE_TYPE, false, conflict_schema_types))) {
|
|
LOG_WARN("fail to check oracle_object exist", K(ret), K(to_table_item.table_name_));
|
|
} else if (conflict_schema_types.count() > 0) {
|
|
// Check whether the name of the new object after rename in oracle mode
|
|
// is already occupied by other objects
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object in oralce mode",
|
|
K(ret), K(to_table_item.table_name_),
|
|
K(conflict_schema_types));
|
|
}
|
|
// check similar to rename table t1 to t3, t2 to t3 in mysql mode
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS == new_table_map.get_refactored(to_table_item, table_id)) {
|
|
//already had t1,t2
|
|
//rename table t1 to t3, t2 to t3(t3 exist!)
|
|
ret = OB_ERR_TABLE_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_TABLE_EXIST, to_table_item.table_name_.length(),
|
|
to_table_item.table_name_.ptr());
|
|
} else if (sequence_exist) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing sequence", K(ret), K(to_table_item));
|
|
} else if (OB_FAIL(schema_guard.get_synonym_info(tenant_id,
|
|
database_schema->get_database_id(),
|
|
to_table_item.table_name_,
|
|
synonym_info))) {
|
|
LOG_WARN("fail to check synonym exist", K(to_table_item), K(ret));
|
|
} else if (NULL != synonym_info) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing synonym", K(ret), K(to_table_item));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
to_table_item.database_name_,
|
|
to_table_item.table_name_,
|
|
false,
|
|
to_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id),
|
|
K(to_table_item.database_name_), K(to_table_item.table_name_));
|
|
} else if (NULL != to_table_schema) {
|
|
int hash_ret = OB_SUCCESS;
|
|
hash_ret = delete_table_set.exist_refactored(to_table_item);
|
|
if (OB_HASH_EXIST != hash_ret && OB_HASH_NOT_EXIST != hash_ret) {
|
|
ret = hash_ret;
|
|
LOG_WARN("delete_table_set check failed!", K(to_table_item), K(ret));
|
|
} else if (OB_HASH_NOT_EXIST == hash_ret) {
|
|
//already had t1,t2
|
|
//rename table t2 to t1 (t1 exist!)
|
|
if (!is_oracle_mode) {
|
|
ret = OB_ERR_TABLE_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_TABLE_EXIST, to_table_item.table_name_.length(),
|
|
to_table_item.table_name_.ptr());
|
|
LOG_WARN("table already exist!", K(to_table_item), K(ret));
|
|
} else {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", K(ret), K(to_table_item));
|
|
}
|
|
} else if (OB_HASH_EXIST == hash_ret) {
|
|
//already had t1,t2
|
|
//rename table t1 to t3, t2 to t1(success!)
|
|
//[delete(t1), new(t3)] -> [delete(t1, t2), new(t3, t1)]
|
|
LOG_INFO("rename to new table name", K(rename_item));
|
|
}
|
|
}
|
|
}
|
|
const ObTableSchema *from_table_schema = NULL;
|
|
const ObSynonymInfo *synonym_info = NULL;
|
|
const ObSequenceSchema *sequence_schema = NULL;
|
|
ObTableItem from_table_item;
|
|
from_table_item.table_name_ = rename_item.origin_table_name_;
|
|
from_table_item.database_name_ = rename_item.origin_db_name_;
|
|
// Determine the object type in oracle mode
|
|
if (is_oracle_mode && OB_SUCC(ret)) {
|
|
if (OB_FAIL(check_rename_object_type(schema_guard,
|
|
tenant_id,
|
|
database_schema->get_database_id(),
|
|
from_table_item.table_name_,
|
|
from_table_schema,
|
|
synonym_info,
|
|
sequence_schema,
|
|
rename_oracle_obj_type))) {
|
|
LOG_WARN("fail to check rename object type!", K(ret), K(tenant_id), K(database_schema->get_database_id()), K(from_table_item.table_name_));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && OB_FAIL(schema_guard.get_tenant_name_case_mode(tenant_id, from_table_item.mode_))) {
|
|
LOG_WARN("failed to get tenant name case mode!", K(tenant_id), K(from_table_item), K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
table_id = OB_INVALID_ID;
|
|
if (OB_SUCCESS != new_table_map.get_refactored(from_table_item, table_id)) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
from_table_item.database_name_,
|
|
from_table_item.table_name_,
|
|
false,
|
|
from_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret));
|
|
} else if (NULL != from_table_schema) {
|
|
bool is_db_in_recyclebin = false;
|
|
if (from_table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("Can not perform operation in recyclebin", K(ret), K(from_table_item));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
from_table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), KPC(from_table_schema));
|
|
} else if (is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not rename table in recyclebin", K(ret), K(from_table_item));
|
|
} else if (OB_HASH_EXIST == delete_table_set.exist_refactored(from_table_item)) {
|
|
//already had t1,t2
|
|
//rename table t1 to table1, t1 to t3 (t1 not exist)
|
|
ret = OB_FILE_NOT_EXIST;
|
|
LOG_WARN("table not exist!", K(from_table_item), K(ret));
|
|
} else if (OB_FAIL(check_enable_sys_table_ddl(*from_table_schema, OB_DDL_TABLE_RENAME))) {
|
|
LOG_WARN("rename table is not allowed on system table", K(ret));
|
|
} else {
|
|
in_new_table_set = false; //has not been rename before
|
|
LOG_WARN("orgin table exist! can do rename", K(rename_item), K(in_new_table_set));
|
|
}
|
|
} else {
|
|
//already had t1,t2
|
|
//rename table t3 to table3 (t3 not exist)
|
|
if (!is_oracle_mode) {
|
|
ret = OB_FILE_NOT_EXIST;
|
|
LOG_WARN("table not exist!", K(rename_item), K(ret));
|
|
} else {
|
|
LOG_INFO("oracle mode rename A to B", K(ret), K(rename_oracle_obj_type));
|
|
}
|
|
}
|
|
} else {
|
|
//rename table t1 to t2, t2 to t3
|
|
//[new(t2), delete(t1)] -> [new(t3), delete(t1)]
|
|
in_new_table_set = true;
|
|
LOG_INFO("origin table exist in new table set", K(rename_item), K(in_new_table_set));
|
|
if (OB_INVALID_ID == table_id) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table id is invalid ", K(table_id), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, from_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_ISNULL(from_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema is null", K(table_id), K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_oracle_mode
|
|
&& OB_FAIL(check_cst_name_dup_for_rename_table_mysql(schema_guard, from_table_schema, database_schema->get_database_id()))) {
|
|
LOG_WARN("check cst_name dup for rename table in mysql mode failed", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObSqlString sql;
|
|
if (!is_oracle_mode) {
|
|
if (OB_FAIL(sql.append_fmt("RENAME TABLE `%.*s`.`%.*s` TO `%.*s`.`%.*s`",
|
|
rename_item.origin_db_name_.length(),
|
|
rename_item.origin_db_name_.ptr(),
|
|
rename_item.origin_table_name_.length(),
|
|
rename_item.origin_table_name_.ptr(),
|
|
rename_item.new_db_name_.length(),
|
|
rename_item.new_db_name_.ptr(),
|
|
rename_item.new_table_name_.length(),
|
|
rename_item.new_table_name_.ptr()))) {
|
|
LOG_WARN("failed to append sql", K(ret));
|
|
}
|
|
} else { // oracle mode
|
|
if (OB_FAIL(sql.append_fmt("RENAME \"%.*s\" TO \"%.*s\"",
|
|
rename_item.origin_table_name_.length(),
|
|
rename_item.origin_table_name_.ptr(),
|
|
rename_item.new_table_name_.length(),
|
|
rename_item.new_table_name_.ptr()))) {
|
|
LOG_WARN("failed to append sql", K(ret));
|
|
}
|
|
}
|
|
bool need_reset_object_status = false;
|
|
if (OB_SUCC(ret) && OB_NOT_NULL(from_table_schema)) {
|
|
for (int64_t i = 0; i < all_dep_objs.count(); ++i) {
|
|
if (from_table_schema->get_table_id() == all_dep_objs.at(i).first) {
|
|
need_reset_object_status = true;
|
|
// reset to invalid id and dependencyinfo will not touch
|
|
all_dep_objs.at(i).first = OB_INVALID_ID;
|
|
// do not break to avoid duplicate obj in all_dep_objs
|
|
}
|
|
}
|
|
}
|
|
|
|
// example case : create sequence s1; rename s1 to s3;
|
|
bool need_table_lock_and_defense = false; // need table lock and rw defense
|
|
if (OB_FAIL(ret) || nullptr == from_table_schema) {
|
|
// OB_FAIL(ret) : error occur
|
|
// OB_SUCC(ret) but nullptr == from_table_schema : oracle mode rename A to B
|
|
} else if (OB_FALSE_IT(need_table_lock_and_defense =
|
|
from_table_schema->is_user_table() &&
|
|
((MOCK_DATA_VERSION_4_2_3_0 <= tenant_data_version && tenant_data_version < DATA_VERSION_4_3_0_0) /* ([4.2.3, 4.3.0)) */
|
|
|| DATA_VERSION_4_3_2_0 <= tenant_data_version /* [4.3.2, ~) */ ))) {
|
|
} else if (need_table_lock_and_defense &&
|
|
OB_FAIL(lock_table(trans, *from_table_schema))) {
|
|
LOG_WARN("lock table failed", K(ret), K(from_table_schema->get_table_id()));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && !is_oracle_mode) {
|
|
ObString rename_sql = sql.string();
|
|
ObSArray<std::pair<uint64_t, int64_t>> idx_schema_versions;
|
|
int64_t new_table_schema_version = 0;
|
|
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(),
|
|
need_reset_object_status,
|
|
trans,
|
|
&rename_sql,
|
|
new_table_schema_version,
|
|
idx_schema_versions))) {
|
|
LOG_WARN("failed to rename table!", K(rename_item), K(table_id), K(ret));
|
|
} else if (need_table_lock_and_defense &&
|
|
!in_new_table_set && // to avoid defense twice on the same table
|
|
OB_FAIL(build_rw_defense_for_table_(
|
|
tenant_data_version,
|
|
*from_table_schema,
|
|
new_table_schema_version,
|
|
idx_schema_versions,
|
|
trans))) {
|
|
LOG_WARN("fail to build rw defense for table", K(ret), K(new_table_schema_version), KPC(from_table_schema), KPC(database_schema));
|
|
} else if (OB_FAIL(rebuild_trigger_package(schema_guard,
|
|
tenant_id,
|
|
from_table_schema->get_trigger_list(),
|
|
database_schema->get_database_name_str(),
|
|
to_table_item.table_name_,
|
|
ddl_operator, trans))) {
|
|
LOG_WARN("failed to rebuild trigger package", K(rename_item), K(ret));
|
|
} else {
|
|
table_id = from_table_schema->get_table_id();
|
|
if (OB_FAIL(new_table_map.set_refactored(to_table_item, table_id))) {
|
|
LOG_WARN("failed to insert into table map", K(to_table_item),
|
|
K(table_id), K(ret));
|
|
} else {
|
|
if (!in_new_table_set) {
|
|
if (OB_FAIL(delete_table_set.set_refactored(from_table_item))) {
|
|
LOG_WARN("failed to add to delete table set", K(from_table_item), K(ret));
|
|
}
|
|
} else {
|
|
//rename table t1 to t3, t3 to t4, t2 to t3;
|
|
//should drop t3, when execute t3 to t4
|
|
if (OB_FAIL(new_table_map.erase_refactored(from_table_item))) {
|
|
LOG_WARN("failed to delete from new table map!", K(from_table_item), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && is_oracle_mode) {
|
|
ObString rename_sql = sql.string();
|
|
ObSArray<std::pair<uint64_t, int64_t>> idx_schema_versions;
|
|
int64_t new_table_schema_version = 0;
|
|
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(),
|
|
false,/*oracle mode can not rename multiple table*/
|
|
trans,
|
|
&rename_sql,
|
|
new_table_schema_version,
|
|
idx_schema_versions))) {
|
|
LOG_WARN("failed to rename table!", K(ret), K(rename_item), K(table_id));
|
|
} else if (need_table_lock_and_defense &&
|
|
OB_FAIL(build_rw_defense_for_table_(
|
|
tenant_data_version,
|
|
*from_table_schema,
|
|
new_table_schema_version,
|
|
idx_schema_versions,
|
|
trans))) {
|
|
LOG_WARN("fail to build rw defense for table", K(ret), K(new_table_schema_version), KPC(from_table_schema), KPC(database_schema));
|
|
} else if (OB_FAIL(rebuild_trigger_package(schema_guard,
|
|
tenant_id,
|
|
from_table_schema->get_trigger_list(),
|
|
database_schema->get_database_name_str(),
|
|
to_table_item.table_name_,
|
|
ddl_operator, trans))) {
|
|
LOG_WARN("failed to rebuild trigger package", K(rename_item), K(ret));
|
|
}
|
|
} else if (RENAME_TYPE_SYNONYM == rename_oracle_obj_type) {
|
|
ObSynonymInfo tmp_synonym_info(*synonym_info);
|
|
if (OB_FAIL(tmp_synonym_info.set_synonym_name(to_table_item.table_name_))) {
|
|
LOG_WARN("failed to set new synonym name to synonym_info", K(ret), K(to_table_item.table_name_), KPC(synonym_info));
|
|
} else if (OB_FAIL(ddl_operator.replace_synonym(tmp_synonym_info, trans, &rename_sql))) {
|
|
LOG_WARN("failed to rename synonym", K(ret), K(tmp_synonym_info));
|
|
}
|
|
} else if (RENAME_TYPE_SEQUENCE == rename_oracle_obj_type) {
|
|
ObSequenceDDLProxy ddl_operator(*schema_service_);
|
|
ObSequenceSchema tmp_sequence_schema;
|
|
if (OB_FAIL(tmp_sequence_schema.assign(*sequence_schema))) {
|
|
LOG_WARN("fail to assign sequence schema", KR(ret));
|
|
} else if (OB_FAIL(tmp_sequence_schema.set_sequence_name(to_table_item.table_name_))) {
|
|
LOG_WARN("failed to set new sequence name to sequence_schema", K(ret), K(to_table_item.table_name_), KPC(sequence_schema));
|
|
} else if (OB_FAIL(ddl_operator.rename_sequence(tmp_sequence_schema, trans, &rename_sql))) {
|
|
LOG_WARN("failed to rename sequence", K(ret), K(tmp_sequence_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ObDependencyInfo::modify_all_obj_status(all_dep_objs, trans,
|
|
tenant_id, ddl_operator,
|
|
*schema_service_))) {
|
|
LOG_WARN("failed to modify all obj status", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_oracle_mode) {
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < rename_table_arg.rename_table_items_.size(); ++i) {
|
|
const ObRenameTableItem &rename_item = rename_table_arg.rename_table_items_.at(i);
|
|
uint64_t from_table_id = OB_INVALID_ID;
|
|
const ObDatabaseSchema *to_database_schema = NULL;
|
|
ObTableItem to_table_item;
|
|
to_table_item.database_name_ = rename_item.new_db_name_;
|
|
to_table_item.table_name_ = rename_item.new_table_name_;
|
|
if (OB_FAIL(schema_guard.get_tenant_name_case_mode(tenant_id, to_table_item.mode_))) {
|
|
LOG_WARN("failed to get tenant name case mode!", K(ret), K(tenant_id), K(to_table_item));
|
|
} else if (OB_FAIL(new_table_map.get_refactored(to_table_item, from_table_id))) {
|
|
if (OB_HASH_NOT_EXIST == ret) {
|
|
ret = OB_SUCCESS; // continue
|
|
} else {
|
|
LOG_WARN("get_refactored from new_table_map failed", K(ret), K(to_table_item), K(to_table_item.mode_), K(from_table_id));
|
|
}
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, to_table_item.database_name_, to_database_schema))) {
|
|
LOG_WARN("get_database_schema failed", K(ret), K(to_table_item));
|
|
} else if (OB_ISNULL(to_database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
SQL_RESV_LOG(WARN, "to_database_schema is null", K(ret), K(rename_item));
|
|
} else {
|
|
const ObMockFKParentTableSchema *ori_mock_parent_table_schema_ptr = NULL;
|
|
if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name(
|
|
tenant_id, to_database_schema->get_database_id(),
|
|
to_table_item.table_name_, ori_mock_parent_table_schema_ptr))) {
|
|
SQL_RESV_LOG(WARN, "failed to check_mock_fk_parent_table_exist_with_name", K(ret));
|
|
} else if (OB_NOT_NULL(ori_mock_parent_table_schema_ptr)) { // will replace mock_parent_table with this new table
|
|
const ObTableSchema *from_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(rename_table_arg.tenant_id_, from_table_id, from_table_schema))) {
|
|
LOG_WARN("fail to get from_table_schema schema", K(ret));
|
|
} else if (OB_ISNULL(from_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("to_table_schema is null", K(ret));
|
|
} else {
|
|
ObArray<ObAuxTableMetaInfo> simple_index_infos;
|
|
ObArray<const share::schema::ObTableSchema*> unique_index_schemas;
|
|
if (OB_FAIL(from_table_schema->get_simple_index_infos(simple_index_infos))) {
|
|
SERVER_LOG(WARN, "get simple_index_infos without delay_deleted_tid failed", K(ret));
|
|
} else {
|
|
const ObTableSchema *index_schema = NULL;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < simple_index_infos.count(); ++j) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(rename_table_arg.tenant_id_, simple_index_infos.at(j).table_id_, index_schema))) {
|
|
LOG_WARN("fail to get to_table_schema schema", K(ret));
|
|
} else if (OB_ISNULL(index_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("index_schema is null", K(ret));
|
|
} else if (index_schema->is_unique_index() && OB_FAIL(unique_index_schemas.push_back(index_schema))) {
|
|
LOG_WARN("fail to push_back index_schema to unique_index_schemas", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObMockFKParentTableSchema mock_fk_parent_table_schema;
|
|
if (OB_FAIL(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table(
|
|
schema_guard, ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id(),
|
|
*from_table_schema, unique_index_schemas, mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table", K(ret));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
if (FAILEDx(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(tenant_id));
|
|
}
|
|
}
|
|
} // trans.start
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
//refresh table schema
|
|
if (OB_SUCC(ret)) {
|
|
DEBUG_SYNC(RENAME_TABLE_BEFORE_PUBLISH_SCHEMA);
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("refresh_schema failed", K(ret));
|
|
}
|
|
}
|
|
} // get_schema_guard
|
|
} // ddl_operator
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::build_single_table_rw_defensive_(
|
|
const uint64_t tenant_id,
|
|
const uint64_t tenant_data_version,
|
|
const ObArray<ObTabletID> &tablet_ids,
|
|
const int64_t schema_version,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id || tablet_ids.empty() || schema_version <= 0)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid args", K(ret), K(tenant_id), K(tablet_ids), K(schema_version));
|
|
} else if (OB_LIKELY(tenant_data_version >= DATA_VERSION_4_3_2_0)) {
|
|
const int64_t abs_timeout_us = THIS_WORKER.is_timeout_ts_valid() ? THIS_WORKER.get_timeout_ts()
|
|
: ObTimeUtility::current_time() + GCONF.rpc_timeout;
|
|
if (OB_FAIL(ObTabletBindingMdsHelper::modify_tablet_binding_for_rw_defensive(tenant_id, tablet_ids, schema_version, abs_timeout_us, trans))) {
|
|
LOG_WARN("failed to modify tablet binding", K(ret), K(abs_timeout_us));
|
|
}
|
|
} else {
|
|
ObArray<ObBatchUnbindTabletArg> args;
|
|
if (OB_FAIL(build_modify_tablet_binding_args(
|
|
tenant_id, tablet_ids, true/*is_hidden_tablets*/, schema_version, args, trans))) {
|
|
LOG_WARN("failed to build reuse index args", K(ret));
|
|
}
|
|
ObArenaAllocator allocator("DDLRWDefens");
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < args.count(); i++) {
|
|
int64_t pos = 0;
|
|
int64_t size = args[i].get_serialize_size();
|
|
char *buf = nullptr;
|
|
allocator.reuse();
|
|
if (OB_ISNULL(buf = static_cast<char *>(allocator.alloc(size)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("failed to allocate", K(ret));
|
|
} else if (OB_FAIL(args[i].serialize(buf, size, pos))) {
|
|
LOG_WARN("failed to serialize arg", K(ret));
|
|
} else if (OB_FAIL(trans.register_tx_data(args[i].tenant_id_, args[i].ls_id_, transaction::ObTxDataSourceType::UNBIND_TABLET_NEW_MDS, buf, pos))) {
|
|
LOG_WARN("failed to register tx data", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/*
|
|
* Emphasize: Build read and write defense by new_schema_version through registing mds.
|
|
* This function is design for building rw defense during renaming table,
|
|
* table_schema may be an origin_table_schema, please be careful when using this interface.
|
|
* (An origin_table_schema has no updated tablet info compared to a new table schema, e.g. tablet split)
|
|
*/
|
|
int ObDDLService::build_rw_defense_for_table_(
|
|
const uint64_t tenant_data_version,
|
|
const ObTableSchema &table_schema,
|
|
const int64_t new_data_table_schema_version,
|
|
const ObIArray<std::pair<uint64_t, int64_t>> &aux_schema_versions, // pair: table_id, schema_version
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
if (!table_schema.is_valid() ||
|
|
OB_INVALID_SCHEMA_VERSION == new_data_table_schema_version) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("argument is invalid", K(ret), K(table_schema), K(new_data_table_schema_version), K(aux_schema_versions));
|
|
} else {
|
|
for (int64_t i = 0; i < aux_schema_versions.count(); ++i) {
|
|
if (OB_INVALID_ID == aux_schema_versions.at(i).first ||
|
|
OB_INVALID_SCHEMA_VERSION == aux_schema_versions.at(i).second) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("argument is in-valid", K(ret), K(aux_schema_versions));
|
|
}
|
|
}
|
|
}
|
|
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObArray<ObTabletID> tablet_ids;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
ObSchemaService *schema_service = nullptr;
|
|
|
|
if (OB_FAIL(ret)) {
|
|
// error occur
|
|
} else if (!table_schema.is_user_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("do not support add rw defense for non-user_table during rename table operations", K(ret), K(table_schema));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_WARN("schema sql service must not be null", K(schema_service), K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(table_schema.get_tablet_ids(tablet_ids))) {
|
|
LOG_WARN("failed to get tablet_ids", K(ret), K(table_schema));
|
|
} else if (OB_FAIL(build_single_table_rw_defensive_(tenant_id, tenant_data_version, tablet_ids, new_data_table_schema_version, trans))) {
|
|
LOG_WARN("failed to build defense ", K(ret), K(table_schema), K(tablet_ids));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < aux_schema_versions.count(); ++i) {
|
|
tablet_ids.reuse();
|
|
const ObTableSchema *aux_table_schema = NULL;
|
|
const uint64_t table_id = aux_schema_versions.at(i).first;
|
|
const int64_t schema_version = aux_schema_versions.at(i).second;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, aux_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_ISNULL(aux_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (OB_FAIL(aux_table_schema->get_tablet_ids(tablet_ids))) {
|
|
LOG_WARN("failed to get tablet_ids", K(ret), KPC(aux_table_schema));
|
|
} else if (OB_FAIL(build_single_table_rw_defensive_(tenant_id, tenant_data_version, tablet_ids, schema_version, trans))) {
|
|
LOG_WARN("failed to build defense ", K(ret), KPC(aux_table_schema), K(i), K(schema_version));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::truncate_table_in_trans(const obrpc::ObTruncateTableArg &arg,
|
|
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 ObString &database_name)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
const 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(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
ObSArray<ObSAuditSchema> audit_schemas;
|
|
ObArray<ObObjPriv> orig_obj_privs_ora;
|
|
const bool to_recyclebin = false;
|
|
if (table_schemas.count() < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schemas have no element", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is null", K(ret));
|
|
} else {
|
|
ObArray<const ObSAuditSchema *> orig_audits;
|
|
if (OB_FAIL(schema_guard.get_audit_schema_in_owner(tenant_id,
|
|
AUDIT_TABLE,
|
|
orig_table_schema.get_table_id(),
|
|
orig_audits))) {
|
|
LOG_WARN("fail to get_audit_schema_in_owner", K(tenant_id), K(orig_table_schema), K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_audits.count(); ++i) {
|
|
const ObSAuditSchema *audit_schema = orig_audits.at(i);
|
|
if (OB_ISNULL(audit_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("audit_schema is NULL", K(ret));
|
|
} else {
|
|
ObSAuditSchema new_audit_schema;
|
|
if (OB_FAIL(new_audit_schema.assign(*audit_schema))) {
|
|
LOG_WARN("fail to assign ObSAuditSchema", KR(ret));
|
|
} else if (OB_FAIL(audit_schemas.push_back(new_audit_schema))) {
|
|
LOG_WARN("failed to add audit_schema!", K(new_audit_schema), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// Save Oracle obj privs on table for later restore
|
|
if (OB_SUCC(ret) && OB_FAIL(get_obj_privs_ora(tenant_id,
|
|
orig_table_schema.get_table_id(),
|
|
static_cast<uint64_t>(ObObjectType::TABLE),
|
|
schema_guard,
|
|
orig_obj_privs_ora))) {
|
|
LOG_WARN("fail to get obj privs ora", KR(ret), K(tenant_id), K(orig_table_schema.get_table_id()));
|
|
}
|
|
if (OB_SUCC(ret) && OB_FAIL(drop_aux_table_in_truncate(
|
|
orig_table_schema, schema_guard, trans, ddl_operator,
|
|
USER_INDEX, to_recyclebin))) {
|
|
LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(drop_aux_table_in_truncate(
|
|
orig_table_schema, schema_guard, trans, ddl_operator,
|
|
AUX_VERTIAL_PARTITION_TABLE, to_recyclebin))) {
|
|
LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(drop_aux_table_in_truncate(
|
|
orig_table_schema, schema_guard, trans, ddl_operator,
|
|
AUX_LOB_META, to_recyclebin))) {
|
|
LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema));
|
|
} else if (OB_FAIL(drop_aux_table_in_truncate(
|
|
orig_table_schema, schema_guard, trans, ddl_operator,
|
|
AUX_LOB_PIECE, to_recyclebin))) {
|
|
LOG_WARN("drop_aux_table_in_truncate failed", K(ret), K(orig_table_schema));
|
|
} else {
|
|
// Two conditions need to be met for placing in the recycle bin:
|
|
// 1. to_recyclebin is marked
|
|
// 2. it is not inner table
|
|
if (OB_FAIL(drop_trigger_in_drop_table(trans, ddl_operator, schema_guard,
|
|
orig_table_schema, to_recyclebin))) {
|
|
LOG_WARN("drop trigger failed,", K(orig_table_schema), K(ret), K(to_recyclebin));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (to_recyclebin && !is_inner_table(orig_table_schema.get_table_id())) {
|
|
if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(orig_table_schema,
|
|
schema_guard, trans,
|
|
ddl_stmt_str, true))) {
|
|
LOG_WARN("ddl_operator drop table to recyclebin failed,",
|
|
K(orig_table_schema), K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ddl_operator.drop_table(orig_table_schema,
|
|
trans, ddl_stmt_str,
|
|
true/*is_truncate_table*/))) {
|
|
LOG_WARN("ddl_operator drop_table failed", K(orig_table_schema), K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
bool is_truncate_table = false;
|
|
bool recycle_cnt = 0;
|
|
for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); i++) {
|
|
if (0 == i) {
|
|
is_truncate_table = true; //record create table with opertion type TRUNCATE_TABLE_CREATE
|
|
} else {
|
|
is_truncate_table = false;
|
|
}
|
|
ObTableSchema &tmp_schema = table_schemas.at(i);
|
|
bool is_table_in_recyclebin = false;
|
|
if (tmp_schema.is_in_recyclebin()) {
|
|
is_table_in_recyclebin = true;
|
|
if (!tmp_schema.is_index_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("normal table is in recyclebin", K(ret));
|
|
continue;
|
|
}
|
|
if (recycle_cnt >= index_recycle_objs.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("index recycle objs indx is wrong", K(recycle_cnt), K(index_recycle_objs.count()), K(ret));
|
|
} else {
|
|
const ObRecycleObject &recycle_obj = index_recycle_objs.at(recycle_cnt);
|
|
if (OB_FAIL(reconstruct_table_schema_from_recyclebin(tmp_schema, recycle_obj, schema_guard))) {
|
|
LOG_WARN("ddl operator reconstruct table schema from recyclebin failed", K(tmp_schema), K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
recycle_cnt++;
|
|
ObString new_index_name;
|
|
ObString new_index_table_name;
|
|
if (OB_FAIL(ObTableSchema::get_index_name(allocator,
|
|
orig_table_schema.get_table_id(),
|
|
tmp_schema.get_table_name_str(),
|
|
new_index_name))) {
|
|
LOG_WARN("failed to build new index table name!", K(tmp_schema), K(ret));
|
|
continue;
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator,
|
|
tmp_schema.get_data_table_id(),
|
|
new_index_name,
|
|
new_index_table_name))) {
|
|
LOG_WARN("failed to build new index table name!", K(tmp_schema.get_data_table_id()));
|
|
continue;
|
|
} else {
|
|
tmp_schema.set_table_name(new_index_table_name);
|
|
}
|
|
const int VISIBLE = 0;
|
|
const uint64_t DROPINDEX = 0;
|
|
const uint64_t INVISIBLEBEFORE = 0;
|
|
tmp_schema.set_drop_index(DROPINDEX);
|
|
if (!tmp_schema.is_invisible_before()) {
|
|
tmp_schema.set_index_visibility(VISIBLE);
|
|
}
|
|
tmp_schema.set_invisible_before(INVISIBLEBEFORE);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && (0 == i)) {
|
|
// truncate table need update child table foreign key ID and foreign_key_id
|
|
ObIArray<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 (orig_table_schema.get_table_id() == foreign_key_info.child_table_id_) {
|
|
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();
|
|
}
|
|
foreign_key_info.child_table_id_ = tmp_schema.get_table_id();
|
|
} else if (orig_table_schema.get_table_id() == foreign_key_info.parent_table_id_) {
|
|
foreign_key_info.parent_table_id_ = tmp_schema.get_table_id();
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig table id is not equal to orig parent_table_id_ or orig child_table_id_ of foreign_key_info ",
|
|
K(ret), K(orig_table_schema.get_table_id()), K(foreign_key_info));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (!is_table_in_recyclebin) {
|
|
if (OB_FAIL(ddl_operator.create_sequence_in_create_table(
|
|
tmp_schema,
|
|
trans,
|
|
schema_guard,
|
|
NULL))) {
|
|
LOG_WARN("failed to create sequence in create table", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.create_table(
|
|
tmp_schema,
|
|
trans,
|
|
0 == i ? ddl_stmt_str : NULL,
|
|
true, /*need_sync_schema_version*/
|
|
is_truncate_table))) {
|
|
LOG_WARN("failed to create table schema, ", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, tmp_schema))) {
|
|
LOG_WARN("failed to insert_temp_table_info!", K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ddl_operator.create_index_in_recyclebin(
|
|
tmp_schema,
|
|
schema_guard,
|
|
trans,
|
|
NULL))) {
|
|
LOG_WARN("failed to create index schema", K(ret));
|
|
}
|
|
}
|
|
}
|
|
// If table truncate is child table which has mock parent tables, will update the schema version of mock parent tables
|
|
if (OB_SUCC(ret) && tmp_schema.get_foreign_key_real_count() > 0) {
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = tmp_schema.get_foreign_key_infos();
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < foreign_key_infos.count(); ++j) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(j);
|
|
if (foreign_key_info.is_parent_table_mock_) {
|
|
const ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL;
|
|
if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_id(
|
|
tenant_id, foreign_key_info.parent_table_id_, mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("get_mock_fk_parent_table_schema_with_id failed", K(ret), K(tenant_id), K(foreign_key_info.parent_table_id_));
|
|
} else if (OB_ISNULL(mock_fk_parent_table_ptr)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mock_fk_parent_table_ptr is null", K(ret));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(*mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("push_back mock_fk_parent_table failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !mock_fk_parent_table_schema_array.empty()) {
|
|
if (!mock_fk_parent_table_schema_array.empty()) {
|
|
for (int64_t k = 0; k < mock_fk_parent_table_schema_array.count(); ++k) {
|
|
mock_fk_parent_table_schema_array.at(k).set_operation_type(MOCK_FK_PARENT_TABLE_OP_UPDATE_SCHEMA_VERSION);
|
|
}
|
|
if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("failed to deal_with_mock_fk_parent_table", K(ret), K(mock_fk_parent_table_schema_array));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && (0 == i)) {
|
|
// truncate table needs to rebuild the audit rules for the newly created table
|
|
common::ObSqlString public_sql_string;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < audit_schemas.count(); ++i) {
|
|
uint64_t new_audit_id = common::OB_INVALID_ID;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("failed to gen new schema_version", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_service->fetch_new_audit_id(tenant_id, new_audit_id))) {
|
|
LOG_WARN("failed to fetch new_audit_id", K(ret));
|
|
} else {
|
|
ObSAuditSchema &new_audit_schema = audit_schemas.at(i);
|
|
new_audit_schema.set_schema_version(new_schema_version);
|
|
new_audit_schema.set_audit_id(new_audit_id);
|
|
new_audit_schema.set_owner_id(tmp_schema.get_table_id());
|
|
if (OB_FAIL(schema_service->get_audit_sql_service().handle_audit_metainfo(
|
|
new_audit_schema,
|
|
AUDIT_MT_ADD,
|
|
false,
|
|
new_schema_version,
|
|
NULL,
|
|
trans,
|
|
public_sql_string))) {
|
|
LOG_WARN("failed to add audit_schema", K(new_audit_schema), K(ret));
|
|
} else {
|
|
LOG_INFO("succ to add audit_schema for truncate", K(new_audit_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && (0 == i)) {
|
|
// truncate table needs to rebuild the audit rules for the newly created table
|
|
if (OB_FAIL(restore_obj_privs_for_table(tmp_schema.get_table_id(),
|
|
database_name,
|
|
tmp_schema.get_table_name_str(),
|
|
ddl_operator,
|
|
trans,
|
|
orig_obj_privs_ora))) {
|
|
LOG_WARN("restore_obj_privs_for_table failed", KR(ret),
|
|
K(tmp_schema.get_table_id()),
|
|
K(database_name),
|
|
K(tmp_schema.get_table_name_str()));
|
|
}
|
|
}
|
|
}
|
|
|
|
SCN frozen_scn;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObTableCreator table_creator(tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(
|
|
tenant_id,
|
|
schema_guard,
|
|
sql_proxy_);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
const ObTablegroupSchema *data_tablegroup_schema = NULL; // keep NULL if no tablegroup
|
|
const ObTableSchema &data_table_schema = table_schemas.at(0);
|
|
|
|
if (OB_FAIL(table_creator.init(false/*need_check_tablet_cnt*/))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
} else if (OB_INVALID_ID != data_table_schema.get_tablegroup_id()) {
|
|
if (OB_FAIL(schema_guard.get_tablegroup_schema(
|
|
data_table_schema.get_tenant_id(),
|
|
data_table_schema.get_tablegroup_id(),
|
|
data_tablegroup_schema))) {
|
|
LOG_WARN("get tablegroup_schema failed", KR(ret), K(data_table_schema));
|
|
} else if (OB_ISNULL(data_tablegroup_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("data_tablegroup_schema is null", KR(ret), K(data_table_schema));
|
|
}
|
|
}
|
|
|
|
ObArray<const ObTableSchema*> schemas;
|
|
ObArray<bool> need_create_empty_majors;
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
const share::schema::ObTableSchema &this_table = table_schemas.at(i);
|
|
const int64_t table_id = this_table.get_table_id();
|
|
if (!this_table.has_tablet()) {
|
|
} else if (!this_table.is_global_index_table()) {
|
|
if (OB_FAIL(schemas.push_back(&this_table))
|
|
|| OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("failed to push_back", KR(ret), K(this_table));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(new_table_tablet_allocator.prepare(trans, this_table, data_tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare ls for index schema tablets");
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(
|
|
ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_table_arg(
|
|
this_table,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
true /*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table partitions failed", KR(ret), K(this_table));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_INVALID_VERSION == last_schema_version
|
|
&& OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("get last schema version failed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_ori_schema_version(
|
|
trans, tenant_id, table_id, last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!",
|
|
K(ret), K(tenant_id), K(table_id), K(last_schema_version));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
}
|
|
// virtual table and view skip
|
|
else if (schemas.count() <= 0) {
|
|
} else if (OB_FAIL(new_table_tablet_allocator.prepare(trans, *schemas.at(0), data_tablegroup_schema))) {
|
|
LOG_WARN("new table tablet allocator prepared failed", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(
|
|
ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg(
|
|
schemas,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors/*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("create table partitions failed", KR(ret), K(orig_table_schema),
|
|
K(last_schema_version));
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("execute create partition failed", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
// Create the trigger after creating the table,table_schemas.at(0) is new table schema
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(create_trigger_for_truncate_table(schema_guard,
|
|
orig_table_schema.get_trigger_list(),
|
|
table_schemas.at(0),
|
|
ddl_operator, trans))) {
|
|
LOG_WARN("failed to create trigger for truncate table", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (arg.task_id_ > 0 && OB_FAIL(ObDDLRetryTask::update_task_status_wait_child_task_finish(trans, tenant_id, arg.task_id_))) {
|
|
LOG_WARN("update ddl task status failed", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::restore_obj_privs_for_table(const uint64_t new_table_id,
|
|
const ObString &database_name,
|
|
const ObString &table_name,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObIArray<ObObjPriv> &orig_obj_privs_ora) {
|
|
int ret = OB_SUCCESS;
|
|
for (int i = 0; OB_SUCC(ret) && i < orig_obj_privs_ora.count(); ++i) {
|
|
ObObjPriv &obj_priv = orig_obj_privs_ora.at(i);
|
|
ObRawObjPrivArray raw_priv_array_option; // privs with grant option
|
|
ObRawObjPrivArray raw_priv_array_no_option; // privs without grant option
|
|
if (OB_FAIL(ObPrivPacker::raw_option_obj_priv_from_pack(
|
|
obj_priv.get_obj_privs(),
|
|
raw_priv_array_option))) {
|
|
LOG_WARN("raw_option_obj_priv_from_pack failed", K(ret), K(obj_priv.get_obj_privs()));
|
|
} else if (OB_FAIL(ObPrivPacker::raw_no_option_obj_priv_from_pack(
|
|
obj_priv.get_obj_privs(),
|
|
raw_priv_array_no_option))) {
|
|
LOG_WARN("raw_no_option_obj_priv_from_pack failed", K(ret), K(obj_priv.get_obj_privs()));
|
|
} else {
|
|
ObTablePrivSortKey table_key(obj_priv.get_tenant_id(),
|
|
obj_priv.get_grantee_id(),
|
|
database_name,
|
|
table_name);
|
|
obj_priv.set_obj_id(new_table_id); // truncate can change table_id
|
|
ObObjPrivSortKey obj_priv_key = obj_priv.get_sort_key();
|
|
// Restore obj privs without grant option
|
|
if (OB_FAIL(ddl_operator.grant_table(
|
|
table_key,
|
|
0, // priv_set is null
|
|
NULL,
|
|
trans,
|
|
raw_priv_array_no_option, // add new priv
|
|
NO_OPTION,
|
|
obj_priv_key))) {
|
|
LOG_WARN("failed to add priv", K(ret), K(table_key));
|
|
} else if (OB_FAIL(ddl_operator.grant_table( // Restore obj priv with grant option
|
|
table_key,
|
|
0, // priv_set is null
|
|
NULL,
|
|
trans,
|
|
raw_priv_array_option, // add new priv
|
|
GRANT_OPTION,
|
|
obj_priv_key))) {
|
|
LOG_WARN("failed to add priv", K(ret), K(table_key));
|
|
} else {
|
|
LOG_INFO("succ to add priv for truncate table", K(ret), K(table_key));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_aux_table_in_truncate(
|
|
const ObTableSchema &orig_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
const ObTableType table_type,
|
|
const bool to_recyclebin)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
const bool is_index = USER_INDEX == table_type;
|
|
ObSEArray<uint64_t, 16> aux_vp_tid_array;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
uint64_t lob_meta_table_id = 0;
|
|
uint64_t lob_piece_table_id = 0;
|
|
int64_t N = 0;
|
|
if (table_type == USER_INDEX) {
|
|
if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type));
|
|
} else {
|
|
N = simple_index_infos.count();
|
|
}
|
|
} else if (table_type == AUX_VERTIAL_PARTITION_TABLE) {
|
|
if (OB_FAIL(orig_table_schema.get_aux_vp_tid_array(aux_vp_tid_array))) {
|
|
LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type));
|
|
} else {
|
|
N = aux_vp_tid_array.count();
|
|
}
|
|
} else if (table_type == AUX_LOB_META) {
|
|
lob_meta_table_id = orig_table_schema.get_aux_lob_meta_tid();
|
|
N = orig_table_schema.has_lob_aux_table() ? 1 : 0;
|
|
} else if (table_type == AUX_LOB_PIECE) {
|
|
lob_piece_table_id = orig_table_schema.get_aux_lob_piece_tid();
|
|
N = orig_table_schema.has_lob_aux_table() ? 1 : 0;
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table type is invalide", K(ret), K(table_type));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
|
|
const ObTableSchema *aux_table_schema = NULL;
|
|
uint64_t tid = 0;
|
|
if (table_type == USER_INDEX) {
|
|
tid = simple_index_infos.at(i).table_id_;
|
|
} else if (table_type == AUX_VERTIAL_PARTITION_TABLE) {
|
|
tid = aux_vp_tid_array.at(i);
|
|
} else if (table_type == AUX_LOB_META) {
|
|
tid = lob_meta_table_id;
|
|
} else if (table_type == AUX_LOB_PIECE) {
|
|
tid = lob_piece_table_id;
|
|
}
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, tid, aux_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", tid, K(ret));
|
|
} else if (OB_ISNULL(aux_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (to_recyclebin && !is_inner_table(orig_table_schema.get_table_id())) {
|
|
// support truncate table when recyclebin on
|
|
if (aux_table_schema->is_in_recyclebin()) {
|
|
LOG_WARN("the aux table is already in recyclebin");
|
|
} else if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(*aux_table_schema,
|
|
schema_guard,
|
|
trans,
|
|
NULL))) {
|
|
LOG_WARN("drop aux table to recycle failed", K(ret));
|
|
}
|
|
} else if (aux_table_schema->is_in_recyclebin()) {
|
|
// if aux table is in recyclebin (not support now), can purge
|
|
if (OB_FAIL(ddl_operator.purge_table_in_recyclebin(
|
|
*aux_table_schema, trans, NULL))) {
|
|
LOG_WARN("purge aux table failed", K(ret), K(*aux_table_schema));
|
|
}
|
|
} else if (OB_FAIL(ddl_operator.drop_table(*aux_table_schema, trans))) {
|
|
LOG_WARN("ddl_operator drop_table failed,", K(*aux_table_schema), K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::truncate_oracle_temp_table(const ObString &db_name,
|
|
const ObString &tab_name,
|
|
const uint64_t tenant_id,
|
|
const uint64_t session_id,
|
|
const int64_t sess_create_time)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSqlString sql;
|
|
int64_t affect_rows = 0;
|
|
common::ObOracleSqlProxy oracle_sql_proxy;
|
|
if (OB_FAIL(oracle_sql_proxy.init(sql_proxy_->get_pool()))) {
|
|
LOG_WARN("init oracle sql proxy failed", K(ret));
|
|
} else if (0 == sess_create_time) {
|
|
ret = sql.assign_fmt("DELETE FROM \"%.*s\".\"%.*s\" WHERE "
|
|
"%s = %lu",
|
|
db_name.length(), db_name.ptr(),
|
|
tab_name.length(), tab_name.ptr(),
|
|
OB_HIDDEN_SESSION_ID_COLUMN_NAME, session_id);
|
|
} else {
|
|
ret = sql.assign_fmt("DELETE FROM \"%.*s\".\"%.*s\" WHERE "
|
|
"%s = %lu AND %s <> %lu",
|
|
db_name.length(), db_name.ptr(),
|
|
tab_name.length(), tab_name.ptr(),
|
|
OB_HIDDEN_SESSION_ID_COLUMN_NAME, session_id,
|
|
OB_HIDDEN_SESS_CREATE_TIME_COLUMN_NAME, sess_create_time);
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("assign sql failed", K(ret));
|
|
} else if (OB_FAIL(oracle_sql_proxy.write(tenant_id, sql.ptr(), affect_rows))) {
|
|
LOG_WARN("execute sql failed", K(ret), K(sql));
|
|
} else {
|
|
LOG_DEBUG("succeed to execute truncate table as delete stmt", K(sql), K(affect_rows));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::maintain_obj_dependency_info(const obrpc::ObDependencyObjDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_TENANT_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (!arg.update_dep_objs_.empty()
|
|
&& OB_FAIL(process_schema_object_dependency(tenant_id, arg.update_dep_objs_,
|
|
schema_guard, trans, ddl_operator, ObReferenceObjTable::UPDATE_OP))) {
|
|
LOG_WARN("failed to process update object dependency", K(ret));
|
|
} else if (!arg.insert_dep_objs_.empty()
|
|
&& OB_FAIL(process_schema_object_dependency(tenant_id, arg.insert_dep_objs_,
|
|
schema_guard, trans, ddl_operator, ObReferenceObjTable::INSERT_OP))) {
|
|
LOG_WARN("failed to process insert object dependency", K(ret));
|
|
} else if (!arg.delete_dep_objs_.empty()
|
|
&& OB_FAIL(process_schema_object_dependency(tenant_id, arg.delete_dep_objs_,
|
|
schema_guard, trans, ddl_operator, ObReferenceObjTable::DELETE_OP))) {
|
|
LOG_WARN("failed to process delete object dependency", K(ret));
|
|
} else if (arg.schema_.is_valid() && OB_FAIL(recompile_view(arg.schema_, arg.reset_view_column_infos_, trans))) {
|
|
LOG_WARN("failed to recompile view", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::process_schema_object_dependency(
|
|
const uint64_t tenant_id,
|
|
const ObReferenceObjTable::DependencyObjKeyItemPairs &dep_objs,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
ObReferenceObjTable::ObSchemaRefObjOp op)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
if (dep_objs.empty()) {
|
|
// do nothing
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
|
|
} else {
|
|
switch (op) {
|
|
case ObReferenceObjTable::INSERT_OP:
|
|
case ObReferenceObjTable::UPDATE_OP:
|
|
OZ (ObReferenceObjTable::batch_execute_insert_or_update_obj_dependency(tenant_id,
|
|
new_schema_version, dep_objs, trans, schema_guard, ddl_operator));
|
|
break;
|
|
case ObReferenceObjTable::DELETE_OP:
|
|
OZ (ObReferenceObjTable::batch_execute_delete_obj_dependency(tenant_id, dep_objs, trans));
|
|
break;
|
|
default:
|
|
break;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_hidden_obj_name(const uint64_t obj_id,
|
|
const uint64_t table_id,
|
|
const uint64_t origin_fk_id,
|
|
ObIAllocator &allocator,
|
|
ObString &hidden_obj_name,
|
|
const ObSchemaType schema_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t pos = 0;
|
|
char *buf = NULL;
|
|
int64_t buf_len = OB_MAX_TABLE_NAME_LENGTH;
|
|
if (OB_ISNULL(buf = static_cast<char *>(allocator.alloc(buf_len)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("failed to alloc memory", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "__hidden_"))) {
|
|
LOG_WARN("append name to buf error", K(ret));
|
|
} else if (FK_SCHEMA == schema_type
|
|
&& OB_FAIL(databuff_printf(buf, buf_len, pos, "%ld_", origin_fk_id))) {
|
|
LOG_WARN("append name to buf error", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%ld", obj_id))) {
|
|
LOG_WARN("append name to buf error", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "_%ld", table_id))) {
|
|
LOG_WARN("append name to buf error", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", schema_type_str(schema_type)))) {
|
|
LOG_WARN("append name to buf error", K(ret));
|
|
} else {
|
|
hidden_obj_name.assign_ptr(buf, pos);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::is_foreign_key_name_prefix_match(const ObForeignKeyInfo &origin_fk_info,
|
|
const ObForeignKeyInfo &hidden_fk_info,
|
|
common::ObIAllocator &allocator,
|
|
bool &is_prefix_match)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_prefix_match = false;
|
|
int64_t pos = 0;
|
|
char *buf = nullptr;
|
|
int64_t buf_len = OB_MAX_TABLE_NAME_LENGTH;
|
|
if (OB_ISNULL(buf = static_cast<char *>(allocator.alloc(buf_len)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("failed to alloc memory", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "__hidden_"))) {
|
|
LOG_WARN("append name to buf error", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%ld_", origin_fk_info.foreign_key_id_))) {
|
|
LOG_WARN("append name to buf error", K(ret));
|
|
} else {
|
|
ObString prefix_hidden_fk_name;
|
|
prefix_hidden_fk_name.assign_ptr(buf, pos);
|
|
is_prefix_match = hidden_fk_info.foreign_key_name_.prefix_match(prefix_hidden_fk_name);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::prepare_hidden_table_schema(const ObTableSchema &orig_table_schema,
|
|
ObIAllocator &allocator,
|
|
ObTableSchema &hidden_table_schema,
|
|
const ObString &index_name)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
uint64_t new_table_id = OB_INVALID_ID;
|
|
ObString new_table_name;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(hidden_table_schema.get_tenant_id(),
|
|
new_table_id))) {
|
|
LOG_WARN("failed to fetch_new_table_id", K(ret));
|
|
} else {
|
|
// in the first stage, create a hidden table without creating constraints, foreign keys
|
|
// and indexes. if it needs to be created, it will be created in the second stage
|
|
hidden_table_schema.clear_constraint();
|
|
hidden_table_schema.clear_foreign_key_infos();
|
|
hidden_table_schema.set_data_table_id(0);
|
|
if (OB_FAIL(gen_hidden_obj_name(new_table_id,
|
|
orig_table_schema.get_table_id(),
|
|
common::OB_INVALID_ID,
|
|
allocator,
|
|
new_table_name,
|
|
TABLE_SCHEMA))) {
|
|
LOG_WARN("failed to gen hidden table name", K(orig_table_schema), K(ret));
|
|
} else if (OB_FAIL(try_format_partition_schema(hidden_table_schema))) {
|
|
LOG_WARN("convert template schema to non template schema", K(ret));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(hidden_table_schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(hidden_table_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(hidden_table_schema))) {
|
|
LOG_WARN("fail to generate tablet id for hidden table", K(ret), K(hidden_table_schema));
|
|
} else if (orig_table_schema.is_ctas_tmp_table() &&
|
|
OB_FAIL(clear_ctas_hidden_table_session_id_(hidden_table_schema))) {
|
|
// for CTAS table, clear its session id, otherwise this table schema will not be visble
|
|
// to the index rebuiding phase.
|
|
LOG_WARN("fail to clear ctas hidden table session id", K(ret), K(orig_table_schema),
|
|
K(hidden_table_schema));
|
|
} else {
|
|
// offline ddl change table_id, so we need to reset truncate_version
|
|
hidden_table_schema.set_truncate_version(OB_INVALID_VERSION);
|
|
hidden_table_schema.set_table_id(new_table_id);
|
|
hidden_table_schema.set_table_name(new_table_name);
|
|
hidden_table_schema.set_association_table_id(orig_table_schema.get_table_id());
|
|
// set the hidden attributes of the table
|
|
hidden_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL);
|
|
if (orig_table_schema.get_tenant_id() != hidden_table_schema.get_tenant_id()) {
|
|
// recover restore table, do not sync log to cdc.
|
|
hidden_table_schema.set_ddl_ignore_sync_cdc_flag(ObDDLIgnoreSyncCdcFlag::DONT_SYNC_LOG_FOR_CDC);
|
|
}
|
|
// in oracle mode, need to add primary key constraints
|
|
if (is_oracle_mode && !hidden_table_schema.is_heap_table()) {
|
|
uint64_t new_cst_id = OB_INVALID_ID;
|
|
ObString pk_name;
|
|
ObNameGeneratedType pk_name_type = GENERATED_TYPE_USER;
|
|
ObConstraint cst;
|
|
cst.set_constraint_type(CONSTRAINT_TYPE_PRIMARY_KEY);
|
|
if (OB_FAIL(schema_service->fetch_new_constraint_id(hidden_table_schema.get_tenant_id(),
|
|
new_cst_id))) {
|
|
LOG_WARN("failed to fetch new constraint id", K(ret));
|
|
} else if (FALSE_IT(cst.set_constraint_id(new_cst_id))) {
|
|
} else if (!orig_table_schema.is_heap_table()) {
|
|
const ObConstraint *pk_cst = orig_table_schema.get_pk_constraint();
|
|
if (OB_NOT_NULL(pk_cst)) {
|
|
pk_name.assign_ptr(
|
|
pk_cst->get_constraint_name_str().ptr(), pk_cst->get_constraint_name_str().length());
|
|
pk_name_type = pk_cst->get_name_generated_type();
|
|
}
|
|
} else {
|
|
if (index_name.length() != 0) {
|
|
if (OB_FAIL(ob_write_string(allocator, index_name, pk_name))) {
|
|
LOG_WARN("fail to write string", K(ret));
|
|
}
|
|
} else if (OB_FAIL(ObTableSchema::create_cons_name_automatically(
|
|
pk_name, orig_table_schema.get_table_name_str(),
|
|
allocator, CONSTRAINT_TYPE_PRIMARY_KEY, is_oracle_mode))) {
|
|
LOG_WARN("create cons name automatically failed", K(ret));
|
|
} else {
|
|
pk_name_type = GENERATED_TYPE_SYSTEM;
|
|
}
|
|
}
|
|
OZ(cst.set_constraint_name(pk_name));
|
|
OX(cst.set_name_generated_type(pk_name_type));
|
|
OZ(hidden_table_schema.add_constraint(cst));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::clear_ctas_hidden_table_session_id_(share::schema::ObTableSchema &hidden_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
if (!hidden_table_schema.is_ctas_tmp_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("hidden table is not a CTAS tmp table", K(ret), K(hidden_table_schema));
|
|
} else {
|
|
hidden_table_schema.set_session_id(0);
|
|
hidden_table_schema.set_create_host("");
|
|
LOG_INFO("clear session_id of hidden table copied from CTAS table", K(hidden_table_schema));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::swap_ctas_hidden_table_session_id_(
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
const share::schema::ObTableSchema &hidden_table_schema,
|
|
share::schema::ObTableSchema &new_orig_table_schema,
|
|
share::schema::ObTableSchema &new_hidden_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
if (!orig_table_schema.is_ctas_tmp_table() || hidden_table_schema.is_ctas_tmp_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("origin table or hidden table state is not valid", K(ret), K(orig_table_schema),
|
|
K(hidden_table_schema));
|
|
} else {
|
|
new_orig_table_schema.set_session_id(hidden_table_schema.get_session_id());
|
|
new_hidden_table_schema.set_session_id(orig_table_schema.get_session_id());
|
|
new_orig_table_schema.set_create_host(hidden_table_schema.get_create_host());
|
|
new_hidden_table_schema.set_create_host(orig_table_schema.get_create_host());
|
|
// since the session id is cleared when we create the hidden table, the temp table info
|
|
// won't be added at the creation time, so we should add it here
|
|
new_hidden_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, new_hidden_table_schema))) {
|
|
LOG_WARN("failed to insert temp table info", K(ret), K(new_hidden_table_schema));
|
|
}
|
|
LOG_INFO("restore session_id of hidden table copied from CTAS table",
|
|
K(new_hidden_table_schema), K(new_orig_table_schema));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_priv(const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArray<ObObjPriv> orig_obj_privs_ora;
|
|
if (OB_FAIL(get_obj_privs_ora(orig_table_schema.get_tenant_id(),
|
|
orig_table_schema.get_table_id(),
|
|
static_cast<uint64_t>(ObObjectType::TABLE),
|
|
schema_guard,
|
|
orig_obj_privs_ora))) {
|
|
LOG_WARN("fail to get obj privs ora", KR(ret), K(orig_table_schema.get_tenant_id()),
|
|
K(orig_table_schema.get_table_id()));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_obj_privs_ora.count(); i++) {
|
|
orig_obj_privs_ora.at(i).set_tenant_id(hidden_table_schema.get_tenant_id());
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(restore_obj_privs_for_table(hidden_table_schema.get_table_id(),
|
|
hidden_table_schema.get_link_database_name(),
|
|
hidden_table_schema.get_table_name_str(),
|
|
ddl_operator,
|
|
trans,
|
|
orig_obj_privs_ora))) {
|
|
// need to rebuild permissions for the newly created table
|
|
LOG_WARN("restore_obj_privs_for_table failed", KR(ret),
|
|
K(hidden_table_schema.get_table_id()),
|
|
K(hidden_table_schema.get_table_name_str()));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_rls_objects(
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObRlsPolicySchema new_rls_policy;
|
|
ObRlsGroupSchema new_rls_group;
|
|
ObRlsContextSchema new_rls_context;
|
|
ObString empty_str;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_table_schema.get_rls_context_ids().count(); ++i) {
|
|
const ObRlsContextSchema *context_schema = NULL;
|
|
uint64_t context_id = orig_table_schema.get_rls_context_ids().at(i);
|
|
OZ (schema_guard.get_rls_context_schema_by_id(tenant_id, context_id, context_schema));
|
|
CK (OB_NOT_NULL(context_schema));
|
|
OZ (new_rls_context.assign(*context_schema));
|
|
OX (new_rls_context.set_tenant_id(hidden_table_schema.get_tenant_id()));
|
|
OX (new_rls_context.set_table_id(hidden_table_schema.get_table_id()));
|
|
OZ (ddl_operator.create_rls_context(new_rls_context, trans, empty_str, false, NULL));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_table_schema.get_rls_group_ids().count(); ++i) {
|
|
const ObRlsGroupSchema *group_schema = NULL;
|
|
uint64_t group_id = orig_table_schema.get_rls_group_ids().at(i);
|
|
OZ (schema_guard.get_rls_group_schema_by_id(tenant_id, group_id, group_schema));
|
|
CK (OB_NOT_NULL(group_schema));
|
|
OZ (new_rls_group.assign(*group_schema));
|
|
OX (new_rls_group.set_tenant_id(hidden_table_schema.get_tenant_id()));
|
|
OX (new_rls_group.set_table_id(hidden_table_schema.get_table_id()));
|
|
OZ (ddl_operator.create_rls_group(new_rls_group, trans, empty_str, false, NULL));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_table_schema.get_rls_policy_ids().count(); ++i) {
|
|
const ObRlsPolicySchema *policy_schema = NULL;
|
|
uint64_t policy_id = orig_table_schema.get_rls_policy_ids().at(i);
|
|
OZ (schema_guard.get_rls_policy_schema_by_id(tenant_id, policy_id, policy_schema));
|
|
CK (OB_NOT_NULL(policy_schema));
|
|
OZ (new_rls_policy.rebuild_with_table_schema(*policy_schema, hidden_table_schema));
|
|
if (OB_FAIL(ret)) {
|
|
} else if (policy_schema->is_column_level_policy()
|
|
&& !new_rls_policy.is_column_level_policy()) {
|
|
// column level policy will be dropped after drop column
|
|
} else {
|
|
OZ (ddl_operator.create_rls_policy(new_rls_policy, trans, empty_str, false, NULL));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_user_hidden_table(const ObTableSchema &orig_table_schema,
|
|
ObTableSchema &hidden_table_schema,
|
|
const obrpc::ObSequenceDDLArg *sequence_ddl_arg,
|
|
const bool bind_tablets,
|
|
share::schema::ObSchemaGetterGuard &src_tenant_schema_guard,
|
|
share::schema::ObSchemaGetterGuard &dst_tenant_schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObIAllocator &allocator,
|
|
const uint64_t tenant_data_version,
|
|
const ObString &index_name/*default ""*/,
|
|
const bool ignore_cs_replica/*= false*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = hidden_table_schema.get_tenant_id();
|
|
ObArray<ObTableSchema> aux_table_schemas;
|
|
ObSEArray<const ObTableSchema*, 3> schemas; // 1(hidden_table) + 2(aux_lob_table)
|
|
ObSEArray<bool, 3> need_create_empty_majors;
|
|
bool need_sync_schema_version = false;
|
|
SCN frozen_scn = SCN::min_scn();
|
|
bool is_add_identity_column = false;
|
|
const bool in_offline_ddl_white_list = orig_table_schema.get_tenant_id() != hidden_table_schema.get_tenant_id() ?
|
|
true : orig_table_schema.check_can_do_ddl();
|
|
hidden_table_schema.set_in_offline_ddl_white_list(in_offline_ddl_white_list); // allow offline ddl execute if there's no offline ddl doing
|
|
bool have_spatial_generated_column = false;
|
|
if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root service is null", KR(ret));
|
|
} else if (OB_UNLIKELY(tenant_data_version <= 0)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(tenant_data_version));
|
|
} else if (OB_FAIL(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("failed to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_is_add_identity_column(orig_table_schema, hidden_table_schema, is_add_identity_column))) {
|
|
LOG_WARN("failed to check is add identity column", K(ret));
|
|
} else if (OB_FAIL(prepare_hidden_table_schema(orig_table_schema,
|
|
allocator,
|
|
hidden_table_schema,
|
|
index_name))) {
|
|
LOG_WARN("failed to prepare hidden table schema", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.create_sequence_in_create_table(hidden_table_schema,
|
|
trans,
|
|
dst_tenant_schema_guard,
|
|
is_add_identity_column ? sequence_ddl_arg : nullptr))) {
|
|
// alter table t1 modify c2 int generated always as identity;
|
|
// alter table t1 add c2 int generated by default on null as identity;
|
|
// alter table t1 add column c6 datetime(6) default '20180224' after c2;
|
|
// alter table t1 add column c1_5 int generated always as identity after c1;
|
|
LOG_WARN("failed to create sequence in create table", K(ret));
|
|
} else if (OB_FAIL(build_aux_lob_table_schema_if_need(hidden_table_schema, aux_table_schemas))) {
|
|
LOG_WARN("failed to build_aux_lob_table_schema_if_need", K(ret), K(hidden_table_schema));
|
|
} else if (OB_FAIL(rebuild_hidden_table_priv(orig_table_schema,
|
|
hidden_table_schema,
|
|
src_tenant_schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("failed to rebuild hidden table priv", K(ret));
|
|
} else if (OB_FAIL(rebuild_hidden_table_rls_objects(orig_table_schema,
|
|
hidden_table_schema,
|
|
src_tenant_schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("failed to rebuild hidden table rls objects", K(ret));
|
|
// to prevent other action to effect table partition info in tablegroup
|
|
} else if (OB_FAIL(check_alter_partition_with_tablegroup(&orig_table_schema, hidden_table_schema, dst_tenant_schema_guard))) {
|
|
LOG_WARN("fail to check alter partition with tablegroup", KR(ret));
|
|
} else if (OB_FAIL(schemas.push_back(&hidden_table_schema))) {
|
|
LOG_WARN("push back schema failed", K(ret));
|
|
} else if (OB_FAIL(need_create_empty_majors.push_back(false))) {
|
|
LOG_WARN("push back flag failed" , K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < aux_table_schemas.count(); i++) {
|
|
ObTableSchema &table_schema = aux_table_schemas.at(i);
|
|
// allow offline ddl execute if there's no offline ddl doing
|
|
table_schema.set_in_offline_ddl_white_list(in_offline_ddl_white_list);
|
|
if (OB_FAIL(schemas.push_back(&table_schema))) {
|
|
LOG_WARN("fail to push back aux table schema" , K(ret));
|
|
} else if (table_schema.is_aux_lob_piece_table()) {
|
|
if (OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
} else if (table_schema.is_aux_lob_meta_table()) {
|
|
if (OB_FAIL(need_create_empty_majors.push_back(DATA_VERSION_4_3_0_0 > tenant_data_version))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected table", K(ret), K(table_schema));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObTableCreator table_creator(
|
|
tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(
|
|
tenant_id,
|
|
dst_tenant_schema_guard,
|
|
sql_proxy_);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
const ObTablegroupSchema *tablegroup_schema = NULL; // keep NULL if no tablegroup
|
|
if (OB_FAIL(table_creator.init(false/*need_tablet_cnt_check*/))) {
|
|
LOG_WARN("fail to init table creator", KR(ret));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
} else if (OB_INVALID_ID != hidden_table_schema.get_tablegroup_id()) {
|
|
if (OB_FAIL(dst_tenant_schema_guard.get_tablegroup_schema(
|
|
hidden_table_schema.get_tenant_id(),
|
|
hidden_table_schema.get_tablegroup_id(),
|
|
tablegroup_schema))) {
|
|
LOG_WARN("get tablegroup_schema failed", KR(ret), K(hidden_table_schema));
|
|
} else if (OB_ISNULL(tablegroup_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tablegroup_schema is null", KR(ret), K(hidden_table_schema));
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < schemas.count(); i++) {
|
|
share::schema::ObTableSchema *table_schema = const_cast<ObTableSchema*>(schemas.at(i));
|
|
if (OB_FAIL(ddl_operator.create_table(*table_schema, trans, NULL,
|
|
i == schemas.count() - 1/*need_sync_schema_version, to update data table schema version*/))) {
|
|
LOG_WARN("failed to create table schema", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_temp_table_info(trans, *table_schema))) {
|
|
LOG_WARN("failed to insert temp table info", K(ret), KPC(table_schema));
|
|
}
|
|
}
|
|
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
if (FAILEDx(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("get last schema version failed", K(ret), K(last_schema_version));
|
|
} else if (OB_UNLIKELY(last_schema_version < schemas.at(schemas.count() - 1)->get_schema_version())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected last schema version", K(ret), K(last_schema_version),
|
|
"table schema version", schemas.at(schemas.count() - 1)->get_schema_version());
|
|
} else {
|
|
// update schema version after sync version by creating lob table.
|
|
hidden_table_schema.set_schema_version(last_schema_version);
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < schemas.count(); i++) {
|
|
share::schema::ObTableSchema *table_schema = const_cast<ObTableSchema*>(schemas.at(i));
|
|
if (OB_FAIL(ddl_operator.insert_ori_schema_version(trans, tenant_id,
|
|
table_schema->get_table_id(), last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!", K(ret), KPC(table_schema), K(last_schema_version));
|
|
}
|
|
}
|
|
// when need bind tablets, schemas array only store aux tables, need remove data schema
|
|
if (OB_SUCC(ret) && bind_tablets) {
|
|
if (OB_FAIL(schemas.remove(0)) || OB_FAIL(need_create_empty_majors.remove(0))) {
|
|
LOG_WARN("failed to remove data schema.", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && hidden_table_schema.has_tablet()) {
|
|
if (bind_tablets && OB_FAIL(new_table_tablet_allocator.prepare_like(orig_table_schema))) {
|
|
LOG_WARN("fail to prepare like", KR(ret), K(orig_table_schema));
|
|
} else if (!bind_tablets && OB_FAIL(new_table_tablet_allocator.prepare(trans, hidden_table_schema, tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare", KR(ret), K(hidden_table_schema));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (!bind_tablets && OB_FAIL(table_creator.add_create_tablets_of_tables_arg(
|
|
schemas,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors/*need_create_empty_major_sstable*/,
|
|
ignore_cs_replica))) {
|
|
LOG_WARN("create table tablets failed", K(ret), K(hidden_table_schema));
|
|
} else if (bind_tablets && OB_FAIL(table_creator.add_create_bind_tablets_of_hidden_table_arg(
|
|
orig_table_schema,
|
|
hidden_table_schema,
|
|
ls_id_array,
|
|
tenant_data_version))) {
|
|
LOG_WARN("failed to add arg", K(ret), K(hidden_table_schema));
|
|
} else if (bind_tablets && schemas.count() > 0 &&
|
|
OB_FAIL(table_creator.add_create_tablets_of_local_aux_tables_arg(
|
|
schemas,
|
|
&hidden_table_schema,
|
|
ls_id_array,
|
|
tenant_data_version,
|
|
need_create_empty_majors/*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("failed to add arg", K(ret), K(aux_table_schemas), K(hidden_table_schema));
|
|
} else if (OB_FAIL(table_creator.execute())) {
|
|
LOG_WARN("fail to execute create tablet", KR(ret));
|
|
}
|
|
}
|
|
// finishing is always invoked for new table tablet allocator
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) {
|
|
LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && orig_table_schema.get_tenant_id() == hidden_table_schema.get_tenant_id()) {
|
|
// For some ddl like restore table, the tenant ids between source table and dest table are different,
|
|
// and we do not change any attribute of the source table.
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
ObTableSchema table_schema;
|
|
if (OB_FAIL(table_schema.assign(orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
// here need to change the state of the orig table to TABLE_STATE_OFFLINE_DDL, and at
|
|
// the same time update association table id
|
|
table_schema.set_in_offline_ddl_white_list(orig_table_schema.check_can_do_ddl());
|
|
table_schema.set_association_table_id(hidden_table_schema.get_table_id());
|
|
table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL);
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(table_schema, trans, operation_type))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::build_aux_lob_table_schema_if_need(ObTableSchema &data_table_schema,
|
|
ObIArray<ObTableSchema> &table_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObLobMetaBuilder lob_meta_builder(*this);
|
|
ObLobPieceBuilder lob_piece_builder(*this);
|
|
bool lob_col_found = false;
|
|
const uint64_t new_table_id = OB_INVALID_ID;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < data_table_schema.get_column_count() && !lob_col_found; ++i) {
|
|
if (is_lob_storage(data_table_schema.get_column_schema_by_idx(i)->get_data_type())) {
|
|
HEAP_VARS_2((ObTableSchema, lob_meta_schema), (ObTableSchema, lob_piece_schema)) {
|
|
if (OB_FAIL(lob_meta_builder.generate_aux_lob_meta_schema(
|
|
schema_service_->get_schema_service(), data_table_schema, new_table_id, lob_meta_schema, true))) {
|
|
LOG_WARN("generate_schema for lob meta table failed", K(data_table_schema), K(ret));
|
|
} else if (OB_FAIL(table_schemas.push_back(lob_meta_schema))) {
|
|
LOG_WARN("push_back lob meta table failed", K(ret));
|
|
} else if (OB_FAIL(lob_piece_builder.generate_aux_lob_piece_schema(
|
|
schema_service_->get_schema_service(), data_table_schema, new_table_id, lob_piece_schema, true))) {
|
|
LOG_WARN("generate_schema for lob data table failed", K(data_table_schema), K(ret));
|
|
} else if (OB_FAIL(table_schemas.push_back(lob_piece_schema))) {
|
|
LOG_WARN("push_back lob data table failed", K(ret));
|
|
} else {
|
|
data_table_schema.set_aux_lob_meta_tid(lob_meta_schema.get_table_id());
|
|
data_table_schema.set_aux_lob_piece_tid(lob_piece_schema.get_table_id());
|
|
}
|
|
}
|
|
lob_col_found = true;
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_new_index_table_name(
|
|
const ObString &orig_index_table_name,
|
|
const uint64_t orig_table_id,
|
|
const uint64_t new_table_id,
|
|
ObIAllocator &allocator,
|
|
ObString &new_index_table_name)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObString index_name;
|
|
if (OB_FAIL(ObTableSchema::get_index_name(
|
|
allocator,
|
|
orig_table_id,
|
|
orig_index_table_name,
|
|
index_name))) {
|
|
LOG_WARN("error get index table name failed",
|
|
K(orig_table_id), K(orig_index_table_name), K(ret));
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(
|
|
allocator,
|
|
new_table_id,
|
|
index_name,
|
|
new_index_table_name))) {
|
|
LOG_WARN("failed to build new index table name!", K(new_table_id),
|
|
K(new_index_table_name), K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// col_name_map must live beyond index_schema
|
|
int ObDDLService::gen_hidden_index_schema_columns(const ObTableSchema &orig_index_schema,
|
|
const common::ObIArray<int64_t> &drop_cols_id_arr,
|
|
const ObColumnNameMap &col_name_map,
|
|
ObTableSchema &new_table_schema,
|
|
ObTableSchema &index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
SMART_VAR(ObCreateIndexArg, create_index_arg) {
|
|
ObTableSchema::const_column_iterator tmp_begin = index_schema.column_begin();
|
|
ObTableSchema::const_column_iterator tmp_end = index_schema.column_end();
|
|
ObArray<std::pair<int64_t, obrpc::ObColumnSortItem>> index_columns;
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) {
|
|
ObColumnSchemaV2 *col = (*tmp_begin);
|
|
ObString col_name;
|
|
bool is_dropped = false;
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is NULL", K(ret));
|
|
} else if (col->is_shadow_column()) {
|
|
// shadown column doesn't exist in data table
|
|
col_name = col->get_column_name_str();
|
|
} else if (OB_FAIL(col_name_map.get(col->get_column_name_str(), col_name))) {
|
|
if (OB_ENTRY_NOT_EXIST == ret) {
|
|
ret = OB_SUCCESS;
|
|
// test whether the column is dropped;
|
|
FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && !is_dropped) {
|
|
if (*dropped_col == col->get_column_id()) {
|
|
is_dropped = true;
|
|
}
|
|
}
|
|
if (!is_dropped) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, column is null in new table", K(ret), K(col->get_column_name_str()));
|
|
}
|
|
} else {
|
|
LOG_WARN("failed to get new name", K(ret), K(col->get_column_name()));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_dropped) {
|
|
// drop in index
|
|
} else if (col->get_index_position() > 0) {
|
|
obrpc::ObColumnSortItem sort_item;
|
|
sort_item.column_name_ = col_name;
|
|
sort_item.order_type_ = col->get_order_in_rowkey();
|
|
if (col->is_prefix_column()) {
|
|
sort_item.prefix_len_ = col->get_data_length();
|
|
}
|
|
if (OB_FAIL(index_columns.push_back(std::make_pair(col->get_index_position(), sort_item)))) {
|
|
LOG_WARN("fail to add index columns", K(ret));
|
|
}
|
|
} else if (col->is_user_specified_storing_column()) {
|
|
if (0 < col->get_index_position() || col->is_hidden()
|
|
|| col->is_hidden_pk_column_id(col->get_column_id())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, these columns can not be user specified storing column", K(ret), KPC(col));
|
|
} else if (OB_FAIL(create_index_arg.store_columns_.push_back(col_name))) {
|
|
LOG_WARN("fail to push back storing column", K(ret), KPC(col));
|
|
} else { /* do nothing. */ }
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
lib::ob_sort(index_columns.begin(), index_columns.end(),
|
|
[](const std::pair<int64_t, obrpc::ObColumnSortItem> &lhs, const std::pair<int64_t, obrpc::ObColumnSortItem> &rhs) -> bool {
|
|
return lhs.first < rhs.first;
|
|
});
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_columns.count(); i++) {
|
|
if (OB_FAIL(create_index_arg.index_columns_.push_back(index_columns[i].second))) {
|
|
LOG_WARN("failed to push back index column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
index_schema.reset_column_info();
|
|
create_index_arg.index_type_ = index_schema.get_index_type();
|
|
if (INDEX_TYPE_NORMAL_LOCAL == create_index_arg.index_type_
|
|
|| INDEX_TYPE_UNIQUE_LOCAL == create_index_arg.index_type_
|
|
|| INDEX_TYPE_UNIQUE_MULTIVALUE_LOCAL == create_index_arg.index_type_
|
|
|| INDEX_TYPE_DOMAIN_CTXCAT_DEPRECATED == create_index_arg.index_type_) {
|
|
if (OB_FAIL(sql::ObResolverUtils::check_unique_index_cover_partition_column(
|
|
new_table_schema, create_index_arg))) {
|
|
LOG_WARN("fail to check unique key cover partition column", K(ret));
|
|
}
|
|
}
|
|
}
|
|
OZ(ObIndexBuilderUtil::set_index_table_columns(create_index_arg, new_table_schema, index_schema));
|
|
tmp_begin = orig_index_schema.column_begin();
|
|
tmp_end = orig_index_schema.column_end();
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end; tmp_begin++) {
|
|
ObColumnSchemaV2 *orig_col = (*tmp_begin);
|
|
ObString col_name;
|
|
if (OB_ISNULL(orig_col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is NULL", K(ret));
|
|
} else if (orig_col->is_tbl_part_key_column()) {
|
|
if (OB_UNLIKELY(orig_col->is_shadow_column())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("shadow column as part key", K(ret), KPC(orig_col));
|
|
} else if (OB_FAIL(col_name_map.get(orig_col->get_column_name_str(), col_name))) {
|
|
LOG_WARN("invalid column name", K(ret), KPC(orig_col));
|
|
} else {
|
|
ObColumnSchemaV2 *col = index_schema.get_column_schema(col_name);
|
|
if (OB_ISNULL(col)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is NULL", K(ret));
|
|
} else {
|
|
col->set_tbl_part_key_pos(orig_col->get_tbl_part_key_pos());
|
|
col->set_part_key_pos(orig_col->get_part_key_pos());
|
|
col->set_subpart_key_pos(orig_col->get_subpart_key_pos());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_new_index_schema(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObSchemaGetterGuard &dest_schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
ObSArray<ObTableSchema> &new_table_schemas,
|
|
ObSArray<uint64_t> &index_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObTableSchema new_table_schema;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (!alter_table_arg.is_alter_columns_) {
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", K(ret), KP(schema_service));
|
|
} else if (OB_FAIL(new_table_schema.assign(hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
ObIndexBuilder index_builder(*this);
|
|
HEAP_VAR(AddIndexNameHashSet, add_index_name_set) {
|
|
HEAP_VAR(DropIndexNameHashSet, drop_index_name_set) {
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
int64_t index_count = new_table_schema.get_index_tid_count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_arg_list.size(); ++i) {
|
|
ObIndexArg *index_arg = const_cast<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 (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()
|
|
&& !new_table_schema.is_auto_partitioned_table()
|
|
&& !create_index_arg->index_schema_.is_partitioned_table()) {
|
|
if (INDEX_TYPE_NORMAL_GLOBAL == create_index_arg->index_type_) {
|
|
create_index_arg->index_type_ = INDEX_TYPE_NORMAL_GLOBAL_LOCAL_STORAGE;
|
|
} else if (INDEX_TYPE_UNIQUE_GLOBAL == create_index_arg->index_type_) {
|
|
create_index_arg->index_type_ = INDEX_TYPE_UNIQUE_GLOBAL_LOCAL_STORAGE;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (create_index_arg->index_type_ == INDEX_TYPE_PRIMARY) {
|
|
if (new_table_schema.get_rowkey_column_num() > 0) {
|
|
if (new_table_schema.is_heap_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to add primary key!", K(ret));
|
|
} else {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret));
|
|
}
|
|
}
|
|
continue;
|
|
}
|
|
}
|
|
if (create_index_arg->index_name_.empty()) {
|
|
// check whether the index table exists based on the origin table rather than the hidden one.
|
|
if (OB_FAIL(generate_index_name(*create_index_arg,
|
|
orig_table_schema,
|
|
add_index_name_set,
|
|
drop_index_name_set,
|
|
schema_guard,
|
|
alter_table_arg.allocator_))) {
|
|
LOG_WARN("generate index name failed", K(ret));
|
|
}
|
|
} else {
|
|
ObString index_name = create_index_arg->index_name_;
|
|
ObString index_table_name;
|
|
bool is_exist = false;
|
|
// check index exist should use orig_table_schema, otherwise cannot see old index
|
|
if (OB_FAIL(check_index_table_exist(orig_table_schema.get_tenant_id(),
|
|
orig_table_schema.get_database_id(),
|
|
orig_table_schema.get_table_id(),
|
|
index_name,
|
|
schema_guard,
|
|
is_exist))) {
|
|
LOG_WARN("failed to check index table", K(ret));
|
|
} else {
|
|
ObIndexNameHashWrapper index_key(index_name);
|
|
if (!is_exist) {
|
|
if (OB_HASH_EXIST == add_index_name_set.exist_refactored(index_key)) {
|
|
//alter table t1 add index c1, add index c1
|
|
is_exist = true;
|
|
}
|
|
} else {
|
|
if (OB_HASH_EXIST == drop_index_name_set.exist_refactored(index_key)) {
|
|
//alter table t1 drop index c1, add index c1
|
|
is_exist = false;
|
|
}
|
|
}
|
|
if (is_exist) {
|
|
ret = OB_ERR_KEY_NAME_DUPLICATE;
|
|
LOG_USER_ERROR(OB_ERR_KEY_NAME_DUPLICATE, index_name.length(), index_name.ptr());
|
|
LOG_WARN("duplicate index name", K(index_name), K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
HEAP_VAR(ObTableSchema, index_schema) {
|
|
ObArray<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) {
|
|
if (OB_FAIL(index_schema.assign(create_index_arg->index_schema_))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (FALSE_IT(index_schema.set_tenant_id(new_table_schema.get_tenant_id()))) {
|
|
}
|
|
}
|
|
bool global_index_without_column_info = create_index_arg->index_schema_.is_partitioned_table() ? false : true;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ObIndexBuilderUtil::adjust_expr_index_args(
|
|
*create_index_arg, new_table_schema, alter_table_arg.allocator_, gen_columns))) {
|
|
LOG_WARN("adjust fulltext args failed", K(ret));
|
|
} else if (OB_FAIL(index_builder.generate_schema(*create_index_arg,
|
|
new_table_schema,
|
|
global_index_without_column_info,
|
|
true, /*generate_id*/
|
|
index_schema))) {
|
|
LOG_WARN("failed to generate index schema!", K(ret));
|
|
} else {
|
|
bool is_exist = false;
|
|
ObString final_index_name;
|
|
if (OB_FAIL(build_hidden_table_index_and_mlog_schema_(
|
|
hidden_table_schema, index_schema))) {
|
|
LOG_WARN("failed to build hidden table index and mlog schema",
|
|
KR(ret), K(hidden_table_schema), K(index_schema));
|
|
} else if (OB_FAIL(index_schema.get_index_name(final_index_name))) {
|
|
LOG_WARN("get index name failed", K(ret));
|
|
} else if (OB_FAIL(check_hidden_index_exist(index_schema.get_tenant_id(),
|
|
index_schema.get_database_id(),
|
|
orig_table_schema,
|
|
final_index_name,
|
|
dest_schema_guard,
|
|
is_exist))) {
|
|
LOG_WARN("failed to check hidden index exist", K(ret));
|
|
} else if (is_exist) {
|
|
ret = OB_EAGAIN;
|
|
LOG_WARN("index name conflict, need retry", K(index_schema.get_table_id()), K(index_schema.get_table_name_str()));
|
|
} else if (OB_FAIL(new_table_schemas.push_back(index_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
} else if (OB_FAIL(index_ids.push_back(index_schema.get_table_id()))) {
|
|
LOG_WARN("failed to add new index id!", K(ret));
|
|
} else {
|
|
// add new function index should add additional SYS_NC column into data table.
|
|
new_table_schema.set_in_offline_ddl_white_list(true);
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < gen_columns.count(); ++i) {
|
|
ObColumnSchemaV2 *new_column_schema = gen_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)) {
|
|
ObIndexNameHashWrapper index_key(create_index_arg->index_name_);
|
|
if (OB_FAIL(create_index_arg->index_schema_.assign(index_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(add_index_name_set.set_refactored(index_key))) {
|
|
LOG_WARN("set index name to hash set failed",
|
|
K(create_index_arg->index_name_), K(ret));
|
|
|
|
} else {
|
|
++index_count;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else if (index_arg->index_action_type_ == ObIndexArg::DROP_FOREIGN_KEY
|
|
|| index_arg->index_action_type_ == ObIndexArg::RENAME_INDEX
|
|
|| index_arg->index_action_type_ == ObIndexArg::ADD_PRIMARY_KEY) {
|
|
// ignore drop foreign key and rename index, it should be handled outside
|
|
} else {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement");
|
|
}
|
|
}
|
|
} //end for
|
|
} // end add_index_name_set
|
|
} // end drop_index_name_set
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::build_hidden_table_index_and_mlog_schema_(
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObTableSchema &index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t new_idx_tid = OB_INVALID_ID;
|
|
ObSchemaService *schema_service = nullptr;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", KR(ret), KP(schema_service));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(hidden_table_schema.get_tenant_id(),
|
|
new_idx_tid))) {
|
|
LOG_WARN("failed to fetch_new_table_id", KR(ret));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(index_schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(index_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(index_schema))) {
|
|
LOG_WARN("fail to generate tablet id for hidden table", KR(ret), K(index_schema));
|
|
} else {
|
|
index_schema.set_table_id(new_idx_tid);
|
|
index_schema.set_data_table_id(hidden_table_schema.get_table_id());
|
|
index_schema.set_tenant_id(hidden_table_schema.get_tenant_id());
|
|
index_schema.set_database_id(hidden_table_schema.get_database_id());
|
|
index_schema.set_index_status(INDEX_STATUS_UNAVAILABLE);
|
|
// set the hidden attributes of the table
|
|
index_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check whether the index table need to be rebuilt.
|
|
// TODO @yiren 2022-05-05 indexes should not be rebuilt if there is no relationship to drop columns.
|
|
int ObDDLService::check_index_table_need_rebuild(const share::schema::ObTableSchema &index_table_schema,
|
|
const common::ObIArray<int64_t> &drop_cols_id_arr,
|
|
const bool is_oracle_mode,
|
|
bool &need_rebuild)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
need_rebuild = true;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (ObIndexStatus::INDEX_STATUS_INDEX_ERROR == index_table_schema.get_index_status()) {
|
|
need_rebuild = false;
|
|
LOG_INFO("skip error index", K(index_table_schema.get_table_id()));
|
|
} else if (is_oracle_mode) {
|
|
// For Oracle mode, index is invalid if its' any rowkey/storing column specified by user is dropped;
|
|
ObSArray<uint64_t> check_columns_id;
|
|
// push back user specified index column.
|
|
if (OB_FAIL(index_table_schema.get_index_info().get_column_ids(check_columns_id))) {
|
|
LOG_WARN("fail to get column ids that need be checked", K(ret));
|
|
} else {
|
|
// push back user specified storing column for index.(unique and non-unique)
|
|
for (ObTableSchema::const_column_iterator iter = index_table_schema.column_begin();
|
|
OB_SUCC(ret) && iter != index_table_schema.column_end(); iter++) {
|
|
const ObColumnSchemaV2 *col_schema = *iter;
|
|
if (OB_ISNULL(col_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, column is null", K(ret));
|
|
} else if (col_schema->is_user_specified_storing_column()
|
|
&& OB_FAIL(check_columns_id.push_back(col_schema->get_column_id()))) {
|
|
LOG_WARN("fail to push back store column id", K(ret), KPC(col_schema));
|
|
} else {/* do nothing. */}
|
|
}
|
|
FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && need_rebuild) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && need_rebuild && i < check_columns_id.count(); i++) {
|
|
if (*dropped_col == check_columns_id.at(i)) {
|
|
need_rebuild = false;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else {
|
|
// For mysql mode, index is invalid if its' rowkey columns specified by user are all dropped.
|
|
int64_t drop_check_column_cnt = 0;
|
|
ObSArray<uint64_t> check_columns_id;
|
|
if (OB_FAIL(index_table_schema.get_index_info().get_column_ids(check_columns_id))) {
|
|
LOG_WARN("fail to get column ids that need be checked", K(ret));
|
|
} else {
|
|
FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && need_rebuild) {
|
|
bool is_drop_check_column = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && !is_drop_check_column && i < check_columns_id.count(); i++) {
|
|
if (*dropped_col == check_columns_id.at(i)) {
|
|
is_drop_check_column = true;
|
|
drop_check_column_cnt++;
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && drop_check_column_cnt == check_columns_id.count()) {
|
|
need_rebuild = false;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::reconstruct_index_schema(obrpc::ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObSchemaGetterGuard &dest_schema_guard,
|
|
const common::ObIArray<int64_t> &drop_cols_id_arr,
|
|
const ObColumnNameMap &col_name_map,
|
|
const common::ObTimeZoneInfo &tz_info,
|
|
ObIAllocator &allocator,
|
|
ObSArray<ObTableSchema> &new_table_schemas,
|
|
ObSArray<uint64_t> &index_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t src_tenant_id = orig_table_schema.get_tenant_id();
|
|
const uint64_t dst_tenant_id = hidden_table_schema.get_tenant_id();
|
|
bool is_oracle_mode = false;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", K(ret), KP(schema_service));
|
|
} else {
|
|
ObTableSchema new_table_schema;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.assign(hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to check if oralce compat mode", K(ret));
|
|
}
|
|
lib::Worker::CompatMode compat_mode = (is_oracle_mode ?
|
|
lib::Worker::CompatMode::ORACLE : lib::Worker::CompatMode::MYSQL);
|
|
lib::CompatModeGuard tmpCompatModeGuard(compat_mode);
|
|
/**
|
|
* For recover restore table ddl that src_tenant_id does not equal to the dest,
|
|
* any index rebuild error can be ignored. And to avoid building a invalid index which has duplicated index name,
|
|
* the table state flag of recoverd index is set to normal, which avoids dropping index when takes effect.
|
|
*
|
|
* As for the check_table_type, offline ddl like modify pk operation only needs to check the hidden schema to avoid retry
|
|
* to rebuild multiple times. And recover restore table ddl should check all non-hidden schema to avoid duplicated name
|
|
* created in other objects.
|
|
*/
|
|
const bool is_recover_restore_table = src_tenant_id != dst_tenant_id;
|
|
const ObTableStateFlag &target_flag = !is_recover_restore_table ?
|
|
hidden_table_schema.get_table_state_flag() :
|
|
ObTableStateFlag::TABLE_STATE_NORMAL;
|
|
const ObSchemaGetterGuard::CheckTableType &check_table_type = !is_recover_restore_table ?
|
|
ObSchemaGetterGuard::USER_HIDDEN_TABLE_TYPE :
|
|
ObSchemaGetterGuard::ALL_NON_HIDDEN_TYPES;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
bool need_rebuild = true;
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
src_tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(src_tenant_id), "table id", simple_index_infos.at(i).table_id_, K(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (OB_FAIL(check_index_table_need_rebuild(*index_table_schema,
|
|
drop_cols_id_arr,
|
|
is_oracle_mode,
|
|
need_rebuild))) {
|
|
LOG_WARN("fail to check index whether to need rebuild", K(ret), KPC(index_table_schema));
|
|
} else if (need_rebuild) {
|
|
ObString new_index_table_name;
|
|
HEAP_VAR(ObTableSchema, new_index_schema) {
|
|
int64_t rebuild_index_arg_schema_id = OB_INVALID_ID;
|
|
// No need to convert hidden table column id since indexes are rebuilt based on column names.
|
|
for (int64_t i = 0; OB_SUCC(ret) && rebuild_index_arg_schema_id == OB_INVALID_ID
|
|
&& i < alter_table_arg.rebuild_index_arg_list_.count(); ++i) {
|
|
const int64_t tmp_index_id = alter_table_arg.rebuild_index_arg_list_.at(i).get_table_id();
|
|
if (tmp_index_id == index_table_schema->get_table_id()) {
|
|
rebuild_index_arg_schema_id = i;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (rebuild_index_arg_schema_id != OB_INVALID_ID) {
|
|
if (OB_FAIL(new_index_schema.assign(alter_table_arg.rebuild_index_arg_list_.at(rebuild_index_arg_schema_id)))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
// set rebuild index arg schema as old index_table_schema
|
|
index_table_schema = &alter_table_arg.rebuild_index_arg_list_.at(rebuild_index_arg_schema_id);
|
|
}
|
|
} else {
|
|
if (OB_FAIL(new_index_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (FALSE_IT(new_index_schema.set_tenant_id(hidden_table_schema.get_tenant_id()))) {
|
|
} else if (OB_FAIL(gen_new_index_table_name(
|
|
index_table_schema->get_table_name_str(),
|
|
orig_table_schema.get_table_id(),
|
|
hidden_table_schema.get_table_id(),
|
|
allocator,
|
|
new_index_table_name))) {
|
|
LOG_WARN("failed to build new index table name!", K(hidden_table_schema.get_table_id()),
|
|
K(new_index_table_name), K(ret));
|
|
} else if (OB_FAIL(new_index_schema.set_table_name(new_index_table_name))) {
|
|
LOG_WARN("set table name failed", K(ret), K(new_index_table_name));
|
|
} else if (is_recover_restore_table && is_oracle_mode
|
|
&& OB_FAIL(check_and_replace_default_index_name_on_demand(is_oracle_mode, allocator, hidden_table_schema,
|
|
alter_table_arg.alter_table_schema_.get_table_name_str(), new_index_schema))) {
|
|
LOG_WARN("replace sys default name failed", K(ret));
|
|
} else if (hidden_table_schema.get_part_level() > 0 && new_index_schema.is_index_local_storage()) {
|
|
if (INDEX_TYPE_NORMAL_GLOBAL_LOCAL_STORAGE == new_index_schema.get_index_type()) {
|
|
new_index_schema.set_index_type(INDEX_TYPE_NORMAL_GLOBAL);
|
|
} else if (INDEX_TYPE_UNIQUE_GLOBAL_LOCAL_STORAGE == new_index_schema.get_index_type()) {
|
|
new_index_schema.set_index_type(INDEX_TYPE_UNIQUE_GLOBAL);
|
|
} else if (INDEX_TYPE_SPATIAL_GLOBAL_LOCAL_STORAGE == new_index_schema.get_index_type()) {
|
|
new_index_schema.set_index_type(INDEX_TYPE_SPATIAL_GLOBAL);
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (new_index_schema.is_global_index_table() && new_index_schema.is_partitioned_table()) {
|
|
ObArray<std::pair<ObString, ObString>> changed_names;
|
|
if (OB_FAIL(col_name_map.get_changed_names(changed_names))) {
|
|
LOG_WARN("failed to get changed names", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < changed_names.count(); i++) {
|
|
if (OB_FAIL(modify_part_func_expr(
|
|
changed_names[i].first,
|
|
changed_names[i].second,
|
|
new_index_schema,
|
|
tz_info,
|
|
allocator))) {
|
|
LOG_WARN("failed to modify part func expr", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(gen_hidden_index_schema_columns(
|
|
*index_table_schema, drop_cols_id_arr, col_name_map, new_table_schema, new_index_schema))) {
|
|
LOG_WARN("failed to gen hidden index schema", K(ret));
|
|
} else if ((hidden_table_schema.get_part_level() > 0 || hidden_table_schema.is_auto_partitioned_table())
|
|
&& new_index_schema.is_index_local_storage()
|
|
&& OB_FAIL(new_index_schema.assign_partition_schema_without_auto_part_attr(hidden_table_schema))) {
|
|
LOG_WARN("fail to assign partition schema", K(ret), K(new_index_schema));
|
|
} else {
|
|
uint64_t new_idx_tid = OB_INVALID_ID;
|
|
ObAlterAutoPartAttrOp alter_auto_part(*this);
|
|
if (new_index_schema.is_global_index_table()
|
|
&& OB_FAIL(alter_auto_part.alter_global_indexes_auto_part_attribute_offline(alter_table_arg, new_index_schema))) {
|
|
LOG_WARN("fail to alter global index auto split perproty", K(ret), K(new_index_schema));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(
|
|
dst_tenant_id, new_idx_tid))) {
|
|
LOG_WARN("failed to fetch_new_table_id", K(ret));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(new_index_schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(new_index_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(new_index_schema))) {
|
|
LOG_WARN("fail to generate tablet id for hidden table", K(ret), K(new_index_schema));
|
|
} else {
|
|
bool is_exist = false;
|
|
new_index_schema.set_max_used_column_id(max(
|
|
new_index_schema.get_max_used_column_id(), hidden_table_schema.get_max_used_column_id()));
|
|
new_index_schema.set_table_id(new_idx_tid);
|
|
new_index_schema.set_data_table_id(hidden_table_schema.get_table_id());
|
|
new_index_schema.set_index_status(INDEX_STATUS_UNAVAILABLE);
|
|
new_index_schema.set_tenant_id(hidden_table_schema.get_tenant_id());
|
|
new_index_schema.set_database_id(hidden_table_schema.get_database_id());
|
|
new_index_schema.set_table_state_flag(target_flag);
|
|
|
|
if (OB_FAIL(ObSchemaUtils::mock_default_cg(new_index_schema.get_tenant_id(), new_index_schema))) {
|
|
LOG_WARN("fail to mock default cg", K(ret), K(new_index_schema));
|
|
} else if (is_recover_restore_table) {
|
|
if (OB_FAIL(new_index_schema.set_encryption_str(hidden_table_schema.get_encryption_str()))) {
|
|
LOG_WARN("set encryption str failed", K(ret), K(hidden_table_schema.get_encryption_str()));
|
|
} else if (OB_FAIL(new_index_schema.set_encrypt_key(hidden_table_schema.get_encrypt_key()))) {
|
|
LOG_WARN("set encrypt key failed", K(ret), K(hidden_table_schema.get_encrypt_key()));
|
|
} else {
|
|
new_index_schema.set_master_key_id(hidden_table_schema.get_master_key_id());
|
|
new_index_schema.set_tablespace_id(hidden_table_schema.get_tablespace_id());
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(adjust_cg_for_offline(new_index_schema))) {
|
|
LOG_WARN("fail to adjust column group for index", K(ret));
|
|
} else if (OB_FAIL(dest_schema_guard.check_table_exist(new_index_schema.get_tenant_id(),
|
|
new_index_schema.get_database_id(),
|
|
new_index_schema.get_table_name_str(),
|
|
true/*is_index*/,
|
|
check_table_type,
|
|
is_exist))) {
|
|
LOG_WARN("failed to check table exist", K(ret));
|
|
} else if (is_exist) {
|
|
LOG_INFO("index already existed, skip", K(new_index_schema.get_table_id()), K(new_index_schema.get_table_name_str()));
|
|
} else if (OB_FAIL(new_table_schemas.push_back(new_index_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
} else if (OB_FAIL(index_ids.push_back(new_idx_tid))) {
|
|
LOG_WARN("failed to add new index id!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} //end for
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_index_and_mlog(
|
|
const uint64_t tenant_id,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObSArray<ObTableSchema> &new_table_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get tenant data version failed", K(ret), K(tenant_data_version));
|
|
} else {
|
|
for (int64_t i = 0; i < new_table_schemas.count() && OB_SUCC(ret); i++) {
|
|
ObTableSchema &tmp_schema = new_table_schemas.at(i);
|
|
tmp_schema.set_in_offline_ddl_white_list(true); // allow rebuild table index of hidden table
|
|
if (OB_FAIL(ddl_operator.create_table(tmp_schema, trans, NULL, true))) {
|
|
LOG_WARN("failed to create table schema, ", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int64_t last_schema_version = OB_INVALID_VERSION;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_table_schemas.count(); i++) {
|
|
const share::schema::ObTableSchema &this_table = new_table_schemas.at(i);
|
|
uint64_t table_id = this_table.get_table_id();
|
|
bool has_tablet = false;
|
|
if (!is_inner_table(table_id)) {
|
|
has_tablet = this_table.has_tablet();
|
|
} else {
|
|
has_tablet = is_system_table(table_id);
|
|
}
|
|
if (!has_tablet) {
|
|
} else if (this_table.is_mlog_table()) {
|
|
if (OB_FAIL(create_mlog_tablet(trans, schema_guard, this_table,
|
|
false/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("failed to create mlog tablet", KR(ret), K(this_table));
|
|
}
|
|
} else if (OB_FAIL(create_index_tablet(this_table, trans, schema_guard, false/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("create table tablets failed", K(ret), K(this_table));
|
|
} else {}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_INVALID_VERSION == last_schema_version
|
|
&& OB_FAIL(get_last_schema_version(last_schema_version))) {
|
|
LOG_WARN("get last schema version failed", K(ret), K(last_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.insert_ori_schema_version(
|
|
trans, tenant_id, table_id, last_schema_version))) {
|
|
LOG_WARN("failed to insert_ori_schema_version!",
|
|
K(ret), K(tenant_id), K(table_id), K(last_schema_version));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_new_mlog_schema(
|
|
const uint64_t tenant_id,
|
|
const share::schema::ObTableSchema &orig_table_schema,
|
|
const share::schema::ObTableSchema &hidden_table_schema,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
share::schema::ObSchemaGetterGuard &dest_schema_guard,
|
|
common::ObMySQLTransaction &trans,
|
|
ObIAllocator &allocator,
|
|
ObSArray<ObTableSchema> &new_table_schemas,
|
|
ObSArray<uint64_t> &index_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_service = nullptr;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", K(ret), KP(schema_service));
|
|
} else {
|
|
// generate mlog schema
|
|
const ObTableSchema *orig_mlog_schema = nullptr;
|
|
ObTableSchema new_mlog_schema;
|
|
const uint64_t orig_mlog_tid = orig_table_schema.get_mlog_tid();
|
|
ObString new_mlog_name;
|
|
bool is_exist = false;
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, orig_mlog_tid, orig_mlog_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), "table id", orig_mlog_tid);
|
|
} else if (OB_ISNULL(orig_mlog_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", KR(ret));
|
|
} else if (OB_FAIL(new_mlog_schema.assign(*orig_mlog_schema))) {
|
|
LOG_WARN("failed to assign schema", KR(ret));
|
|
} else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to check if oracle compat mode", KR(ret));
|
|
} else if (OB_FAIL(ObTableSchema::build_mlog_table_name(allocator,
|
|
hidden_table_schema.get_table_name_str(), new_mlog_name, is_oracle_mode))) {
|
|
LOG_WARN("failed to build mlog table name", KR(ret), K(hidden_table_schema.get_table_name_str()));
|
|
} else if (OB_FAIL(new_mlog_schema.set_table_name(new_mlog_name))) {
|
|
LOG_WARN("failed to set table name", KR(ret), K(new_mlog_name));
|
|
} else if (OB_FAIL(build_hidden_table_index_and_mlog_schema_(
|
|
hidden_table_schema, new_mlog_schema))) {
|
|
LOG_WARN("failed to build hidden table index and mlog schema",
|
|
KR(ret), K(hidden_table_schema), K(new_mlog_schema));
|
|
} else if (OB_FAIL(dest_schema_guard.check_table_exist(new_mlog_schema.get_tenant_id(),
|
|
new_mlog_schema.get_database_id(),
|
|
new_mlog_schema.get_table_name_str(),
|
|
false/*is_index*/,
|
|
ObSchemaGetterGuard::USER_HIDDEN_TABLE_TYPE/*check_type*/,
|
|
is_exist))) {
|
|
LOG_WARN("failed to check table exist", KR(ret), K(new_mlog_schema));
|
|
} else if (is_exist) {
|
|
LOG_INFO("mlog already rebuilt, skip",
|
|
K(new_mlog_schema.get_table_id()), K(new_mlog_schema.get_table_name_str()));
|
|
} else if (OB_FAIL(new_table_schemas.push_back(new_mlog_schema))) {
|
|
LOG_WARN("failed to add table schema!", KR(ret), K(new_mlog_schema));
|
|
} else if (OB_FAIL(index_ids.push_back(new_mlog_schema.get_table_id()))) {
|
|
LOG_WARN("failed to add new index id!", KR(ret), K(new_mlog_schema));
|
|
} else { // insert new mlog_info to all_mlog
|
|
ObMLogInfo mlog_info;
|
|
uint64_t new_mlog_tid = new_mlog_schema.get_table_id();
|
|
if (OB_FAIL(ObMLogInfo::fetch_mlog_info(trans,
|
|
tenant_id, orig_mlog_tid, mlog_info, false/*for_update*/))) {
|
|
LOG_WARN("failed to fetch mlog info", KR(ret), K(tenant_id), K(orig_mlog_tid));
|
|
} else if (OB_FALSE_IT(mlog_info.set_mlog_id(new_mlog_tid))) {
|
|
} else if (OB_FAIL(ObMLogInfo::insert_mlog_info(trans, mlog_info))) {
|
|
LOG_WARN("fail to insert mlog info", KR(ret), K(mlog_info));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_index_in_trans(obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObSArray<uint64_t> &index_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSArray<ObTableSchema> new_table_schemas;
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t src_tenant_id = alter_table_schema.get_tenant_id();
|
|
const uint64_t dst_tenant_id = alter_table_arg.exec_tenant_id_;
|
|
ObColumnNameMap col_name_map;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(tz_info_wrap.get_time_zone_info())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard hold_buf_src_tenant_schema_guard;
|
|
ObSchemaGetterGuard hold_buf_dst_tenant_schema_guard;
|
|
ObSchemaGetterGuard *src_tenant_schema_guard = nullptr;
|
|
ObSchemaGetterGuard *dst_tenant_schema_guard = nullptr;
|
|
hold_buf_src_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
hold_buf_dst_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
const ObTableSchema *orig_table_schema = nullptr;
|
|
const ObTableSchema *hidden_table_schema = nullptr;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObArray<int64_t> drop_cols_id_arr;
|
|
bool has_mlog_table = false;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(src_tenant_id, dst_tenant_id,
|
|
hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard,
|
|
src_tenant_schema_guard, dst_tenant_schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(src_tenant_id), K(dst_tenant_id));
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->get_schema_version(dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(dst_tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", K(ret), K(dst_tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("failed to get orig and hidden table schema", K(ret));
|
|
} else if (OB_FALSE_IT(has_mlog_table = orig_table_schema->has_mlog_table())) {
|
|
} else if (hidden_table_schema->get_simple_index_infos().count() > 0
|
|
|| hidden_table_schema->has_mlog_table()) {
|
|
// if there is any index in the hidden table, all indexes are already rebuilt because the rebuild is wrap in a transaction
|
|
const common::ObIArray<ObAuxTableMetaInfo> &index_infos = hidden_table_schema->get_simple_index_infos();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_infos.count(); ++i) {
|
|
if (OB_FAIL(index_ids.push_back(index_infos.at(i).table_id_))) {
|
|
LOG_WARN("push back index id failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && hidden_table_schema->has_mlog_table()) {
|
|
if (OB_FAIL(index_ids.push_back(hidden_table_schema->get_mlog_tid()))) {
|
|
LOG_WARN("failed to push back mlog tid", KR(ret), K(hidden_table_schema->get_mlog_tid()));
|
|
}
|
|
}
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(col_name_map.init(*orig_table_schema, *hidden_table_schema, alter_table_schema))) {
|
|
LOG_WARN("failed to init column name map", K(ret), K(alter_table_schema), KPC(orig_table_schema));
|
|
} else if (OB_FAIL(get_all_dropped_column_ids(alter_table_arg, *orig_table_schema, drop_cols_id_arr))) {
|
|
LOG_WARN("fail to get drop cols id set", K(ret));
|
|
} else if (OB_FAIL(reconstruct_index_schema(alter_table_arg,
|
|
*orig_table_schema,
|
|
*hidden_table_schema,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
drop_cols_id_arr,
|
|
col_name_map,
|
|
*tz_info_wrap.get_time_zone_info(),
|
|
alter_table_arg.allocator_,
|
|
new_table_schemas,
|
|
index_ids))) {
|
|
LOG_WARN("failed to reconstruct index schema", K(ret));
|
|
} else if (OB_FAIL(add_new_index_schema(alter_table_arg,
|
|
*orig_table_schema,
|
|
*hidden_table_schema,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
new_table_schemas,
|
|
index_ids))) {
|
|
LOG_WARN("failed to add new index schema", K(ret));
|
|
} else if (has_mlog_table && OB_FAIL(add_new_mlog_schema(
|
|
dst_tenant_id,
|
|
*orig_table_schema,
|
|
*hidden_table_schema,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
trans,
|
|
alter_table_arg.allocator_,
|
|
new_table_schemas,
|
|
index_ids))) {
|
|
LOG_WARN("failed to add new mlog schema", KR(ret));
|
|
} else if (OB_FAIL(rebuild_hidden_table_index_and_mlog(dst_tenant_id,
|
|
*dst_tenant_schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
new_table_schemas))) {
|
|
LOG_WARN("failed to rebuild hidden table index and mlog", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(dst_tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_hidden_table_column_id_by_orig_column_id(
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const ObColumnNameMap &col_name_map,
|
|
const uint64_t orig_column_id,
|
|
uint64_t &hidden_column_id) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObString orig_column_name;
|
|
ObString col_name;
|
|
const ObColumnSchemaV2 *orig_column = nullptr;
|
|
const ObColumnSchemaV2 *hidden_column = nullptr;
|
|
if (OB_ISNULL(orig_column = orig_table_schema.get_column_schema(orig_column_id))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get column from orig table schema", K(ret));
|
|
} else if (OB_FALSE_IT(orig_column_name = orig_column->get_column_name_str())) {
|
|
} else if (OB_FAIL(col_name_map.get(orig_column_name, col_name))) {
|
|
LOG_WARN("invalid column name", K(ret), K(orig_column_name));
|
|
} else if (OB_ISNULL(hidden_column = hidden_table_schema.get_column_schema(col_name))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get column from hidden table schema", K(ret), K(col_name));
|
|
} else {
|
|
hidden_column_id = hidden_column->get_column_id();
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::copy_constraint_for_hidden_table(
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const ObColumnNameMap &col_name_map,
|
|
const ObConstraint &orig_constraint,
|
|
ObConstraint &hidden_constraint)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSEArray<uint64_t, 16> column_ids;
|
|
if (OB_FAIL(hidden_constraint.assign(orig_constraint))) {
|
|
LOG_WARN("failed to assign constraint", K(ret));
|
|
}
|
|
for (ObConstraint::const_cst_col_iterator it = hidden_constraint.cst_col_begin();
|
|
OB_SUCC(ret) && it != hidden_constraint.cst_col_end(); it++) {
|
|
uint64_t orig_column_id = *it;
|
|
uint64_t column_id = 0;
|
|
if (OB_FAIL(get_hidden_table_column_id_by_orig_column_id(orig_table_schema, hidden_table_schema, col_name_map, orig_column_id, column_id))) {
|
|
LOG_WARN("failed to get column id", K(ret), K(orig_column_id));
|
|
} else if (OB_FAIL(column_ids.push_back(column_id))) {
|
|
LOG_WARN("failed to push back column id", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(hidden_constraint.assign_column_ids(column_ids))) {
|
|
LOG_WARN("failed to assign new column ids to constraint", K(ret));
|
|
} else {
|
|
// do nothing
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check whether the csts have been rebuilt, and get rebuilds constraints.
|
|
int ObDDLService::check_and_get_rebuild_constraints(
|
|
const ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &new_table_schema,
|
|
ObIArray<ObConstraint> &rebuild_constraints)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArray<int64_t> drop_cols_id_arr;
|
|
rebuild_constraints.reset();
|
|
bool has_rebuilt = false;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
// check whether the csts have been rebuilt.
|
|
for (ObTableSchema::const_constraint_iterator iter = orig_table_schema.constraint_begin();
|
|
OB_SUCC(ret) && !has_rebuilt && iter != orig_table_schema.constraint_end(); ++iter) {
|
|
if (OB_ISNULL(*iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("iter is NULL", K(ret));
|
|
} else if (CONSTRAINT_TYPE_PRIMARY_KEY == (*iter)->get_constraint_type()) {
|
|
// the primary key has been added when creating the hidden table, so won't add it again
|
|
} else {
|
|
const ObString &cst_name = (*iter)->get_constraint_name();
|
|
has_rebuilt = nullptr != new_table_schema.get_constraint(cst_name) ?
|
|
true : has_rebuilt;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (has_rebuilt) {
|
|
} else if (OB_FAIL(get_all_dropped_column_ids(alter_table_arg, orig_table_schema, drop_cols_id_arr))) {
|
|
LOG_WARN("fail to get drop cols id set", K(ret));
|
|
} else {
|
|
// remove invalid constraint caused by drop column.
|
|
for (ObTableSchema::const_constraint_iterator iter = orig_table_schema.constraint_begin();
|
|
OB_SUCC(ret) && iter != orig_table_schema.constraint_end(); ++iter) {
|
|
if (OB_ISNULL(*iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("iter is NULL", K(ret));
|
|
} else if (CONSTRAINT_TYPE_PRIMARY_KEY == (*iter)->get_constraint_type()) {
|
|
// the primary key has been added when creating the hidden table, so won't add it again
|
|
} else {
|
|
bool need_rebuild = true;
|
|
const ObString &cst_name = (*iter)->get_constraint_name_str();
|
|
// check whether the constraint is invalid caused by drop constraint.
|
|
if (obrpc::ObAlterTableArg::DROP_CONSTRAINT == alter_table_arg.alter_constraint_type_) {
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
for (ObTableSchema::const_constraint_iterator dropped_cst = alter_table_schema.constraint_begin();
|
|
OB_SUCC(ret) && need_rebuild && dropped_cst != alter_table_schema.constraint_end(); dropped_cst++) {
|
|
if (0 == cst_name.case_compare((*dropped_cst)->get_constraint_name_str())) {
|
|
need_rebuild = false;
|
|
}
|
|
}
|
|
}
|
|
// check whether the constraint is invalid caused by drop column.
|
|
for (ObConstraint::const_cst_col_iterator it_col = (*iter)->cst_col_begin();
|
|
OB_SUCC(ret) && need_rebuild && it_col != (*iter)->cst_col_end(); it_col++) {
|
|
FOREACH_CNT_X(dropped_col, drop_cols_id_arr, OB_SUCC(ret) && need_rebuild) {
|
|
if (*dropped_col == *it_col) {
|
|
need_rebuild = false;
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (need_rebuild && OB_FAIL(rebuild_constraints.push_back(**iter))) {
|
|
LOG_WARN("failed to push back constraint", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_constraints(
|
|
const ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObMySQLTransaction &trans,
|
|
ObSArray<uint64_t> &cst_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema new_table_schema;
|
|
ObColumnNameMap col_name_map;
|
|
ObArray<ObConstraint> rebuild_constraints;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.assign(hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(check_and_get_rebuild_constraints(alter_table_arg, orig_table_schema, new_table_schema, rebuild_constraints))) {
|
|
LOG_WARN("fail to get constraints that need to rebuild", K(ret));
|
|
} else if (rebuild_constraints.count() == 0) {
|
|
// no constraints, or constraints have been rebuilt.
|
|
} else if (OB_FAIL(col_name_map.init(orig_table_schema, hidden_table_schema, alter_table_arg.alter_table_schema_))) {
|
|
LOG_WARN("failed to init column name map", K(ret));
|
|
} else {
|
|
new_table_schema.clear_constraint();
|
|
new_table_schema.set_in_offline_ddl_white_list(true);
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < rebuild_constraints.size(); i++) {
|
|
ObConstraint constraint;
|
|
if (OB_FAIL(copy_constraint_for_hidden_table(orig_table_schema, hidden_table_schema, col_name_map, rebuild_constraints.at(i), constraint))) {
|
|
LOG_WARN("failed to assign constraint", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.add_constraint(constraint))) {
|
|
LOG_WARN("failed to add constraint", K(ret));
|
|
} else {/* do nothing. */}
|
|
}
|
|
if (OB_SUCC(ret) && new_table_schema.has_constraint()) {
|
|
if (OB_FAIL(ddl_operator.add_table_constraints(new_table_schema,
|
|
new_table_schema,
|
|
trans,
|
|
&cst_ids))) {
|
|
LOG_WARN("failed to add constraints", K(ret));
|
|
} else {
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(new_table_schema,
|
|
trans,
|
|
operation_type))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_constraints_in_trans(ObAlterTableArg &alter_table_arg,
|
|
ObSArray<uint64_t> &cst_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t src_tenant_id = alter_table_schema.get_tenant_id();
|
|
const uint64_t dst_tenant_id = alter_table_arg.exec_tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard hold_buf_src_tenant_schema_guard;
|
|
ObSchemaGetterGuard hold_buf_dst_tenant_schema_guard;
|
|
ObSchemaGetterGuard *src_tenant_schema_guard = nullptr;
|
|
ObSchemaGetterGuard *dst_tenant_schema_guard = nullptr;
|
|
hold_buf_src_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
hold_buf_dst_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
const ObTableSchema *orig_table_schema = nullptr;
|
|
const ObTableSchema *hidden_table_schema = nullptr;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(src_tenant_id, dst_tenant_id,
|
|
hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard,
|
|
src_tenant_schema_guard, dst_tenant_schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(src_tenant_id), K(dst_tenant_id));
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->get_schema_version(dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(src_tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(dst_tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("failed to get orig and hidden table schema", K(ret));
|
|
} else if (OB_FAIL(rebuild_hidden_table_constraints(alter_table_arg,
|
|
*orig_table_schema,
|
|
*hidden_table_schema,
|
|
trans,
|
|
cst_ids))) {
|
|
LOG_WARN("failed to rebuild hidden table constraints", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(dst_tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_triggers_on_hidden_table(
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &src_tenant_schema_guard,
|
|
ObSchemaGetterGuard &dst_tenant_schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
const uint64_t src_tenant_id = orig_table_schema.get_tenant_id();
|
|
const uint64_t dst_tenant_id = hidden_table_schema.get_tenant_id();
|
|
const bool is_across_tenant = src_tenant_id != dst_tenant_id;
|
|
const ObIArray<uint64_t> &trigger_list = orig_table_schema.get_trigger_list();
|
|
const ObTriggerInfo *trigger_info = NULL;
|
|
ObTriggerInfo new_trigger_info;
|
|
ObErrorInfo error_info;
|
|
for (int i = 0; OB_SUCC(ret) && i < trigger_list.count(); i++) {
|
|
const ObTriggerInfo *check_exist_trigger = nullptr;
|
|
OZ (src_tenant_schema_guard.get_trigger_info(src_tenant_id, trigger_list.at(i), trigger_info));
|
|
OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_list.at(i));
|
|
OZ (dst_tenant_schema_guard.get_trigger_info(dst_tenant_id, hidden_table_schema.get_database_id(),
|
|
trigger_info->get_trigger_name(), check_exist_trigger));
|
|
if (nullptr != check_exist_trigger && is_across_tenant) {
|
|
LOG_INFO("duplicated trigger name, ignore to rebuild", K(dst_tenant_id), "db_id", hidden_table_schema.get_database_id(), KPC(trigger_info));
|
|
} else {
|
|
OZ (new_trigger_info.assign(*trigger_info));
|
|
OX (new_trigger_info.set_base_object_id(hidden_table_schema.get_table_id()));
|
|
OX (new_trigger_info.set_trigger_id(OB_INVALID_ID));
|
|
OX (new_trigger_info.set_tenant_id(dst_tenant_id));
|
|
OX (new_trigger_info.set_database_id(hidden_table_schema.get_database_id()));
|
|
if (!is_across_tenant) {
|
|
// triiger under source tenant will be dropped when drop tenant.
|
|
OZ (ddl_operator.drop_trigger(*trigger_info, trans,
|
|
nullptr, false/*is_update_table_schema_version*/));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObSEArray<ObDependencyInfo, 1> dep_infos;
|
|
int64_t table_schema_version = OB_INVALID_VERSION;
|
|
OZ (ddl_operator.create_trigger(new_trigger_info, trans, error_info, dep_infos,
|
|
table_schema_version, nullptr, false/*is_update_table_schema_version*/));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::swap_child_table_fk_name(
|
|
const uint64_t child_table_id,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObTableSchema new_table_schema;
|
|
const ObTableSchema *child_table_schema;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, child_table_id, child_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(child_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get child table schema", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.assign(*child_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
ObIArray<ObForeignKeyInfo> &new_foreign_key_infos = new_table_schema.get_foreign_key_infos();
|
|
const ObIArray<ObForeignKeyInfo> &orig_foreign_key_infos = child_table_schema->get_foreign_key_infos();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_foreign_key_infos.count(); i++) {
|
|
ObForeignKeyInfo &fk_info1 = new_foreign_key_infos.at(i);
|
|
// rename hidden fk_name to orig fk_name
|
|
if (fk_info1.parent_table_id_ == hidden_table_schema.get_table_id()) {
|
|
bool is_prefix_match = false;
|
|
for (int64_t j = 0; OB_SUCC(ret) && !is_prefix_match && j < orig_foreign_key_infos.count(); j++) {
|
|
const ObForeignKeyInfo &fk_info2 = orig_foreign_key_infos.at(j);
|
|
if (OB_FAIL(is_foreign_key_name_prefix_match(fk_info2, fk_info1, allocator, is_prefix_match))) {
|
|
LOG_WARN("is foreign key name prefix match", K(ret));
|
|
} else if (is_prefix_match
|
|
&& fk_info1.child_table_id_ == fk_info2.child_table_id_
|
|
&& fk_info2.parent_table_id_ == orig_table_schema.get_table_id()) {
|
|
fk_info1.set_is_modify_fk_state(true);
|
|
fk_info1.set_is_modify_fk_name_flag(true);
|
|
if (OB_FAIL(ob_write_string(allocator,
|
|
fk_info2.foreign_key_name_,
|
|
fk_info1.foreign_key_name_))) {
|
|
LOG_WARN("Can not malloc space for fk name", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_prefix_match) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, can not found match hidden fk info", K(ret), K(fk_info1), K(orig_foreign_key_infos));
|
|
}
|
|
} else if (fk_info1.parent_table_id_ == orig_table_schema.get_table_id()) {
|
|
// rename orig fk_name to hidden fk_name
|
|
bool is_prefix_match = false;
|
|
for (int64_t j = 0; OB_SUCC(ret) && !is_prefix_match && j < orig_foreign_key_infos.count(); j++) {
|
|
const ObForeignKeyInfo &fk_info2 = orig_foreign_key_infos.at(j);
|
|
if (OB_FAIL(is_foreign_key_name_prefix_match(fk_info1, fk_info2, allocator, is_prefix_match))) {
|
|
LOG_WARN("is foreign key name prefix match", K(ret));
|
|
} else if (is_prefix_match
|
|
&& fk_info1.child_table_id_ == fk_info2.child_table_id_
|
|
&& fk_info2.parent_table_id_ == hidden_table_schema.get_table_id()) {
|
|
fk_info1.set_is_modify_fk_state(true);
|
|
fk_info1.set_is_modify_fk_name_flag(true);
|
|
if (OB_FAIL(ob_write_string(allocator,
|
|
fk_info2.foreign_key_name_,
|
|
fk_info1.foreign_key_name_))) {
|
|
LOG_WARN("Can not malloc space for fk name", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_prefix_match) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, can not found match hidden fk info", K(ret), K(fk_info1), K(orig_foreign_key_infos));
|
|
}
|
|
}
|
|
}
|
|
// update foreign key name
|
|
if (OB_SUCC(ret)) {
|
|
new_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (OB_FAIL(new_table_schema.add_depend_table_id(orig_table_schema.get_table_id()))
|
|
|| OB_FAIL(new_table_schema.add_depend_table_id(hidden_table_schema.get_table_id()))) {
|
|
LOG_WARN("failed to add depend table id", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_table_foreign_keys(
|
|
new_table_schema, trans, true/*in_offline_ddl_white_list*/))) {
|
|
LOG_WARN("failed to update foreign key", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_child_table_fk(
|
|
const ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObArray<uint64_t> cst_ids;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else {
|
|
common::ObSEArray<uint64_t, 16> child_table_ids;
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = orig_table_schema.get_foreign_key_infos();
|
|
const bool prev_in_offline_ddl_white_list = orig_table_schema.get_in_offline_ddl_white_list();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i);
|
|
if (foreign_key_info.parent_table_id_ == orig_table_schema.get_table_id()
|
|
&& foreign_key_info.child_table_id_ != orig_table_schema.get_table_id()) {
|
|
ObDropForeignKeyArg drop_fk_arg;
|
|
drop_fk_arg.foreign_key_name_ = foreign_key_info.foreign_key_name_;
|
|
const ObTableSchema *child_table_schema = nullptr;
|
|
const ObForeignKeyInfo *parent_table_mock_fk_info = nullptr;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, foreign_key_info.child_table_id_, child_table_schema))) {
|
|
LOG_WARN("failed to get child table schema", K(ret));
|
|
} else if (OB_ISNULL(child_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid child table schema", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_table_drop_foreign_key(*child_table_schema, drop_fk_arg, trans, parent_table_mock_fk_info,
|
|
true/*parent_table_in_offline_ddl_white_list*/))) {
|
|
LOG_WARN("failed to drop child table foreign key", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::swap_all_child_table_fk_name(
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else {
|
|
common::ObSEArray<uint64_t, 16> child_table_ids;
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = orig_table_schema.get_foreign_key_infos();
|
|
// get all child table id
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i);
|
|
if (foreign_key_info.parent_table_id_ == orig_table_schema.get_table_id()
|
|
&& foreign_key_info.child_table_id_ != orig_table_schema.get_table_id()) {
|
|
if (OB_FAIL(child_table_ids.push_back(foreign_key_info.child_table_id_))) {
|
|
LOG_WARN("failed to add table id", K(ret));
|
|
}
|
|
}
|
|
}
|
|
// modify all child table foreign key
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < child_table_ids.count(); i++) {
|
|
uint64_t child_table_id = child_table_ids.at(i);
|
|
if (OB_FAIL(swap_child_table_fk_name(child_table_id,
|
|
orig_table_schema,
|
|
hidden_table_schema,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
allocator))) {
|
|
LOG_WARN("fail to swap child table fk name", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::build_hidden_index_table_map(
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
hash::ObHashMap<common::ObString, uint64_t> &new_index_table_map)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = hidden_table_schema.get_tenant_id();
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
if (OB_FAIL(hidden_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
const ObSimpleTableSchemaV2 *index_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_simple_table_schema(
|
|
tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", simple_index_infos.at(i).table_id_, K(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (OB_FAIL(new_index_table_map.set_refactored(index_table_schema->get_table_name_str(),
|
|
index_table_schema->get_table_id()))) {
|
|
LOG_WARN("failed to insert into table map", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::convert_hidden_table_column_ids_by_orig_column_ids(
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const ObColumnNameMap &col_name_map,
|
|
ObIArray<uint64_t> &column_ids) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < column_ids.count(); i++) {
|
|
const uint64_t orig_column_id = column_ids.at(i);
|
|
uint64_t &hidden_column_id = column_ids.at(i);
|
|
if (OB_FAIL(get_hidden_table_column_id_by_orig_column_id(orig_table_schema, hidden_table_schema, col_name_map, orig_column_id, hidden_column_id))) {
|
|
LOG_WARN("failed to get column id", K(ret), K(orig_column_id));
|
|
} else {
|
|
// do nothing
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// remove invalid foreign key, caused by drop column and drop fk.
|
|
int ObDDLService::get_rebuild_foreign_key_infos(
|
|
const ObAlterTableArg &alter_table_arg,
|
|
share::schema::ObSchemaGetterGuard &src_tenant_schema_guard,
|
|
share::schema::ObSchemaGetterGuard &dst_tenant_schema_guard,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const bool rebuild_child_table_fk,
|
|
ObIArray<ObForeignKeyInfo> &original_fk_infos,
|
|
ObIArray<ObForeignKeyInfo> &rebuild_fk_infos)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
original_fk_infos.reset();
|
|
rebuild_fk_infos.reset();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
|
|
const ObIArray<ObForeignKeyInfo> &all_fk_infos = orig_table_schema.get_foreign_key_infos();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < all_fk_infos.count(); i++) {
|
|
bool need_rebuild = true;
|
|
ObForeignKeyInfo new_fk_info = all_fk_infos.at(i);
|
|
for (int64_t j = 0; OB_SUCC(ret) && need_rebuild && j < index_arg_list.size(); j++) {
|
|
ObIndexArg *index_arg = const_cast<ObIndexArg *>(index_arg_list.at(j));
|
|
if (OB_ISNULL(index_arg)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument, index arg should not be null", K(ret));
|
|
} else if (ObIndexArg::DROP_FOREIGN_KEY == index_arg->index_action_type_) {
|
|
const ObDropForeignKeyArg *drop_fk_arg = static_cast<ObDropForeignKeyArg *>(index_arg);
|
|
if (0 == (drop_fk_arg->foreign_key_name_).case_compare(new_fk_info.foreign_key_name_)) {
|
|
need_rebuild = false;
|
|
} else {/* do nothing.*/ }
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && need_rebuild) {
|
|
const int64_t orig_table_id = orig_table_schema.get_table_id();
|
|
const uint64_t src_tenant_id = orig_table_schema.get_tenant_id();
|
|
const uint64_t dst_tenant_id = alter_table_arg.exec_tenant_id_;
|
|
if (new_fk_info.child_table_id_ != orig_table_id && new_fk_info.parent_table_id_ == orig_table_id) {
|
|
need_rebuild = rebuild_child_table_fk;
|
|
} else if (new_fk_info.child_table_id_ == orig_table_id) {
|
|
need_rebuild = !rebuild_child_table_fk;
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected fk", K(ret), K(new_fk_info), K(orig_table_id));
|
|
}
|
|
if (OB_SUCC(ret) && need_rebuild && (src_tenant_id != dst_tenant_id)) {
|
|
// replace foreign key info for recover restore table ddl task.
|
|
const bool is_recover_child_table = new_fk_info.child_table_id_ == orig_table_id; // child table or self-reference table.
|
|
if (OB_FAIL(check_and_replace_fk_info_on_demand(
|
|
src_tenant_schema_guard,
|
|
dst_tenant_schema_guard,
|
|
hidden_table_schema,
|
|
is_recover_child_table,
|
|
new_fk_info))) {
|
|
LOG_INFO("check and replace fk info on demand failed, ignore to rebuild", K(ret));
|
|
need_rebuild = false;
|
|
ret = OB_SUCCESS; // override error code is expected, ignore to rebuild it.
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret) || !need_rebuild) {
|
|
} else if (OB_FAIL(original_fk_infos.push_back(all_fk_infos.at(i)))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
} else if (OB_FAIL(rebuild_fk_infos.push_back(new_fk_info))) {
|
|
LOG_WARN("fail to push back fk infos that need to rebuild", K(ret));
|
|
} else {/* do nothing. */}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_foreign_key(
|
|
ObAlterTableArg &alter_table_arg,
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const bool rebuild_child_table_fk,
|
|
ObSchemaGetterGuard &src_tenant_schema_guard,
|
|
ObSchemaGetterGuard &dst_tenant_schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
ObSArray<uint64_t> &cst_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
ObTableSchema inc_table_schema;
|
|
ObArray<ObForeignKeyInfo> original_fk_infos;
|
|
ObArray<ObForeignKeyInfo> rebuild_fk_infos;
|
|
ObColumnNameMap col_name_map;
|
|
common::hash::ObHashMap<ObString, uint64_t> new_index_table_map;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const uint64_t tenant_id = hidden_table_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else if (OB_FAIL(inc_table_schema.assign(hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(new_index_table_map.create(16, "HashBucIdxTabMa"))) {
|
|
LOG_WARN("failed to add create ObHashMap", K(ret));
|
|
} else if (OB_FAIL(build_hidden_index_table_map(hidden_table_schema,
|
|
dst_tenant_schema_guard,
|
|
new_index_table_map))) {
|
|
LOG_WARN("failed to build hidden index table map", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to check if oralce compat mode", K(ret));
|
|
} else if (OB_FAIL(get_rebuild_foreign_key_infos(alter_table_arg,
|
|
src_tenant_schema_guard,
|
|
dst_tenant_schema_guard,
|
|
orig_table_schema,
|
|
hidden_table_schema,
|
|
rebuild_child_table_fk,
|
|
original_fk_infos,
|
|
rebuild_fk_infos))) {
|
|
LOG_WARN("fail to get fk infos that need to rebuild", K(ret));
|
|
} else if (OB_FAIL(inc_table_schema.set_foreign_key_infos(rebuild_fk_infos))) {
|
|
LOG_WARN("fail to set fk infos", K(ret));
|
|
} else if (OB_FAIL(col_name_map.init(orig_table_schema, hidden_table_schema, alter_table_arg.alter_table_schema_))) {
|
|
LOG_WARN("failed to init column name map", K(ret));
|
|
} else {
|
|
ObIArray<ObForeignKeyInfo> &foreign_key_infos = inc_table_schema.get_foreign_key_infos();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i);
|
|
const ObForeignKeyInfo &original_fk_info = original_fk_infos.at(i);
|
|
const uint64_t origin_fk_id = foreign_key_info.foreign_key_id_;
|
|
foreign_key_info.foreign_key_id_ = OB_INVALID_ID;
|
|
|
|
// add depend table id
|
|
if (original_fk_info.parent_table_id_ != orig_table_schema.get_table_id()) {
|
|
if (!foreign_key_info.is_parent_table_mock_) {
|
|
if (OB_FAIL(inc_table_schema.add_depend_table_id(foreign_key_info.parent_table_id_))) {
|
|
LOG_WARN("fail to add depend table id", K(ret));
|
|
}
|
|
} else if (OB_FAIL(inc_table_schema.add_depend_mock_fk_parent_table_id(foreign_key_info.parent_table_id_))) {
|
|
LOG_WARN("fail to add depend table id", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (original_fk_info.child_table_id_ != orig_table_schema.get_table_id()
|
|
&& OB_FAIL(inc_table_schema.add_depend_table_id(foreign_key_info.child_table_id_))) {
|
|
LOG_WARN("fail to add depend table id", K(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_new_constraint_id(
|
|
tenant_id, foreign_key_info.foreign_key_id_))) {
|
|
LOG_WARN("failed to fetch new foreign key id", K(ret), K(tenant_id));
|
|
} else if (original_fk_info.parent_table_id_ == orig_table_schema.get_table_id()) {
|
|
// update referenced constraint id
|
|
if (CONSTRAINT_TYPE_PRIMARY_KEY == foreign_key_info.ref_cst_type_) {
|
|
if (is_oracle_mode) {
|
|
const ObConstraint *pk_cst = hidden_table_schema.get_pk_constraint();
|
|
if (OB_ISNULL(pk_cst)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("primary key cst is null", K(ret));
|
|
} else {
|
|
foreign_key_info.ref_cst_id_ = pk_cst->get_constraint_id();
|
|
}
|
|
} else {
|
|
foreign_key_info.ref_cst_id_ = common::OB_INVALID_ID;
|
|
}
|
|
} else {
|
|
const ObSimpleTableSchemaV2 *orig_index_table_schema = NULL;
|
|
ObString new_index_table_name;
|
|
uint64_t new_ref_cst_id = OB_INVALID_ID;
|
|
if (OB_FAIL(dst_tenant_schema_guard.get_simple_table_schema(
|
|
tenant_id, foreign_key_info.ref_cst_id_, orig_index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", foreign_key_info.ref_cst_id_, K(ret));
|
|
} else if (OB_ISNULL(orig_index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (OB_FAIL(gen_new_index_table_name(
|
|
orig_index_table_schema->get_table_name_str(),
|
|
orig_table_schema.get_table_id(),
|
|
hidden_table_schema.get_table_id(),
|
|
alter_table_arg.allocator_,
|
|
new_index_table_name))) {
|
|
LOG_WARN("failed to build new index table name!", K(hidden_table_schema.get_table_id()),
|
|
K(new_index_table_name), K(ret));
|
|
} else if (OB_FAIL(new_index_table_map.get_refactored(new_index_table_name,
|
|
new_ref_cst_id))) {
|
|
LOG_WARN("failed to get new ref cst id", K(new_index_table_name), K(ret));
|
|
} else {
|
|
foreign_key_info.ref_cst_id_ = new_ref_cst_id;
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (original_fk_info.parent_table_id_ == orig_table_schema.get_table_id()) {
|
|
// self-dependent
|
|
if (original_fk_info.child_table_id_ == original_fk_info.parent_table_id_) {
|
|
foreign_key_info.parent_table_id_ = hidden_table_schema.get_table_id();
|
|
foreign_key_info.child_table_id_ = hidden_table_schema.get_table_id();
|
|
if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.parent_column_ids_))) {
|
|
LOG_WARN("failed to convert parent column ids of foreign key", K(ret));
|
|
} else if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.child_column_ids_))) {
|
|
LOG_WARN("failed to convert child column ids of foreign key", K(ret));
|
|
}
|
|
} else {
|
|
foreign_key_info.parent_table_id_ = hidden_table_schema.get_table_id();
|
|
if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.parent_column_ids_))) {
|
|
LOG_WARN("failed to convert parent column ids of foreign key", K(ret));
|
|
}
|
|
}
|
|
} else {
|
|
foreign_key_info.child_table_id_ = hidden_table_schema.get_table_id();
|
|
if (OB_FAIL(convert_hidden_table_column_ids_by_orig_column_ids(orig_table_schema, hidden_table_schema, col_name_map, foreign_key_info.child_column_ids_))) {
|
|
LOG_WARN("failed to convert child column ids of foreign key", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && foreign_key_info.validate_flag_) {
|
|
if (OB_FAIL(cst_ids.push_back(foreign_key_info.foreign_key_id_))) {
|
|
LOG_WARN("failed to add new foreign key id!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
inc_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (OB_FAIL(ddl_operator.add_table_foreign_keys(hidden_table_schema,
|
|
inc_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to add foreign key", K(ret));
|
|
} else {
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(inc_table_schema,
|
|
trans,
|
|
operation_type))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_hidden_table_foreign_key_in_trans(ObAlterTableArg &alter_table_arg,
|
|
ObSArray<uint64_t> &cst_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t src_tenant_id = alter_table_schema.get_tenant_id();
|
|
const uint64_t dst_tenant_id = alter_table_arg.exec_tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard hold_buf_src_tenant_schema_guard;
|
|
ObSchemaGetterGuard hold_buf_dst_tenant_schema_guard;
|
|
ObSchemaGetterGuard *src_tenant_schema_guard = nullptr;
|
|
ObSchemaGetterGuard *dst_tenant_schema_guard = nullptr;
|
|
hold_buf_src_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
hold_buf_dst_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
const ObTableSchema *orig_table_schema = nullptr;
|
|
const ObTableSchema *hidden_table_schema = nullptr;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(src_tenant_id, dst_tenant_id,
|
|
hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard,
|
|
src_tenant_schema_guard, dst_tenant_schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(src_tenant_id), K(dst_tenant_id));
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->get_schema_version(dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(dst_tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(dst_tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("failed to get orig and hidden table schema", K(ret));
|
|
} else if (!hidden_table_schema->get_foreign_key_infos().empty()) {
|
|
// not empty means already rebuilt.
|
|
} else if (OB_FAIL(rebuild_hidden_table_foreign_key(alter_table_arg,
|
|
*orig_table_schema,
|
|
*hidden_table_schema,
|
|
false/*rebuild_child_table_fk*/,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
trans,
|
|
cst_ids))) {
|
|
LOG_WARN("failed to rebuild hidden table foreign key", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(dst_tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_orig_and_hidden_table_schema(
|
|
const ObAlterTableArg &alter_table_arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
share::schema::ObSchemaGetterGuard &dest_schema_guard,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const ObTableSchema *&orig_table_schema,
|
|
const ObTableSchema *&hidden_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t orig_table_id = alter_table_arg.table_id_;
|
|
const int64_t hidden_table_id = alter_table_arg.hidden_table_id_;
|
|
const uint64_t tenant_id = alter_table_arg.alter_table_schema_.get_tenant_id();
|
|
const uint64_t dest_tenant_id = alter_table_arg.exec_tenant_id_;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, orig_table_id, orig_table_schema))) {
|
|
LOG_WARN("failed to get orig table schema", K(ret), K(orig_table_id));
|
|
} else if (OB_FAIL(dest_schema_guard.get_table_schema(dest_tenant_id, hidden_table_id, hidden_table_schema))) {
|
|
LOG_WARN("fail to get hidden table schema", K(ret), K(hidden_table_id));
|
|
} else if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("failed to get orig table schema", K(ret), K(tenant_id), K(orig_table_id));
|
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("fail to get hidden table schema", K(ret), K(dest_tenant_id), K(hidden_table_id));
|
|
} else if (orig_table_schema->get_tenant_id() != hidden_table_schema->get_tenant_id()) {
|
|
// do nothing
|
|
} else if (OB_UNLIKELY(orig_table_schema->get_association_table_id() != hidden_table_schema->get_table_id())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("not in offline ddl", K(ret), K(orig_table_id), K(hidden_table_id), K(hidden_table_schema->get_table_id()), K(orig_table_schema->get_association_table_id()));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_tablets(
|
|
const uint64_t tenant_id,
|
|
const ObArray<common::ObTabletID> &tablet_ids,
|
|
ObIArray<LSTabletID> &tablets,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArray<ObLSID> ls_ids;
|
|
if (OB_FAIL(ObTabletToLSTableOperator::batch_get_ls(trans, tenant_id, tablet_ids, ls_ids))) {
|
|
LOG_WARN("failed to batch get ls", K(ret));
|
|
} else if (OB_UNLIKELY(tablet_ids.count() != ls_ids.count())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid tablet ids ls ids", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); i++) {
|
|
if (OB_FAIL(tablets.push_back({ls_ids[i], tablet_ids[i]}))) {
|
|
LOG_WARN("failed to push back tablet id and ls id", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::build_modify_tablet_binding_args(
|
|
const uint64_t tenant_id,
|
|
const ObArray<ObTabletID> &tablet_ids,
|
|
const bool is_hidden_tablets,
|
|
const int64_t schema_version,
|
|
ObIArray<ObBatchUnbindTabletArg> &modify_args,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArray<LSTabletID> tablets;
|
|
if (OB_FAIL(get_tablets(tenant_id, tablet_ids, tablets, trans))) {
|
|
LOG_WARN("failed to get tablet ids of orig table", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < tablets.count(); i++) {
|
|
const ObLSID &ls_id = tablets[i].first;
|
|
int64_t j = 0;
|
|
for (; j < modify_args.count(); j++) {
|
|
if (modify_args.at(j).ls_id_ == ls_id && modify_args.at(j).tenant_id_ == tenant_id) {
|
|
break;
|
|
}
|
|
}
|
|
if (j == modify_args.count()) {
|
|
ObBatchUnbindTabletArg modify_arg;
|
|
modify_arg.tenant_id_ = tenant_id;
|
|
modify_arg.ls_id_ = ls_id;
|
|
modify_arg.schema_version_ = schema_version;
|
|
if (OB_FAIL(modify_args.push_back(modify_arg))) {
|
|
LOG_WARN("failed to push back modify arg", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObBatchUnbindTabletArg &modify_arg = modify_args.at(j);
|
|
const ObTabletID &tablet_id = tablets[i].second;
|
|
if (is_hidden_tablets) {
|
|
if (OB_FAIL(modify_arg.hidden_tablet_ids_.push_back(tablet_id))) {
|
|
LOG_WARN("failed to push back", K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(modify_arg.orig_tablet_ids_.push_back(tablet_id))) {
|
|
LOG_WARN("failed to push back", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
LOG_DEBUG("build modify tablet binding args", K(ret), K(modify_args));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::unbind_hidden_tablets(
|
|
const ObTableSchema &orig_table_schema,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const int64_t schema_version,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
uint64_t tenant_data_version = 0;
|
|
ObArray<ObTabletID> orig_tablet_ids;
|
|
ObArray<ObTabletID> hidden_tablet_ids;
|
|
if (OB_FAIL(orig_table_schema.get_tablet_ids(orig_tablet_ids))) {
|
|
LOG_WARN("get tablet ids failed", K(ret));
|
|
} else if (OB_FAIL(hidden_table_schema.get_tablet_ids(hidden_tablet_ids))) {
|
|
LOG_WARN("get tablet ids failed", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else if (OB_LIKELY(tenant_data_version >= DATA_VERSION_4_3_2_0)) {
|
|
const int64_t abs_timeout_us = THIS_WORKER.is_timeout_ts_valid() ? THIS_WORKER.get_timeout_ts()
|
|
: ObTimeUtility::current_time() + GCONF.rpc_timeout;
|
|
if (OB_FAIL(ObTabletBindingMdsHelper::modify_tablet_binding_for_unbind(tenant_id, orig_tablet_ids, hidden_tablet_ids, schema_version, abs_timeout_us, trans))) {
|
|
LOG_WARN("failed to modify tablet binding", K(ret), K(abs_timeout_us));
|
|
}
|
|
} else {
|
|
ObArray<ObBatchUnbindTabletArg> args;
|
|
if (OB_FAIL(build_modify_tablet_binding_args(
|
|
tenant_id, orig_tablet_ids, false/*is_hidden_tablets*/, schema_version, args, trans))) {
|
|
LOG_WARN("failed to build reuse index args", K(ret));
|
|
} else if (OB_FAIL(build_modify_tablet_binding_args(
|
|
tenant_id, hidden_tablet_ids, true/*is_hidden_tablets*/, schema_version, args, trans))) {
|
|
LOG_WARN("failed to build reuse index args", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < args.count(); i++) {
|
|
int64_t pos = 0;
|
|
int64_t size = args[i].get_serialize_size();
|
|
ObArenaAllocator allocator;
|
|
char *buf = nullptr;
|
|
if (OB_ISNULL(buf = static_cast<char *>(allocator.alloc(size)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("failed to allocate", K(ret));
|
|
} else if (OB_FAIL(args[i].serialize(buf, size, pos))) {
|
|
LOG_WARN("failed to serialize arg", K(ret));
|
|
} else if (OB_FAIL(trans.register_tx_data(args[i].tenant_id_, args[i].ls_id_, transaction::ObTxDataSourceType::UNBIND_TABLET_NEW_MDS, buf, pos))) {
|
|
LOG_WARN("failed to register tx data", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::write_ddl_barrier(
|
|
const ObTableSchema &hidden_table_schema,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = hidden_table_schema.get_tenant_id();
|
|
ObArray<ObTabletID> tablet_ids;
|
|
ObArray<LSTabletID> hidden_tablets;
|
|
if (OB_FAIL(hidden_table_schema.get_tablet_ids(tablet_ids))) {
|
|
LOG_WARN("failed to get tablets", K(ret));
|
|
} else if (OB_FAIL(get_tablets(tenant_id, tablet_ids, hidden_tablets, trans))) {
|
|
LOG_WARN("failed to get tablet", K(ret));
|
|
} else {
|
|
ObArenaAllocator allocator("DDLSrvBarrier");
|
|
ObArray<ObDDLBarrierLog> logs;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < hidden_tablets.count(); i++) {
|
|
const ObLSID &ls_id = hidden_tablets[i].first;
|
|
const ObTabletID &tablet_id = hidden_tablets[i].second;
|
|
int64_t j = 0;
|
|
for (; j < logs.count(); j++) {
|
|
if (logs[j].ls_id_ == ls_id) {
|
|
break;
|
|
}
|
|
}
|
|
if (j == logs.count()) {
|
|
ObDDLBarrierLog log;
|
|
log.ls_id_ = ls_id;
|
|
if (OB_FAIL(logs.push_back(log))) {
|
|
LOG_WARN("failed to push back log", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(logs[j].hidden_tablet_ids_.push_back(tablet_id))) {
|
|
LOG_WARN("failed to push back hidden tablet", K(ret));
|
|
}
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < logs.count(); i++) {
|
|
int64_t pos = 0;
|
|
int64_t size = logs[i].get_serialize_size();
|
|
char *buf = nullptr;
|
|
allocator.reuse();
|
|
if (OB_UNLIKELY(!logs[i].is_valid())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid ddl barrier log", K(ret), K(logs[i]));
|
|
} else if (OB_ISNULL(buf = static_cast<char *>(allocator.alloc(size)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("failed to allocate", K(ret));
|
|
} else if (OB_FAIL(logs[i].serialize(buf, size, pos))) {
|
|
LOG_WARN("failed to serialize arg", K(ret));
|
|
} else if (OB_FAIL(trans.register_tx_data(tenant_id, logs[i].ls_id_, transaction::ObTxDataSourceType::DDL_BARRIER, buf, pos))) {
|
|
LOG_WARN("failed to register tx data", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLSQLTransaction::register_tx_data(
|
|
const uint64_t tenant_id,
|
|
const share::ObLSID &ls_id,
|
|
const transaction::ObTxDataSourceType &type,
|
|
const char *buf,
|
|
const int64_t buf_len)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
sqlclient::ObISQLConnection *isql_conn = nullptr;
|
|
if (OB_ISNULL(isql_conn = get_connection())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid connection", K(ret));
|
|
} else {
|
|
observer::ObInnerSQLConnection *conn = static_cast<observer::ObInnerSQLConnection *>(isql_conn);
|
|
if (OB_FAIL(conn->register_multi_data_source(tenant_id, ls_id, type, buf, buf_len))) {
|
|
LOG_WARN("failed to register tx data", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_hidden_table_constraint_exist(
|
|
const ObTableSchema *hidden_table_schema,
|
|
const ObTableSchema *orig_table_schema,
|
|
ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
for (ObTableSchema::const_constraint_iterator iter = hidden_table_schema->constraint_begin();
|
|
OB_SUCC(ret) && iter != hidden_table_schema->constraint_end(); ++iter) {
|
|
if (OB_ISNULL(*iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("iter is NULL", K(ret));
|
|
} else {
|
|
const ObString &cst_name = (*iter)->get_constraint_name();
|
|
bool is_check_constraint_name_exist = false;
|
|
const ObConstraint *cst = orig_table_schema->get_constraint(cst_name);
|
|
if (cst == nullptr) { // duplicate name which is not from orig table
|
|
if (OB_FAIL(check_constraint_name_is_exist(schema_guard,
|
|
*hidden_table_schema,
|
|
cst_name,
|
|
false,
|
|
is_check_constraint_name_exist))) {
|
|
LOG_WARN("fail to check constraint name is exist or not", K(ret), K(cst_name));
|
|
} else if (is_check_constraint_name_exist) {
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_WARN("check constraint name is duplicate", K(ret), K(cst_name));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (alter_table_arg.mview_refresh_info_.is_mview_complete_refresh_ &&
|
|
OB_UNLIKELY(!alter_table_arg.mview_refresh_info_.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid mview refresh info", K(ret), K(alter_table_arg.mview_refresh_info_));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
const ObTableSchema *hidden_table_schema = NULL;
|
|
HEAP_VARS_2((ObTableSchema, new_orig_table_schema),
|
|
(ObTableSchema, new_hidden_table_schema)) {
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObSArray<ObTableSchema> table_schemas;
|
|
ObSArray<uint64_t> fk_cst_ids;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> orig_simple_index_infos;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> new_simple_index_infos;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
ObTableLockOwnerID owner_id;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("failed to get orig and hidden table schema", K(ret));
|
|
} else if (OB_FAIL(orig_table_schema->get_simple_index_infos(
|
|
orig_simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
} else if (OB_FAIL(hidden_table_schema->get_simple_index_infos(
|
|
new_simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
} else if (OB_FAIL(new_orig_table_schema.assign(*orig_table_schema))
|
|
|| OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (0 == hidden_table_schema->get_table_name_str().case_compare(alter_table_arg.alter_table_schema_.get_origin_table_name())) {
|
|
// ignore case sensitivity is expected.
|
|
ret = OB_NO_NEED_UPDATE;
|
|
LOG_WARN("already swapped", K(ret));
|
|
} else if (OB_FAIL(drop_child_table_fk(alter_table_arg, new_orig_table_schema,
|
|
*hidden_table_schema, schema_guard, trans))) {
|
|
LOG_WARN("failed to drop origin table fk", K(ret));
|
|
} else if (OB_FAIL(rebuild_hidden_table_foreign_key(alter_table_arg,
|
|
*orig_table_schema, *hidden_table_schema, true/*rebuild_child_table_fk*/, schema_guard, schema_guard, trans, fk_cst_ids))) {
|
|
LOG_WARN("failed to rebuild hidden table fk", K(ret));
|
|
} else if (OB_FAIL(check_hidden_table_constraint_exist(hidden_table_schema,
|
|
orig_table_schema,
|
|
schema_guard))) {
|
|
LOG_WARN("failed to check hidden table constraint existence", K(ret));
|
|
} else {
|
|
if (OB_SUCC(ret) && alter_table_arg.need_rebuild_trigger_) {
|
|
if (OB_FAIL(rebuild_triggers_on_hidden_table(*orig_table_schema,
|
|
*hidden_table_schema,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans))) {
|
|
LOG_WARN("failed to create triggers on hidden table", K(ret));
|
|
}
|
|
}
|
|
// prepare schema
|
|
if (OB_SUCC(ret)) {
|
|
ObTableSchema tmp_schema;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
tmp_schema.reset();
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, orig_simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id),
|
|
"table id", orig_simple_index_infos.at(i).table_id_, K(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (OB_FAIL(tmp_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
// modify the state of the all original index tables to hidden
|
|
tmp_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL);
|
|
if (OB_FAIL(table_schemas.push_back(tmp_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
tmp_schema.reset();
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, new_simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", new_simple_index_infos.at(i).table_id_, K(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (OB_FAIL(tmp_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
// modify the state of the all hidden index tables to non-hidden
|
|
tmp_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL);
|
|
if (OB_FAIL(table_schemas.push_back(tmp_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
// materialized view log
|
|
if (OB_SUCC(ret) && alter_table_arg.mview_refresh_info_.is_mview_complete_refresh_) {
|
|
if (orig_table_schema->has_mlog_table() && hidden_table_schema->has_mlog_table()) {
|
|
if (OB_FAIL(swap_orig_and_hidden_table_mlog_state_(schema_guard,
|
|
tenant_id,
|
|
orig_table_schema->get_mlog_tid(),
|
|
hidden_table_schema->get_mlog_tid(),
|
|
table_schemas))) {
|
|
LOG_WARN("failed to swap orig and hidden table mlog state", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
new_orig_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL);
|
|
new_hidden_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL);
|
|
new_orig_table_schema.set_table_name(hidden_table_schema->get_table_name_str());
|
|
new_hidden_table_schema.set_table_name(orig_table_schema->get_table_name_str());
|
|
// in prepare_hidden_table_schema, we clear the session id for hidden table of
|
|
// CTAS tmp table. now, data loading stage is finished, we are ready to swap hidden table
|
|
// with CTAS tmp table. we should exchange the session id(and create_host) property to
|
|
// ensure the new CTAS tmp table has correct state.
|
|
if (orig_table_schema->is_ctas_tmp_table() &&
|
|
OB_FAIL(swap_ctas_hidden_table_session_id_(
|
|
*orig_table_schema, *hidden_table_schema, new_orig_table_schema,
|
|
new_hidden_table_schema, ddl_operator, trans))) {
|
|
LOG_WARN("failed to swap ctas hidden table session id", K(ret));
|
|
} else if (OB_FAIL(table_schemas.push_back(new_orig_table_schema)) ||
|
|
OB_FAIL(table_schemas.push_back(new_hidden_table_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
const ObString *ddl_stmt_str = NULL;
|
|
ObTableSchema &tmp_schema = table_schemas.at(i);
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
tmp_schema.set_in_offline_ddl_white_list(true);
|
|
if (tmp_schema.get_table_id() == orig_table_schema->get_table_id()) {
|
|
ddl_stmt_str = &alter_table_arg.ddl_stmt_str_;
|
|
}
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(tmp_schema,
|
|
trans,
|
|
operation_type,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int64_t schema_version = table_schemas[table_schemas.count()-1].get_schema_version();
|
|
if (orig_table_schema->mv_major_refresh()) {
|
|
// for major refresh mv skip modify ddl_info
|
|
// because of we read majoar mv data by specified snapshot
|
|
// last_refresh_scn may less then ddl snapshot
|
|
} else if (OB_FAIL(unbind_hidden_tablets(*orig_table_schema, *hidden_table_schema,
|
|
schema_version, trans))) {
|
|
LOG_WARN("failed to unbind hidden tablets", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(write_ddl_barrier(*hidden_table_schema, trans))) {
|
|
LOG_WARN("failed to write ddl barrier", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_table_arg.mview_refresh_info_.is_mview_complete_refresh_) {
|
|
// update mview table_schema
|
|
const uint64_t mview_table_id = alter_table_arg.mview_refresh_info_.mview_table_id_;
|
|
const ObTableSchema *mview_table_schema = nullptr;
|
|
bool is_based_schema_version_consistent = true;
|
|
int64_t based_schema_version = OB_INVALID_VERSION;
|
|
int64_t max_dependency_version = 0;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, mview_table_id, mview_table_schema))) {
|
|
LOG_WARN("failed to get mview table schema", KR(ret), K(mview_table_id));
|
|
} else if (OB_ISNULL(mview_table_schema)) {
|
|
ret = OB_ERR_MVIEW_NOT_EXIST;
|
|
LOG_WARN("fail to get mview table schema", KR(ret), K(mview_table_id));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && is_based_schema_version_consistent && i < alter_table_arg.based_schema_object_infos_.count(); ++i) {
|
|
const ObBasedSchemaObjectInfo &based_info = alter_table_arg.based_schema_object_infos_.at(i);
|
|
const ObTableSchema *based_table_schema = nullptr;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, based_info.schema_id_,
|
|
based_table_schema))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(based_info));
|
|
} else if (OB_ISNULL(based_table_schema)) {
|
|
// ignore ret
|
|
LOG_WARN("based table is not exist", KR(ret), K(based_info));
|
|
is_based_schema_version_consistent = false;
|
|
} else if (OB_UNLIKELY(based_table_schema->get_schema_version() !=
|
|
based_info.schema_version_)) {
|
|
// ignore ret
|
|
LOG_WARN("based table schema version is changed", KR(ret), K(based_info),
|
|
KPC(based_table_schema));
|
|
is_based_schema_version_consistent = false;
|
|
} else {
|
|
max_dependency_version = MAX(max_dependency_version, based_info.schema_version_);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
HEAP_VAR(ObTableSchema, new_mview_table_schema) {
|
|
if (OB_FAIL(new_mview_table_schema.assign(*mview_table_schema))) {
|
|
LOG_WARN("fail to assign table schema", KR(ret));
|
|
} else {
|
|
new_mview_table_schema.set_data_table_id(alter_table_arg.hidden_table_id_);
|
|
if (is_based_schema_version_consistent) {
|
|
new_mview_table_schema.set_object_status(ObObjectStatus::VALID);
|
|
new_mview_table_schema.set_max_dependency_version(max_dependency_version);
|
|
} else {
|
|
new_mview_table_schema.set_object_status(ObObjectStatus::INVALID);
|
|
new_mview_table_schema.set_max_dependency_version(OB_INVALID_VERSION);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(new_mview_table_schema, trans, operation_type))) {
|
|
LOG_WARN("failed to update mview table schema attribute", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// update dependency
|
|
if (OB_SUCC(ret) && is_based_schema_version_consistent) {
|
|
uint64_t property = 0;
|
|
ObString dep_attrs, dep_reason;
|
|
ObArray<ObDependencyInfo> deps;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ObDependencyInfo::collect_dep_infos(
|
|
alter_table_arg.based_schema_object_infos_, deps, tenant_id,
|
|
ObObjectType::VIEW, mview_table_id, mview_table_id, property, dep_attrs,
|
|
dep_reason, new_schema_version))) {
|
|
LOG_WARN("fail to collect dep infos", KR(ret), K(alter_table_arg));
|
|
} else if (OB_FAIL(ObDependencyInfo::delete_schema_object_dependency(
|
|
trans, tenant_id, mview_table_id, new_schema_version,
|
|
ObObjectType::VIEW))) {
|
|
LOG_WARN("fail to delete schema object dependency", KR(ret), K(mview_table_id));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < deps.count(); ++i) {
|
|
ObDependencyInfo &dep = deps.at(i);
|
|
if (OB_FAIL(dep.insert_schema_object_dependency(trans))) {
|
|
LOG_WARN("fail to insert schema object dependency", KR(ret), K(dep));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObMViewDependencyService mv_dep_service(*schema_service_);
|
|
if (OB_FAIL(mv_dep_service.update_mview_dep_infos(
|
|
trans, schema_guard, tenant_id, mview_table_id, deps))) {
|
|
LOG_WARN("failed to update mview dep infos", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
// update mview_info
|
|
if (OB_SUCC(ret)) {
|
|
const uint64_t last_refresh_scn_val =
|
|
alter_table_arg.mview_refresh_info_.last_refresh_scn_.get_val_for_inner_table_field();
|
|
const uint64_t refresh_scn_val =
|
|
alter_table_arg.mview_refresh_info_.refresh_scn_.get_val_for_inner_table_field();
|
|
const int64_t start_time = alter_table_arg.mview_refresh_info_.start_time_;
|
|
ObMViewInfo mview_info;
|
|
if (OB_FAIL(ObMViewInfo::fetch_mview_info(trans, tenant_id, mview_table_id, mview_info,
|
|
true /*for_update*/, true /*nowait*/))) {
|
|
LOG_WARN("fail to fetch mview info", KR(ret), K(mview_table_id));
|
|
} else if (OB_UNLIKELY(mview_info.get_last_refresh_scn() != last_refresh_scn_val)) {
|
|
ret = OB_VERSION_NOT_MATCH;
|
|
LOG_WARN("mview version is old", KR(ret), K(alter_table_arg.mview_refresh_info_),
|
|
K(mview_info));
|
|
} else {
|
|
DEBUG_SYNC(BEFORE_MV_FINISH_COMPLETE_REFRESH);
|
|
mview_info.set_last_refresh_scn(refresh_scn_val);
|
|
mview_info.set_last_refresh_type(share::schema::ObMVRefreshType::COMPLETE);
|
|
mview_info.set_last_refresh_date(start_time);
|
|
mview_info.set_last_refresh_time((ObTimeUtil::current_time() - start_time) / 1000 / 1000);
|
|
if (OB_FAIL(mview_info.set_last_refresh_trace_id(ObCurTraceId::get_trace_id_str()))) {
|
|
LOG_WARN("fail to set last refresh trace id", KR(ret));
|
|
} else if (OB_FAIL(ObMViewInfo::update_mview_last_refresh_info(trans, mview_info))) {
|
|
LOG_WARN("fail to update mview last refresh info", KR(ret), K(mview_info));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE, alter_table_arg.task_id_))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(alter_table_arg.task_id_));
|
|
} else if (OB_FAIL(ObDDLLock::unlock_for_offline_ddl(tenant_id,
|
|
orig_table_schema->get_table_id(),
|
|
nullptr/*hidden_tablet_ids_alone*/,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to unlock ddl", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
DEBUG_SYNC(SWAP_ORIG_AND_HIDDEN_TABLE_BEFORE_PUBLISH_SCHEMA);
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
if (OB_NO_NEED_UPDATE == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::swap_orig_and_hidden_table_partitions(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
ObPartition** part_array = alter_table_schema.get_part_array();
|
|
ObArray<ObTabletID> tablet_ids;
|
|
ObArray<ObTabletID> inc_tablet_ids;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_NOT_NULL(part_array)) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && (i < alter_table_schema.get_partition_num()); ++i) {
|
|
ObPartition *part = part_array[i];
|
|
if (OB_NOT_NULL(part)) {
|
|
if (OB_FAIL(tablet_ids.push_back(part->get_tablet_id()))) {
|
|
LOG_WARN("failed to add tablet id", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && !tablet_ids.empty()) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const ObTableSchema *orig_table_schema = nullptr;
|
|
const ObTableSchema *hidden_table_schema = nullptr;
|
|
HEAP_VARS_2((ObTableSchema, new_orig_table_schema),
|
|
(ObTableSchema, new_hidden_table_schema)) {
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
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("failed to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg,
|
|
schema_guard,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("failed to get orig and hidden table schema", KR(ret), K(alter_table_arg));
|
|
} else if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("failed to get orig table schema", KR(ret), K(alter_table_arg), KP(orig_table_schema));
|
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("failed to get hidden table schema", KR(ret), K(alter_table_arg), KP(hidden_table_schema));
|
|
} else if (OB_FAIL(new_orig_table_schema.assign(*orig_table_schema))
|
|
|| OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) {
|
|
LOG_WARN("failed to assign schema", KR(ret), KPC(orig_table_schema), KPC(hidden_table_schema));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(check_hidden_table_constraint_exist(hidden_table_schema,
|
|
orig_table_schema,
|
|
schema_guard))) {
|
|
LOG_WARN("failed to check hidden table constraint existence", KR(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && (i < tablet_ids.count()); ++i) {
|
|
const ObTabletID &orig_tablet_id = tablet_ids.at(i);
|
|
int64_t part_idx = OB_INVALID_INDEX;
|
|
int64_t subpart_idx = OB_INVALID_INDEX;
|
|
ObTabletID inc_tablet_id;
|
|
ObObjectID inc_object_id;
|
|
ObObjectID first_level_part_id;
|
|
if (OB_FAIL(orig_table_schema->get_part_idx_by_tablet(orig_tablet_id, part_idx, subpart_idx))) {
|
|
LOG_WARN("failed to get part idx by tablet", KR(ret), K(orig_tablet_id));
|
|
} else if (OB_FAIL(hidden_table_schema->get_tablet_and_object_id_by_index(
|
|
part_idx, subpart_idx, inc_tablet_id, inc_object_id, first_level_part_id))) {
|
|
LOG_WARN("failed to get tablet and object id by index", KR(ret), K(part_idx), K(subpart_idx));
|
|
} else if (OB_FAIL(inc_tablet_ids.push_back(inc_tablet_id))) {
|
|
LOG_WARN("failed to add inc tablet id", KR(ret), K(inc_tablet_id));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t tenant_data_version = 0;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDirectLoadPartitionExchange dl_part_exchange(*this, tenant_data_version);
|
|
new_orig_table_schema.set_in_offline_ddl_white_list(true);
|
|
new_hidden_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (OB_FAIL(dl_part_exchange.exchange_multipart_table_partitions(
|
|
tenant_id,
|
|
trans,
|
|
schema_guard,
|
|
new_orig_table_schema,
|
|
new_hidden_table_schema,
|
|
tablet_ids,
|
|
inc_tablet_ids))) {
|
|
LOG_WARN("failed to exchange multipart table partitions", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObTableLockOwnerID owner_id;
|
|
if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE, alter_table_arg.task_id_))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(alter_table_arg.task_id_));
|
|
} else if (OB_FAIL(ObDDLLock::unlock_for_offline_ddl(tenant_id,
|
|
orig_table_schema->get_table_id(),
|
|
nullptr/*hidden_tablet_ids_alone*/,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to unlock ddl", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
if (OB_NO_NEED_UPDATE == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::swap_orig_and_hidden_table_mlog_state_(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const uint64_t orig_table_mlog_tid,
|
|
const uint64_t hidden_table_mlog_tid,
|
|
ObIArray<ObTableSchema> &table_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObTableSchema *orig_table_mlog_schema = nullptr;
|
|
const ObTableSchema *hidden_table_mlog_schema = nullptr;
|
|
HEAP_VARS_2((ObTableSchema, new_orig_table_mlog_schema),
|
|
(ObTableSchema, new_hidden_table_mlog_schema)) {
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, orig_table_mlog_tid, orig_table_mlog_schema))) {
|
|
LOG_WARN("failed to get orig table mlog schema", K(ret), K(orig_table_mlog_tid));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, hidden_table_mlog_tid, hidden_table_mlog_schema))) {
|
|
LOG_WARN("failed to get hidden table mlog schema", K(ret), K(hidden_table_mlog_tid));
|
|
} else if (OB_ISNULL(orig_table_mlog_schema) || OB_ISNULL(hidden_table_mlog_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("mlog table schema should not be null",
|
|
KR(ret), KP(orig_table_mlog_schema), KP(hidden_table_mlog_schema));
|
|
} else if (OB_FAIL(new_orig_table_mlog_schema.assign(*orig_table_mlog_schema))) {
|
|
LOG_WARN("failed to assign schema", KR(ret));
|
|
} else if (OB_FAIL(new_hidden_table_mlog_schema.assign(*hidden_table_mlog_schema))) {
|
|
LOG_WARN("failed to assign schema", KR(ret));
|
|
} else {
|
|
new_orig_table_mlog_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL);
|
|
new_hidden_table_mlog_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL);
|
|
new_orig_table_mlog_schema.set_table_name(hidden_table_mlog_schema->get_table_name_str());
|
|
new_hidden_table_mlog_schema.set_table_name(orig_table_mlog_schema->get_table_name_str());
|
|
if (OB_FAIL(table_schemas.push_back(new_orig_table_mlog_schema))
|
|
|| OB_FAIL(table_schemas.push_back(new_hidden_table_mlog_schema))) {
|
|
LOG_WARN("failed to add table schemas", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_and_replace_default_index_name_on_demand(
|
|
const bool is_oracle_mode,
|
|
common::ObIAllocator &allocator,
|
|
const ObTableSchema &hidden_data_schema/*without target table name*/,
|
|
const ObString &target_data_table_name,
|
|
ObTableSchema &new_index_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_UNLIKELY(!is_oracle_mode || target_data_table_name.empty() || !new_index_schema.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid args", K(ret), K(is_oracle_mode), K(target_data_table_name), K(new_index_schema));
|
|
} else {
|
|
const bool is_unique_index = new_index_schema.is_unique_index();
|
|
if (new_index_schema.is_sys_generated_name(true/*check_unknown*/)) {
|
|
// the index name is generated by the sys default function, then replace it with new default name.
|
|
ObString new_index_name;
|
|
ObString new_index_name_postfix;
|
|
if (is_unique_index && OB_FAIL(ObTableSchema::create_cons_name_automatically(new_index_name_postfix,
|
|
target_data_table_name, allocator, CONSTRAINT_TYPE_UNIQUE_KEY, is_oracle_mode))) {
|
|
LOG_WARN("create cons name automatically failed", K(ret));
|
|
} else if (!is_unique_index && OB_FAIL(ObTableSchema::create_idx_name_automatically_oracle(
|
|
new_index_name_postfix, target_data_table_name, allocator))) {
|
|
LOG_WARN("create idx name automatically oracle failed", K(ret));
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(
|
|
allocator, hidden_data_schema.get_table_id(), new_index_name_postfix, new_index_name))) {
|
|
LOG_WARN("build_index_table_name failed", K(ret));
|
|
} else if (OB_FAIL(new_index_schema.set_table_name(new_index_name))) {
|
|
LOG_WARN("set new table name failed", K(ret));
|
|
} else {
|
|
new_index_schema.set_name_generated_type(GENERATED_TYPE_SYSTEM);
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_and_replace_dup_constraint_name_on_demand(
|
|
const bool is_oracle_mode,
|
|
ObSchemaGetterGuard &tenant_schema_guard,
|
|
ObTableSchema &hidden_data_schema/*data_table_schema with target normal name*/,
|
|
common::ObIAllocator &allocator,
|
|
ObDDLOperator &ddl_operator,
|
|
ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
// drop all constraints firstly, then check constraint name duplicated, add the constraint finally.
|
|
HEAP_VAR(ObTableSchema, tmp_schema) {
|
|
if (OB_FAIL(tmp_schema.assign(hidden_data_schema))) {
|
|
LOG_WARN("assign schema failed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_table_constraints(hidden_data_schema, tmp_schema/*inc_schema*/,
|
|
hidden_data_schema, trans))) {
|
|
LOG_WARN("drop table all constraints failed", K(ret), K(hidden_data_schema));
|
|
} else {
|
|
hidden_data_schema.clear_constraint();
|
|
// to decide which constraint should be rebuilt and replce its' name on need.
|
|
for (ObTableSchema::const_constraint_iterator iter = tmp_schema.constraint_begin();
|
|
OB_SUCC(ret) && iter != tmp_schema.constraint_end(); ++iter) {
|
|
if (OB_ISNULL(*iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("iter is NULL", K(ret));
|
|
} else {
|
|
ObString new_cst_name;
|
|
ObConstraint new_constraint;
|
|
bool is_constraint_name_exist = false;
|
|
const ObConstraintType &cst_type = (*iter)->get_constraint_type();
|
|
if (OB_FAIL(new_constraint.assign(**iter))) {
|
|
LOG_WARN("assign failed", K(ret));
|
|
} else if (new_constraint.is_sys_generated_name(true/*check_unknown*/)) {
|
|
if (OB_FAIL(ObTableSchema::create_cons_name_automatically(new_cst_name,
|
|
tmp_schema.get_table_name_str(), allocator, cst_type, is_oracle_mode))) {
|
|
LOG_WARN("create cons name automatically failed", K(ret));
|
|
} else if (OB_FAIL(new_constraint.set_constraint_name(new_cst_name))) {
|
|
LOG_WARN("set constraint name failed", K(ret), K(new_cst_name));
|
|
} else {
|
|
new_constraint.set_name_generated_type(GENERATED_TYPE_SYSTEM);
|
|
}
|
|
}
|
|
if (FAILEDx(check_constraint_name_is_exist(tenant_schema_guard,
|
|
tmp_schema, new_constraint.get_constraint_name(), false/*is_foreign_key*/, is_constraint_name_exist))) {
|
|
LOG_WARN("check constraint name is exist failed", K(ret));
|
|
} else if (is_constraint_name_exist) {
|
|
if (CONSTRAINT_TYPE_PRIMARY_KEY == new_constraint.get_constraint_type()) {
|
|
// duplicated primary key name, the recover restore table task should fail finally.
|
|
ret = OB_ERR_CONSTRAINT_NAME_DUPLICATE;
|
|
LOG_WARN("duplicated pk name, should fail the recover restore table task", K(ret), K(new_constraint));
|
|
} else if (CONSTRAINT_TYPE_NOT_NULL == new_constraint.get_constraint_type()) {
|
|
// duplicated not null cst name, should delete the not null flag, and ignore to rebuild the cst.
|
|
const uint64_t column_id = *(new_constraint.cst_col_begin());
|
|
ObColumnSchemaV2 *not_null_column = hidden_data_schema.get_column_schema(column_id);
|
|
if (OB_ISNULL(not_null_column) || OB_UNLIKELY(!not_null_column->has_not_null_constraint())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected status", K(ret), KPC(not_null_column), K(new_constraint), K(hidden_data_schema));
|
|
} else if (OB_FALSE_IT(not_null_column->drop_not_null_cst())) {
|
|
} else if (OB_FAIL(ddl_operator.update_single_column(trans, hidden_data_schema, hidden_data_schema, *not_null_column))) {
|
|
LOG_WARN("update single column failed", K(ret), KPC(not_null_column), K(hidden_data_schema));
|
|
}
|
|
LOG_INFO("duplicated not null name, remove the cst for the restore table task", K(ret), K(new_constraint));
|
|
}
|
|
} else if (OB_FAIL(hidden_data_schema.add_constraint(new_constraint))) {
|
|
LOG_WARN("failed to add constraint", K(ret));
|
|
} else {/* do nothing. */}
|
|
}
|
|
} // end iter constraint.
|
|
}
|
|
}
|
|
if (FAILEDx(ddl_operator.add_table_constraints(hidden_data_schema, hidden_data_schema, trans))) {
|
|
LOG_WARN("add table constraints failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_and_replace_fk_info_on_demand(
|
|
ObSchemaGetterGuard &src_tenant_schema_guard,
|
|
ObSchemaGetterGuard &dst_tenant_schema_guard,
|
|
const ObTableSchema &hidden_table_schema,
|
|
const bool is_recover_child_table,
|
|
ObForeignKeyInfo &new_fk_info)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_cst_name_exist = false;
|
|
const uint64_t src_tenant_id = src_tenant_schema_guard.get_tenant_id();
|
|
const uint64_t dst_tenant_id = dst_tenant_schema_guard.get_tenant_id();
|
|
const ObTableSchema *src_parent_schema = nullptr;
|
|
const ObTableSchema *src_child_schema = nullptr;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_UNLIKELY(src_tenant_id == dst_tenant_id
|
|
|| OB_INVALID_TENANT_ID == src_tenant_id
|
|
|| OB_INVALID_TENANT_ID == dst_tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(src_tenant_id), K(dst_tenant_id));
|
|
} else if (OB_FAIL(src_tenant_schema_guard.get_table_schema(src_tenant_id, new_fk_info.parent_table_id_, src_parent_schema))) {
|
|
LOG_WARN("get child schema failed", K(ret), K(src_tenant_id), K(new_fk_info));
|
|
} else if (OB_ISNULL(src_parent_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("associated parent schema does not exist", K(ret), K(new_fk_info));
|
|
} else if (OB_FAIL(src_tenant_schema_guard.get_table_schema(src_tenant_id, new_fk_info.child_table_id_, src_child_schema))) {
|
|
LOG_WARN("get child schema failed", K(ret), K(src_tenant_id), K(new_fk_info));
|
|
} else if (OB_ISNULL(src_child_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("associated parent schema does not exist", K(ret), K(new_fk_info));
|
|
} else if (src_parent_schema->get_database_id() != src_child_schema->get_database_id()) {
|
|
ret = OB_ERR_CANNOT_ADD_FOREIGN;
|
|
LOG_INFO("child and parent schems are not under same db space, ignore to rebuild", K(ret), K(new_fk_info));
|
|
} else if (new_fk_info.is_parent_table_mock_) {
|
|
ret = OB_ERR_CANNOT_ADD_FOREIGN;
|
|
LOG_INFO("mock fk, ignore to rebuild", K(ret), K(new_fk_info));
|
|
} else if (is_recover_child_table &&
|
|
OB_FAIL(check_constraint_name_is_exist(dst_tenant_schema_guard,
|
|
hidden_table_schema, new_fk_info.foreign_key_name_, true/*is_foreign_key*/, is_cst_name_exist))) {
|
|
LOG_WARN("check cst name exist failed", K(ret), K(new_fk_info));
|
|
} else if (is_cst_name_exist) {
|
|
ret = OB_ERR_CANNOT_ADD_FOREIGN;
|
|
LOG_INFO("fk name exist, ignore to rebuild", K(ret), K(new_fk_info));
|
|
} else {
|
|
// step 1. to find parent table if recover child table now, to find child table if recover parent table now.
|
|
const uint64_t dst_db_id = hidden_table_schema.get_database_id();
|
|
const ObDatabaseSchema *dst_db_schema = nullptr;
|
|
const ObTableSchema *dst_parent_schema = is_recover_child_table ? nullptr : &hidden_table_schema;
|
|
const ObTableSchema *dst_child_schema = is_recover_child_table ? &hidden_table_schema : nullptr;
|
|
const ObString &to_find_schema_name = is_recover_child_table ? src_parent_schema->get_table_name_str() : src_child_schema->get_table_name_str();
|
|
const ObTableSchema *to_find_schema = nullptr;
|
|
if (OB_FAIL(dst_tenant_schema_guard.get_database_schema(dst_tenant_id, dst_db_id, dst_db_schema))) {
|
|
LOG_WARN("get database schema failed", K(ret), K(dst_tenant_id), K(dst_db_id));
|
|
} else if (OB_ISNULL(dst_db_schema)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("database is invalid", K(ret), K(dst_tenant_id), K(dst_db_id));
|
|
} else if (new_fk_info.parent_table_id_ == new_fk_info.child_table_id_) {
|
|
// self reference foreign key.
|
|
// remap table will make table name change, keep self-reference attribute.
|
|
dst_parent_schema = &hidden_table_schema;
|
|
} else if (OB_FAIL(dst_tenant_schema_guard.get_table_schema(dst_tenant_id, dst_db_id,
|
|
to_find_schema_name, false/*is_index*/, to_find_schema, false/*is_hidden*/))) {
|
|
LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id), K(dst_db_id), K(to_find_schema_name));
|
|
} else if (OB_ISNULL(to_find_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("associated parent/child schema does not exist", K(ret), K(dst_tenant_id), K(dst_db_id), K(to_find_schema_name));
|
|
} else {
|
|
dst_parent_schema = is_recover_child_table ? to_find_schema : dst_parent_schema;
|
|
dst_child_schema = is_recover_child_table ? dst_child_schema : to_find_schema;
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_UNLIKELY(nullptr == dst_parent_schema || nullptr == dst_child_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null ptr schema", K(ret), K(is_recover_child_table),
|
|
K(dst_db_id), K(to_find_schema_name), KP(dst_parent_schema), KP(dst_child_schema));
|
|
} else {
|
|
// step 2. to check whether the fk info is valid under the destination tenant space.
|
|
new_fk_info.parent_table_id_ = dst_parent_schema->get_table_id();
|
|
new_fk_info.child_table_id_ = dst_child_schema->get_table_id();
|
|
obrpc::ObCreateForeignKeyArg create_fk_arg;
|
|
ObSchemaChecker schema_checker;
|
|
create_fk_arg.parent_database_ = dst_db_schema->get_database_name_str();
|
|
create_fk_arg.parent_table_ = dst_parent_schema->get_table_name_str();
|
|
create_fk_arg.foreign_key_name_ = new_fk_info.foreign_key_name_;
|
|
create_fk_arg.is_parent_table_mock_ = new_fk_info.is_parent_table_mock_;
|
|
create_fk_arg.update_action_ = new_fk_info.update_action_;
|
|
create_fk_arg.delete_action_ = new_fk_info.delete_action_;
|
|
ARRAY_FOREACH (new_fk_info.parent_column_ids_, idx) {
|
|
const ObColumnSchemaV2 *col = nullptr;
|
|
const uint64_t column_id = new_fk_info.parent_column_ids_.at(idx);
|
|
if (OB_ISNULL(col = dst_parent_schema->get_column_schema(column_id))) {
|
|
ret = OB_ERR_BAD_TABLE;
|
|
LOG_WARN("bad table", K(ret), K(dst_tenant_id), "table_id", dst_parent_schema->get_table_id(), K(column_id));
|
|
} else if (OB_FAIL(create_fk_arg.parent_columns_.push_back(col->get_column_name_str()))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
}
|
|
ARRAY_FOREACH (new_fk_info.child_column_ids_, idx) {
|
|
const ObColumnSchemaV2 *col = nullptr;
|
|
const uint64_t column_id = new_fk_info.child_column_ids_.at(idx);
|
|
if (OB_ISNULL(col = dst_child_schema->get_column_schema(column_id))) {
|
|
ret = OB_ERR_BAD_TABLE;
|
|
LOG_WARN("bad table", K(ret), K(dst_tenant_id), "table_id", dst_child_schema->get_table_id(), K(column_id));
|
|
} else if (OB_FAIL(create_fk_arg.child_columns_.push_back(col->get_column_name_str()))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
}
|
|
if (FAILEDx(schema_checker.init(dst_tenant_schema_guard))) {
|
|
LOG_WARN("init schema checker failed", K(ret));
|
|
} else if (OB_FAIL(check_rebuild_foreign_key_satisfy(create_fk_arg, *dst_parent_schema, *dst_child_schema,
|
|
schema_checker, new_fk_info.ref_cst_type_))) {
|
|
LOG_WARN("check rebuild foreign key satisfy failed", K(ret), K(create_fk_arg));
|
|
} else {
|
|
// used to fetch unique index schema when reference to unique column.
|
|
new_fk_info.ref_cst_id_ = create_fk_arg.ref_cst_id_;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_rebuild_foreign_key_satisfy(
|
|
obrpc::ObCreateForeignKeyArg &create_fk_arg,
|
|
const ObTableSchema &parent_table_schema,
|
|
const ObTableSchema &child_table_schema,
|
|
sql::ObSchemaChecker &schema_checker,
|
|
const ObConstraintType &expected_cst_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
bool is_matched = false;
|
|
const bool is_self_reference = (&parent_table_schema == &child_table_schema);
|
|
ObSArray<ObCreateIndexArg> index_arg_list;
|
|
if (OB_UNLIKELY(!parent_table_schema.is_valid() || !child_table_schema.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(create_fk_arg), K(parent_table_schema), K(child_table_schema));
|
|
} else if (OB_UNLIKELY(create_fk_arg.is_parent_table_mock_)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to recover mock foreign key", K(ret), K(create_fk_arg));
|
|
} else if (OB_UNLIKELY(!parent_table_schema.is_user_table() || !child_table_schema.is_user_table())) {
|
|
ret = OB_ERR_CANNOT_ADD_FOREIGN;
|
|
LOG_WARN("foreign key cannot be based on non-user table", K(ret), "parent_table_is_user_table", parent_table_schema.is_user_table(),
|
|
"child_table_is_user_table", child_table_schema.is_user_table());
|
|
} else if (OB_FAIL(child_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("check if oracle compat mode failed", K(ret));
|
|
} else if (is_self_reference &&
|
|
OB_FAIL(ObResolverUtils::check_self_reference_fk_columns_satisfy(create_fk_arg))) {
|
|
LOG_WARN("check self reference foreign key columns satisfy failed", K(ret), K(create_fk_arg));
|
|
} else if (OB_FAIL(ObResolverUtils::check_foreign_key_columns_type(!is_oracle_mode/*is_mysql_compat_mode*/,
|
|
child_table_schema,
|
|
parent_table_schema,
|
|
create_fk_arg.child_columns_,
|
|
create_fk_arg.parent_columns_,
|
|
nullptr))) {
|
|
LOG_WARN("Failed to check_foreign_key_columns_type", K(ret));
|
|
} else if (OB_FAIL(ObResolverUtils::foreign_key_column_match_uk_pk_column(
|
|
parent_table_schema, schema_checker, create_fk_arg.parent_columns_, index_arg_list/*without initialization is expected*/,
|
|
is_oracle_mode, create_fk_arg.ref_cst_type_, create_fk_arg.ref_cst_id_, is_matched))) {
|
|
LOG_WARN("Failed to check reference columns in parent table");
|
|
} else if (!is_matched || expected_cst_type != create_fk_arg.ref_cst_type_) {
|
|
if (!is_oracle_mode) {
|
|
ret = OB_ERR_CANNOT_ADD_FOREIGN;
|
|
LOG_WARN("reference to pk or uk in parent table failed or cst type mismatched", K(ret),
|
|
K(expected_cst_type), "real_cst_type", create_fk_arg.ref_cst_type_);
|
|
} else { // oracle mode
|
|
ret = OB_ERR_NO_MATCHING_UK_PK_FOR_COL_LIST;
|
|
LOG_WARN("reference to pk or uk in parent table failed or cst type mismatched", K(ret),
|
|
K(expected_cst_type), "real_cst_type", create_fk_arg.ref_cst_type_);
|
|
}
|
|
} else if (OB_FAIL(ObResolverUtils::check_foreign_key_set_null_satisfy(create_fk_arg, child_table_schema, !is_oracle_mode))) {
|
|
LOG_WARN("check fk set null satisfy failed", K(ret), K(create_fk_arg), K(is_oracle_mode));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::make_recover_restore_tables_visible(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
common::ObArenaAllocator allocator(lib::ObLabel("RebuildCST"));
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSArray<uint64_t> unused_cst_ids;
|
|
const uint64_t src_tenant_id = alter_table_arg.alter_table_schema_.get_tenant_id();
|
|
const uint64_t dst_tenant_id = alter_table_arg.exec_tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
bool is_oracle_mode = false;
|
|
const ObTableSchema *orig_table_schema = nullptr;
|
|
const ObTableSchema *hidden_table_schema = nullptr;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObSchemaGetterGuard hold_buf_src_tenant_schema_guard;
|
|
ObSchemaGetterGuard hold_buf_dst_tenant_schema_guard;
|
|
ObSchemaGetterGuard *src_tenant_schema_guard = nullptr;
|
|
ObSchemaGetterGuard *dst_tenant_schema_guard = nullptr;
|
|
hold_buf_src_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
hold_buf_dst_tenant_schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(src_tenant_id, dst_tenant_id,
|
|
hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard,
|
|
src_tenant_schema_guard, dst_tenant_schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(src_tenant_id), K(dst_tenant_id));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg,
|
|
*src_tenant_schema_guard,
|
|
*dst_tenant_schema_guard,
|
|
alter_table_arg.alter_table_schema_,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("failed to get orig and hidden table schema", K(ret));
|
|
} else if (!hidden_table_schema->is_offline_ddl_table()) {
|
|
ret = OB_NO_NEED_UPDATE;
|
|
LOG_WARN("already swapped", K(ret));
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->get_schema_version(dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(dst_tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, dst_tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(dst_tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(hidden_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("check if oracle mode failed", K(ret));
|
|
} else {
|
|
HEAP_VAR(ObTableSchema, tmp_schema) {
|
|
tmp_schema.reset();
|
|
const ObString &new_data_table_name = alter_table_arg.alter_table_schema_.get_table_name_str();
|
|
// make main table take effect.
|
|
if (OB_FAIL(tmp_schema.assign(*hidden_table_schema))) {
|
|
LOG_WARN("assign failed", K(ret));
|
|
} else if (OB_FAIL(tmp_schema.set_table_name(new_data_table_name))) {
|
|
LOG_WARN("set new table name failed", K(ret), K(new_data_table_name));
|
|
} else {
|
|
tmp_schema.set_association_table_id(OB_INVALID_ID);
|
|
tmp_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL);
|
|
tmp_schema.set_in_offline_ddl_white_list(true);
|
|
tmp_schema.set_ddl_ignore_sync_cdc_flag(ObDDLIgnoreSyncCdcFlag::DO_SYNC_LOG_FOR_CDC); // reset.
|
|
ObArray<ObSchemaType> conflict_schema_types;
|
|
uint64_t synonym_id = OB_INVALID_ID;
|
|
bool object_exist = false;
|
|
bool is_data_table_name_exist = false;
|
|
if (OB_FAIL(dst_tenant_schema_guard->check_oracle_object_exist(tmp_schema.get_tenant_id(),
|
|
tmp_schema.get_database_id(), tmp_schema.get_table_name_str(),
|
|
TABLE_SCHEMA, INVALID_ROUTINE_TYPE, false/*if_not_exist_*/, conflict_schema_types))) {
|
|
LOG_WARN("fail to check oracle_object exist", K(ret), K(tmp_schema));
|
|
} else if (OB_UNLIKELY(conflict_schema_types.count() > 0)) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", K(ret), K(tmp_schema), K(conflict_schema_types));
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->check_synonym_exist_with_name(tmp_schema.get_tenant_id(),
|
|
tmp_schema.get_database_id(), tmp_schema.get_table_name_str(), object_exist, synonym_id))) {
|
|
LOG_WARN("fail to check synonym exist", K(ret), K(tmp_schema));
|
|
} else if (object_exist) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", K(ret), K(tmp_schema));
|
|
} else if (OB_FAIL(dst_tenant_schema_guard->check_table_exist(
|
|
tmp_schema.get_tenant_id(), tmp_schema.get_database_id(),
|
|
new_data_table_name, false/*is_index*/, ObSchemaGetterGuard::ALL_NON_HIDDEN_TYPES, is_data_table_name_exist))) {
|
|
LOG_WARN("check table exist failed", K(ret), K(tmp_schema));
|
|
} else if (is_data_table_name_exist) {
|
|
ret = OB_ERR_TABLE_EXIST;
|
|
LOG_WARN("table exist", K(ret), K(tmp_schema));
|
|
} else if (OB_FAIL(check_and_replace_dup_constraint_name_on_demand(is_oracle_mode,
|
|
*dst_tenant_schema_guard, tmp_schema, allocator, ddl_operator, trans))) {
|
|
LOG_WARN("check dup and replace cst name failed", K(ret));
|
|
} else if (OB_FAIL(rebuild_triggers_on_hidden_table(*orig_table_schema, *hidden_table_schema,
|
|
*src_tenant_schema_guard, *dst_tenant_schema_guard, ddl_operator, trans))) {
|
|
LOG_WARN("rebuild triggers failed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_table_attribute(tmp_schema, trans, OB_DDL_ALTER_TABLE/*operation_type*/, nullptr/*ddl_stmt_str*/))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
} else {
|
|
// Notice that, all index have already built and set to normal state flag when rebuild them.
|
|
// TODO yiren, rebuild trigger and foreign key, and check object duplicated too.
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(write_ddl_barrier(*hidden_table_schema, trans))) {
|
|
LOG_WARN("failed to write ddl barrier", K(ret));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
DEBUG_SYNC(SWAP_ORIG_AND_HIDDEN_TABLE_BEFORE_PUBLISH_SCHEMA);
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(dst_tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
ret = OB_SUCCESS == ret ? tmp_ret : ret;
|
|
}
|
|
if (OB_NO_NEED_UPDATE == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_hidden_table_fk_state(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (alter_table_arg.foreign_key_arg_list_.empty()) {
|
|
// do nothing
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool is_oracle_mode = false;
|
|
bool to_recyclebin = false;
|
|
const ObTableSchema *hidden_table_schema = NULL;
|
|
ObTableSchema new_hidden_table_schema;
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, alter_table_arg.hidden_table_id_, hidden_table_schema))) {
|
|
LOG_WARN("get schema failed", K(ret), K(tenant_id), "tid", alter_table_arg.hidden_table_id_);
|
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table not exist", K(ret), K(tenant_id), "tid", alter_table_arg.hidden_table_id_);
|
|
} else if (OB_FAIL(hidden_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to check if oralce compat mode", K(ret));
|
|
} else if (OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
new_hidden_table_schema.reset_foreign_key_infos(); // must reset, to avoid insert all fk records into all_foreign_key.
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.foreign_key_arg_list_.count(); i++) {
|
|
const ObCreateForeignKeyArg &foreign_key_arg = alter_table_arg.foreign_key_arg_list_.at(i);
|
|
if (foreign_key_arg.is_modify_fk_state_) {
|
|
bool is_found = false;
|
|
ObForeignKeyInfo found_foreign_key_info;
|
|
found_foreign_key_info.reset();
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = hidden_table_schema->get_foreign_key_infos();
|
|
for (int64_t j = 0; OB_SUCC(ret) && !is_found && j < foreign_key_infos.count(); ++j) {
|
|
if (is_oracle_mode) {
|
|
// case sensitive.
|
|
is_found = 0 == foreign_key_arg.foreign_key_name_.compare(foreign_key_infos.at(j).foreign_key_name_) ? true : false;
|
|
} else {
|
|
is_found = 0 == foreign_key_arg.foreign_key_name_.case_compare(foreign_key_infos.at(j).foreign_key_name_) ? true : false;
|
|
}
|
|
if (is_found) {
|
|
if (OB_FAIL(found_foreign_key_info.assign(foreign_key_infos.at(j)))) {
|
|
LOG_WARN("fail to assign foreign key info", K(ret));
|
|
} else {
|
|
found_foreign_key_info.is_modify_fk_state_ = true;
|
|
if (foreign_key_arg.is_modify_enable_flag_) {
|
|
found_foreign_key_info.is_modify_enable_flag_ = true;
|
|
found_foreign_key_info.enable_flag_ = foreign_key_arg.enable_flag_;
|
|
}
|
|
if (foreign_key_arg.is_modify_validate_flag_) {
|
|
found_foreign_key_info.is_modify_validate_flag_ = true;
|
|
found_foreign_key_info.validate_flag_ = foreign_key_arg.validate_flag_;
|
|
}
|
|
if (foreign_key_arg.is_modify_rely_flag_) {
|
|
found_foreign_key_info.is_modify_rely_flag_ = true;
|
|
found_foreign_key_info.rely_flag_ = foreign_key_arg.rely_flag_;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (!is_found) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("no such fk constraint", K(ret), K(foreign_key_arg.foreign_key_name_));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (found_foreign_key_info.child_table_id_ != found_foreign_key_info.parent_table_id_) {
|
|
// If the reference table is itself, there is no need to update sync_versin_for_cascade_table
|
|
if (new_hidden_table_schema.get_table_id() != found_foreign_key_info.parent_table_id_
|
|
&& new_hidden_table_schema.get_table_id() != found_foreign_key_info.child_table_id_) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected new hidden table schema", K(ret), K(new_hidden_table_schema), K(found_foreign_key_info));
|
|
} else if (new_hidden_table_schema.get_table_id() == found_foreign_key_info.parent_table_id_
|
|
&& OB_FAIL(new_hidden_table_schema.add_depend_table_id(found_foreign_key_info.child_table_id_))) {
|
|
LOG_WARN("failed to add dependent table id", K(ret), K(found_foreign_key_info));
|
|
} else if (new_hidden_table_schema.get_table_id() == found_foreign_key_info.child_table_id_) {
|
|
if (!found_foreign_key_info.is_parent_table_mock_) {
|
|
if (OB_FAIL(new_hidden_table_schema.add_depend_table_id(found_foreign_key_info.parent_table_id_))) {
|
|
LOG_WARN("failed to add depend table id", K(ret), K(found_foreign_key_info));
|
|
}
|
|
} else if (OB_FAIL(new_hidden_table_schema.add_depend_mock_fk_parent_table_id(found_foreign_key_info.parent_table_id_))) {
|
|
LOG_WARN("failed to add depend table id", K(ret), K(found_foreign_key_info));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(new_hidden_table_schema.add_foreign_key_info(found_foreign_key_info))) {
|
|
LOG_WARN("fail to add foreign key info", K(ret), K(found_foreign_key_info));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
new_hidden_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (OB_FAIL(alter_table_foreign_keys(*hidden_table_schema, new_hidden_table_schema,
|
|
ddl_operator, trans))) {
|
|
LOG_WARN("alter table foreign keys failed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_table_attribute(
|
|
new_hidden_table_schema,
|
|
trans,
|
|
operation_type))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_hidden_table_not_null_column_state(const obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool is_oracle_mode = false;
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
const ObTableSchema *hidden_table_schema = NULL;
|
|
ObTableSchema new_hidden_table_schema;
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, alter_table_arg.hidden_table_id_, hidden_table_schema))) {
|
|
LOG_WARN("failed to and hidden table schema", K(ret));
|
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("hidden table schema is null", K(ret));
|
|
} else if (OB_FAIL(hidden_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to check if oralce compat mode", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
hidden_table_schema->get_association_table_id(),
|
|
orig_table_schema))) {
|
|
LOG_WARN("failed to and hidden table schema", K(ret));
|
|
} else if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig table schema is null", K(ret));
|
|
} else if (OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
ObColumnNameMap col_name_map;
|
|
const ObConstraint *cst = *alter_table_schema.constraint_begin_for_non_const_iter();
|
|
const uint64_t orig_column_id = *(cst->cst_col_begin());
|
|
uint64_t hidden_column_id = 0;
|
|
if (nullptr == orig_table_schema->get_column_schema(orig_column_id)) {
|
|
hidden_column_id = orig_column_id;
|
|
} else if (OB_FAIL(col_name_map.init(*orig_table_schema, *hidden_table_schema, alter_table_schema))) {
|
|
LOG_WARN("failed to init column name map", K(ret));
|
|
} else if (OB_FAIL(get_hidden_table_column_id_by_orig_column_id(*orig_table_schema,
|
|
*hidden_table_schema,
|
|
col_name_map,
|
|
orig_column_id,
|
|
hidden_column_id))) {
|
|
LOG_WARN("failed to get hidden table column id", K(ret), K(orig_column_id));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
const ObColumnSchemaV2 *col_schema = new_hidden_table_schema.get_column_schema(hidden_column_id);
|
|
ObColumnSchemaV2 new_col_schema;
|
|
if (OB_ISNULL(col_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column schema not found", K(ret), K(hidden_column_id));
|
|
} else if (OB_FAIL(new_col_schema.assign(*col_schema))) {
|
|
LOG_WARN("fail to assign column schema", KR(ret));
|
|
} else {
|
|
new_col_schema.set_nullable(false);
|
|
new_col_schema.drop_not_null_cst();
|
|
new_hidden_table_schema.set_in_offline_ddl_white_list(true);
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
bool for_view = false;
|
|
if (OB_FAIL(new_hidden_table_schema.alter_column(new_col_schema, ObTableSchema::CHECK_MODE_ONLINE, for_view))) {
|
|
LOG_WARN("failed to alter column", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column(trans,
|
|
*hidden_table_schema,
|
|
new_hidden_table_schema,
|
|
new_col_schema))) {
|
|
LOG_WARN("failed to update single column", K(ret));
|
|
} else if (!is_oracle_mode && OB_FAIL(ddl_operator.drop_table_constraints(*hidden_table_schema,
|
|
alter_table_schema,
|
|
new_hidden_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failed to drop constraint", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_table_attribute(new_hidden_table_schema, trans, operation_type))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
DEBUG_SYNC(MODIFY_HIDDEN_TABLE_NOT_NULL_COLUMN_STATE_BEFORE_PUBLISH_SCHEMA);
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::restore_the_table_to_split_completed_state(obrpc::ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = alter_table_arg.alter_table_schema_.get_tenant_id();
|
|
const uint64_t dest_tenant_id = alter_table_arg.alter_table_schema_.get_tenant_id();
|
|
const int64_t data_table_id = alter_table_arg.table_id_;
|
|
int64_t new_task_id = 0;
|
|
uint64_t tenant_data_version = 0;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObDDLTaskRecord task_record;
|
|
ObRootService *root_service = GCTX.root_service_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *orig_data_table_schema = nullptr;
|
|
const ObDatabaseSchema *orig_database_schema = nullptr;
|
|
ObArray<int64_t> task_ids;
|
|
ObArray<uint64_t> table_ids;
|
|
ObTableLockOwnerID new_owner_id;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_UNLIKELY(!alter_table_arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("alter_table_arg is invalid", K(ret), K(alter_table_arg));
|
|
} else if (OB_ISNULL(root_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("error unexpected, root service must not be nullptr", K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, orig_data_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_ISNULL(orig_data_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("orig data table schema is null", K(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_UNLIKELY(orig_data_table_schema->is_aux_table())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig data table schema is aux table", K(ret), K(tenant_id), K(data_table_id));
|
|
} else if (OB_INVALID_ID != orig_data_table_schema->get_association_table_id()) {
|
|
LOG_INFO("the two tables offline ddl are being executed.", K(ret), K(orig_data_table_schema->get_association_table_id()), K(orig_data_table_schema->get_table_id()));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, orig_data_table_schema->get_database_id(), orig_database_schema))) {
|
|
LOG_WARN("fail to get orig database schema", K(ret), K(tenant_id), K(orig_data_table_schema->get_database_id()));
|
|
} else if (OB_ISNULL(orig_database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_database_schema is null", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", K(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObDDLTask::fetch_new_task_id(root_service->get_sql_proxy(), tenant_id, new_task_id))) {
|
|
LOG_WARN("fetch new task id failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(new_owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE, new_task_id))) {
|
|
LOG_WARN("failed to get new owner id", K(ret), K(new_task_id));
|
|
} else if (OB_FAIL(ObDDLUtil::get_global_index_table_ids(*orig_data_table_schema, table_ids, schema_guard))) {
|
|
LOG_WARN("fail to get global index table ids", K(ret), K(orig_data_table_schema));
|
|
} else if (OB_FAIL(table_ids.push_back(orig_data_table_schema->get_table_id()))) {
|
|
LOG_WARN("fail to push back", K(ret), K(orig_data_table_schema->get_table_id()));
|
|
} else if (OB_FAIL(ObDDLTaskRecordOperator::get_partition_split_task_ids(trans, tenant_id, table_ids, task_ids))) {
|
|
LOG_WARN("failed to get partition split task ids", K(ret), K(tenant_id), K(table_ids));
|
|
} else {
|
|
//1、设置当前table的处于DDL_PARTITION_SPLIT状态的task任务状态设置为WAIT_PARTITION_SPLIT_RECOVERY_TASK_FINISH, 等待新表完成补数据任务
|
|
//2、依次根据task_id获取task_record, 根据record在ObPartitionSplitTask中初始化, 解除相应ddl锁, 并且替换掉表级锁
|
|
common::ObArenaAllocator allocator;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < task_ids.count(); i++) {
|
|
ObDDLTaskStatus new_status = WAIT_PARTITION_SPLIT_RECOVERY_TASK_FINISH;
|
|
if (OB_FAIL(ObDDLTaskRecordOperator::update_task_status(trans, dest_tenant_id, task_ids.at(i), static_cast<int64_t>(new_status)))) {
|
|
LOG_WARN("update task status failed", K(ret), K(dest_tenant_id), K(task_ids.at(i)), K(new_status));
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < task_ids.count(); i++) {
|
|
const ObTableSchema *table_schema = nullptr;
|
|
ObDDLTaskRecord old_split_task_record;
|
|
ObArray<ObTabletID> src_tablet_ids;
|
|
ObArray<ObTabletID> dst_tablet_ids;
|
|
ObTableLockOwnerID old_owner_id;
|
|
HEAP_VAR(ObPartitionSplitTask, split_task) {
|
|
if (OB_FAIL(ObDDLTaskRecordOperator::get_ddl_task_record(tenant_id,
|
|
task_ids.at(i),
|
|
root_service->get_sql_proxy(),
|
|
allocator,
|
|
old_split_task_record))) {
|
|
LOG_WARN("get ddl task record failed", K(ret), K(tenant_id), K(task_ids.at(i)));
|
|
} else if (OB_FAIL(split_task.init(old_split_task_record))) {
|
|
LOG_WARN("init partition split task failed", K(ret), K(old_split_task_record));
|
|
} else if (OB_FAIL(split_task.get_src_tablet_ids(src_tablet_ids))) {
|
|
LOG_WARN("fail to get all src tablet ids", K(ret));
|
|
} else if (OB_FAIL(split_task.get_dest_tablet_ids(dst_tablet_ids))) {
|
|
LOG_WARN("fail to get all dest tablet ids", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, split_task.get_object_id(), table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(split_task.get_object_id()));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table schema is null", K(ret), K(tenant_id), K(table_schema));
|
|
} else if (OB_FAIL(old_owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE, task_ids.at(i)))) {
|
|
LOG_WARN("failed to get old owner id", K(ret), K(task_ids.at(i)));
|
|
} else if (OB_FAIL(ObDDLLock::replace_lock_for_split_partition(*table_schema,
|
|
src_tablet_ids,
|
|
dst_tablet_ids,
|
|
old_owner_id,
|
|
new_owner_id,
|
|
trans))) {
|
|
LOG_WARN("fail to replace lock for split partition", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
HEAP_VAR(ObTableSchema, new_table_schema) {
|
|
if (OB_FAIL(new_table_schema.assign(*orig_data_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
new_table_schema.set_tenant_id(dest_tenant_id);
|
|
new_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_OFFLINE_DDL);
|
|
new_table_schema.reset_hidden_partition_array();
|
|
bool bind_tablets = true;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(create_user_hidden_table(
|
|
*orig_data_table_schema,
|
|
new_table_schema,
|
|
nullptr,
|
|
bind_tablets,
|
|
schema_guard,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
allocator,
|
|
tenant_data_version))) {
|
|
LOG_WARN("fail to create hidden table", K(ret));
|
|
} else {
|
|
LOG_INFO("create hidden table success!", K(ret));
|
|
common::ObArenaAllocator allocator_for_restore(lib::ObLabel("RestoreSplit"));
|
|
alter_table_arg.task_id_ = new_task_id;
|
|
ObCreateDDLTaskParam param(tenant_id,
|
|
ObDDLType::DDL_PARTITION_SPLIT_RECOVERY_TABLE_REDEFINITION,
|
|
orig_data_table_schema,
|
|
&new_table_schema,
|
|
data_table_id,
|
|
new_table_schema.get_schema_version(),
|
|
alter_table_arg.parallelism_,
|
|
alter_table_arg.consumer_group_id_,
|
|
&allocator_for_restore,
|
|
&alter_table_arg,
|
|
0,
|
|
new_task_id);
|
|
param.tenant_data_version_ = tenant_data_version;
|
|
if (OB_FAIL(root_service->get_ddl_scheduler().create_ddl_task(param, trans, task_record))) {
|
|
LOG_WARN("submit ddl task failed", K(ret));
|
|
} else if (ObTableStateFlag::TABLE_STATE_OFFLINE_DDL == orig_data_table_schema->get_table_state_flag()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("offline ddl is being executed, other ddl operations are not allowed, create hidden table fail", K(ret), K(alter_table_arg));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
} else if (OB_TMP_FAIL(root_service->get_ddl_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record));
|
|
} else {
|
|
LOG_INFO("schedule ddl task success");
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::cleanup_garbage(ObAlterTableArg &alter_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool to_recyclebin = false;
|
|
const ObTableSchema *old_orig_table_schema = NULL;
|
|
const ObTableSchema *old_hidden_table_schema = NULL;
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
const ObTableSchema *hidden_table_schema = NULL;
|
|
HEAP_VARS_2((ObTableSchema, new_orig_table_schema),
|
|
(ObTableSchema, new_hidden_table_schema)) {
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
int64_t refreshed_schema_version = 0;
|
|
bool ddl_succ = false;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, /* get orig_table_schema through its name, thus the hidden_table_schema is orign table schema */
|
|
schema_guard,
|
|
schema_guard,
|
|
alter_table_schema,
|
|
orig_table_schema,
|
|
hidden_table_schema))) {
|
|
LOG_WARN("failed to get orig and hidden table schema", K(ret));
|
|
} else {
|
|
const ObString &orig_table_name = alter_table_arg.alter_table_schema_.get_origin_table_name();
|
|
// ignore case sensitivity is expected.
|
|
if (0 == orig_table_schema->get_table_name_str().case_compare(orig_table_name)) {
|
|
ddl_succ = false;
|
|
} else if (0 == hidden_table_schema->get_table_name_str().case_compare(orig_table_name)) {
|
|
ddl_succ = true;
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected orig and hidden table name", K(ret), K(orig_table_schema->get_table_name_str()),
|
|
K(hidden_table_schema->get_table_name_str()), K(orig_table_name));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (ddl_succ) {
|
|
if (alter_table_arg.is_direct_load_partition_) {
|
|
if (OB_FAIL(new_orig_table_schema.assign(*orig_table_schema))
|
|
|| OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(new_orig_table_schema.assign(*hidden_table_schema))
|
|
|| OB_FAIL(new_hidden_table_schema.assign(*orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
}
|
|
}
|
|
} else {
|
|
if (OB_FAIL(new_orig_table_schema.assign(*orig_table_schema))
|
|
|| OB_FAIL(new_hidden_table_schema.assign(*hidden_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// if the previous step fails, the newly created hidden table should be deleted.
|
|
// if the previous steps are successful, the original table is changed to a hidden table
|
|
// so regardless of success or failure, only need to delete the hidden table
|
|
if (share::PARTITION_SPLIT_RECOVERY_CLEANUP_GARBAGE_TASK != alter_table_arg.ddl_task_type_) {
|
|
new_hidden_table_schema.set_association_table_id(OB_INVALID_ID);
|
|
}
|
|
new_hidden_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (!ddl_succ) {
|
|
if (OB_FAIL(unbind_hidden_tablets(*orig_table_schema, *hidden_table_schema,
|
|
OB_INVALID_VERSION, trans))) {
|
|
LOG_WARN("failed to unbind hidden tablets", K(ret));
|
|
}
|
|
} else {
|
|
// previous step succeeds, no need to unbind
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(lock_table(trans, new_hidden_table_schema))) {
|
|
LOG_WARN("fail to lock_table", KR(ret), K(new_hidden_table_schema));
|
|
} else if (OB_FAIL(drop_table_in_trans(schema_guard,
|
|
new_hidden_table_schema,
|
|
false,
|
|
new_hidden_table_schema.is_index_table(),
|
|
to_recyclebin,
|
|
NULL,
|
|
&trans,
|
|
NULL,
|
|
NULL))) {
|
|
LOG_WARN("failed to drop table in trans", K(ret));
|
|
} else {
|
|
// update table state flag
|
|
ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE;
|
|
new_orig_table_schema.set_association_table_id(OB_INVALID_ID);
|
|
new_orig_table_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_NORMAL);
|
|
new_orig_table_schema.set_in_offline_ddl_white_list(true);
|
|
if (OB_FAIL(ddl_operator.update_table_attribute(
|
|
new_orig_table_schema,
|
|
trans,
|
|
operation_type))) {
|
|
LOG_WARN("failed to update data table schema attribute", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObTableLockOwnerID owner_id;
|
|
if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE,
|
|
alter_table_arg.task_id_))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(alter_table_arg.task_id_));
|
|
} else if (share::PARTITION_SPLIT_RECOVERY_CLEANUP_GARBAGE_TASK == alter_table_arg.ddl_task_type_) {
|
|
ObArray<ObTabletID> tablet_ids;
|
|
const ObPartitionLevel part_level = orig_table_schema->get_part_level();
|
|
const int64_t part_num = orig_table_schema->get_hidden_partition_num();
|
|
ObPartition **part_array = orig_table_schema->get_hidden_part_array();
|
|
if (OB_ISNULL(part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", K(ret), KPC(orig_table_schema));
|
|
} 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("NULL ptr", K(ret), K(i), KPC(orig_table_schema));
|
|
} else if (PARTITION_LEVEL_ONE == part_level) {
|
|
if (OB_FAIL(tablet_ids.push_back(part_array[i]->get_tablet_id()))) {
|
|
LOG_WARN("fail to push back tablet id", K(ret), K(i), K(part_array[i]->get_tablet_id()));
|
|
}
|
|
} else if (PARTITION_LEVEL_TWO == part_level) {
|
|
ObSubPartition **sub_part_array = part_array[i]->get_hidden_subpart_array();
|
|
int64_t sub_part_num = part_array[i]->get_hidden_subpartition_num();
|
|
if (OB_ISNULL(sub_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", K(ret), KPC(orig_table_schema));
|
|
} else {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < sub_part_num; j++) {
|
|
if (OB_ISNULL(sub_part_array[j])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("NULL ptr", K(ret), K(j), KPC(orig_table_schema));
|
|
} else if (OB_FAIL(tablet_ids.push_back(sub_part_array[j]->get_tablet_id()))) {
|
|
LOG_WARN("fail to push back tablet id", K(ret), K(j), K(sub_part_array[j]->get_tablet_id()));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ObDDLLock::unlock_for_offline_ddl(tenant_id,
|
|
orig_table_schema->get_table_id(),
|
|
&tablet_ids,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to unlock ddl", K(ret));
|
|
}
|
|
}
|
|
} else if (OB_FAIL(ObDDLLock::unlock_for_offline_ddl(tenant_id,
|
|
orig_table_schema->get_table_id(),
|
|
nullptr/*hidden_tablet_ids_alone*/,
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to unlock ddl", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(tmp_ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = tmp_ret;
|
|
}
|
|
if (OB_NO_NEED_UPDATE == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_last_schema_version(int64_t &last_schema_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
auto *tsi_oper = GET_TSI(share::schema::TSILastOper);
|
|
if (OB_ISNULL(tsi_oper)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tsi_oper is null", KR(ret));
|
|
} else {
|
|
last_schema_version = tsi_oper->last_operation_schema_version_;
|
|
if (OB_INVALID_VERSION == last_schema_version) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid schema version", KR(ret), K(last_schema_version));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_db_and_table_is_exist(const obrpc::ObTruncateTableArg &arg,
|
|
ObMySQLTransaction &trans,
|
|
uint64_t &database_id,
|
|
uint64_t &table_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSqlString sql;
|
|
uint64_t tenant_id = arg.tenant_id_;
|
|
table_id = OB_INVALID_ID;
|
|
database_id = OB_INVALID_ID;
|
|
ObString database_name = arg.database_name_;
|
|
ObString table_name = arg.table_name_;
|
|
uint64_t tmp_session_id = OB_INVALID_ID;
|
|
uint64_t session_id = arg.session_id_;
|
|
bool is_oracle_mode = lib::Worker::CompatMode::ORACLE == arg.compat_mode_
|
|
? true : false;
|
|
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
|
common::sqlclient::ObMySQLResult *result = NULL;
|
|
bool skip_escape = false;
|
|
// Before checking the table name, we should use mysql mode to escape the table name anyway,
|
|
// otherwise we may not find the table name in select sql
|
|
bool do_oracle_mode_escape = false;
|
|
const char *tmp_table_name = to_cstring(ObHexEscapeSqlStr(table_name, skip_escape, do_oracle_mode_escape));
|
|
const char *tmp_database_name = to_cstring(ObHexEscapeSqlStr(database_name, skip_escape, do_oracle_mode_escape));
|
|
if (OB_ISNULL(tmp_table_name)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table name is NULL", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(tmp_database_name)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database name is NULL", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(sql.assign_fmt("SELECT session_id, a.database_id, table_id, database_name, table_name "
|
|
"FROM %s a JOIN (SELECT session_id, database_id, table_id, table_name FROM %s "
|
|
"UNION ALL SELECT session_id, database_id, table_id, table_name FROM %s WHERE tenant_id = %ld) c "
|
|
"ON a.database_id = c.database_id WHERE a.database_name = '%s' AND table_name = '%s' "
|
|
"AND (session_id = 0 or session_id = %lu) order by session_id desc",
|
|
OB_ALL_DATABASE_TNAME, OB_ALL_TABLE_TNAME,
|
|
OB_ALL_VIRTUAL_CORE_ALL_TABLE_TNAME, tenant_id,
|
|
tmp_database_name, tmp_table_name, session_id))) {
|
|
LOG_WARN("failed assing sql", KR(ret), K(table_name), K(database_name), K(session_id));
|
|
} else if (OB_FAIL(trans.read(res, tenant_id, sql.ptr()))) {
|
|
LOG_WARN("failed to execute sql", KR(ret), K(tenant_id), K(table_name), K(database_name), K(session_id), K(sql));
|
|
} else if (OB_ISNULL(result = res.get_result())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get result", KR(ret), K(tenant_id), K(table_name), K(database_name), K(session_id), K(sql));
|
|
} else {
|
|
ObString new_database_name;
|
|
ObString new_table_name;
|
|
bool not_find_table = true;
|
|
while(OB_SUCC(ret) && OB_SUCC(result->next())) {
|
|
EXTRACT_INT_FIELD_MYSQL(*result, "session_id", tmp_session_id, uint64_t);
|
|
EXTRACT_INT_FIELD_MYSQL(*result, "database_id", database_id, uint64_t);
|
|
EXTRACT_INT_FIELD_MYSQL(*result, "table_id", table_id, uint64_t);
|
|
EXTRACT_VARCHAR_FIELD_MYSQL(*result, "database_name", new_database_name);
|
|
EXTRACT_VARCHAR_FIELD_MYSQL(*result, "table_name", new_table_name);
|
|
if (OB_SUCC(ret) && is_oracle_mode){
|
|
if (0 != new_database_name.compare(database_name)) {
|
|
//do nothing
|
|
} else if (0 != new_table_name.compare(table_name)) {
|
|
//do nothing
|
|
} else {
|
|
if (0 != tmp_session_id) {
|
|
//do nothing
|
|
} else {
|
|
not_find_table = false;
|
|
break;
|
|
}
|
|
}
|
|
} else {
|
|
// If it is MySQL mode, the case is not sensitive,
|
|
// i.e. it can be uniquely determined and the first result can be output directly
|
|
not_find_table = false;
|
|
break;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (not_find_table) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("can not find table",
|
|
KR(ret), K(tenant_id), K(table_name), K(database_name));
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(database_name),
|
|
to_cstring(table_name));
|
|
} else if (0 != tmp_session_id && is_oracle_mode) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate oracle tmp table not supported",
|
|
KR(ret), K(tenant_id), K(table_name), K(database_name));
|
|
}
|
|
} else if (OB_FAIL(ret) && OB_ITER_END == ret) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("can not find table",
|
|
KR(ret), K(tenant_id), K(table_name), K(database_name));
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(database_name),
|
|
to_cstring(table_name));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_and_create_tablet(const int64_t &schema_version,
|
|
const ObIArray<const ObTableSchema*> &orig_table_schemas,
|
|
const ObIArray<ObTableSchema*> &new_table_schemas,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArray<const ObTableSchema*> orig_normal_schemas;
|
|
ObArray<ObTableSchema*> new_normal_schemas;
|
|
|
|
int64_t orig_table_schemas_count = orig_table_schemas.count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_table_schemas_count; ++i) {
|
|
if (orig_table_schemas.at(i)->is_global_index_table()) {
|
|
// global index table need to handle specailly
|
|
ObArray<const ObTableSchema*> tmp_orig_table_schema;
|
|
ObArray<ObTableSchema*> tmp_new_table_schema;
|
|
if (OB_FAIL(tmp_orig_table_schema.push_back(orig_table_schemas.at(i)))) {
|
|
LOG_WARN("fail to push back tmp_orig_table_schema", KR(ret));
|
|
} else if (OB_FAIL(tmp_new_table_schema.push_back(new_table_schemas.at(i)))) {
|
|
LOG_WARN("fail to push back tmp_new_table_schema", KR(ret));
|
|
} else if (OB_FAIL(inner_drop_and_create_tablet_(schema_version, tmp_orig_table_schema, tmp_new_table_schema, trans))) {
|
|
LOG_WARN("fail to drop and create global table's tablet", K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(orig_normal_schemas.push_back(orig_table_schemas.at(i)))) {
|
|
LOG_WARN("fail to push back orig_normal_schemas", KR(ret));
|
|
} else if (OB_FAIL(new_normal_schemas.push_back(new_table_schemas.at(i)))) {
|
|
LOG_WARN("fail to push back new_normal_schemas", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
if (FAILEDx(inner_drop_and_create_tablet_(schema_version, orig_normal_schemas, new_normal_schemas, trans))) {
|
|
LOG_WARN("fail to drop and create normal tables' tablet ", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::inner_drop_and_create_tablet_(const int64_t &schema_version,
|
|
const ObIArray<const ObTableSchema*> &orig_table_schemas,
|
|
const ObIArray<ObTableSchema*> &new_table_schemas,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = OB_INVALID_ID;
|
|
common::ObArray<share::ObLSID> orig_ls_id_array;
|
|
const ObTableSchema *tmp_table_schema = NULL;
|
|
int64_t del_tablet_count = orig_table_schemas.count();
|
|
int64_t create_table_count = new_table_schemas.count();
|
|
common::ObArray<bool> need_create_empty_majors;
|
|
common::ObArray<const ObTableSchema*> create_table_schema_ptrs;
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
|
|
if (0 == del_tablet_count || 0 == create_table_count) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("del or create tablet count can not be zero",
|
|
KR(ret), K(del_tablet_count), K(create_table_count));
|
|
} else if (del_tablet_count != create_table_count) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("delete tablet count is not equal to create tablet count", KR(ret), K(schema_version));
|
|
} else if (OB_ISNULL(orig_table_schemas.at(0))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig table schema first element is NULL", K(ret), K(schema_version));
|
|
} else if (OB_ISNULL(new_table_schemas.at(0))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("new table schema first element is NULL", K(ret), K(schema_version));
|
|
} else if (OB_ISNULL(GCTX.lst_operator_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("GCTX last_operator is NULL", KR(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
tenant_id = orig_table_schemas.at(0)->get_tenant_id();
|
|
LOG_INFO("start to drop_and_create_tablet", K(tenant_id), K(schema_version), K(orig_table_schemas.at(0)->get_table_id()));
|
|
|
|
ObTabletDrop tablet_drop(tenant_id, trans, schema_version);
|
|
// drop tablet
|
|
if (FAILEDx(tablet_drop.init())) {
|
|
LOG_WARN("fail to init tablet_drop", KR(ret), K(tenant_id));
|
|
// get old ls
|
|
} else if (OB_FAIL(tablet_drop.get_ls_from_table(*orig_table_schemas.at(0), orig_ls_id_array))) {
|
|
LOG_WARN("failed to get drop tablet's ls", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(tablet_drop.add_drop_tablets_of_table_arg(orig_table_schemas))) {
|
|
LOG_WARN("fail to add drop tablets table args", KR(ret), K(tenant_id));
|
|
// delete all_tablet_to_ls/insert into all_tablet_to_table_history
|
|
} else if (OB_FAIL(tablet_drop.execute())) {
|
|
LOG_WARN("failed to execute tablet drop", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
tenant_id = new_table_schemas.at(0)->get_tenant_id();
|
|
if (OB_FAIL(create_table_schema_ptrs.reserve(create_table_count))
|
|
|| OB_FAIL(need_create_empty_majors.reserve(create_table_count))) {
|
|
LOG_WARN("fail to reserve create_table_schema_ptrs", KR(ret), K(tenant_id), K(schema_version));
|
|
} else {
|
|
for (int64_t i = 0; i < create_table_count && OB_SUCC(ret); ++i) {
|
|
tmp_table_schema = new_table_schemas.at(i);
|
|
if (OB_ISNULL(tmp_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tmp table schema is NULL", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(create_table_schema_ptrs.push_back(tmp_table_schema))
|
|
|| OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("fail to push back to new_table_schema_ptrs", KR(ret), K(tenant_id), KPC(tmp_table_schema));
|
|
}
|
|
}
|
|
// create tablet
|
|
SCN frozen_scn;
|
|
uint64_t tenant_data_version = 0;
|
|
if (FAILEDx(ObMajorFreezeHelper::get_frozen_scn(tenant_id, frozen_scn))) {
|
|
LOG_WARN("fail to get frozen status for create tablet", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else {
|
|
ObTableCreator table_creator(tenant_id, frozen_scn, trans);
|
|
if (OB_FAIL(table_creator.init(true/*need_check_tablet_cnt*/))) {
|
|
LOG_WARN("table_creator init failed", KR(ret), K(tenant_id));
|
|
} else if (1 == create_table_count && create_table_schema_ptrs.at(0)->is_global_index_table()) {
|
|
if (OB_FAIL(table_creator.add_create_tablets_of_table_arg(*create_table_schema_ptrs.at(0), orig_ls_id_array,
|
|
tenant_data_version, true/*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("fail to add_create_tablets_of_tables_arg", KR(ret), K(tenant_id));
|
|
}
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg(create_table_schema_ptrs, orig_ls_id_array,
|
|
tenant_data_version, need_create_empty_majors/*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("fail to add_create_tablets_of_tables_arg", KR(ret), K(tenant_id));
|
|
}
|
|
if (FAILEDx(table_creator.execute())) {
|
|
LOG_WARN("execute table_creator failed", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
int64_t finish_time = ObTimeUtility::current_time();
|
|
LOG_INFO("finish drop_and_create_tablet", KR(ret), "cost_ts", finish_time - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_table_schemas(const ObIArray<const ObTableSchema*> &orig_table_schemas,
|
|
ObIArray<ObTableSchema*> &new_table_schemas,
|
|
ObIArray<int64_t> &gen_schema_version_array,
|
|
ObArenaAllocator &allocator,
|
|
int64_t &task_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = OB_INVALID;
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
int64_t need_schema_version_count = OB_INVALID_COUNT;
|
|
int64_t orig_table_count = orig_table_schemas.count();
|
|
if (0 == orig_table_count) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_table_schemas count is zero", KR(ret));
|
|
} else if (OB_ISNULL(orig_table_schemas.at(0))){
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig table schema is NULL", KR(ret));
|
|
} else {
|
|
tenant_id = orig_table_schemas.at(0)->get_tenant_id();
|
|
need_schema_version_count = orig_table_schemas.count() + 2;
|
|
// calculate needed schema version count
|
|
// main + index + lob + main_latest + 1503
|
|
if (OB_FAIL(schema_service_->get_ddl_trans_controller()
|
|
.create_task_and_assign_schema_version(tenant_id, need_schema_version_count,
|
|
task_id, gen_schema_version_array))) {
|
|
LOG_WARN("fail to gen batch new schema version", KR(ret), K(tenant_id));
|
|
// check allocated schema version count
|
|
} else if (need_schema_version_count != gen_schema_version_array.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("gen_schema_version_array count is not equal to need_schema_version_count",
|
|
KR(ret), K(tenant_id), K(gen_schema_version_array.count()), K(need_schema_version_count));
|
|
}
|
|
}
|
|
new_table_schemas.reset();
|
|
void *new_schema_ptr = NULL;
|
|
ObTableSchema *new_table_schema = NULL;
|
|
const ObTableSchema *tmp_table_schema = NULL;
|
|
int64_t truncate_version = OB_INVALID_VERSION;
|
|
//use first new schema version as truncate version instead of orig_table_schema schema version
|
|
//orig table schema version as refresh schema low boundary will load more useless info
|
|
if (OB_SUCC(ret) && 0 != gen_schema_version_array.count()) {
|
|
truncate_version = gen_schema_version_array.at(0);
|
|
}
|
|
// construnct new table_schemas
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_table_count; ++i) {
|
|
new_schema_ptr = allocator.alloc(sizeof(ObTableSchema));
|
|
tmp_table_schema = orig_table_schemas.at(i);
|
|
if (OB_ISNULL(new_schema_ptr)) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail alloc memory", KR(ret), K(tenant_id), KP(new_schema_ptr));
|
|
} else if (OB_ISNULL(tmp_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema is NULL", KR(ret), K(tenant_id));
|
|
} else {
|
|
new_table_schema = new (new_schema_ptr)ObTableSchema(&allocator);
|
|
if (OB_FAIL(new_table_schema->assign(*tmp_table_schema))) {
|
|
LOG_WARN("fail to assign orig table schema to new table schema",
|
|
KR(ret), K(tenant_id), K(tmp_table_schema->get_table_id()));
|
|
} else if (i == 0 && FALSE_IT(new_table_schema->set_auto_increment(1))) {
|
|
} else if (FALSE_IT(new_table_schema->set_truncate_version(truncate_version))) {
|
|
} else if (OB_FAIL(new_table_schemas.push_back(new_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema",
|
|
KR(ret), K(tenant_id), K(new_table_schema->get_table_id()));
|
|
}
|
|
}
|
|
}
|
|
// generate tablet_id and object_id for new_table_schema
|
|
int64_t schema_index = 0;
|
|
ObTableSchema *tmp_schema = NULL;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
int64_t new_table_count = new_table_schemas.count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_table_count; ++i) {
|
|
tmp_schema = new_table_schemas.at(i);
|
|
if (OB_ISNULL(tmp_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("new table schema is NULL", KR(ret), K(i));
|
|
} else if (schema_index >= need_schema_version_count) {
|
|
ret = OB_ERROR_OUT_OF_RANGE;
|
|
LOG_WARN("schema_index is out of range", KR(ret), K(schema_index),
|
|
K(need_schema_version_count));
|
|
} else {
|
|
new_schema_version = gen_schema_version_array.at(schema_index++);
|
|
tmp_schema->set_schema_version(new_schema_version);
|
|
// to improve
|
|
if (OB_FAIL(generate_tablet_id(*tmp_schema))) {
|
|
LOG_WARN("fail to fetch new table id",
|
|
KR(ret), K(tmp_schema->get_table_id()), K(new_schema_version));
|
|
} else if (tmp_schema->is_partitioned_table()) {
|
|
// replace object id for schema history recycle
|
|
// if not, only replace tablet id, schema history will miss is_delete flag to recycle
|
|
if (OB_FAIL(generate_object_id_for_partition_schema(*tmp_schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema",
|
|
KR(ret), K(tmp_schema->get_table_id()), K(new_schema_version));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
int64_t finish_time = ObTimeUtility::current_time();
|
|
LOG_INFO("finish generate_table_schema", KR(ret), "cost_ts", finish_time - start_time, K(task_id));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::new_truncate_table_in_trans(const ObIArray<const ObTableSchema*> &orig_table_schemas,
|
|
ObDDLSQLTransaction &trans,
|
|
const ObString *ddl_stmt_str,
|
|
obrpc::ObDDLRes &ddl_res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObString table_name;
|
|
int64_t task_id = OB_INVALID_ID;
|
|
uint64_t tenant_id = OB_INVALID_ID;
|
|
ObArenaAllocator allocator("TruncInTrans");
|
|
int64_t tablet_cost = OB_INVALID_ID;
|
|
int64_t before_fetch_schema = OB_INVALID_ID;
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
ObSchemaService *schema_service = NULL;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (orig_table_schemas.count() < 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schemas have no element", KR(ret));
|
|
} else if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is null", KR(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get schema_service is null", KR(ret));
|
|
} else {
|
|
int64_t schema_index = 0;
|
|
ObArray<ObTableSchema*> new_table_schemas;
|
|
ObArray<int64_t> gen_schema_version_array;
|
|
int64_t first_schema_version = OB_INVALID_VERSION;
|
|
int64_t boundary_schema_version = OB_INVALID_VERSION;
|
|
int64_t latest_table_schema_version = OB_INVALID_VERSION;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_ISNULL(orig_table_schemas.at(0))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig table schema first element is NULL", KR(ret));
|
|
} else {
|
|
tenant_id = orig_table_schemas.at(0)->get_tenant_id();
|
|
table_name = orig_table_schemas.at(0)->get_table_name();
|
|
}
|
|
if (FAILEDx(generate_table_schemas(orig_table_schemas, new_table_schemas,
|
|
gen_schema_version_array, allocator, task_id))) {
|
|
LOG_WARN("fail to generate table schema", KR(ret), K(tenant_id), K(table_name), K(task_id));
|
|
} else if (new_table_schemas.count() != orig_table_schemas.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orig_table schemas count is not equal to new_table_schemas",
|
|
KR(ret), K(table_name), K(new_table_schemas.count()), K(orig_table_schemas.count()));
|
|
} else if (3 > gen_schema_version_array.count()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("gen schema array count is less than 3", KR(ret), K(tenant_id), K(table_name), K(task_id));
|
|
} else {
|
|
int gen_array_count = gen_schema_version_array.count();
|
|
first_schema_version = gen_schema_version_array.at(0);
|
|
boundary_schema_version = gen_schema_version_array.at(gen_array_count - 1);
|
|
latest_table_schema_version = gen_schema_version_array.at(gen_array_count - 2);
|
|
}
|
|
// in mysql mode
|
|
// reinit auto_increment table value
|
|
if (FAILEDx(ddl_operator.reinit_autoinc_row(*new_table_schemas.at(0), trans))) {
|
|
LOG_WARN("fail to reinit autoinc row", KR(ret), K(table_name));
|
|
} else if (OB_FAIL(drop_and_create_tablet(first_schema_version, orig_table_schemas, new_table_schemas, trans))) {
|
|
LOG_WARN("fail to drop or create tablet", KR(ret), K(table_name), K(first_schema_version));
|
|
} else {
|
|
ObTableSchema *new_table_schema = NULL;
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
int64_t table_schema_count = new_table_schemas.count();
|
|
tablet_cost = ObTimeUtility::current_time();
|
|
const ObString *ddl_str = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schema_count; ++i) {
|
|
ddl_str = (i == 0) ? ddl_stmt_str : NULL;
|
|
new_table_schema = new_table_schemas.at(i);
|
|
orig_table_schema = orig_table_schemas.at(i);
|
|
if (OB_ISNULL(new_table_schema) || OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_ERR_NULL_VALUE;
|
|
LOG_WARN("table schema is NULL", KR(ret), K(tenant_id), K(table_name));
|
|
} else if (OB_FAIL(ddl_operator.truncate_table(ddl_str,
|
|
*orig_table_schema,
|
|
*new_table_schema,
|
|
trans))) {
|
|
LOG_WARN("failt to truncate part", KR(ret), K(orig_table_schema->get_table_name()),
|
|
K(orig_table_schema->get_schema_version()), K(new_table_schema->get_schema_version()));
|
|
}
|
|
}
|
|
// upate main table schema version
|
|
ObSchemaOperationType operation_type = OB_DDL_UPDATE_TABLE_SCHEMA_VERSION;
|
|
if (OB_SUCC(ret)) {
|
|
new_table_schemas.at(0)->set_schema_version(latest_table_schema_version);
|
|
if (OB_FAIL(schema_service->get_table_sql_service()
|
|
.update_table_attribute(trans,
|
|
*new_table_schemas.at(0),
|
|
operation_type,
|
|
false))) {
|
|
LOG_WARN("failed to update table schema attribute", KR(ret), K(table_name), K(latest_table_schema_version));
|
|
}// 1503 boundary
|
|
else if (OB_FAIL(ddl_operator.update_boundary_schema_version(tenant_id,
|
|
boundary_schema_version,
|
|
trans))) {
|
|
LOG_WARN("fail to update boundary schema version", KR(ret), K(tenant_id), K(boundary_schema_version));
|
|
}
|
|
}
|
|
} // else
|
|
|
|
before_fetch_schema = ObTimeUtility::current_time();
|
|
LOG_INFO("truncate cost after truncate part and update attribute", KR(ret), "cost_ts", before_fetch_schema - tablet_cost);
|
|
|
|
// serialize increment table schemas
|
|
if (FAILEDx(trans.serialize_inc_schemas(first_schema_version - 1))) {
|
|
LOG_WARN("fail to serialize inc schemas", KR(ret), K(tenant_id), "start_schema_version", first_schema_version - 1);
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ddl_res.tenant_id_ = tenant_id;
|
|
ddl_res.schema_id_ = new_table_schemas.at(0)->get_table_id();
|
|
ddl_res.task_id_ = boundary_schema_version;
|
|
}
|
|
|
|
} // else
|
|
|
|
int64_t before_wait_task = ObTimeUtility::current_time();
|
|
// Serial Submit
|
|
if (FAILEDx(schema_service_->get_ddl_trans_controller().wait_task_ready(tenant_id, task_id, THIS_WORKER.get_timeout_remain()))) {
|
|
LOG_WARN("wait_task_ready", KR(ret), K(table_name), K(task_id));
|
|
}
|
|
int64_t wait_task = ObTimeUtility::current_time();
|
|
const bool is_commit = OB_SUCC(ret);
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(is_commit))) {
|
|
LOG_WARN("trans end failed", K(is_commit), K(temp_ret));
|
|
ret = is_commit ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_INVALID_ID != task_id) {
|
|
int tmp_ret = schema_service_->get_ddl_trans_controller().remove_task(tenant_id, task_id);
|
|
if (OB_SUCCESS != tmp_ret) {
|
|
LOG_WARN("remove_task fail", KR(ret), KR(tmp_ret), K(tenant_id), K(task_id));
|
|
}
|
|
}
|
|
int64_t trans_end = ObTimeUtility::current_time();
|
|
LOG_INFO("truncate cost after truncate_in_trans finish", KR(ret), K(tenant_id) , K(task_id),
|
|
"trans_cost", trans_end - start_time,
|
|
"fetch_schema_cost", before_wait_task - before_fetch_schema,
|
|
"wait_task_cost", wait_task - before_wait_task,
|
|
"trans_end_cost", trans_end - wait_task);
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_index_lob_table_schema(const ObTableSchema &orig_table_schema,
|
|
const ObRefreshSchemaStatus &schema_status,
|
|
common::ObArray<const ObTableSchema*> &table_schemas,
|
|
ObArenaAllocator &allocator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
table_schemas.reset();
|
|
ObArray<uint64_t> table_ids;
|
|
ObSchemaService *schema_service = NULL;
|
|
ObArray<ObAuxTableMetaInfo> simple_index_infos;
|
|
common::ObArray<ObTableSchema*> tmp_table_schemas;
|
|
uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
int64_t schema_version = orig_table_schema.get_schema_version();
|
|
uint64_t database_id = OB_INVALID;
|
|
ObString table_name = orig_table_schema.get_table_name();
|
|
uint64_t orig_database_id = orig_table_schema.get_database_id();
|
|
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is NULL", KR(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get schema service is NULL", KR(ret));
|
|
} else if (OB_FAIL(table_schemas.push_back(&orig_table_schema))) {
|
|
LOG_WARN("fail to push back orign table schema", KR(ret), K(table_name), K(schema_version));
|
|
}// get index table id
|
|
else if (OB_FAIL(orig_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", KR(ret), K(table_name), K(schema_version));
|
|
} else {
|
|
ObIndexType index_type = INDEX_TYPE_IS_NOT;
|
|
ObTableType table_type = MAX_TABLE_TYPE;
|
|
// get all index table id
|
|
int64_t index_count = simple_index_infos.count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < index_count; ++i) {
|
|
index_type = simple_index_infos.at(i).index_type_;
|
|
table_type = simple_index_infos.at(i).table_type_;
|
|
if ((USER_INDEX == table_type) && index_has_tablet(index_type)) {
|
|
if (OB_FAIL(table_ids.push_back(simple_index_infos.at(i).table_id_))) {
|
|
LOG_WARN("failed to push index id to index_ids",
|
|
KR(ret), K(i), K(simple_index_infos.at(i).table_id_), K(table_name), K(schema_version));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// get lob table id
|
|
if (OB_SUCC(ret) && orig_table_schema.has_lob_column()) {
|
|
uint64_t mtid = orig_table_schema.get_aux_lob_meta_tid();
|
|
uint64_t ptid = orig_table_schema.get_aux_lob_piece_tid();
|
|
if (OB_INVALID_ID == mtid || OB_INVALID_ID == ptid) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Expect meta tid and piece tid valid",
|
|
KR(ret), K(table_name), K(schema_version), K(mtid), K(ptid));
|
|
} else if (OB_FAIL(table_ids.push_back(mtid))) {
|
|
LOG_WARN("fail to push back lob meta tid", KR(ret),K(table_name), K(schema_version), K(mtid));
|
|
} else if (OB_FAIL(table_ids.push_back(ptid))) {
|
|
LOG_WARN("fail to push back lob piece tid", KR(ret), K(table_name), K(schema_version), K(ptid));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && 0 != table_ids.count()) {
|
|
// this batch impl lost foreign key and trigger etc.
|
|
if (OB_FAIL(schema_service->get_batch_table_schema(schema_status, schema_version, table_ids,
|
|
trans, allocator, tmp_table_schemas))) {
|
|
LOG_WARN("failed to get batch table schema", KR(ret), K(table_name), K(schema_version));
|
|
} else {
|
|
const ObTableSchema *tmp_schema = NULL;
|
|
ObIndexStatus index_status = INDEX_STATUS_AVAILABLE;
|
|
int64_t tmp_table_schema_count = tmp_table_schemas.count();
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < tmp_table_schema_count; ++i) {
|
|
tmp_schema = tmp_table_schemas.at(i);
|
|
if (OB_ISNULL(tmp_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tmp schema is NULL", KR(ret));
|
|
} else {
|
|
table_name = tmp_schema->get_table_name();
|
|
database_id = tmp_schema->get_database_id();
|
|
index_status = tmp_schema->get_index_status();
|
|
schema_version = tmp_schema->get_schema_version();
|
|
if (orig_database_id != database_id) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("orign table database_id is not equal to index table database_id",
|
|
KR(ret), K(orig_database_id), K(database_id), K(table_name), K(schema_version));
|
|
} else if (tmp_schema->is_index_table() && !is_available_index_status(index_status)) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("index table's index status is not available",
|
|
KR(ret), K(table_name), K(database_id), K(schema_version));
|
|
} else if (OB_FAIL(table_schemas.push_back(tmp_schema))) {
|
|
LOG_WARN("push back schema failed", KR(ret), K(table_name), K(database_id), K(schema_version));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check if the table is foreign_key's parent table
|
|
int ObDDLService::check_is_foreign_key_parent_table(const ObTableSchema &table_schema, ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObRefreshSchemaStatus schema_status;
|
|
schema_status.tenant_id_ = table_schema.get_tenant_id();
|
|
ObArenaAllocator allocator("CheckForeign");
|
|
if (table_schema.is_parent_table()) {
|
|
char *err_description_buf = NULL;
|
|
int64_t err_description_buf_len = OB_MAX_VARCHAR_LENGTH;
|
|
int64_t pos = 0;
|
|
// Check if it is self-referential
|
|
if (table_schema.is_child_table()) {
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = table_schema.get_foreign_key_infos();
|
|
FOREACH_CNT_X(foreign_key_info, foreign_key_infos, OB_SUCC(ret)) {
|
|
if (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_description_buf = static_cast<char *>(allocator.alloc(err_description_buf_len)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
SERVER_LOG(ERROR, "fail to alloc table_def_buf", KR(ret));
|
|
} else if (OB_FAIL(fill_truncate_table_fk_err_msg_without_schema_guard(*foreign_key_info,
|
|
table_schema,
|
|
schema_status,
|
|
trans,
|
|
err_description_buf,
|
|
err_description_buf_len,
|
|
pos))) {
|
|
LOG_WARN("Cannot print err def of truncate table with foreign keys", KR(ret), K(table_schema.get_table_name_str()));
|
|
} else {
|
|
ret = OB_ERR_TRUNCATE_ILLEGAL_FK;
|
|
ObString truncate_table_err_description(static_cast<int32_t>(pos), static_cast<int32_t>(pos), err_description_buf);
|
|
LOG_USER_ERROR(OB_ERR_TRUNCATE_ILLEGAL_FK, truncate_table_err_description.length(), truncate_table_err_description.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 = table_schema.get_foreign_key_infos();
|
|
FOREACH_CNT_X(foreign_key_info, foreign_key_infos, OB_SUCC(ret)) {
|
|
if (table_schema.get_table_id() == foreign_key_info->parent_table_id_) {
|
|
if (OB_UNLIKELY(NULL == (err_description_buf = static_cast<char *>(allocator.alloc(err_description_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_without_schema_guard(*foreign_key_info,
|
|
table_schema,
|
|
schema_status,
|
|
trans,
|
|
err_description_buf,
|
|
err_description_buf_len,
|
|
pos))) {
|
|
LOG_WARN("Cannot print err def of truncate table with foreign keys", KR(ret), K(table_schema.get_table_name_str()));
|
|
} else {
|
|
ret = OB_ERR_TRUNCATE_ILLEGAL_FK;
|
|
ObString truncate_table_err_description(static_cast<int32_t>(pos), static_cast<int32_t>(pos), err_description_buf);
|
|
LOG_USER_ERROR(OB_ERR_TRUNCATE_ILLEGAL_FK, truncate_table_err_description.length(), truncate_table_err_description.ptr());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_table_schema_is_legal(const obrpc::ObTruncateTableArg &arg,
|
|
const ObDatabaseSchema & database_schema,
|
|
const ObTableSchema &table_schema,
|
|
const bool check_foreign_key,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t table_id = table_schema.get_table_id();
|
|
ObString table_name = table_schema.get_table_name();
|
|
ObString database_name = database_schema.get_database_name();
|
|
|
|
if (OB_UNLIKELY(table_schema.get_database_id() != database_schema.get_database_id())) {
|
|
ret = OB_ERR_PARALLEL_DDL_CONFLICT;
|
|
LOG_WARN("table databse id not equal to database schema", KR(ret),
|
|
K(table_schema.get_database_id()),
|
|
K(database_schema.get_database_id()));
|
|
} else if (OB_UNLIKELY(database_schema.get_database_name_str() != arg.database_name_)) {
|
|
ret = OB_ERR_PARALLEL_DDL_CONFLICT;
|
|
LOG_WARN("database_schema's database name not equal to arg", KR(ret), K(database_schema.get_database_name_str()), K_(arg.database_name));
|
|
} else if (table_schema.is_in_recyclebin() || database_schema.is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not truncate table in recyclebin",
|
|
KR(ret), K(table_name), K(table_id), K(database_name));
|
|
} else if (table_schema.is_user_table() || table_schema.is_mysql_tmp_table()) {
|
|
if (table_schema.required_by_mview_refresh()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate table required by materialized view refresh is not supported",
|
|
KR(ret), K(table_schema), K(table_id));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate table required by materialized view refresh is");
|
|
} else if (check_foreign_key &&
|
|
OB_FAIL(check_is_foreign_key_parent_table(table_schema, trans))) {
|
|
LOG_WARN("failed to check table is foreign key's parent table", KR(ret), K(table_name), K(table_id));
|
|
}
|
|
} else if (0 != table_schema.get_autoinc_column_id()) {
|
|
ret = OB_ERR_PARALLEL_DDL_CONFLICT;
|
|
LOG_WARN("table with autoinc column should not get in new_truncate_table", KR(ret), K(table_id), K(table_name), K(database_name));
|
|
} else if (table_schema.is_sys_table() || table_schema.is_external_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate table is not supported on system table", KR(ret), K(table_id), K(table_name));
|
|
} else if (table_schema.is_index_table() || table_schema.is_aux_vp_table() || table_schema.is_aux_lob_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate table is not supported on index or aux vp table", KR(ret) ,K(table_name), K(table_id));
|
|
} else if (table_schema.is_mlog_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate materialized view log is not supported",
|
|
KR(ret), K(table_name), K(table_id));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate materialized view log is");
|
|
} else if (table_schema.is_materialized_view()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate materialized view is not supported",
|
|
KR(ret), K(table_name), K(table_id));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate materialized view is");
|
|
} else if (table_schema.mv_container_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate materialized view container table is not supported",
|
|
KR(ret), K(table_name), K(table_id));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate materialized view container table is");
|
|
} else {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("truncate table not exist", KR(ret), K(table_name), K(table_id), K(database_name));
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(database_name), to_cstring(table_name));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_parallel_ddl_conflict(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const obrpc::ObDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t schema_version = OB_INVALID_VERSION;
|
|
|
|
if (arg.is_need_check_based_schema_objects()) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && (i < arg.based_schema_object_infos_.count()); ++i) {
|
|
const ObBasedSchemaObjectInfo &info = arg.based_schema_object_infos_.at(i);
|
|
if (OB_FAIL(schema_guard.get_schema_version(
|
|
info.schema_type_,
|
|
info.schema_tenant_id_ == OB_INVALID_TENANT_ID ? arg.exec_tenant_id_: info.schema_tenant_id_,
|
|
info.schema_id_,
|
|
schema_version))) {
|
|
LOG_WARN("failed to get_schema_version", K(ret), K(arg.exec_tenant_id_), K(info));
|
|
} else if (OB_INVALID_VERSION == schema_version) {
|
|
ret = OB_ERR_PARALLEL_DDL_CONFLICT;
|
|
LOG_WARN("schema_version is OB_INVALID_VERSION", K(ret), K(info));
|
|
} else if (schema_version != info.schema_version_) {
|
|
ret = OB_ERR_PARALLEL_DDL_CONFLICT;
|
|
LOG_WARN("schema_version is not equal to info.schema_version_", K(ret), K(schema_version), K(info));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
int ObDDLService::new_truncate_table(const obrpc::ObTruncateTableArg &arg,
|
|
obrpc::ObDDLRes &ddl_res,
|
|
const SCN &frozen_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t database_id = OB_INVALID_ID;
|
|
uint64_t table_id = OB_INVALID_ID;
|
|
// if need 1503 boundary
|
|
bool need_end_signal = false;
|
|
bool enable_query_stash = false;
|
|
bool enable_ddl_parallel = true;
|
|
bool with_snapshot = false;
|
|
//unused
|
|
int64_t fake_schema_version = 1000;
|
|
ObArenaAllocator allocator("Truncate");
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
observer::ObInnerSQLConnection *conn = NULL;
|
|
ObDDLSQLTransaction trans(schema_service_, need_end_signal, enable_query_stash, enable_ddl_parallel);
|
|
ObSchemaService *schema_service = NULL;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", KR(ret));
|
|
} else if (OB_INVALID_ID == tenant_id || arg.database_name_.empty()
|
|
|| arg.table_name_.empty()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(arg));
|
|
//tenant share lock
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, fake_schema_version, with_snapshot))) {
|
|
LOG_WARN("failed to start trans", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(conn = dynamic_cast<observer::ObInnerSQLConnection *>
|
|
(trans.get_connection()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("trans conn is NULL", KR(ret), K(arg));
|
|
// To verify the existence of database and table
|
|
} else if (OB_FAIL(ObDDLHelper::obj_lock_database_name(trans, tenant_id, arg.database_name_, transaction::tablelock::SHARE))) {
|
|
LOG_WARN("fail to lock database name", KR(ret), K(tenant_id), K_(arg.database_name));
|
|
} else if (OB_FAIL(ObDDLHelper::obj_lock_obj_name(trans, tenant_id, arg.database_name_, arg.table_name_, transaction::tablelock::EXCLUSIVE))) {
|
|
LOG_WARN("fail to lock table name", KR(ret), K(tenant_id), K_(arg.database_name), K_(arg.table_name));
|
|
} else if (OB_FAIL(check_db_and_table_is_exist(arg, trans, database_id, table_id))) {
|
|
LOG_WARN("failed to check database and table exist", KR(ret), K(arg.database_name_), K(arg.table_name_));
|
|
} else if (OB_FAIL(ObDDLHelper::obj_lock_obj_id(trans, tenant_id, database_id, transaction::tablelock::SHARE))) {
|
|
LOG_WARN("fail to lock databse id", KR(ret), K(tenant_id), K(database_id));
|
|
} else {
|
|
// table lock
|
|
ObTableSchema orig_table_schema;
|
|
ObDatabaseSchema orig_db_schema;
|
|
ObArray<const ObTableSchema*> table_schema_array;
|
|
ObArray<ObDatabaseSchema> database_schema_array;
|
|
ObRefreshSchemaStatus schema_status;
|
|
schema_status.tenant_id_ = tenant_id;
|
|
int64_t before_table_lock = ObTimeUtility::current_time();
|
|
bool lock_table_not_allow = false;
|
|
LOG_INFO("truncate cost after trans start, lock database name, lock table name, and check_db_table_is_exist", KR(ret), "cost_ts", before_table_lock - start_time);
|
|
// try lock
|
|
if (OB_FAIL(ObDDLHelper::obj_lock_obj_id(trans, tenant_id, table_id, transaction::tablelock::EXCLUSIVE))) {
|
|
LOG_WARN("fail to lock table id", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_FAIL(ObInnerConnectionLockUtil::lock_table(tenant_id,
|
|
table_id,
|
|
EXCLUSIVE,
|
|
0,
|
|
conn))) {
|
|
LOG_WARN("failed to lock table", KR(ret), K(arg.table_name_), K(table_id));
|
|
// for error code convert
|
|
if (OB_OP_NOT_ALLOW == ret) {
|
|
ret = OB_SUCCESS;
|
|
lock_table_not_allow = true;
|
|
}
|
|
}
|
|
int64_t after_table_lock = ObTimeUtility::current_time();
|
|
LOG_INFO("truncate cost after lock table id and lock table", KR(ret), "cost_ts", after_table_lock - before_table_lock);
|
|
if (FAILEDx(schema_service->get_db_schema_from_inner_table(schema_status, database_id, database_schema_array, trans))){
|
|
LOG_WARN("fail to get database schema", KR(ret), K(arg.database_name_), K(database_id));
|
|
// get table full scehma
|
|
} else if (OB_FAIL(schema_service->get_full_table_schema_from_inner_table(schema_status, table_id, orig_table_schema, allocator, trans))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(arg.table_name_), K(table_id));
|
|
} else if (OB_FAIL(check_table_schema_is_legal(arg, database_schema_array.at(0), orig_table_schema, arg.foreign_key_checks_, trans))) {
|
|
LOG_WARN("failed to check table schema is legal",
|
|
KR(ret), K(arg.table_name_), K(table_id), K(orig_table_schema.get_schema_version()));
|
|
} else if (lock_table_not_allow) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("fail to lock table", KR(ret), K(orig_table_schema));
|
|
}// get index and lob schema
|
|
else if (OB_FAIL(get_index_lob_table_schema(orig_table_schema, schema_status,
|
|
table_schema_array, allocator, trans))) {
|
|
LOG_WARN("fail to get index or lob table schema",
|
|
KR(ret), K(arg.table_name_), K(table_id), K(orig_table_schema.get_schema_version()));
|
|
}
|
|
int64_t after_get_schema = ObTimeUtility::current_time();
|
|
LOG_INFO("truncate cost after get schema and check legal",
|
|
KR(ret), "cost_ts", after_get_schema - after_table_lock);
|
|
if (FAILEDx(new_truncate_table_in_trans(table_schema_array, trans, &arg.ddl_stmt_str_, ddl_res))) {
|
|
LOG_WARN("truncate table in trans failed",
|
|
KR(ret), K(arg.table_name_), K(table_id), K(orig_table_schema.get_schema_version()));
|
|
}
|
|
int64_t finish_truncate_table = ObTimeUtility::current_time();
|
|
LOG_INFO("truncate cost after finish truncate", KR(ret), K(tenant_id), K(table_id), "cost_ts", finish_truncate_table - start_time);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::truncate_table(const ObTruncateTableArg &arg,
|
|
const obrpc::ObDDLRes &ddl_res,
|
|
const SCN &frozen_scn)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(arg.session_id_);
|
|
ObSchemaService *schema_service = NULL;
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
uint64_t tenant_id = arg.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else if (OB_INVALID_ID == tenant_id || arg.database_name_.empty()
|
|
|| arg.table_name_.empty()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(arg), K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
bool is_index = false;
|
|
bool is_db_in_recyclebin = false;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, arg.database_name_,
|
|
arg.table_name_, is_index, orig_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(arg.database_name_), K(arg.table_name_));
|
|
} else if (NULL == orig_table_schema) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(arg.database_name_), to_cstring(arg.table_name_));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
orig_table_schema->get_database_id(), is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), K(*orig_table_schema));
|
|
} else if (orig_table_schema->is_in_recyclebin() || is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("Can not truncate table in recyclebin", K(ret), K(arg), K(*orig_table_schema));
|
|
} else if(orig_table_schema->is_index_table() || orig_table_schema->is_aux_vp_table() || orig_table_schema->is_aux_lob_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate table is not supported on index or aux vp table", K(ret));
|
|
} else if (orig_table_schema->is_materialized_view()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate materialized view is not supported",
|
|
KR(ret), K(*orig_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate materialized view is");
|
|
} else if (orig_table_schema->mv_container_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate materialized view container table is not supported",
|
|
KR(ret), K(*orig_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate materialized view container table is");
|
|
} else if (orig_table_schema->is_mlog_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate materialized view log is not supported", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate materialized view log is");
|
|
} else if (!orig_table_schema->is_user_table() && !orig_table_schema->is_tmp_table()) {
|
|
if (orig_table_schema->is_sys_table() || orig_table_schema->is_external_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->required_by_mview_refresh()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate table required by materialized view refresh is not supported",
|
|
KR(ret), KPC(orig_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "truncate table with materialized view log is");
|
|
} else if (!orig_table_schema->check_can_do_ddl()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("offline ddl is being executed, other ddl operations are not allowed",
|
|
K(orig_table_schema), K(ret));
|
|
} else { // else-start
|
|
|
|
if (OB_SUCC(ret) && arg.foreign_key_checks_) {
|
|
// 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());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// lock table when drop data table
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(lock_table(trans, *orig_table_schema))) {
|
|
LOG_WARN("fail to lock_table", KR(ret), KPC(orig_table_schema));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && orig_table_schema->is_oracle_tmp_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("truncate temp table not supported on RS", K(ret));
|
|
} else if (OB_SUCC(ret)) {
|
|
ObSArray<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(tenant_id, new_table_id))) {
|
|
LOG_WARN("failed to fetch_new_table_id", K(ret));
|
|
} else {
|
|
new_table_schema.set_table_id(new_table_id);
|
|
// reset auto increment after truncate table
|
|
new_table_schema.set_auto_increment(1);
|
|
if (orig_table_schema->is_primary_vp_table()) {
|
|
// After truncate table, the new table is still a VP table
|
|
new_table_schema.set_data_table_id(new_table_id);
|
|
}
|
|
if (OB_FAIL(table_schemas.push_back(new_table_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
//reconstruct index schema
|
|
if (OB_SUCC(ret)) {
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
if (OB_FAIL(orig_table_schema->get_simple_index_infos(
|
|
simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", simple_index_infos.at(i).table_id_, K(ret));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else {
|
|
ObString index_name;
|
|
ObString new_index_table_name;
|
|
ObTableSchema new_index_schema;
|
|
if (OB_FAIL(new_index_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(ObTableSchema::get_index_name(allocator,
|
|
orig_table_schema->get_table_id(), index_table_schema->get_table_name_str(),
|
|
index_name))) {
|
|
const ObString &index_table_name = index_table_schema->get_table_name_str();
|
|
uint64_t orig_tid = orig_table_schema->get_table_id();
|
|
LOG_WARN("error get index table name failed",
|
|
K(orig_tid), K(index_table_name), K(ret));
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator,
|
|
new_table_id,
|
|
index_name,
|
|
new_index_table_name))) {
|
|
LOG_WARN("failed to build new index table name!", K(new_table_id),
|
|
K(new_index_table_name), K(ret));
|
|
} else {
|
|
uint64_t new_idx_tid = OB_INVALID_ID;
|
|
if (OB_FAIL(schema_service->fetch_new_table_id(tenant_id, new_idx_tid))) {
|
|
LOG_WARN("failed to fetch_new_table_id", K(ret));
|
|
} else {
|
|
new_index_schema.set_table_id(new_idx_tid);
|
|
new_index_schema.set_data_table_id(new_table_id);
|
|
// The index table generated by truncate is empty, set directly to AVAILABLE
|
|
new_index_schema.set_index_status(INDEX_STATUS_AVAILABLE);
|
|
if (new_index_schema.is_in_recyclebin()) {
|
|
new_index_schema.set_table_name(index_table_schema->get_table_name_str());
|
|
ObArray<ObRecycleObject> recycle_objs;
|
|
ObRecycleObject::RecycleObjType recycle_type = ObRecycleObject::get_type_by_table_schema(new_index_schema);
|
|
if (OB_INVALID_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is invalid", K(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_recycle_object(
|
|
tenant_id,
|
|
new_index_schema.get_table_name_str(),
|
|
recycle_type,
|
|
trans,
|
|
recycle_objs))) {
|
|
LOG_WARN("get recycle object failed", K(tenant_id), K(ret));
|
|
} else if (recycle_objs.size() != 1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected recycle object num", K(ret),
|
|
"table_name", new_index_schema.get_table_name_str(),
|
|
"size", recycle_objs.size());
|
|
} else {
|
|
ObRecycleObject &recycle_obj = recycle_objs.at(0);
|
|
if (OB_FAIL(index_recycle_objs.push_back(recycle_obj))) {
|
|
LOG_WARN("fail to push back recyclebin object", KR(ret), K(recycle_obj));
|
|
}
|
|
}
|
|
} else {
|
|
new_index_schema.set_table_name(new_index_table_name);
|
|
}
|
|
if (OB_FAIL(table_schemas.push_back(new_index_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} //end for
|
|
}
|
|
|
|
if (OB_SUCC(ret) && new_table_schema.has_lob_column()) {
|
|
ObLobMetaBuilder lob_meta_builder(*this);
|
|
ObLobPieceBuilder lob_data_builder(*this);
|
|
const uint64_t new_table_id = OB_INVALID_ID;
|
|
ObTableSchema lob_meta_schema;
|
|
ObTableSchema lob_piece_schema;
|
|
if (OB_FAIL(lob_meta_builder.generate_aux_lob_meta_schema(
|
|
schema_service, new_table_schema, new_table_id, lob_meta_schema, false))) {
|
|
LOG_WARN("generate_schema for lob meta table failed", K(new_table_schema), K(ret));
|
|
} else if (OB_FAIL(table_schemas.push_back(lob_meta_schema))) {
|
|
LOG_WARN("push_back lob meta table failed", K(ret));
|
|
} else if (OB_FAIL(lob_data_builder.generate_aux_lob_piece_schema(
|
|
schema_service, new_table_schema, new_table_id, lob_piece_schema, false))) {
|
|
LOG_WARN("generate_schema for lob data table failed", K(new_table_schema), K(ret));
|
|
} else if (OB_FAIL(table_schemas.push_back(lob_piece_schema))) {
|
|
LOG_WARN("push_back lob data table failed", K(ret));
|
|
} else {
|
|
table_schemas.at(0).set_aux_lob_meta_tid(lob_meta_schema.get_table_id());
|
|
table_schemas.at(0).set_aux_lob_piece_tid(lob_piece_schema.get_table_id());
|
|
}
|
|
}
|
|
|
|
if (FAILEDx(generate_object_id_for_partition_schemas(table_schemas))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(table_schemas));
|
|
} else if (OB_FAIL(generate_tables_tablet_id(table_schemas))) {
|
|
LOG_WARN("failed to generate_tables_id", KR(ret), K(table_schemas));
|
|
}
|
|
|
|
// rebuild VP table
|
|
if (OB_SUCC(ret) && orig_table_schema->is_primary_vp_table()) {
|
|
ObSEArray<uint64_t, 16> aux_vp_tid_array;
|
|
if (OB_FAIL(orig_table_schema->get_aux_vp_tid_array(aux_vp_tid_array))) {
|
|
LOG_WARN("get_aux_tid_array failed", K(ret));
|
|
}
|
|
int64_t N = aux_vp_tid_array.count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
|
|
const ObTableSchema *aux_vp_table_schema = NULL;
|
|
uint64_t new_aux_vp_tid = OB_INVALID_ID;
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, aux_vp_tid_array.at(i), aux_vp_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", aux_vp_tid_array.at(i), K(ret));
|
|
} else if (OB_ISNULL(aux_vp_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(
|
|
tenant_id, new_aux_vp_tid))) {
|
|
LOG_WARN("failed to fetch_new_table_id", K(ret));
|
|
} else {
|
|
ObVertialPartitionBuilder vp_builder(*this);
|
|
const int64_t buf_size = 64;
|
|
char buf[buf_size];
|
|
MEMSET(buf, 0, buf_size);
|
|
int64_t pos = 0;
|
|
if (OB_FAIL(vp_builder.generate_vp_table_name(new_aux_vp_tid, buf, buf_size, pos))) {
|
|
LOG_WARN("failed to generate_vp_table_name", K(ret), K(new_table_id));
|
|
} else {
|
|
ObString aux_vp_table_name(pos, buf);
|
|
ObTableSchema new_aux_vp_schema;
|
|
if (OB_FAIL(new_aux_vp_schema.assign(*aux_vp_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else {
|
|
new_aux_vp_schema.set_table_id(new_aux_vp_tid);
|
|
new_aux_vp_schema.set_table_name(aux_vp_table_name);
|
|
new_aux_vp_schema.set_data_table_id(new_table_id);
|
|
if (OB_FAIL(table_schemas.push_back(new_aux_vp_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
//check table
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(truncate_table_in_trans(arg, *orig_table_schema, table_schemas, index_recycle_objs, schema_guard, trans, &arg.ddl_stmt_str_, arg.database_name_))) {
|
|
LOG_WARN("truncate table in trans failed", K(ret));
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
} // else-end
|
|
}
|
|
}
|
|
allocator.clear();
|
|
return ret;
|
|
}
|
|
|
|
// description: when error code is OB_ERR_TRUNCATE_ILLEGAL_FK, print error
|
|
// Because there are very detailed error messages in mysql mode when truncate a parent table with foreign key constraints,
|
|
// so in order to be compatible with mysql, the following interface is used to generate error messages
|
|
//
|
|
// @param [in] foreign_key_info ObForeignKeyInfo
|
|
// @param [in] schema_guard ObSchemaGetterGuard
|
|
// @param [in] buf print buff
|
|
// @param [in] buf_len OB_MAX_VARCHAR_LENGTH
|
|
// @param [in] pos eror info last byte offset
|
|
//
|
|
// @return oceanbase error code defined in lib/ob_errno.def
|
|
int ObDDLService::fill_truncate_table_fk_err_msg(const ObForeignKeyInfo &foreign_key_info,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &parent_table_schema,
|
|
char *buf,
|
|
const int64_t &buf_len,
|
|
int64_t &pos) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaPrinter schema_printer(schema_guard);
|
|
const ObDatabaseSchema *parent_db_schema = NULL;
|
|
const ObDatabaseSchema *child_db_schema = NULL;
|
|
const ObTableSchema *child_table_schema = NULL;
|
|
const uint64_t tenant_id = parent_table_schema.get_tenant_id();
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, foreign_key_info.child_table_id_, child_table_schema))) {
|
|
LOG_WARN("fail to get child table schema", K(ret), K(tenant_id), K(foreign_key_info.child_table_id_));
|
|
} else if (NULL == child_table_schema) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unknow table", K(ret), K(foreign_key_info.parent_table_id_));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, child_table_schema->get_database_id(), child_db_schema))) {
|
|
LOG_WARN("failed to get database", K(ret), K(tenant_id), K(child_table_schema->get_database_id()));
|
|
} else if (NULL == child_db_schema) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unknow database", K(ret), K(child_table_schema->get_database_id()));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "(`%s`.`%s`, ", child_db_schema->get_database_name(),
|
|
child_table_schema->get_table_name()))) {
|
|
// print "(`child_db_name`.`child_table_name`, "
|
|
LOG_WARN("fail to print database and table name", K(ret),
|
|
K(child_db_schema->get_database_name_str()), K(child_table_schema->get_table_name_str()));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "CONSTRAINT "))) {
|
|
// print "CONSTRAINT "
|
|
LOG_WARN("fail to print CONSTRAINT", K(ret));
|
|
} else if (!foreign_key_info.foreign_key_name_.empty() &&
|
|
OB_FAIL(databuff_printf(buf, buf_len, pos, "`%.*s` ",
|
|
foreign_key_info.foreign_key_name_.length(),
|
|
foreign_key_info.foreign_key_name_.ptr()))) {
|
|
// print "`constraint_name` "
|
|
LOG_WARN("fail to print foreign key name", K(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "FOREIGN KEY ("))) {
|
|
// print "FOREIGN KEY ("
|
|
LOG_WARN("fail to print FOREIGN KEY (", K(ret));
|
|
} else if (OB_FAIL(schema_printer.print_column_list(*child_table_schema, foreign_key_info.child_column_ids_, buf, buf_len, pos))) {
|
|
// print "child table foreign key column list", eg: "`c1`, `c2`"
|
|
LOG_WARN("fail to print_column_list", K(ret), K(child_table_schema->get_table_name_str()));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, ") REFERENCES "))) {
|
|
// print ") REFERENCES "
|
|
LOG_WARN("fail to print ) REFERENCES ", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, parent_table_schema.get_database_id(), parent_db_schema))) {
|
|
SHARE_SCHEMA_LOG(WARN, "failed to get database", K(ret), K(tenant_id), K(parent_table_schema.get_database_id()));
|
|
} else if(OB_FAIL(databuff_printf(buf, buf_len, pos, "`%s`.`%s` (", parent_db_schema->get_database_name(), parent_table_schema.get_table_name()))) {
|
|
// print "`child_db_name`.`child_table_name`"
|
|
LOG_WARN("fail to print parent_database_name.parent_table_name", K(ret));
|
|
} else if (OB_FAIL(schema_printer.print_column_list(parent_table_schema, foreign_key_info.parent_column_ids_, buf, buf_len, pos))) {
|
|
// print "parent table foreign key column list" and "))", eg: "`c1`, `c2`))"
|
|
LOG_WARN("fail to print_column_list", K(ret), K(parent_table_schema.get_table_name_str()));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "))"))) {
|
|
LOG_WARN("fail to print ) ", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fill_truncate_table_fk_err_msg_without_schema_guard(const ObForeignKeyInfo &foreign_key_info,
|
|
const ObTableSchema &parent_table_schema,
|
|
const ObRefreshSchemaStatus &schema_status,
|
|
ObMySQLTransaction &trans,
|
|
char *buf,
|
|
const int64_t &buf_len,
|
|
int64_t &pos) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObSchemaFetcher schema_fetcher;
|
|
ObSchemaPrinter schema_printer(schema_guard);
|
|
ObArenaAllocator allocator("FillFkErrMsg");
|
|
ObTableSchema child_table_schema;
|
|
ObArray<ObDatabaseSchema> child_db_array;
|
|
ObArray<ObDatabaseSchema> parent_db_array;
|
|
const int64_t table_id = foreign_key_info.child_table_id_;
|
|
const uint64_t tenant_id = parent_table_schema.get_tenant_id();
|
|
ObSchemaService *schema_service = NULL;
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is NULL", KR(ret));
|
|
} else if (OB_ISNULL(schema_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get schema service is NULL", KR(ret));
|
|
} else if (OB_FAIL(schema_service->get_full_table_schema_from_inner_table(schema_status, table_id,
|
|
child_table_schema, allocator, trans))) {
|
|
LOG_WARN("fail to get child table schema", KR(ret), K(tenant_id), K(foreign_key_info.child_table_id_));
|
|
} else if (OB_FAIL(schema_service->get_db_schema_from_inner_table(schema_status, child_table_schema.get_database_id(), child_db_array, trans))) {
|
|
LOG_WARN("failed to get database", KR(ret), K(tenant_id), K(child_table_schema.get_database_id()));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "(`%s`.`%s`, ", child_db_array.at(0).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", KR(ret),
|
|
K(child_db_array.at(0).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", KR(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", KR(ret));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "FOREIGN KEY ("))) {
|
|
// print "FOREIGN KEY ("
|
|
LOG_WARN("fail to print FOREIGN KEY (", KR(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", KR(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 ", KR(ret));
|
|
} else if (OB_FAIL(schema_service->get_db_schema_from_inner_table(schema_status, parent_table_schema.get_database_id(), parent_db_array, trans))) {
|
|
LOG_WARN("failed to get database schema", KR(ret), K(tenant_id), K(parent_table_schema.get_database_id()));
|
|
} else if(OB_FAIL(databuff_printf(buf, buf_len, pos, "`%s`.`%s` (", parent_db_array.at(0).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", KR(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", KR(ret), K(parent_table_schema.get_table_name_str()));
|
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "))"))) {
|
|
LOG_WARN("fail to print ) ", KR(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
//the first schema in new_schemas is data table schema
|
|
//other is the index table schema
|
|
//only used create table like
|
|
int ObDDLService::rebuild_table_schema_with_new_id(const ObTableSchema &orig_table_schema,
|
|
const ObDatabaseSchema &new_database_schema,
|
|
const ObString &new_table_name,
|
|
const ObString &create_host,
|
|
const int64_t session_id,
|
|
const share::schema::ObTableType table_type_,
|
|
ObSchemaService &schema_service,
|
|
ObIArray<ObTableSchema> &new_schemas,
|
|
ObArenaAllocator &allocator,
|
|
const uint64_t define_user_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t new_table_id = OB_INVALID_ID;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
|
|
ObTableSchema new_table_schema;
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(new_table_schema.assign(orig_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(schema_service.fetch_new_table_id(orig_table_schema.get_tenant_id(),
|
|
new_table_id))) {
|
|
LOG_WARN("failed to fetch new table id", K(ret), K(orig_table_schema.get_tenant_id()));
|
|
} else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to check_if_oracle_compat_mode", K(ret), K(orig_table_schema.get_tenant_id()));
|
|
} else {
|
|
new_table_schema.set_table_id(new_table_id);
|
|
new_table_schema.set_table_name(new_table_name);
|
|
new_table_schema.set_database_id(new_database_schema.get_database_id());
|
|
new_table_schema.set_define_user_id(define_user_id);
|
|
new_table_schema.reset_foreign_key_infos();
|
|
new_table_schema.reset_trigger_list();
|
|
if (new_table_schema.has_constraint()) {
|
|
// reset check constraint name in mysql mode
|
|
ObTableSchema::const_constraint_iterator iter = new_table_schema.constraint_begin();
|
|
ObTableSchema::const_constraint_iterator iter_last = iter;
|
|
ObString new_constraint_name;
|
|
bool is_constraint_name_exist = false;
|
|
for (; OB_SUCC(ret) && iter != new_table_schema.constraint_end();++iter) {
|
|
(*iter)->set_table_id(new_table_id);
|
|
(*iter)->set_tenant_id(tenant_id);
|
|
do {
|
|
if (OB_FAIL(ObTableSchema::create_cons_name_automatically(
|
|
new_constraint_name, new_table_name, allocator,
|
|
(*iter)->get_constraint_type(), is_oracle_mode))) {
|
|
SQL_RESV_LOG(WARN, "create cons name automatically failed", K(ret));
|
|
} else if (OB_UNLIKELY(0 == new_constraint_name.case_compare((*iter_last)->get_constraint_name_str()))) {
|
|
is_constraint_name_exist = true;
|
|
} else if (OB_FAIL(check_constraint_name_is_exist(
|
|
schema_guard, new_table_schema, new_constraint_name, false, is_constraint_name_exist))) {
|
|
LOG_WARN("fail to check check constraint name is exist or not", K(ret), K(new_constraint_name));
|
|
}
|
|
} while (OB_SUCC(ret) && is_constraint_name_exist);
|
|
if (OB_SUCC(ret)) {
|
|
(*iter)->set_constraint_name(new_constraint_name);
|
|
(*iter)->set_name_generated_type(GENERATED_TYPE_SYSTEM);
|
|
}
|
|
iter_last = iter;
|
|
}
|
|
}
|
|
if (orig_table_schema.is_sys_table() || orig_table_schema.is_vir_table()) {
|
|
new_table_schema.set_table_type(USER_TABLE);
|
|
} else if (orig_table_schema.is_sys_view()) {
|
|
new_table_schema.set_table_type(USER_VIEW);
|
|
} else if (orig_table_schema.is_external_table()) {
|
|
new_table_schema.set_table_type(EXTERNAL_TABLE);
|
|
}
|
|
if (new_table_schema.is_user_table()
|
|
&& (TMP_TABLE == table_type_ || TMP_TABLE_ORA_SESS == table_type_)) {
|
|
new_table_schema.set_table_type(table_type_);
|
|
new_table_schema.set_create_host(create_host);
|
|
new_table_schema.set_sess_active_time(ObTimeUtility::current_time());
|
|
new_table_schema.set_session_id(session_id);
|
|
}
|
|
if (orig_table_schema.is_primary_vp_table()) {
|
|
new_table_schema.set_data_table_id(0); // VP not support
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(new_schemas.push_back(new_table_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
//reconstruct index schema
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
if (OB_FAIL(new_table_schema.get_simple_index_infos(
|
|
simple_index_infos))) {
|
|
LOG_WARN("get simple_index_infos failed", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < simple_index_infos.count(); ++i) {
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, simple_index_infos.at(i).table_id_, index_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret), K(tenant_id), "table id", simple_index_infos.at(i).table_id_);
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (index_table_schema->is_in_recyclebin()) {
|
|
continue;
|
|
} else {
|
|
ObString index_name;
|
|
ObString new_index_table_name;
|
|
HEAP_VAR(ObTableSchema, new_index_schema) {
|
|
if (OB_FAIL(new_index_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign schema", K(ret));
|
|
} else if (OB_FAIL(ObTableSchema::get_index_name(allocator,
|
|
orig_table_schema.get_table_id(), index_table_schema->get_table_name_str(),
|
|
index_name))) {
|
|
const ObString &index_table_name = index_table_schema->get_table_name_str();
|
|
uint64_t orig_tid = orig_table_schema.get_table_id();
|
|
LOG_WARN("error get index table name failed",
|
|
K(orig_tid), K(index_table_name), K(ret));
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator,
|
|
new_table_id,
|
|
index_name,
|
|
new_index_table_name))) {
|
|
LOG_WARN("failed to build new index table name!", K(new_table_id),
|
|
K(new_index_table_name), K(ret));
|
|
} else {
|
|
uint64_t new_idx_tid = OB_INVALID_ID;
|
|
if (OB_FAIL(schema_service.fetch_new_table_id(orig_table_schema.get_tenant_id(),
|
|
new_idx_tid))) {
|
|
LOG_WARN("failed to fetch_new_table_id", K(ret));
|
|
} else {
|
|
new_index_schema.set_table_id(new_idx_tid);
|
|
new_index_schema.set_data_table_id(new_table_id);
|
|
new_index_schema.set_table_name(new_index_table_name);
|
|
new_index_schema.set_database_id(new_database_schema.get_database_id());
|
|
//create table like, index always is valid
|
|
new_index_schema.set_index_status(INDEX_STATUS_AVAILABLE);
|
|
if (OB_FAIL(new_schemas.push_back(new_index_schema))) {
|
|
LOG_WARN("failed to add table schema!", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} //end for
|
|
|
|
if (OB_SUCC(ret) && new_table_schema.has_lob_column()) {
|
|
ObLobMetaBuilder lob_meta_builder(*this);
|
|
ObLobPieceBuilder lob_data_builder(*this);
|
|
const uint64_t new_table_id = OB_INVALID_ID;
|
|
HEAP_VARS_2((ObTableSchema, lob_meta_schema), (ObTableSchema, lob_piece_schema)) {
|
|
if (OB_FAIL(lob_meta_builder.generate_aux_lob_meta_schema(
|
|
&schema_service, new_table_schema, new_table_id, lob_meta_schema, false))) {
|
|
LOG_WARN("generate_schema for lob meta table failed", K(new_table_schema), K(ret));
|
|
} else if (OB_FAIL(new_schemas.push_back(lob_meta_schema))) {
|
|
LOG_WARN("push_back lob meta table failed", K(ret));
|
|
} else if (OB_FAIL(lob_data_builder.generate_aux_lob_piece_schema(
|
|
&schema_service, new_table_schema, new_table_id, lob_piece_schema, false))) {
|
|
LOG_WARN("generate_schema for lob data table failed", K(new_table_schema), K(ret));
|
|
} else if (OB_FAIL(new_schemas.push_back(lob_piece_schema))) {
|
|
LOG_WARN("push_back lob data table failed", K(ret));
|
|
} else {
|
|
new_schemas.at(0).set_aux_lob_meta_tid(lob_meta_schema.get_table_id());
|
|
new_schemas.at(0).set_aux_lob_piece_tid(lob_piece_schema.get_table_id());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_table_like(const ObCreateTableLikeArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObSchemaService *schema_service = NULL;
|
|
bool object_exist = false;
|
|
uint64_t synonym_id = OB_INVALID_ID;
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
uint64_t tenant_id = arg.tenant_id_;
|
|
schema_guard.set_session_id(arg.session_id_);
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
int64_t ddl_task_id = 0;
|
|
if (arg.sequence_ddl_arg_.get_stmt_type() != common::OB_INVALID_ID) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("create table like not suppotted identity column", K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else if (OB_INVALID_ID == tenant_id || arg.origin_db_name_.empty()
|
|
|| arg.origin_table_name_.empty()
|
|
|| arg.new_db_name_.empty()
|
|
|| arg.new_table_name_.empty()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(arg), K(ret));
|
|
} else {
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
const ObTableSchema *new_table_schema = NULL;
|
|
const ObDatabaseSchema *new_db_schema = NULL;
|
|
bool is_db_in_recyclebin = false;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
arg.origin_db_name_,
|
|
arg.origin_table_name_,
|
|
false,
|
|
orig_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(arg), K(tenant_id));
|
|
} else if (NULL == orig_table_schema) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(arg.origin_db_name_), to_cstring(arg.origin_table_name_));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(
|
|
tenant_id, orig_table_schema->get_database_id(), is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id), K(*orig_table_schema));
|
|
} else if (orig_table_schema->is_in_recyclebin() || is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can create table like table in recyclebin", K(ret),
|
|
K(*orig_table_schema), K(is_db_in_recyclebin));
|
|
} else if (!orig_table_schema->is_user_table() && !orig_table_schema->is_sys_table()) {
|
|
ret = OB_ERR_WRONG_OBJECT;
|
|
LOG_USER_ERROR(OB_ERR_WRONG_OBJECT, to_cstring(arg.origin_db_name_), to_cstring(arg.origin_table_name_),
|
|
"BASE TABLE");
|
|
} else if (is_inner_table(orig_table_schema->get_table_id())) {
|
|
// tablegroup of system table is oceanbase,
|
|
// Including the user table in it may cause some unexpected problems, please ban it here
|
|
//
|
|
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 (orig_table_schema->required_by_mview_refresh()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN(
|
|
"create table like on table required by materialized view refresh is not supported",
|
|
KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED,
|
|
"create table like on table required by materialized view refresh is");
|
|
} 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;
|
|
share::schema::ObErrorInfo error_info;
|
|
if (OB_FAIL(rebuild_table_schema_with_new_id(*orig_table_schema,
|
|
*new_db_schema,
|
|
arg.new_table_name_,
|
|
arg.create_host_,
|
|
arg.session_id_,
|
|
arg.table_type_,
|
|
*schema_service,
|
|
table_schemas,
|
|
allocator,
|
|
arg.define_user_id_))) {
|
|
LOG_WARN("failed to rebuild table schema with new id", KR(ret));
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schemas(table_schemas))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(table_schemas));
|
|
} else if (OB_FAIL(generate_tables_tablet_id(table_schemas))) {
|
|
LOG_WARN("failed to generate_tables_id", KR(ret), K(table_schemas));
|
|
}
|
|
if (OB_SUCC(ret)) { // deal with mock parent table
|
|
const ObMockFKParentTableSchema *ori_mock_parent_table_schema_ptr = NULL;
|
|
if (OB_FAIL(schema_guard.get_mock_fk_parent_table_schema_with_name(
|
|
new_db_schema->get_tenant_id(),
|
|
new_db_schema->get_database_id(),
|
|
arg.new_table_name_,
|
|
ori_mock_parent_table_schema_ptr))) {
|
|
SQL_RESV_LOG(WARN, "failed to check_mock_fk_parent_table_exist_with_name", K(ret));
|
|
} else if (OB_NOT_NULL(ori_mock_parent_table_schema_ptr)) {
|
|
// will replace mock_parent_table with this new table
|
|
ObMockFKParentTableSchema mock_fk_parent_table_schema;
|
|
ObArray<const share::schema::ObTableSchema*> index_schemas;
|
|
for (int64_t i = 1; OB_SUCC(ret) && i < table_schemas.count(); ++i) {
|
|
if (table_schemas.at(i).is_unique_index()
|
|
&& OB_FAIL(index_schemas.push_back(&table_schemas.at(i)))) {
|
|
LOG_WARN("failed to push back index_schemas", K(ret));
|
|
}
|
|
}
|
|
if (FAILEDx(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table(
|
|
schema_guard, ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id(),
|
|
table_schemas.at(0), index_schemas, mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table",
|
|
K(ret), K(ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id()));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema));
|
|
}
|
|
}
|
|
}
|
|
if (FAILEDx(create_user_tables(arg.if_not_exist_,
|
|
arg.ddl_stmt_str_,
|
|
error_info,
|
|
table_schemas,
|
|
schema_guard,
|
|
arg.sequence_ddl_arg_,
|
|
0,
|
|
NULL,
|
|
mock_fk_parent_table_schema_array,
|
|
ddl_task_id))) {
|
|
LOG_WARN("failed to create user tables");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
allocator.clear();
|
|
return ret;
|
|
}
|
|
|
|
// Delete index information and write to system tables in a transaction.
|
|
// If sql_trans is NULL, you need to create a transaction inside the function
|
|
int ObDDLService::drop_table_in_trans(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &table_schema,
|
|
const bool is_rebuild_index,
|
|
const bool is_index,
|
|
const bool to_recyclebin,
|
|
const ObString *ddl_stmt_str,
|
|
ObMySQLTransaction *sql_trans,
|
|
DropTableIdHashSet *drop_table_set,
|
|
ObMockFKParentTableSchema *mock_fk_parent_table_ptr, /* will use it when drop a fk_parent_table */
|
|
const bool delete_priv)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
UNUSED(is_index);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", KR(ret));
|
|
}
|
|
//do drop table in trans
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction tmp_trans(schema_service_);
|
|
ObMySQLTransaction &trans = OB_ISNULL(sql_trans) ? tmp_trans : *sql_trans;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(sql_trans)
|
|
&& OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (table_schema.is_materialized_view()) {
|
|
// drop mv container table's index
|
|
uint64_t container_table_id = table_schema.get_data_table_id();
|
|
const ObTableSchema *container_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, container_table_id, container_table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), K(container_table_id));
|
|
} else if (OB_ISNULL(container_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("container table schema should not be null",
|
|
KR(ret), K(tenant_id), K(container_table_id));
|
|
} else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard,
|
|
*container_table_schema, USER_INDEX, false)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
} else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard,
|
|
*container_table_schema, AUX_LOB_META, false)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
} else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard,
|
|
*container_table_schema, AUX_LOB_PIECE, false)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
} else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard, // drop mv container table
|
|
table_schema, MATERIALIZED_VIEW, false)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
const uint64_t mview_table_id = table_schema.get_table_id();
|
|
ObMViewDependencyService mv_dep_service(*schema_service_);
|
|
if (OB_FAIL(mv_dep_service.remove_mview_dep_infos(
|
|
trans, schema_guard, tenant_id, mview_table_id))) {
|
|
LOG_WARN("failed to remove mview dep infos", 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));
|
|
} else if (table_schema.mv_container_table()
|
|
&& OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard,
|
|
table_schema, MATERIALIZED_VIEW_LOG, to_recyclebin)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
} else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard,
|
|
table_schema, AUX_VERTIAL_PARTITION_TABLE, to_recyclebin)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
} else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard,
|
|
table_schema, AUX_LOB_META, to_recyclebin)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
} else if (OB_FAIL((drop_aux_table_in_drop_table(trans, ddl_operator, schema_guard,
|
|
table_schema, AUX_LOB_PIECE, to_recyclebin)))) {
|
|
LOG_WARN("drop_aux_table_in_drop_table failed", KR(ret));
|
|
} else if (OB_FAIL(drop_trigger_in_drop_table(trans, ddl_operator, schema_guard,
|
|
table_schema, to_recyclebin))) {
|
|
LOG_WARN("drop_trigger_in_drop_table failed", KR(ret));
|
|
}
|
|
}
|
|
|
|
// delete error info added when create force view
|
|
if (OB_SUCC(ret) && table_schema.is_user_view()) {
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check if tenant mode is oracle mode", K(ret));
|
|
} else if (is_oracle_mode) {
|
|
ObErrorInfo error_info;
|
|
error_info.set_obj_id(table_schema.get_table_id());
|
|
error_info.set_obj_type(static_cast<uint64_t>(ObObjectType::VIEW));
|
|
error_info.set_database_id(table_schema.get_database_id());
|
|
error_info.set_tenant_id(table_schema.get_tenant_id());
|
|
error_info.set_schema_version(table_schema.get_schema_version());
|
|
error_info.set_error_status(ERROR_STATUS_NO_ERROR);// set as no error to delete exists error
|
|
if (OB_FAIL(error_info.handle_error_info(trans, NULL))) {
|
|
LOG_WARN("insert create error info failed.", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (to_recyclebin && !table_schema.is_index_table()
|
|
&& !is_inner_table(table_schema.get_table_id())
|
|
&& !table_schema.is_aux_lob_table()
|
|
&& !table_schema.is_aux_vp_table()
|
|
&& !table_schema.is_materialized_view()
|
|
&& !table_schema.is_mlog_table()) { //index/aux_vp/aux_lob/mview table and inner table will drop directly
|
|
if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(table_schema,
|
|
schema_guard,
|
|
trans,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("drop table to recyclebin failed", KR(ret));
|
|
}
|
|
} else {
|
|
if (!is_rebuild_index && table_schema.is_index_table()) {
|
|
ObTableSchema tmp_schema;
|
|
tmp_schema.reset();
|
|
if (OB_FAIL(ddl_operator.drop_inner_generated_index_column(
|
|
trans, schema_guard, table_schema, tmp_schema))) {
|
|
LOG_WARN("drop inner generated index column failed", KR(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && OB_FAIL(ddl_operator.drop_table(
|
|
table_schema, trans, ddl_stmt_str,
|
|
false/*is_truncate_table*/, drop_table_set, false, delete_priv))) {
|
|
LOG_WARN("ddl_operator drop_table failed", K(table_schema), KR(ret));
|
|
}
|
|
}
|
|
}
|
|
// deal with mock_fk_parent_table in drop_table begin
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
// If table dropped is child table which has mock parent tables, will gen_mock_fk_parent_tables_for_drop_fks
|
|
if (OB_SUCC(ret) && table_schema.get_foreign_key_real_count() > 0) {
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = table_schema.get_foreign_key_infos();
|
|
ObArray<const ObForeignKeyInfo*> parent_table_mock_foreign_key_info_array;
|
|
ObArray<const ObMockFKParentTableSchema*> mock_fk_parent_table_schema_ptr_array;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i);
|
|
if (foreign_key_info.is_parent_table_mock_) {
|
|
// TODO:@xiaofeng.lby, delete this restriction,
|
|
if (OB_NOT_NULL(drop_table_set)) {
|
|
if (drop_table_set->size() > 1) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("drop multi tables with mock fks in one sql is not supported ", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(parent_table_mock_foreign_key_info_array.push_back(&foreign_key_info))) {
|
|
LOG_WARN("fail to push back to parent_table_mock_foreign_key_info_array", K(ret), K(foreign_key_info));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !parent_table_mock_foreign_key_info_array.empty()) {
|
|
if (OB_FAIL(prepare_gen_mock_fk_parent_tables_for_drop_fks(
|
|
schema_guard, table_schema.get_tenant_id(),
|
|
parent_table_mock_foreign_key_info_array,
|
|
mock_fk_parent_table_schema_ptr_array,
|
|
mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("failed to prepare_gen_mock_fk_parent_tables_for_drop_fks", K(ret));
|
|
} else if (!mock_fk_parent_table_schema_array.empty()){
|
|
if (OB_FAIL(gen_mock_fk_parent_tables_for_drop_fks(
|
|
schema_guard, mock_fk_parent_table_schema_ptr_array, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("failed to gen_mock_fk_parent_tables_for_drop_fks", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// If table dropped is real parent table, create mock fk parent table
|
|
if (OB_SUCC(ret) && OB_NOT_NULL(mock_fk_parent_table_ptr)) {
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema version", K(ret), K(tenant_id));
|
|
} else if (FALSE_IT(mock_fk_parent_table_ptr->set_schema_version(new_schema_version))) {
|
|
} else if (FALSE_IT(mock_fk_parent_table_ptr->set_operation_type(MOCK_FK_PARENT_TABLE_OP_CREATE_TABLE_BY_DROP_PARENT_TABLE))) {
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(*mock_fk_parent_table_ptr))) {
|
|
LOG_WARN("fail to push mock_fk_parent_table_schema_array", K(ret), KPC(mock_fk_parent_table_ptr));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !mock_fk_parent_table_schema_array.empty()) {
|
|
if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, schema_guard, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("failed to deal_with_mock_fk_parent_table", K(ret), K(mock_fk_parent_table_schema_array));
|
|
}
|
|
}
|
|
// deal with mock_fk_parent_table_schema in drop_table end
|
|
if (OB_ISNULL(sql_trans) && trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_aux_table_in_drop_table(
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &table_schema,
|
|
ObTableType table_type,
|
|
const bool to_recyclebin)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
ObSEArray<uint64_t, 16> aux_tid_array; // for aux_vp or aux_lob
|
|
bool is_index = false;
|
|
|
|
if (USER_INDEX == table_type) {
|
|
is_index = true;
|
|
if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get_simple_index_infos failed", K(ret));
|
|
}
|
|
} else if (AUX_LOB_META == table_type) {
|
|
const uint64_t aux_lob_meta_tid = table_schema.get_aux_lob_meta_tid();
|
|
if (OB_INVALID_ID != aux_lob_meta_tid && OB_FAIL(aux_tid_array.push_back(aux_lob_meta_tid))) {
|
|
LOG_WARN("push back aux_lob_meta_tid failed", K(ret));
|
|
}
|
|
} else if (AUX_LOB_PIECE == table_type) {
|
|
const uint64_t aux_lob_piece_tid = table_schema.get_aux_lob_piece_tid();
|
|
if (OB_INVALID_ID != aux_lob_piece_tid && OB_FAIL(aux_tid_array.push_back(aux_lob_piece_tid))) {
|
|
LOG_WARN("push back aux_lob_meta_tid failed", K(ret));
|
|
}
|
|
} else if (AUX_VERTIAL_PARTITION_TABLE == table_type) {
|
|
if (OB_FAIL(table_schema.get_aux_vp_tid_array(aux_tid_array))) {
|
|
LOG_WARN("get_aux_vp_tid_array failed", K(ret));
|
|
}
|
|
} else if (MATERIALIZED_VIEW == table_type) {
|
|
if (OB_FAIL(aux_tid_array.push_back(table_schema.get_data_table_id()))) {
|
|
LOG_WARN("push container table id failed", KR(ret));
|
|
}
|
|
} else if (MATERIALIZED_VIEW_LOG == table_type) {
|
|
if (table_schema.has_mlog_table() && OB_FAIL(aux_tid_array.push_back(table_schema.get_mlog_tid()))) {
|
|
LOG_WARN("push materialized view log table id failed", KR(ret));
|
|
}
|
|
} else {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invaid aux table type", K(ret), K(table_type));
|
|
}
|
|
|
|
ObTableSchema new_table_schema;
|
|
int64_t N = is_index ? simple_index_infos.count() : aux_tid_array.count();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
|
|
const ObTableSchema *aux_table_schema = NULL;
|
|
uint64_t tid = is_index ? simple_index_infos.at(i).table_id_ : aux_tid_array.at(i);
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, tid, aux_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", tid, K(ret));
|
|
} else if (OB_ISNULL(aux_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(tenant_id), K(tid), KR(ret), K(table_type));
|
|
} else if (OB_FAIL(new_table_schema.assign(*aux_table_schema))) {
|
|
LOG_WARN("assign table schema failed", K(ret));
|
|
} else {
|
|
// If the data table of the delayed index table is placed in the recycle bin,
|
|
// the delayed index will also go in, and a row of data will be inserted into __all_recyclebin
|
|
new_table_schema.set_in_offline_ddl_white_list(table_schema.get_in_offline_ddl_white_list());
|
|
if (to_recyclebin && !is_inner_table(table_schema.get_table_id())) {
|
|
if (new_table_schema.is_in_recyclebin()) {
|
|
LOG_INFO("aux table is already in recyclebin");
|
|
} else if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(new_table_schema,
|
|
schema_guard,
|
|
trans,
|
|
NULL /* ddl_stmt_str */))) {
|
|
LOG_WARN("drop aux table to recycle failed", K(ret));
|
|
}
|
|
} else if (OB_FAIL(ddl_operator.drop_table(new_table_schema, trans))) {
|
|
LOG_WARN("ddl_operator drop_table failed", K(*aux_table_schema), K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_trigger_in_drop_table(ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &table_schema,
|
|
const bool drop_to_recyclebin)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t trigger_id = OB_INVALID_ID;
|
|
const ObTriggerInfo *trigger_info = NULL;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const ObIArray<uint64_t> &trigger_id_list = table_schema.get_trigger_list();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < trigger_id_list.count(); i++) {
|
|
OX (trigger_id = trigger_id_list.at(i));
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trigger_id, trigger_info), trigger_id);
|
|
OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_id);
|
|
OV (!trigger_info->is_in_recyclebin(), OB_ERR_UNEXPECTED, trigger_id);
|
|
if (drop_to_recyclebin && !table_schema.is_view_table()) {
|
|
// 兼容oracle, drop view的时候trigger不进回收站
|
|
OZ (ddl_operator.drop_trigger_to_recyclebin(*trigger_info, schema_guard, trans));
|
|
} else {
|
|
OZ (ddl_operator.drop_trigger(*trigger_info, trans, NULL, true /*is_update_table_schema_version, default true*/,
|
|
table_schema.get_in_offline_ddl_white_list()));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_table_from_recyclebin_in_trans(const ObTableSchema &table_schema,
|
|
const uint64_t new_db_id,
|
|
const ObString &new_table_name,
|
|
const ObString &ddl_stmt_str,
|
|
ObSchemaGetterGuard &guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator,
|
|
new_db_id, USER_INDEX))) {
|
|
LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema));
|
|
} else if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator,
|
|
new_db_id, AUX_VERTIAL_PARTITION_TABLE))) {
|
|
LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema));
|
|
} else if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator,
|
|
new_db_id, AUX_LOB_META))) {
|
|
LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema));
|
|
} else if (OB_FAIL(flashback_aux_table(table_schema, guard, trans, ddl_operator,
|
|
new_db_id, AUX_LOB_PIECE))) {
|
|
LOG_WARN("flashback_aux_table failed", K(ret), K(table_schema));
|
|
} else if (OB_FAIL(flashback_trigger(table_schema, new_db_id, new_table_name,
|
|
guard, trans, ddl_operator))) {
|
|
LOG_WARN("flashback_trigger failed", K(ret), K(table_schema));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObTableSchema new_table_schema;
|
|
if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin(table_schema,
|
|
new_table_schema,
|
|
trans,
|
|
new_db_id,
|
|
new_table_name,
|
|
&ddl_stmt_str,
|
|
guard))) {
|
|
LOG_WARN("flashback table failed", K(ret));
|
|
} else {
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
if (OB_FAIL(guard.get_tenant_compat_mode(tenant_id, compat_mode))) {
|
|
LOG_WARN("fail to get tenant compat mode", K(ret), K(tenant_id), K(compat_mode));
|
|
} else if (lib::Worker::CompatMode::MYSQL == compat_mode) {
|
|
const ObMockFKParentTableSchema *ori_mock_parent_table_schema_ptr = NULL;
|
|
if (OB_FAIL(guard.get_mock_fk_parent_table_schema_with_name(tenant_id,
|
|
new_table_schema.get_database_id(), new_table_schema.get_table_name_str(),
|
|
ori_mock_parent_table_schema_ptr))) {
|
|
SQL_RESV_LOG(WARN, "failed to check_mock_fk_parent_table_exist_with_name", K(ret));
|
|
} else if (OB_NOT_NULL(ori_mock_parent_table_schema_ptr)) {
|
|
ObArray<ObAuxTableMetaInfo> simple_index_infos;
|
|
ObArray<const share::schema::ObTableSchema*> unique_index_schemas;
|
|
if (OB_FAIL(new_table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
SERVER_LOG(WARN, "get simple_index_infos without delay_deleted_tid failed", K(ret));
|
|
} else {
|
|
const ObTableSchema *index_schema = NULL;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < simple_index_infos.count(); ++j) {
|
|
if (OB_FAIL(guard.get_table_schema(ori_mock_parent_table_schema_ptr->get_tenant_id(), simple_index_infos.at(j).table_id_, index_schema))) {
|
|
LOG_WARN("fail to get to_table_schema schema", K(ret));
|
|
} else if (OB_ISNULL(index_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("index_schema is null", K(ret));
|
|
} else if (index_schema->is_unique_index() && OB_FAIL(unique_index_schemas.push_back(index_schema))) {
|
|
LOG_WARN("fail to push_back index_schema to unique_index_schemas", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
ObMockFKParentTableSchema mock_fk_parent_table_schema;
|
|
if (OB_FAIL(gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table(
|
|
guard, ori_mock_parent_table_schema_ptr->get_mock_fk_parent_table_id(),
|
|
new_table_schema, unique_index_schemas, mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to gen_mock_fk_parent_table_for_replacing_mock_fk_parent_table", K(ret));
|
|
} else if (OB_FAIL(mock_fk_parent_table_schema_array.push_back(mock_fk_parent_table_schema))) {
|
|
LOG_WARN("failed to push mock_fk_parent_table_schema", K(ret), K(mock_fk_parent_table_schema));
|
|
} else if (OB_FAIL(ddl_operator.deal_with_mock_fk_parent_tables(trans, guard, mock_fk_parent_table_schema_array))) {
|
|
LOG_WARN("fail to deal_with_mock_fk_parent_tables", K(ret), K(tenant_id), K(mock_fk_parent_table_schema_array));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_aux_table(
|
|
const ObTableSchema &table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
const uint64_t new_db_id,
|
|
const ObTableType table_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
ObSEArray<uint64_t, 16> aux_vp_tid_array;
|
|
ObSEArray<ObAuxTableMetaInfo, 16> simple_index_infos;
|
|
uint64_t lob_meta_table_id = 0;
|
|
uint64_t lob_piece_table_id = 0;
|
|
int64_t N = 0;
|
|
if (table_type == USER_INDEX) {
|
|
if (OB_FAIL(table_schema.get_simple_index_infos(simple_index_infos))) {
|
|
LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type));
|
|
} else {
|
|
N = simple_index_infos.count();
|
|
}
|
|
} else if (table_type == AUX_VERTIAL_PARTITION_TABLE) {
|
|
if (OB_FAIL(table_schema.get_aux_vp_tid_array(aux_vp_tid_array))) {
|
|
LOG_WARN("get_aux_tid_array failed", K(ret), K(table_type));
|
|
} else {
|
|
N = aux_vp_tid_array.count();
|
|
}
|
|
} else if (table_type == AUX_LOB_META) {
|
|
lob_meta_table_id = table_schema.get_aux_lob_meta_tid();
|
|
N = table_schema.has_lob_aux_table() ? 1 : 0;
|
|
} else if (table_type == AUX_LOB_PIECE) {
|
|
lob_piece_table_id = table_schema.get_aux_lob_piece_tid();
|
|
N = table_schema.has_lob_aux_table() ? 1 : 0;
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Invalid table type.", K(ret), K(table_type));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
|
|
const ObTableSchema *aux_table_schema = NULL;
|
|
ObTableSchema new_table_schema;
|
|
uint64_t tid = 0;
|
|
if (table_type == USER_INDEX) {
|
|
tid = simple_index_infos.at(i).table_id_;
|
|
} else if (table_type == AUX_VERTIAL_PARTITION_TABLE) {
|
|
tid = aux_vp_tid_array.at(i);
|
|
} else if (table_type == AUX_LOB_META) {
|
|
tid = lob_meta_table_id;
|
|
} else if (table_type == AUX_LOB_PIECE) {
|
|
tid = lob_piece_table_id;
|
|
}
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, tid, aux_table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), "table id", tid, K(ret));
|
|
} else if (OB_ISNULL(aux_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (aux_table_schema->is_drop_index()) { // Temporarily keep the drop index into the recycle bin code
|
|
ret = OB_SUCCESS;
|
|
LOG_INFO("index table is dropped, can't flashback", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin(
|
|
*aux_table_schema,
|
|
new_table_schema,
|
|
trans,
|
|
new_db_id,
|
|
//set empty, because ensure that the index table flashback, use your own name
|
|
ObString(),
|
|
NULL, //ddl_stmt_str
|
|
schema_guard))) {
|
|
LOG_WARN("flashback table failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_trigger(const ObTableSchema &table_schema,
|
|
const uint64_t new_database_id,
|
|
const ObString &new_table_name,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObMySQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const ObIArray<uint64_t> &trigger_id_list = table_schema.get_trigger_list();
|
|
const ObTriggerInfo *trigger_info = NULL;
|
|
for (int i = 0; OB_SUCC(ret) && i < trigger_id_list.count(); i++) {
|
|
uint64_t trigger_id = trigger_id_list.at(i);
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trigger_id, trigger_info), trigger_id);
|
|
OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_id);
|
|
OZ (ddl_operator.flashback_trigger(*trigger_info, new_database_id,
|
|
new_table_name, schema_guard, trans));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_table_to_time_point(const obrpc::ObFlashBackTableToScnArg &arg)
|
|
{
|
|
UNUSEDx(arg);
|
|
return OB_NOT_SUPPORTED;
|
|
}
|
|
|
|
// check whether org db of object name and arg.origin_db_name_ is same
|
|
int ObDDLService::check_object_name_matches_db_name(
|
|
const uint64_t tenant_id,
|
|
const ObString &origin_table_name,
|
|
const uint64_t database_id,
|
|
bool &is_match)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSqlString sql;
|
|
HEAP_VAR(ObMySQLProxy::MySQLResult, res) {
|
|
common::sqlclient::ObMySQLResult *result = NULL;
|
|
is_match = false;
|
|
|
|
if (OB_FAIL(sql.append_fmt(
|
|
"SELECT 1 FROM __all_recyclebin WHERE OBJECT_NAME = '%.*s' \
|
|
AND DATABASE_ID = %lu",
|
|
static_cast<int>(origin_table_name.length()), origin_table_name.ptr(),
|
|
database_id))) {
|
|
LOG_WARN("failed to append sql",
|
|
K(ret), K(tenant_id), K(origin_table_name), K(database_id));
|
|
} else if (OB_FAIL(sql_proxy_->read(res, tenant_id, sql.ptr()))) {
|
|
LOG_WARN("failed to execute sql", K(sql), K(ret));
|
|
} else if (OB_ISNULL(result = res.get_result())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get result", K(ret));
|
|
} else if (OB_FAIL(result->next())) {
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_SUCCESS;
|
|
is_match = false;
|
|
} else {
|
|
LOG_WARN("iterate next result fail", K(ret), K(sql));
|
|
}
|
|
} else {
|
|
is_match = true;
|
|
}
|
|
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_object_name_with_origin_name_in_recyclebin(
|
|
const uint64_t tenant_id,
|
|
const ObString &origin_table_name,
|
|
const uint64_t database_id,
|
|
const ObRecycleObject::RecycleObjType recycle_type,
|
|
ObString &object_name,
|
|
const bool is_newest,
|
|
common::ObIAllocator *allocator,
|
|
common::ObMySQLProxy *sql_proxy)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSqlString sql;
|
|
|
|
HEAP_VAR(ObMySQLProxy::MySQLResult, res) {
|
|
common::sqlclient::ObMySQLResult *result = NULL;
|
|
if (ObRecycleObject::TABLE != recycle_type
|
|
&& ObRecycleObject::INDEX != recycle_type) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("recycle object type is wrong", K(ret), K(recycle_type));
|
|
} else if (ObRecycleObject::TABLE == recycle_type
|
|
&& OB_FAIL(sql.append_fmt("select database_id, object_name from oceanbase.__all_recyclebin where (type = %lu or type = %lu) and database_id = %lu and original_name = '%.*s' ",
|
|
static_cast<uint64_t>(ObRecycleObject::TABLE), static_cast<uint64_t>(ObRecycleObject::VIEW), database_id,
|
|
static_cast<int>(origin_table_name.length()), origin_table_name.ptr()))) {
|
|
LOG_WARN("failed to append sql", K(ret), K(tenant_id), K(origin_table_name), K(database_id), K(is_newest), K(sql));
|
|
} else if (ObRecycleObject::INDEX == recycle_type
|
|
&& OB_FAIL(sql.append_fmt("select database_id, object_name from oceanbase.__all_recyclebin where type = %lu and database_id = %lu and substr(original_name, 7 + instr(substr(original_name, 7), '_')) = '%.*s' ",
|
|
static_cast<uint64_t>(ObRecycleObject::INDEX), database_id,
|
|
static_cast<int>(origin_table_name.length()), origin_table_name.ptr()))) {
|
|
LOG_WARN("failed to append sql", K(ret), K(tenant_id), K(origin_table_name), K(database_id), K(is_newest), K(sql));
|
|
} else if (OB_FAIL(sql.append_fmt("order by gmt_create %s limit 1", is_newest ? "desc" : "asc"))) {
|
|
LOG_WARN("failed to append sql",
|
|
K(ret), K(tenant_id), K(origin_table_name), K(database_id), K(is_newest), K(sql));
|
|
} else if (OB_FAIL(sql_proxy->read(res, tenant_id, sql.ptr()))) {
|
|
LOG_WARN("failed to execute sql", K(sql), K(ret));
|
|
} else if (OB_ISNULL(result = res.get_result())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get result", K(ret));
|
|
} else if (OB_FAIL(result->next())) {
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("origin table_name not exist in recyclebin", K(ret), K(sql));
|
|
} else {
|
|
LOG_WARN("iterate next result fail", K(ret), K(sql));
|
|
}
|
|
} else {
|
|
ObString tmp_object_name;
|
|
EXTRACT_VARCHAR_FIELD_MYSQL(*result, "object_name", tmp_object_name);
|
|
if (OB_FAIL(deep_copy_ob_string(*allocator, tmp_object_name, object_name))) {
|
|
LOG_WARN("failed to deep copy member list", K(ret), K(object_name));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ITER_END != result->next()) {
|
|
// No more than one line
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("result failed", K(ret), K(sql));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_table_from_recyclebin(const ObFlashBackTableFromRecyclebinArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *table_schema = NULL;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t database_id = OB_INVALID_ID;
|
|
bool is_match = false;
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_database_id(tenant_id,
|
|
arg.origin_db_name_,
|
|
database_id))) {
|
|
LOG_WARN("get database id with database name failed", K(ret));
|
|
} else if (OB_INVALID_ID == database_id) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("database does not exist", K(ret), K(arg.origin_db_name_));
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, arg.origin_db_name_.length(),
|
|
arg.origin_db_name_.ptr());
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id,
|
|
OB_RECYCLEBIN_SCHEMA_ID,
|
|
arg.origin_table_name_,
|
|
false, /* is_index */
|
|
table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
// use origin table name to flashback
|
|
ObString object_name;
|
|
if (OB_FAIL(get_object_name_with_origin_name_in_recyclebin(
|
|
tenant_id,
|
|
arg.origin_table_name_,
|
|
database_id,
|
|
ObRecycleObject::TABLE,
|
|
object_name,
|
|
true,
|
|
&allocator,
|
|
sql_proxy_))) {
|
|
LOG_WARN("get object_name with origin_name in recyclebin failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id,
|
|
OB_RECYCLEBIN_SCHEMA_ID,
|
|
object_name,
|
|
false, /*is_index*/
|
|
table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_shema is null", K(ret), K(database_id), K(arg.origin_table_name_), K(object_name));
|
|
}
|
|
} else if (OB_FAIL(check_object_name_matches_db_name(tenant_id,
|
|
arg.origin_table_name_,
|
|
database_id,
|
|
is_match))) {
|
|
// use object name to flashback
|
|
LOG_WARN("fail to check object name matches db name", K(ret));
|
|
} else if (!is_match) {
|
|
// The specified object name does not exist under the specified db
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("origin table_name not exist in recyclebin",
|
|
K(ret),
|
|
K(arg.origin_table_name_),
|
|
K(database_id));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t new_db_id = OB_INVALID_ID;
|
|
uint64_t synonym_id = OB_INVALID_ID;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
|
|
if (OB_FAIL(schema_guard.get_tenant_compat_mode(arg.tenant_id_, compat_mode))) {
|
|
LOG_WARN("fail to get tenant compat mode", K(ret), K(tenant_id), K(compat_mode));
|
|
} else if (!arg.new_db_name_.empty() && !arg.new_table_name_.empty()) {
|
|
bool is_table_exist = false;
|
|
bool object_exist = false;
|
|
const ObDatabaseSchema *new_db_schema = NULL;
|
|
if (ObString(OB_RECYCLEBIN_SCHEMA_NAME) == arg.new_db_name_
|
|
|| ObString(OB_PUBLIC_SCHEMA_NAME) == arg.new_db_name_) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("flashback table to __recyclebin database is not allowed", K(arg), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(arg.tenant_id_,
|
|
arg.new_db_name_,
|
|
new_db_schema))) {
|
|
LOG_WARN("check database exist failed", K(ret), K(arg));
|
|
} else if (OB_ISNULL(new_db_schema)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, arg.new_db_name_.length(), arg.new_db_name_.ptr());
|
|
} else if (new_db_schema->is_in_recyclebin()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("flashback table to recyclebin db is not allowed", K(ret), K(arg));
|
|
}
|
|
if (OB_SUCC(ret) && lib::Worker::CompatMode::ORACLE == compat_mode) {
|
|
ObArray<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,
|
|
INVALID_ROUTINE_TYPE, false, conflict_schema_types))) {
|
|
LOG_WARN("fail to check oracle_object exist", K(ret), K(table_schema));
|
|
} else if (conflict_schema_types.count() > 0) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", K(ret), K(table_schema),
|
|
K(conflict_schema_types));
|
|
} else {
|
|
new_db_id = new_db_schema->get_database_id();
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && lib::Worker::CompatMode::MYSQL == compat_mode) {
|
|
if (OB_FAIL(schema_guard.check_synonym_exist_with_name(arg.tenant_id_,
|
|
new_db_schema->get_database_id(),
|
|
arg.new_table_name_,
|
|
object_exist,
|
|
synonym_id))) {
|
|
LOG_WARN("fail to check synonym exist", K(arg), K(ret));
|
|
} else if (object_exist) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", K(arg), K(ret));
|
|
} else if (OB_FAIL(schema_guard.check_table_exist(arg.tenant_id_,
|
|
new_db_schema->get_database_id(),
|
|
arg.new_table_name_,
|
|
table_schema->is_index_table(),
|
|
ObSchemaGetterGuard::ALL_NON_HIDDEN_TYPES,
|
|
is_table_exist))) {
|
|
LOG_WARN("check table exist failed,", K(ret), K(arg), K(new_db_id));
|
|
} else if (is_table_exist) {
|
|
ret = OB_ERR_TABLE_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_TABLE_EXIST, arg.new_table_name_.length(),
|
|
arg.new_table_name_.ptr());
|
|
LOG_WARN("talbe exist", K(new_db_id), K(arg), K(ret));
|
|
} else {
|
|
new_db_id = new_db_schema->get_database_id();
|
|
}
|
|
}
|
|
} else {
|
|
if (OB_SUCC(ret) && lib::Worker::CompatMode::ORACLE == compat_mode) {
|
|
ObArray<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, INVALID_ROUTINE_TYPE, false, conflict_schema_types))) {
|
|
LOG_WARN("fail to check oracle_object exist", K(ret), K(table_schema));
|
|
} else if (conflict_schema_types.count() > 0) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", K(ret), K(table_schema));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (table_schema->is_index_table()
|
|
|| table_schema->is_aux_vp_table()
|
|
|| table_schema->is_aux_lob_table()
|
|
|| table_schema->is_mlog_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("flash back index or materialized view log table is not supported now", K(ret));
|
|
} else if (OB_FAIL(flashback_table_from_recyclebin_in_trans(*table_schema,
|
|
new_db_id,
|
|
arg.new_table_name_,
|
|
arg.ddl_stmt_str_,
|
|
schema_guard))) {
|
|
LOG_WARN("flashback table from recyclebin in trans failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_index(const ObFlashBackIndexArg &arg) {
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *index_schema = NULL;
|
|
const ObTableSchema *data_table_schema = NULL;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
OB_RECYCLEBIN_SCHEMA_ID,
|
|
arg.origin_table_name_,
|
|
true, /*is_index*/
|
|
index_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret));
|
|
} else if (OB_ISNULL(index_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table schema is null", K(arg), K(ret));
|
|
} else if (!index_schema->is_index_table()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("the table is not index, flashback index failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, index_schema->get_data_table_id(), data_table_schema))) {
|
|
LOG_WARN("get data table schema failed", K(ret));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("data table schema is null", K(ret));
|
|
} else if (data_table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("the data table is in recyclebin", K(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
uint64_t synonym_id = OB_INVALID_ID;
|
|
if (!arg.new_table_name_.empty()) {
|
|
bool is_table_exist = false;
|
|
bool object_exist = false;
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, data_table_schema->get_database_id(), database_schema))) {
|
|
LOG_WARN("get database schema failed", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database schema is null", K(ret));
|
|
} else if (ObString(OB_RECYCLEBIN_SCHEMA_NAME) == database_schema->get_database_name() ||
|
|
ObString(OB_PUBLIC_SCHEMA_NAME) == database_schema->get_database_name()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("flashback table to __recyclebin database is not allowed", K(ret), K(*database_schema));
|
|
} else if (OB_FAIL(schema_guard.check_synonym_exist_with_name(arg.tenant_id_,
|
|
data_table_schema->get_database_id(),
|
|
arg.new_table_name_,
|
|
object_exist,
|
|
synonym_id))) {
|
|
LOG_WARN("fail to check synonym exist", K(arg), K(ret));
|
|
} else if (object_exist) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("Name is already used by an existing object", K(arg), K(ret));
|
|
} else if (OB_FAIL(schema_guard.check_table_exist(arg.tenant_id_,
|
|
data_table_schema->get_database_id(),
|
|
arg.new_table_name_,
|
|
index_schema->is_index_table(),
|
|
ObSchemaGetterGuard::ALL_NON_HIDDEN_TYPES,
|
|
is_table_exist))) {
|
|
LOG_WARN("check table exist failed,", K(ret), K(arg), K(data_table_schema->get_database_id()));
|
|
} else if (is_table_exist) {
|
|
ret = OB_ERR_TABLE_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_TABLE_EXIST, arg.new_table_name_.length(),
|
|
arg.new_table_name_.ptr());
|
|
LOG_WARN("talbe exist", K(data_table_schema->get_database_id()), K(arg), K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(flashback_index_in_trans(schema_guard,
|
|
*index_schema,
|
|
data_table_schema->get_database_id(),
|
|
arg.new_table_name_,
|
|
arg.ddl_stmt_str_))) {
|
|
LOG_WARN("falshback index in trans failed", K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_index_in_trans(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObTableSchema &table_schema,
|
|
const uint64_t new_db_id,
|
|
const common::ObString &new_table_name,
|
|
const common::ObString &ddl_stmt_str) {
|
|
int ret = OB_SUCCESS;
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
ObString new_index_table_name;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObTableSchema::build_index_table_name(allocator,
|
|
table_schema.get_data_table_id(),
|
|
new_table_name,
|
|
new_index_table_name))) {
|
|
LOG_WARN("build index table name failed", K(ret));
|
|
} else {
|
|
ObTableSchema new_table_schema;
|
|
if (OB_FAIL(ddl_operator.flashback_table_from_recyclebin(table_schema, new_table_schema,
|
|
trans,
|
|
new_db_id,
|
|
new_index_table_name,
|
|
&ddl_stmt_str,
|
|
schema_guard))) {
|
|
LOG_WARN("flashback table in trans failed", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::purge_index(const obrpc::ObPurgeIndexArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *table_schema = NULL;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t database_id = arg.database_id_;
|
|
bool is_match = false;
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
OB_RECYCLEBIN_SCHEMA_ID,
|
|
arg.table_name_,
|
|
true, /*is_index*/
|
|
table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret));
|
|
} else if (NULL == table_schema) { // try to purge with original table name
|
|
ObString object_name;
|
|
if (OB_FAIL(get_object_name_with_origin_name_in_recyclebin(tenant_id, arg.table_name_,
|
|
database_id, ObRecycleObject::INDEX, object_name, false /* the oldest one */, &allocator, sql_proxy_))) {
|
|
LOG_WARN("get object_name with origin_name in recyclebin failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id,
|
|
OB_RECYCLEBIN_SCHEMA_ID,
|
|
object_name,
|
|
true, /*is_index*/
|
|
table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_shema is null", K(ret), K(database_id), K(arg.table_name_), K(object_name));
|
|
}
|
|
} else if (OB_FAIL(check_object_name_matches_db_name(tenant_id, arg.table_name_,
|
|
database_id, is_match))) {
|
|
// purge with object name
|
|
LOG_WARN("fail to check object name matches db name", K(ret));
|
|
} else if (!is_match) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("table is not in recyclebin", K(ret), K(arg));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
const ObTableSchema *data_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_schema->get_data_table_id(), data_table_schema))) {
|
|
LOG_WARN("failed to get data table schema", K(ret));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_shema is null", K(ret), K(table_schema->get_data_table_id()));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_add_drop_index_in_trans(*data_table_schema, *table_schema, trans))) {
|
|
LOG_WARN("failed to lock for purge index", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::purge_table(
|
|
const ObPurgeTableArg &arg,
|
|
ObMySQLTransaction *pr_trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *table_schema = NULL;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t database_id = arg.database_id_;
|
|
bool is_match = false;
|
|
ObArenaAllocator allocator(ObModIds::OB_RS_PARTITION_TABLE_TEMP);
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
OB_RECYCLEBIN_SCHEMA_ID,
|
|
arg.table_name_,
|
|
false, /*is_index*/
|
|
table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret));
|
|
} else if (NULL == table_schema) { // try to purge with original table name
|
|
ObString object_name;
|
|
if (OB_FAIL(get_object_name_with_origin_name_in_recyclebin(tenant_id, arg.table_name_,
|
|
database_id, ObRecycleObject::TABLE, object_name, false /* the oldest one */, &allocator, sql_proxy_))) {
|
|
LOG_WARN("get object_name with origin_name in recyclebin failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id,
|
|
OB_RECYCLEBIN_SCHEMA_ID,
|
|
object_name,
|
|
false, /*is_index */
|
|
table_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(ret));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_shema is null", K(ret), K(database_id), K(arg.table_name_), K(object_name));
|
|
} else if (OB_UNLIKELY(table_schema->is_materialized_view() || table_schema->mv_container_table())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected table type", KR(ret), K(arg), KPC(table_schema));
|
|
}
|
|
} else if (OB_UNLIKELY(table_schema->is_materialized_view() || table_schema->mv_container_table())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected table type", KR(ret), K(arg), KPC(table_schema));
|
|
} else if (OB_FAIL(check_object_name_matches_db_name(tenant_id,
|
|
arg.table_name_,
|
|
database_id,
|
|
is_match))) {
|
|
// purge with object name
|
|
LOG_WARN("fail to check object name matches db name", K(ret));
|
|
} else if (!is_match) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("table is not in recyclebin", K(ret), K(arg));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(pr_trans)
|
|
&& OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(lock_table(OB_ISNULL(pr_trans) ? trans : *pr_trans, *table_schema))) {
|
|
LOG_WARN("fail to lock_table", KR(ret), KPC(table_schema));
|
|
} else if (OB_FAIL(ddl_operator.purge_table_with_aux_table(*table_schema,
|
|
schema_guard,
|
|
OB_ISNULL(pr_trans) ? trans : *pr_trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("purge table with aux table failed", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_ISNULL(pr_trans)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_database_in_trans(const ObDatabaseSchema &db_schema,
|
|
const ObString &new_db_name,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObString &ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const uint64_t tenant_id = db_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.flashback_database_from_recyclebin(db_schema,
|
|
trans,
|
|
new_db_name,
|
|
schema_guard,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("flashback database from recyclebin failed", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_database(const ObFlashBackDatabaseArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id,
|
|
arg.origin_db_name_,
|
|
database_schema))) {
|
|
LOG_WARN("get_database_schema failed", K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("database schema is not in recyclebin", K(ret), K(arg));
|
|
} else if (!database_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("database schema is not in recyclebin", K(ret), K(arg), K(*database_schema));
|
|
} else {
|
|
ObString new_db_name;
|
|
if (!arg.new_db_name_.empty()) {
|
|
bool is_db_exist = false;
|
|
if (OB_FAIL(schema_guard.check_database_exist(arg.tenant_id_,
|
|
arg.new_db_name_,
|
|
is_db_exist))) {
|
|
LOG_WARN("check database exist failed", K(ret), K(arg));
|
|
} else if (is_db_exist) {
|
|
ret = OB_DATABASE_EXIST;
|
|
LOG_USER_ERROR(OB_DATABASE_EXIST, arg.new_db_name_.length(), arg.new_db_name_.ptr());
|
|
LOG_WARN("new db name already exist", K(arg), K(ret));
|
|
} else {
|
|
new_db_name = arg.new_db_name_;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(flashback_database_in_trans(*database_schema,
|
|
new_db_name,
|
|
schema_guard,
|
|
arg.ddl_stmt_str_))) {
|
|
LOG_WARN("flashback table in trans failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::purge_database(
|
|
const ObPurgeDatabaseArg &arg,
|
|
ObMySQLTransaction *pr_trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id,
|
|
arg.db_name_,
|
|
database_schema))) {
|
|
LOG_WARN("get_database_schema failed", K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("database not in recyclebin, can not be purge", K(arg), K(ret));
|
|
} else if (!database_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("database not in recyclebin, can not be purge", K(arg), K(*database_schema), K(ret));
|
|
} else {/*do nothing*/}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(pr_trans)
|
|
&& OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(lock_tables_of_database(*database_schema, OB_ISNULL(pr_trans) ? trans : *pr_trans))) {
|
|
LOG_WARN("failed to lock tables of database", K(ret));
|
|
} else if (OB_FAIL(lock_tables_in_recyclebin(*database_schema, OB_ISNULL(pr_trans) ? trans : *pr_trans))) {
|
|
LOG_WARN("failed to lock tables in recyclebin", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.purge_database_in_recyclebin(*database_schema,
|
|
OB_ISNULL(pr_trans) ? trans : *pr_trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("purge database failed", K(ret));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (recycle_objs.count() == 0) {
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(
|
|
tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
for (int i = 0; OB_SUCC(ret) && i < recycle_objs.count() && purged_objects < arg.purge_num_; ++i) {
|
|
ddl_stmt.reset();
|
|
const ObRecycleObject &recycle_obj = recycle_objs.at(i);
|
|
switch(recycle_obj.get_type()) {
|
|
case ObRecycleObject::TRIGGER:
|
|
case ObRecycleObject::INDEX:
|
|
case ObRecycleObject::AUX_LOB_META:
|
|
case ObRecycleObject::AUX_LOB_PIECE:
|
|
case ObRecycleObject::AUX_VP: {
|
|
// Indexes/VP/AUX_LOB tables do not enter the recycle bin separately,
|
|
// and the purge table will synchronize purge index/aux_vp/aux_lob,
|
|
// so there is no need for a separate purge index trigger as above
|
|
continue;
|
|
}
|
|
case ObRecycleObject::VIEW:
|
|
case ObRecycleObject::TABLE: {
|
|
//purge table
|
|
if (OB_FAIL(ddl_stmt.assign_fmt("PURGE TABLE %.*s",
|
|
recycle_obj.get_object_name().length(),
|
|
recycle_obj.get_object_name().ptr()))) {
|
|
LOG_WARN("append sql failed", K(ret));
|
|
} else {
|
|
ObPurgeTableArg purge_table_arg;
|
|
purge_table_arg.tenant_id_ = arg.tenant_id_;
|
|
purge_table_arg.database_id_ = recycle_obj.get_database_id();
|
|
purge_table_arg.table_name_ = recycle_obj.get_object_name();
|
|
purge_table_arg.ddl_stmt_str_ = ddl_stmt.string();
|
|
if (OB_FAIL(purge_table(purge_table_arg, &trans))) {
|
|
if (OB_ERR_OBJECT_NOT_IN_RECYCLEBIN == ret) {
|
|
LOG_WARN("recycle object maybe purge by database", K(ret), K(recycle_obj));
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("purge table failed", K(purge_table_arg), K(recycle_obj), K(ret));
|
|
}
|
|
} else {
|
|
++purged_objects;
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case ObRecycleObject::DATABASE: {
|
|
if (OB_FAIL(ddl_stmt.assign_fmt("PURGE DATABASE %.*s",
|
|
recycle_obj.get_object_name().length(),
|
|
recycle_obj.get_object_name().ptr()))) {
|
|
LOG_WARN("append sql failed", K(ret));
|
|
} else {
|
|
ObPurgeDatabaseArg purge_database_arg;
|
|
purge_database_arg.tenant_id_ = arg.tenant_id_;
|
|
purge_database_arg.db_name_ = recycle_obj.get_object_name();
|
|
purge_database_arg.ddl_stmt_str_ = ddl_stmt.string();
|
|
if (OB_FAIL(purge_database(purge_database_arg, &trans))) {
|
|
LOG_WARN("purge database failed", K(purge_database_arg), K(recycle_obj), K(ret));
|
|
} else {
|
|
++purged_objects;
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case ObRecycleObject::TENANT: {
|
|
//nothing todo. Unified processing in purge_recyclebin_tenant
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unknown recycle object type", K(recycle_obj));
|
|
}
|
|
}
|
|
LOG_INFO("purge expire recycle object execpt tenant finished",
|
|
K(recycle_obj), K(arg), K(ret));
|
|
}
|
|
}
|
|
LOG_INFO("purge expire recycle object of tenant finished", K(arg), K(ret));
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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()) {
|
|
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;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
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_INVALID_ID == table_item.table_id_) {
|
|
if (OB_FAIL(guard.get_table_schema(tenant_id,
|
|
database_id,
|
|
table_item.table_name_,
|
|
USER_INDEX == expected_table_type,
|
|
tmp_table_schema,
|
|
table_item.is_hidden_))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id),
|
|
KT(database_id), K(table_item), K(expected_table_type), K(ret));
|
|
}
|
|
} else if (OB_FAIL(guard.get_table_schema(tenant_id, table_item.table_id_, tmp_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(table_item));
|
|
}
|
|
if (OB_FAIL(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() && !tmp_table_schema->is_external_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 (MATERIALIZED_VIEW == expected_table_type) {
|
|
if (!tmp_table_schema->is_materialized_view()) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("Table type not equal!", K(expected_table_type), K(table_item), K(ret));
|
|
}
|
|
} else if (is_view) {
|
|
if (SYSTEM_VIEW == tmp_table_schema->get_table_type()) {
|
|
// let it go, for case compatible
|
|
} else if (expected_table_type != tmp_table_schema->get_table_type()) {
|
|
ret = OB_ERR_WRONG_OBJECT;
|
|
LOG_USER_ERROR(OB_ERR_WRONG_OBJECT, to_cstring(table_item.database_name_),
|
|
to_cstring(table_item.table_name_), "VIEW");
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Unknow table type", K(expected_table_type),
|
|
"table_type", tmp_table_schema->get_table_type());
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (table_schema != NULL) {
|
|
*table_schema = tmp_table_schema;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::construct_drop_sql(const ObTableItem &table_item,
|
|
const ObTableType table_type,
|
|
ObSqlString &sql,
|
|
bool is_oracle_mode,
|
|
bool is_cascade_constrains)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArenaAllocator allocator("TmpDropSql");
|
|
ObString table_type_name = table_type == USER_VIEW ? "VIEW" : "TABLE";
|
|
ObString new_db_name;
|
|
ObString new_tbl_name;
|
|
|
|
if (OB_FAIL(sql::ObSQLUtils::generate_new_name_with_escape_character(
|
|
allocator,
|
|
table_item.database_name_,
|
|
new_db_name,
|
|
is_oracle_mode))) {
|
|
LOG_WARN("fail to generate new name with escape character",
|
|
K(ret), K(table_item.database_name_));
|
|
} else if (OB_FAIL(sql::ObSQLUtils::generate_new_name_with_escape_character(
|
|
allocator,
|
|
table_item.table_name_,
|
|
new_tbl_name,
|
|
is_oracle_mode))) {
|
|
LOG_WARN("fail to generate new name with escape character",
|
|
K(ret), K(table_item.database_name_));
|
|
} else if (OB_FAIL(sql.append_fmt(
|
|
is_oracle_mode ? "DROP %.*s \"%.*s\".\"%.*s\"" : "DROP %.*s `%.*s`.`%.*s`",
|
|
table_type_name.length(), table_type_name.ptr(),
|
|
new_db_name.length(), new_db_name.ptr(),
|
|
new_tbl_name.length(), new_tbl_name.ptr()))) {
|
|
LOG_WARN("failed to append sql", K(ret));
|
|
} else if (is_cascade_constrains && OB_FAIL(sql.append_fmt(" CASCADE CONSTRAINTS"))) {
|
|
LOG_WARN("failed to append CASCADE CONSTRAINTS", K(ret));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
/**
|
|
* @table_item table to drop
|
|
* @if_exists drop table if exists ...
|
|
* @err_table_list use for if_exists == false
|
|
*/
|
|
int ObDDLService::log_drop_warn_or_err_msg(const ObTableItem table_item,
|
|
bool if_exists,
|
|
ObSqlString &err_table_list)
|
|
{
|
|
//if exists == true, log to warning
|
|
//else log to err table list
|
|
int ret = OB_SUCCESS;
|
|
if (if_exists) {
|
|
ObSqlString warning_str;
|
|
if (OB_FAIL(warning_str.append_fmt("%.*s.%.*s",
|
|
table_item.database_name_.length(),
|
|
table_item.database_name_.ptr(),
|
|
table_item.table_name_.length(),
|
|
table_item.table_name_.ptr()))) {
|
|
LOG_WARN("append warning str failed", K(ret), K(table_item), K(if_exists));
|
|
} else {
|
|
LOG_USER_NOTE(OB_ERR_BAD_TABLE, static_cast<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", 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 the oracle temporary table has data for the specified session id
|
|
int ObDDLService::check_sessid_exist_in_temp_table(const ObString &db_name,
|
|
const ObString &tab_name,
|
|
const uint64_t tenant_id,
|
|
const uint64_t session_id,
|
|
bool &exists)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
HEAP_VAR(common::ObMySQLProxy::MySQLResult, res) {
|
|
common::sqlclient::ObMySQLResult *result = NULL;
|
|
ObSqlString sql;
|
|
exists = false;
|
|
if (OB_FAIL(sql.assign_fmt("SELECT 1 FROM `%.*s`.`%.*s` WHERE %s = %ld LIMIT 1",
|
|
db_name.length(), db_name.ptr(),
|
|
tab_name.length(), tab_name.ptr(),
|
|
OB_HIDDEN_SESSION_ID_COLUMN_NAME, session_id))) {
|
|
LOG_WARN("assign sql failed", K(ret));
|
|
} else if (OB_FAIL(sql_proxy_->read(res, tenant_id, sql.ptr()))) {
|
|
LOG_WARN("execute sql failed", K(ret), K(sql));
|
|
} else if (OB_ISNULL(result = res.get_result())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
SERVER_LOG(WARN, "result set from read is NULL", K(ret));
|
|
} else if (OB_FAIL(result->next())) {
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_SUCCESS; /* empty table */
|
|
} else {
|
|
LOG_WARN("get next result failed", K(ret));
|
|
}
|
|
} else {
|
|
exists = true;
|
|
}
|
|
LOG_INFO("check temp table has session id specified", K(exists), K(sql), K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
// Whether the current temporary table needs to be cleaned up,
|
|
// the session id can be the same under mysql mode
|
|
// For oracle, it is necessary to make sure that T has the data of the session
|
|
// or a table under the same tenant as T(session id should only be cleaned up for one tenant)
|
|
int ObDDLService::need_collect_current_temp_table(ObSchemaGetterGuard &schema_guard,
|
|
ObDropTableArg &drop_table_arg,
|
|
const ObSimpleTableSchemaV2 *table_schema,
|
|
bool &need_collect)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
need_collect = false;
|
|
if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table schema should not be null", K(ret));
|
|
} else if (drop_table_arg.session_id_ == table_schema->get_session_id()) {
|
|
need_collect = true;
|
|
drop_table_arg.tenant_id_ = table_schema->get_tenant_id();//maybe it's alter system ...
|
|
} else if ((TMP_TABLE_ORA_SESS == drop_table_arg.table_type_ && table_schema->is_oracle_tmp_table())
|
|
|| (TMP_TABLE_ORA_TRX == drop_table_arg.table_type_ && table_schema->is_oracle_trx_tmp_table())) {
|
|
if (drop_table_arg.tenant_id_ == table_schema->get_tenant_id()) {
|
|
need_collect = true;
|
|
}
|
|
} else if (TMP_TABLE_ALL == drop_table_arg.table_type_ && table_schema->is_oracle_tmp_table()) {
|
|
if (drop_table_arg.tenant_id_ == table_schema->get_tenant_id()) {
|
|
need_collect = true;
|
|
} else {
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(table_schema->get_tenant_id(),
|
|
table_schema->get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get database schema", K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database schema is null", K(ret));
|
|
} else if (OB_FAIL(check_sessid_exist_in_temp_table(database_schema->get_database_name_str(),
|
|
table_schema->get_table_name_str(),
|
|
table_schema->get_tenant_id(),
|
|
drop_table_arg.session_id_,
|
|
need_collect))) {
|
|
LOG_WARN("failed to check sessid exists", K(ret), K(table_schema->get_table_name_str()));
|
|
} else if (need_collect) {
|
|
drop_table_arg.tenant_id_ = table_schema->get_tenant_id();
|
|
drop_table_arg.table_type_ = TMP_TABLE_ORA_SESS;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// The syntax for cleaning up temporary tables when the session is disconnected is similar to
|
|
// DROP TABLES IN SESSION #1, so first collect all temporary tables created by SESSION #1
|
|
// The session requested by this DDL is still in a state of being disconnected
|
|
// regardless of whether it is directly connected or obproxy.
|
|
// The session id cannot be reused, so the situation of table_schema->session_id reuse is not considered
|
|
// alter system drop tables in session 1234 is special,
|
|
// it will delete all temporary tables under the specified session 1234(mysql & oracle)
|
|
// FIXME: this function should move to observer
|
|
int ObDDLService::collect_temporary_tables_in_session(const ObDropTableArg &const_drop_table_arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableItem table_item;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
ObSEArray<const ObSimpleTableSchemaV2 *, 512> table_schemas;
|
|
ObDropTableArg &drop_table_arg = const_cast<ObDropTableArg &>(const_drop_table_arg);
|
|
ObArray<uint64_t> tenant_ids;
|
|
if (0 == drop_table_arg.session_id_) {
|
|
//do nothing, in case of alter system drop tables in session 0
|
|
} else if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is null", KR(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_ids(tenant_ids))) {
|
|
LOG_WARN("fail to get tenant_ids", KR(ret));
|
|
} else {
|
|
bool found = false;
|
|
FOREACH_CNT_X(tenant_id, tenant_ids, OB_SUCC(ret) && !found) {
|
|
if (OB_FAIL(schema_service_->get_tenant_schema_guard(*tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), "tenant_id", *tenant_id);
|
|
} else if (OB_FAIL(schema_guard.get_table_schemas_in_tenant(*tenant_id, table_schemas))) {
|
|
LOG_WARN("fail to get table schema", K(ret), "tenant_id", *tenant_id);
|
|
} else {
|
|
for (int64_t i = 0; i < table_schemas.count() && OB_SUCC(ret); i++) {
|
|
const ObSimpleTableSchemaV2 *table_schema = table_schemas.at(i);
|
|
bool need_collect = false;
|
|
if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("got invalid schema", K(ret), K(i));
|
|
} else if (OB_FAIL(need_collect_current_temp_table(schema_guard, drop_table_arg, table_schema, need_collect))) {
|
|
LOG_WARN("check need collect temporary table failed", K(ret), K(table_schema->get_table_name_str()));
|
|
} else if (need_collect) {
|
|
found = true; // session_id should not across tenant
|
|
database_schema = NULL;
|
|
table_item.mode_ = table_schema->get_name_case_mode();
|
|
if (OB_FAIL(schema_guard.get_database_schema(*tenant_id, table_schema->get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get database schema", K(ret), "tenant_id", tenant_id);
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database schema is null", K(ret));
|
|
} else if (database_schema->is_in_recyclebin() || table_schema->is_in_recyclebin()) {
|
|
LOG_INFO("skip table schema in recyclebin", K(*table_schema));
|
|
} else if (OB_FAIL(ob_write_string(drop_table_arg.allocator_, database_schema->get_database_name_str(),
|
|
table_item.database_name_))
|
|
|| OB_FAIL(ob_write_string(drop_table_arg.allocator_, table_schema->get_table_name_str(),
|
|
table_item.table_name_))) {
|
|
LOG_WARN("Can not malloc space for table/db name", K(ret));
|
|
} else if (OB_FAIL(drop_table_arg.tables_.push_back(table_item))) {
|
|
LOG_WARN("failed to add table item!", K(table_item), K(ret));
|
|
} else {
|
|
LOG_DEBUG("a temporary table is about to be dropped since session disconnected", K(table_item), K(drop_table_arg));
|
|
}
|
|
}
|
|
} // end for
|
|
}
|
|
} // end FOREACH_CNT_X
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
//same api for drop table, drop index, drop view, drop materialized view
|
|
//
|
|
// mv rule:
|
|
// If the deleted table has mv, you must first delete mv to delete this table
|
|
// If it is mv, you need to update the version of the base/depend table, and mv does not enter the recycle bin
|
|
// drop table process (after add temporary table featrue)
|
|
// drop table t1; The temporary table is matched first, and the ordinary table cannot be found to be matched again;
|
|
// drop temporary table t1; must by temp table;
|
|
// drop tables in session 123456; --> collect all temp table in 23456, and drop
|
|
int ObDDLService::drop_table(const ObDropTableArg &drop_table_arg, const obrpc::ObDDLRes &ddl_res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
// Record whether the reason for the drop table failure is due to foreign key constraints
|
|
bool fail_for_fk_cons = false;
|
|
ObSqlString err_table_list;
|
|
ObSchemaGetterGuard schema_guard;
|
|
bool check_tmp_table_only = false; //drop temporary table
|
|
schema_guard.set_session_id(drop_table_arg.session_id_);
|
|
uint64_t tenant_id = drop_table_arg.tenant_id_;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObTabletAutoincCacheCleaner tablet_autoinc_cleaner(tenant_id);
|
|
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if ((MATERIALIZED_VIEW == drop_table_arg.table_type_
|
|
|| MATERIALIZED_VIEW_LOG == drop_table_arg.table_type_)
|
|
&& compat_version < DATA_VERSION_4_3_0_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "mview before 4.3 is");
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat error", K(ret));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service must not null", K(ret));
|
|
} else {
|
|
bool is_db_in_recyclebin = false;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
DropTableIdHashSet drop_table_set;
|
|
const int64_t drop_table_count = drop_table_arg.tables_.count();
|
|
if (OB_FAIL(drop_table_set.create(drop_table_count))) {
|
|
LOG_WARN("fail to add create ObHashSet", KR(ret));
|
|
} else if (TMP_TABLE == drop_table_arg.table_type_
|
|
|| TMP_TABLE_ORA_TRX == drop_table_arg.table_type_
|
|
|| TMP_TABLE_ORA_SESS == drop_table_arg.table_type_
|
|
|| TMP_TABLE_ALL == drop_table_arg.table_type_) {
|
|
if (drop_table_arg.tables_.count() == 0) {
|
|
if (OB_FAIL(collect_temporary_tables_in_session(drop_table_arg))) {
|
|
LOG_WARN("failed to collect temporary tables in session", K(ret));
|
|
} else {
|
|
LOG_INFO("collect temporary tables to be dropped succeed", K(drop_table_arg));
|
|
}
|
|
} else {
|
|
check_tmp_table_only = true;
|
|
}
|
|
tenant_id = drop_table_arg.tenant_id_;
|
|
LOG_DEBUG("drop temporary tables", K(drop_table_arg), K(check_tmp_table_only));
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", KR(ret), KR(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
// lock table when drop data table
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < drop_table_arg.tables_.count(); ++i) {
|
|
const ObTableItem &table_item = drop_table_arg.tables_.at(i);
|
|
const ObTableSchema *table_schema = NULL;
|
|
|
|
//ensure use the newest schema of each table
|
|
if (OB_FAIL(check_table_exists(
|
|
tenant_id,
|
|
table_item,
|
|
drop_table_arg.table_type_,
|
|
schema_guard,
|
|
&table_schema))) {
|
|
LOG_WARN("check_table_exist failed", KR(ret));
|
|
if (OB_TABLE_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret) {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schema should not be null", KR(ret));
|
|
} else if (OB_FAIL(drop_table_set.set_refactored(table_schema->get_table_id()))) {
|
|
LOG_WARN("set table_id to hash set failed", K(table_schema->get_table_id()), K(ret));
|
|
} else if (OB_FAIL(lock_table(trans, *table_schema))) {
|
|
LOG_WARN("fail to lock_table", KR(ret), KPC(table_schema));
|
|
} else if (table_schema->is_materialized_view()) {
|
|
// lock mv and mv container table
|
|
uint64_t container_table_id = table_schema->get_data_table_id();
|
|
const ObTableSchema *container_table_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, container_table_id, container_table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), K(container_table_id));
|
|
} else if (OB_ISNULL(container_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("container table schema should not be null",
|
|
KR(ret), K(tenant_id), K(container_table_id));
|
|
} else if (OB_FAIL(lock_mview(trans, *table_schema))) {
|
|
LOG_WARN("fail to lock mview", KR(ret), KPC(table_schema));
|
|
} else if (OB_FAIL(lock_table(trans, *container_table_schema))) {
|
|
LOG_WARN("fail to lock_table", KR(ret), KPC(container_table_schema));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
ObMockFKParentTableSchema mock_fk_parent_table_schema;
|
|
SMART_VAR(ObTableSchema, tmp_table_schema) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < drop_table_arg.tables_.count(); ++i) {
|
|
ObMockFKParentTableSchema *mock_fk_parent_table_ptr = NULL; // will use it when drop a fk_parent_table
|
|
mock_fk_parent_table_schema.reset();
|
|
const ObTableItem &table_item = drop_table_arg.tables_.at(i);
|
|
const ObTableSchema *table_schema = nullptr;
|
|
const ObTableSchema *data_table_schema = nullptr;
|
|
tmp_table_schema.reset();
|
|
is_db_in_recyclebin = false;
|
|
const bool is_drop_index_or_mlog = (MATERIALIZED_VIEW_LOG == drop_table_arg.table_type_)
|
|
|| (USER_INDEX == drop_table_arg.table_type_);
|
|
//ensure use the newest schema of each table
|
|
if (OB_FAIL(check_table_exists(tenant_id,
|
|
table_item,
|
|
drop_table_arg.table_type_,
|
|
schema_guard,
|
|
&table_schema))) {
|
|
LOG_WARN("check_table_exist failed", K(ret));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schema should not be null", KR(ret));
|
|
} else if (check_tmp_table_only && false == table_schema->is_tmp_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schema should be temporary", KR(ret));
|
|
} else if (!drop_table_arg.force_drop_ && table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not drop table in recyclebin, use purge instead", K(ret), K(table_item));
|
|
} else if (table_schema->is_materialized_view() && OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, table_schema->get_data_table_id(), data_table_schema))) {
|
|
LOG_WARN("failed to get mview container table schema", KR(ret), KPC(table_schema));
|
|
} else if (table_schema->is_materialized_view() && OB_ISNULL(data_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("data_table_schema should not be null", KR(ret));
|
|
} else if ((table_schema->required_by_mview_refresh() && !table_schema->is_index_table()) ||
|
|
(table_schema->is_materialized_view() && data_table_schema->required_by_mview_refresh())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("drop table required by materialized view refresh is not supported", KR(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop table required by materialized view refresh is");
|
|
} else if (OB_FAIL(tmp_table_schema.assign(*table_schema))) {
|
|
LOG_WARN("fail to assign table schema", K(ret));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(
|
|
tenant_id, table_schema->get_database_id(), is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", K(ret));
|
|
} else if (!drop_table_arg.force_drop_ && is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not drop table in recyclebin", K(ret), K(*table_schema));
|
|
} else if (OB_FAIL(check_enable_sys_table_ddl(*table_schema, OB_DDL_DROP_TABLE))) {
|
|
LOG_WARN("ddl is not allowed on sys table", K(ret));
|
|
} else if (is_drop_index_or_mlog && OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, tmp_table_schema.get_data_table_id(), data_table_schema))) {
|
|
LOG_WARN("failed to get data table schema", K(ret));
|
|
} else if (is_drop_index_or_mlog && OB_ISNULL(data_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("data table not found", K(ret), K(tmp_table_schema.get_data_table_id()));
|
|
} else if (FALSE_IT(tmp_table_schema.set_in_offline_ddl_white_list(table_item.is_hidden_ ||
|
|
(nullptr != data_table_schema && ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL == data_table_schema->get_table_state_flag())))) {
|
|
// to drop a data table, in_offline_ddl_white_list is decided by the `table_item.is_hidden_`.
|
|
// to drop a index table, in_offline_ddl_white_list is decided by the `table_item.is_hidden_` or data tables' state flag.
|
|
} else if (!tmp_table_schema.check_can_do_ddl()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("offline ddl is being executed, other ddl operations are not allowed",
|
|
K(tmp_table_schema), K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObString ddl_stmt_str;
|
|
ObSqlString drop_sql;
|
|
// do not record drop table stmt for recover restore table.
|
|
const bool is_offline_ddl_hidden_data_table =
|
|
ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL == table_schema->get_table_state_flag();
|
|
const bool use_drop_table_stmt_in_arg = (is_drop_index_or_mlog || is_offline_ddl_hidden_data_table);
|
|
bool is_cascade_constrains = false;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::MYSQL;
|
|
if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) {
|
|
LOG_WARN("fail to get tenant mode", K(ret), K(tenant_id));
|
|
} else if (lib::Worker::CompatMode::ORACLE == compat_mode) {
|
|
// oracle cascade constarints use if_exist_ flag
|
|
is_cascade_constrains = drop_table_arg.if_exist_;
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (use_drop_table_stmt_in_arg) {
|
|
ddl_stmt_str = drop_table_arg.ddl_stmt_str_;
|
|
} else if (OB_FAIL(construct_drop_sql(table_item,
|
|
drop_table_arg.table_type_,
|
|
drop_sql,
|
|
lib::Worker::CompatMode::ORACLE == compat_mode,
|
|
is_cascade_constrains))) {
|
|
LOG_WARN("construct_drop_sql failed", K(ret));
|
|
} else {
|
|
ddl_stmt_str = drop_sql.string();
|
|
}
|
|
// Check foreign key constraints
|
|
if (OB_SUCC(ret)) {
|
|
// If it is the parent table in the foreign key constraint, drop is not allowed
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!is_cascade_constrains && table_schema->is_parent_table()) {
|
|
int64_t violated_fk_index = -1;
|
|
const ObIArray<ObForeignKeyInfo> &foreign_key_infos = table_schema->get_foreign_key_infos();
|
|
if (table_schema->is_child_table()) { // deal with self reference fk
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i);
|
|
if (OB_HASH_EXIST == drop_table_set.exist_refactored(foreign_key_info.child_table_id_)) {
|
|
// do-nothing, Because multiple transactions update the schema, but the schema cache is not refreshed,
|
|
// you need to check that the filter sub-table has been dropped, and the current table can be dropped
|
|
} else if (table_schema->get_table_id() != foreign_key_info.child_table_id_) {
|
|
violated_fk_index = i;
|
|
ret = OB_ERR_TABLE_IS_REFERENCED;
|
|
LOG_WARN("Cannot drop table that is referenced by foreign key", K(ret), K(table_schema->get_table_name_str()));
|
|
} else { } // do-nothing This table is only a child table, not a parent table of any table, can drop
|
|
}
|
|
} else {
|
|
// Check whether the sub-tables are all drop
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < foreign_key_infos.count(); i++) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(i);
|
|
if (OB_HASH_EXIST == drop_table_set.exist_refactored(foreign_key_info.child_table_id_)) {
|
|
// do-nothing
|
|
} else {
|
|
violated_fk_index = i;
|
|
ret = OB_ERR_TABLE_IS_REFERENCED;
|
|
LOG_WARN("Cannot drop table that is referenced by foreign key", K(ret), K(table_schema->get_table_name_str()));
|
|
}
|
|
}
|
|
}
|
|
if (lib::Worker::CompatMode::MYSQL == compat_mode && OB_ERR_TABLE_IS_REFERENCED == ret) {
|
|
const ObForeignKeyInfo &foreign_key_info = foreign_key_infos.at(violated_fk_index);
|
|
if (OB_FAIL(gen_mock_fk_parent_table_for_drop_table(
|
|
schema_service, schema_guard, drop_table_arg, drop_table_set, foreign_key_infos,
|
|
foreign_key_info, table_schema, mock_fk_parent_table_schema))) {
|
|
LOG_WARN("gen_mock_fk_parent_table_for_drop_table failed", K(ret), K(table_schema->get_table_name_str()));
|
|
} else {
|
|
mock_fk_parent_table_ptr = &mock_fk_parent_table_schema;
|
|
}
|
|
}
|
|
} else {
|
|
// normal table or table_schema->is_child_table()
|
|
// do-nothing
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// The oracle temporary table is only to clean up the data of the session unless you explicitly drop the table
|
|
if (TMP_TABLE_ORA_SESS == drop_table_arg.table_type_ || TMP_TABLE_ORA_TRX == drop_table_arg.table_type_) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to clear oracle temp table", K(ret));
|
|
/*
|
|
if (OB_FAIL(truncate_oracle_temp_table(table_item.database_name_,
|
|
table_item.table_name_,
|
|
drop_table_arg.tenant_id_,
|
|
drop_table_arg.session_id_,
|
|
drop_table_arg.sess_create_time_))) {
|
|
LOG_WARN("delete data from oracle temporary table failed", K(ret), K(table_item));
|
|
}
|
|
*/
|
|
} else {
|
|
bool to_recyclebin = drop_table_arg.to_recyclebin_;
|
|
bool has_conflict_ddl = false;
|
|
if (table_schema->get_table_type() == MATERIALIZED_VIEW
|
|
|| table_schema->get_table_type() == MATERIALIZED_VIEW_LOG
|
|
|| table_schema->is_tmp_table()
|
|
|| table_schema->is_external_table()) {
|
|
to_recyclebin = false;
|
|
}
|
|
if (drop_table_arg.table_type_ == USER_TABLE && OB_FAIL(ObDDLTaskRecordOperator::check_has_conflict_ddl(
|
|
sql_proxy_,
|
|
drop_table_arg.tenant_id_,
|
|
table_schema->get_table_id(),
|
|
drop_table_arg.task_id_,
|
|
ObDDLType::DDL_DROP_TABLE,
|
|
has_conflict_ddl))) {
|
|
LOG_WARN("failed to check has conflict ddl", K(ret));
|
|
} else if (has_conflict_ddl) {
|
|
ret = OB_EAGAIN;
|
|
LOG_WARN("failed to drop table that has conflict ddl", K(ret), K(table_schema->get_table_id()));
|
|
} else if (OB_FAIL(drop_table_in_trans(schema_guard,
|
|
tmp_table_schema,
|
|
false,
|
|
USER_INDEX == drop_table_arg.table_type_,
|
|
to_recyclebin,
|
|
&ddl_stmt_str,
|
|
&trans,
|
|
&drop_table_set,
|
|
mock_fk_parent_table_ptr /* will use it when drop a fk_parent_table */))) {
|
|
LOG_WARN("ddl_service_ drop_table failed", K(table_item), K(tenant_id), K(ret));
|
|
} else if (drop_table_arg.task_id_ != 0 && is_drop_index_or_mlog) {
|
|
ObTableLockOwnerID owner_id;
|
|
if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE,
|
|
drop_table_arg.task_id_))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(drop_table_arg.task_id_));
|
|
} else if (OB_FAIL(ObDDLLock::unlock_for_add_drop_index(*data_table_schema,
|
|
tmp_table_schema.get_table_id(),
|
|
tmp_table_schema.is_global_index_table(),
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to unlock for add drop index", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(tablet_autoinc_cleaner.add_table(schema_guard, *table_schema))) {
|
|
LOG_WARN("failed to add table to tablet autoinc cleaner", K(tmp_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) {
|
|
if (MATERIALIZED_VIEW == drop_table_arg.table_type_) {
|
|
if (!drop_table_arg.if_exist_) {
|
|
// OB_ERR_MVIEW_NOT_EXIST只能打印一个mview, 遇到第一个不存在的mview就报错退出循环
|
|
ret = OB_ERR_MVIEW_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_MVIEW_NOT_EXIST, to_cstring(table_item.database_name_), to_cstring(table_item.table_name_));
|
|
} else {
|
|
LOG_USER_NOTE(OB_ERR_MVIEW_NOT_EXIST, to_cstring(table_item.database_name_), to_cstring(table_item.table_name_));
|
|
ret = OB_SUCCESS;
|
|
}
|
|
} else {
|
|
int tmp_ret = OB_SUCCESS;
|
|
ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = log_drop_warn_or_err_msg(table_item,
|
|
drop_table_arg.if_exist_,
|
|
err_table_list))) {
|
|
ret = tmp_ret;
|
|
LOG_WARN("log_drop_warn_or_err_msg failed", K(ret));
|
|
} else {
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (fail_for_fk_cons || (!err_table_list.empty())) {
|
|
if ((!fail_for_fk_cons) && drop_table_arg.if_exist_) {
|
|
//should produce warning //todo
|
|
} else {
|
|
switch(drop_table_arg.table_type_) {
|
|
case TMP_TABLE:
|
|
case USER_TABLE:
|
|
case USER_VIEW: {
|
|
if (fail_for_fk_cons) {
|
|
ret = OB_ERR_TABLE_IS_REFERENCED;
|
|
LOG_WARN("Cannot drop table that is referenced by foreign key", K(ret));
|
|
} else {
|
|
ret = OB_ERR_BAD_TABLE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_TABLE, (int32_t)err_table_list.length() - 1, err_table_list.ptr());
|
|
LOG_WARN("failed to drop table/view ", K(ret), K(err_table_list));
|
|
}
|
|
break;
|
|
}
|
|
case USER_INDEX: {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("failed to drop index table", K(err_table_list), K(ret));
|
|
break;
|
|
}
|
|
case MATERIALIZED_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_MVIEW_NOT_EXIST;
|
|
LOG_WARN("failed to drop mview ", K(ret), K(drop_table_arg));
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
SQL_RESV_LOG(WARN, "Unknown table type", K(drop_table_arg), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// to avoid drop table again in some scenarios like succeed to drop table but RPC timeout,
|
|
// drop table and update ddl task status should be done in single trans.
|
|
if (OB_FAIL(ret)) {
|
|
} else if (drop_table_arg.task_id_ > 0 && drop_table_arg.table_type_ != USER_INDEX
|
|
&& drop_table_arg.table_type_ != MATERIALIZED_VIEW_LOG
|
|
&& OB_FAIL(ObDDLRetryTask::update_task_status_wait_child_task_finish(trans, tenant_id, drop_table_arg.task_id_))) {
|
|
LOG_WARN("update task status of drop table failed", K(ret));
|
|
}
|
|
//no matter success or not, we should publish schema
|
|
//drop table t1, __all_table, may be t1 has been dropped successfully
|
|
//but __all_table failed, and we should refresh schema to see t1 dropped
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
bool commit = OB_SUCC(ret);
|
|
if (OB_SUCCESS != (temp_ret = trans.end(commit))) {
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
LOG_WARN("trans end failed", K(commit), K(temp_ret));
|
|
}
|
|
}
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) {
|
|
ret = tmp_ret;
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_TMP_FAIL(tablet_autoinc_cleaner.commit())) {
|
|
LOG_WARN("failed to commit tablet autoinc cleaner", K(tmp_ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_vec_index(const ObRebuildIndexArg &arg, obrpc::ObAlterTableRes &res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
LOG_DEBUG("RS start to rebuild vec index", K(arg));
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
ret = OB_INNER_STAT_ERROR;
|
|
LOG_WARN("check_inner_stat error", K(ret), K(is_inited()));
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(arg));
|
|
} else {
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(arg.session_id_);
|
|
bool is_db_in_recyclebin = false;
|
|
int64_t refreshed_schema_version = 0;
|
|
const bool is_index = false;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const ObTableSchema *table_schema = NULL;
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("get_schema_guard failed", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, arg.database_name_, arg.table_name_, is_index, table_schema))) {
|
|
LOG_WARN("failed to get data table schema", K(ret), K(arg));
|
|
} 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), K(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.", K(ret), K(arg));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", K(ret), K(tenant_id));
|
|
} else if (is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("Can not truncate index of db in recyclebin", K(ret), K(arg));
|
|
} else {
|
|
const uint64_t table_id = table_schema->get_table_id();
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
ObIndexBuilder index_builder(*this);
|
|
uint64_t tenant_data_version = 0;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, arg.index_table_id_, index_table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(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(ret), K(arg.index_name_));
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY, arg.index_name_.length(), arg.index_name_.ptr());
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else if (tenant_data_version < DATA_VERSION_4_3_3_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "rebuild vec index before 4.3.3 is");
|
|
} else {
|
|
ObDDLTaskRecord task_record;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
SMART_VAR(ObRebuildIndexArg, rebuild_index_arg) {
|
|
if (OB_FAIL(rebuild_index_arg.assign(arg))) {
|
|
LOG_WARN("fail to assign rebuild index arg", K(ret));
|
|
} else if (OB_FAIL(ObVectorIndexUtil::generate_new_index_name(allocator, rebuild_index_arg.index_name_))) {
|
|
LOG_WARN("fail to generate new index name", K(ret));
|
|
} else if (OB_FAIL(trans.start(&GCTX.root_service_->get_sql_proxy(), tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to start trans", K(ret));
|
|
} else if (OB_FAIL(index_builder.submit_rebuild_index_task(trans,
|
|
rebuild_index_arg,
|
|
table_schema,
|
|
nullptr/*inc_data_tablet_ids*/,
|
|
nullptr/*del_data_tablet_ids*/,
|
|
index_table_schema,
|
|
rebuild_index_arg.parallelism_,
|
|
rebuild_index_arg.consumer_group_id_,
|
|
tenant_data_version,
|
|
allocator,
|
|
task_record))) {
|
|
LOG_WARN("fail to submit rebuild vec index task", K(ret), K(index_table_schema));
|
|
} else {
|
|
res.task_id_ = task_record.task_id_;
|
|
LOG_INFO("succ submit rebuild task", K(res.task_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_SUCCESS == ret, K(ret), K(temp_ret));
|
|
ret = OB_SUCC(ret) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("fail to publish schema", K(ret), K(tenant_id));
|
|
} else if (OB_TMP_FAIL(GCTX.root_service_->get_ddl_task_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
ObSqlString err_table_list;
|
|
if (OB_TABLE_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = log_rebuild_warn_or_err_msg(arg, err_table_list))) {
|
|
ret = tmp_ret;
|
|
LOG_WARN("log_drop_warn_or_err_msg failed", KR(ret));
|
|
} else {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("failed to drop index table", K(ret), K(err_table_list));
|
|
}
|
|
}
|
|
}
|
|
LOG_DEBUG("finish rebuild vec index", K(ret), K(arg));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_index(const ObRebuildIndexArg &arg, obrpc::ObAlterTableRes &res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
|
|
const bool is_index = false;
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
const ObTableSchema *table_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
// bool to_recyclebin = arg.to_recyclebin();
|
|
bool is_db_in_recyclebin = false;
|
|
int64_t refreshed_schema_version = 0;
|
|
schema_guard.set_session_id(arg.session_id_);
|
|
|
|
if (check_inner_stat()) {
|
|
ret = OB_INNER_STAT_ERROR;
|
|
LOG_WARN("check_inner_stat error", K(is_inited()), KR(ret));
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(arg), KR(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("get_schema_guard failed", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(
|
|
tenant_id, arg.database_name_, arg.table_name_,
|
|
is_index, table_schema))) {
|
|
LOG_WARN("failed to get data table schema", K(arg), KR(ret));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TABLE_NOT_EXIST, to_cstring(arg.database_name_), to_cstring(arg.table_name_));
|
|
LOG_WARN("table not found", K(arg), KR(ret));
|
|
} else if (table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not truncate index of table in recyclebin.", KR(ret), K(arg));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", KR(ret), K(tenant_id));
|
|
} else if (is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("Can not truncate index of db in recyclebin", KR(ret), K(arg));
|
|
} else {
|
|
const uint64_t table_id = table_schema->get_table_id();
|
|
const ObTableSchema *index_table_schema = NULL;
|
|
ObString index_table_name;
|
|
ObIndexBuilder index_builder(*this);
|
|
uint64_t tenant_data_version = 0;
|
|
|
|
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 if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(tenant_id));
|
|
} else {
|
|
ObString ddl_stmt_str = arg.ddl_stmt_str_;
|
|
ObTableSchema new_table_schema;
|
|
SMART_VAR(ObCreateIndexArg, create_index_arg) {
|
|
ObDDLTaskRecord task_record;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(trans.start(&GCTX.root_service_->get_sql_proxy(), tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to start trans", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.assign(*index_table_schema))) {
|
|
LOG_WARN("fail to assign schema", KR(ret));
|
|
} else if (OB_FAIL(drop_directly_and_create_index_schema_(schema_guard,
|
|
*table_schema,
|
|
new_table_schema,
|
|
&ddl_stmt_str,
|
|
&trans,
|
|
tenant_data_version))) {
|
|
LOG_WARN("ddl_service_ rebuild_index failed", K(tenant_id), KR(ret));
|
|
} else if (OB_FAIL(index_builder.submit_build_index_task(trans,
|
|
create_index_arg,
|
|
table_schema,
|
|
nullptr/*inc_data_tablet_ids*/,
|
|
nullptr/*del_data_tablet_ids*/,
|
|
&new_table_schema,
|
|
arg.parallelism_,
|
|
arg.consumer_group_id_,
|
|
tenant_data_version,
|
|
allocator,
|
|
task_record))) {
|
|
LOG_WARN("fail to submit build global index task", KR(ret));
|
|
} else {
|
|
res.index_table_id_ = new_table_schema.get_table_id();
|
|
res.schema_version_ = new_table_schema.get_schema_version();
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(ret), K(temp_ret));
|
|
ret = OB_SUCC(ret) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("fail to publish schema", K(ret), K(tenant_id));
|
|
} else if (OB_TMP_FAIL(GCTX.root_service_->get_ddl_task_scheduler().schedule_ddl_task(task_record))) {
|
|
LOG_WARN("fail to schedule ddl task", K(tmp_ret), K(task_record));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
ObSqlString err_table_list;
|
|
if (OB_TABLE_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = log_rebuild_warn_or_err_msg(arg, err_table_list))) {
|
|
ret = tmp_ret;
|
|
LOG_WARN("log_drop_warn_or_err_msg failed", KR(ret));
|
|
} else {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("failed to drop index table", K(err_table_list), KR(ret));
|
|
}
|
|
}
|
|
LOG_INFO("finish rebuild index", K(arg), KR(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::clean_splitted_tablet(const obrpc::ObCleanSplittedTabletArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
ObArenaAllocator allocator;
|
|
ObArray<const share::schema::ObTableSchema*> splitting_table_schemas;
|
|
ObArray<share::schema::ObTableSchema*> del_table_schemas;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSEArray<ObTabletID, 1> src_data_tablet_id;
|
|
|
|
if (check_inner_stat()) {
|
|
ret = OB_INNER_STAT_ERROR;
|
|
LOG_WARN("check_inner_stat error", K(is_inited()), KR(ret));
|
|
} else if (OB_FAIL(generate_splitted_schema_array(arg, allocator,
|
|
splitting_table_schemas,
|
|
del_table_schemas,
|
|
refreshed_schema_version))) {
|
|
LOG_WARN("fail to generate splitted schema array", KR(ret), K(arg));
|
|
} else if (OB_UNLIKELY(splitting_table_schemas.count() != del_table_schemas.count())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema's number is different", KR(ret),
|
|
K(splitting_table_schemas.count()),
|
|
K(del_table_schemas.count()));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
// modify schema
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < splitting_table_schemas.count(); i++) {
|
|
if (OB_ISNULL(splitting_table_schemas.at(i)) || OB_ISNULL(del_table_schemas.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", KR(ret), KPC(splitting_table_schemas.at(i)),
|
|
KPC(del_table_schemas.at(i)));
|
|
} else if (OB_FAIL(ddl_operator.drop_table_splitted_partitions(*splitting_table_schemas.at(i),
|
|
*del_table_schemas.at(i),
|
|
trans))) {
|
|
LOG_WARN("failed to drop split source partitions", KR(ret),
|
|
KPC(splitting_table_schemas.at(i)),
|
|
KPC(del_table_schemas.at(i)));
|
|
}
|
|
}
|
|
// inc data_table's schema version when index or lob schema has been updated
|
|
if (OB_SUCC(ret)) {
|
|
if (splitting_table_schemas.empty() || OB_ISNULL(splitting_table_schemas.at(0))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid array", K(splitting_table_schemas.count()), KPC(splitting_table_schemas.at(0)));
|
|
} else if (splitting_table_schemas.count() > 1 || splitting_table_schemas.at(0)->is_global_index_table()) {
|
|
uint64_t data_table_id = splitting_table_schemas.at(0)->is_global_index_table() ?
|
|
splitting_table_schemas.at(0)->get_data_table_id() :
|
|
splitting_table_schemas.at(0)->get_table_id();
|
|
if (OB_FAIL(ddl_operator.inc_table_schema_version(trans, tenant_id, data_table_id))) {
|
|
LOG_WARN("fail to inc data table schema version", KR(ret), K(arg), K(data_table_id),
|
|
KPC(splitting_table_schemas.at(0)));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObTableLockOwnerID owner_id;
|
|
owner_id.convert_from_value(arg.task_id_);
|
|
if (OB_FAIL(src_data_tablet_id.push_back(arg.src_table_tablet_id_))) {
|
|
LOG_WARN("failed to push back", K(ret));
|
|
} else if (OB_FAIL(ObDDLLock::unlock_for_split_partition(*splitting_table_schemas.at(0), src_data_tablet_id, arg.dest_tablet_ids_, owner_id, trans))) {
|
|
LOG_WARN("failed to unlock for split partition", K(ret));
|
|
}
|
|
}
|
|
|
|
// drop tablets
|
|
if (OB_SUCC(ret)) {
|
|
int64_t new_schema_version = 0;
|
|
if (OB_FAIL(schema_service_->get_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("failed to get new schema_version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ObSplitPartitionHelper::clean_split_src_and_dst_tablet(arg, splitting_table_schemas.at(0)->get_auto_part_size(), new_schema_version, trans))) {
|
|
LOG_WARN("failed to clean split dst tablet mds", K(ret));
|
|
}
|
|
}
|
|
|
|
const bool is_commit = OB_SUCC(ret);
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(is_commit))) {
|
|
LOG_WARN("trans end failed", K(is_commit), K(temp_ret));
|
|
ret = is_commit ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// "splitting_table_schemas" records the current table_schema.
|
|
// the first table_schema in the array is of data_table/global_index and the others are local_index schema or lob schema.
|
|
// "del_table_schemas" records the deleting partition of table_schemas.
|
|
int ObDDLService::generate_splitted_schema_array(
|
|
const obrpc::ObCleanSplittedTabletArg &arg,
|
|
ObArenaAllocator& allocator,
|
|
common::ObIArray<const share::schema::ObTableSchema*> &splitting_table_schemas,
|
|
common::ObIArray<share::schema::ObTableSchema*> &del_table_schemas,
|
|
int64_t &refreshed_schema_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t table_id = arg.table_id_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTableSchema *splitting_table_schema = NULL;
|
|
bool is_db_in_recyclebin = false;
|
|
bool is_index = false;
|
|
int64_t del_table_schema_num = 1 + arg.local_index_table_ids_.count() + arg.lob_table_ids_.count();
|
|
splitting_table_schemas.reset();
|
|
del_table_schemas.reset();
|
|
refreshed_schema_version = 0;
|
|
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("get_schema_guard failed", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, splitting_table_schema))) {
|
|
LOG_WARN("failed to get table schema", K(arg), KR(ret));
|
|
} else if (OB_ISNULL(splitting_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table not found", K(arg), KR(ret));
|
|
} else if (splitting_table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("the table is in recyclebin.", KR(ret), K(arg), KPC(splitting_table_schema));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
splitting_table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("check database in recyclebin failed", KR(ret), K(tenant_id), KPC(splitting_table_schema));
|
|
} else if (is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("the database in recyclebin", KR(ret), K(arg), KPC(splitting_table_schema));
|
|
} else if (OB_UNLIKELY(splitting_table_schema->get_part_level() == PARTITION_LEVEL_TWO)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("split subpartition is not supported", KR(ret), KPC(splitting_table_schema));
|
|
} else if (OB_UNLIKELY(splitting_table_schema->get_part_level() == PARTITION_LEVEL_ZERO)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("the splitting table is non-partitioned", KR(ret), KPC(splitting_table_schema));
|
|
} else if (!splitting_table_schema->is_user_table() && !splitting_table_schema->is_global_index_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid table type", K(ret), KPC(splitting_table_schema));
|
|
} else if (splitting_table_schema->is_global_index_table() && del_table_schema_num > 1 ) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("global index should not have aux tables", KR(ret), K(arg), KPC(splitting_table_schema));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < del_table_schema_num; ++i) {
|
|
int64_t index = 0;
|
|
ObTabletID splitted_tablet_id;
|
|
const ObTableSchema *splitting_schema = nullptr;
|
|
|
|
if (i == 0) {
|
|
// data table or global index
|
|
splitting_schema = splitting_table_schema;
|
|
splitted_tablet_id = arg.src_table_tablet_id_;
|
|
} else if (i < arg.local_index_table_ids_.count() + 1) {
|
|
index = i - 1;
|
|
splitted_tablet_id = arg.src_local_index_tablet_ids_.at(index);
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
arg.local_index_table_ids_.at(index),
|
|
splitting_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), K(splitted_tablet_id), KR(ret));
|
|
}
|
|
} else {
|
|
index = i - arg.local_index_table_ids_.count() - 1;
|
|
splitted_tablet_id = arg.src_lob_tablet_ids_.at(index);
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
arg.lob_table_ids_.at(index),
|
|
splitting_schema))) {
|
|
LOG_WARN("get_table_schema failed", K(tenant_id), K(splitted_tablet_id), KR(ret));
|
|
}
|
|
}
|
|
|
|
ObTableSchema *del_schema = nullptr;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(splitting_schema) || !splitted_tablet_id.is_valid()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid argument", KR(ret), K(splitted_tablet_id), KPC(splitting_schema));
|
|
} else if (OB_FAIL(splitting_table_schemas.push_back(splitting_schema))) {
|
|
LOG_WARN("fail to push back", KR(ret), KPC(splitting_schema));
|
|
} else if (OB_FAIL(generate_splitted_schema_from_partitioned_table(
|
|
allocator, splitted_tablet_id,
|
|
*splitting_schema, del_schema))) {
|
|
LOG_WARN("fail to generate splitted schema from partitioned table", KR(ret), K(splitted_tablet_id),
|
|
KPC(splitting_schema));
|
|
} else if (OB_ISNULL(del_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", KR(ret), K(splitted_tablet_id), KPC(splitting_schema));
|
|
} else if (OB_FAIL(del_table_schemas.push_back(del_schema))) {
|
|
LOG_WARN("fail to push back", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_splitted_schema_from_partitioned_table(
|
|
ObArenaAllocator& allocator,
|
|
const ObTabletID splitted_tablet_id,
|
|
const share::schema::ObTableSchema &splitting_table_schema,
|
|
share::schema::ObTableSchema *&del_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
del_table_schema = NULL;
|
|
ObPartition** hidden_part_array = splitting_table_schema.get_hidden_part_array();
|
|
const ObPartition* splitted_part = nullptr;
|
|
|
|
if (OB_UNLIKELY(splitting_table_schema.get_part_level() != PARTITION_LEVEL_ONE)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("the splitting table is not partitioned table", KR(ret), K(splitting_table_schema));
|
|
} else if (OB_ISNULL(hidden_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", KR(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && splitted_part == nullptr
|
|
&& i < splitting_table_schema.get_hidden_partition_num(); i++) {
|
|
if (OB_ISNULL(hidden_part_array[i])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", KR(ret));
|
|
} else if (hidden_part_array[i]->get_tablet_id() == splitted_tablet_id) {
|
|
splitted_part = hidden_part_array[i];
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(splitted_part)) {
|
|
ret = OB_UNKNOWN_PARTITION;
|
|
LOG_WARN("fail to find splitted partition in hidden part array", KR(ret), K(splitted_tablet_id),
|
|
K(splitting_table_schema));
|
|
} else if (splitted_part->get_partition_type() != PARTITION_TYPE_SPLIT_SOURCE) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("the partition is not split source partition", KR(ret), KPC(splitted_part), K(splitting_table_schema));
|
|
}
|
|
}
|
|
|
|
void *del_schema_ptr = allocator.alloc(sizeof(ObTableSchema));
|
|
ObPartition mock_part;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(del_schema_ptr)) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("fail alloc memory", KR(ret));
|
|
} else if (FALSE_IT(del_table_schema = new (del_schema_ptr)ObTableSchema(&allocator))) {
|
|
} else if (OB_FAIL(del_table_schema->assign(splitting_table_schema))) {
|
|
LOG_WARN("failed to push back table_schema", KR(ret), K(splitting_table_schema));
|
|
} else if (FALSE_IT(del_table_schema->reset_partition_array())) {
|
|
} else if (FALSE_IT(del_table_schema->reset_hidden_partition_array())) {
|
|
} else if (FALSE_IT(del_table_schema->get_part_option().set_part_num(1))) {
|
|
} else if (OB_FAIL(mock_part.assign(*splitted_part))) {
|
|
LOG_WARN("fail to assign partition", KR(ret), KPC(splitted_part));
|
|
} else if (FALSE_IT(mock_part.set_partition_type(PARTITION_TYPE_NORMAL))) {
|
|
// we mock the hidden source partition as a normal partition for adapting to
|
|
// the function of dropping partitions in inner_table and dropping tablets which
|
|
// will traverse all "normal" partitions and drop them based on identifier.
|
|
} else if (OB_FAIL(del_table_schema->add_partition(mock_part))) {
|
|
LOG_WARN("fail to add hidden partition", KR(ret), K(mock_part));
|
|
}
|
|
|
|
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::drop_directly_and_create_index_schema_(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &data_table_schema,
|
|
ObTableSchema &index_schema,
|
|
const ObString *ddl_stmt_str,
|
|
ObMySQLTransaction *sql_trans,
|
|
const uint64_t tenant_data_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t new_table_id = index_schema.get_table_id();
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLSQLTransaction tmp_trans(schema_service_);
|
|
ObMySQLTransaction &trans = OB_ISNULL(sql_trans) ? tmp_trans : *sql_trans;
|
|
const uint64_t tenant_id = index_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(sql_trans)
|
|
&& OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObDDLLock::lock_for_add_drop_index_in_trans(data_table_schema, index_schema, trans))) {
|
|
LOG_WARN("failed to lock for drop index", KR(ret));
|
|
} else if (OB_FAIL(drop_table_in_trans(
|
|
schema_guard, index_schema, true, true,
|
|
false, ddl_stmt_str, &trans, NULL, NULL))) {
|
|
LOG_WARN("drop_table failed", K(index_schema), KR(ret));
|
|
} else if (FALSE_IT(new_table_id = OB_INVALID_ID)) {
|
|
} else if (OB_FAIL(schema_service->fetch_new_table_id(
|
|
index_schema.get_tenant_id(), new_table_id))) {
|
|
LOG_WARN("fail to fetch new table id", KR(ret));
|
|
} else {
|
|
index_schema.set_index_status(INDEX_STATUS_UNAVAILABLE);
|
|
index_schema.set_table_id(new_table_id);
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(generate_tablet_id(index_schema))) {
|
|
LOG_WARN("failed to generate tablet id", K(ret));
|
|
} else if (OB_FAIL(create_index_or_mlog_table_in_trans(index_schema,
|
|
ddl_stmt_str, &trans, schema_guard,
|
|
false/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("create_table_in_trans failed", K(index_schema), KR(ret), K(ddl_stmt_str));
|
|
}
|
|
if (OB_ISNULL(sql_trans) && trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_index_status(const obrpc::ObUpdateIndexStatusArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObTableSchema *table = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
share::schema::ObIndexStatus new_status = arg.status_;
|
|
const uint64_t table_id = arg.index_table_id_;
|
|
const uint64_t tenant_id = arg.exec_tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == table_id
|
|
|| new_status <= INDEX_STATUS_NOT_FOUND
|
|
|| new_status >= INDEX_STATUS_MAX) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arugment", KT(table_id), K(new_status));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, table))) {
|
|
LOG_WARN("get table schema failed", KR(ret), K(tenant_id), KT(table_id));
|
|
} else if (NULL == table) {
|
|
// maybe table has already been deleted, do-nothing
|
|
} else if (INDEX_STATUS_INDEX_ERROR == new_status && arg.convert_status_) {
|
|
// Distinguish the failure of the physical recovery index rebuild
|
|
const ObSimpleTenantSchema *tenant_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("fail to get tenant info", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", KR(ret), K(tenant_id));
|
|
} else if (tenant_schema->is_restore()) {
|
|
new_status = INDEX_STATUS_RESTORE_INDEX_ERROR;
|
|
LOG_INFO("convert error index status", KR(ret), K(new_status));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && NULL != table) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
const ObString *ddl_stmt_str = arg.ddl_stmt_str_.empty() ? nullptr : &arg.ddl_stmt_str_;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (is_available_index_status(new_status) && !table->is_unavailable_index()) {
|
|
ret = OB_EAGAIN;
|
|
LOG_WARN("set index status to available, but previous status is not unavailable, which is not expected behavior", KR(ret), KPC(table), K(new_status));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.update_index_status(
|
|
tenant_id, table->get_data_table_id(), table_id,
|
|
new_status, arg.in_offline_ddl_white_list_, trans, ddl_stmt_str))) {
|
|
LOG_WARN("update index status failed", K(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && arg.task_id_ != 0) {
|
|
if (table->get_index_status() != new_status && new_status == INDEX_STATUS_AVAILABLE) {
|
|
const ObTableSchema *data_table_schema = nullptr;
|
|
ObTableLockOwnerID owner_id;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table->get_data_table_id(), data_table_schema))) {
|
|
LOG_WARN("failed to get data table schema", K(ret));
|
|
} else if (nullptr == data_table_schema) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("data table has been deleted", K(ret), K(table->get_data_table_id()));
|
|
} else if (OB_FAIL(owner_id.convert_from_value(ObLockOwnerType::DEFAULT_OWNER_TYPE,
|
|
arg.task_id_))) {
|
|
LOG_WARN("failed to get owner id", K(ret), K(arg.task_id_));
|
|
} else if (OB_FAIL(ObDDLLock::unlock_for_add_drop_index(*data_table_schema,
|
|
table->get_table_id(),
|
|
table->is_global_index_table(),
|
|
owner_id,
|
|
trans))) {
|
|
LOG_WARN("failed to unlock ddl lock", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
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;
|
|
}
|
|
}
|
|
DEBUG_SYNC(AFTER_UPDATE_INDEX_STATUS);
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_mview_status(const obrpc::ObUpdateMViewStatusArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObTableSchema *mview_schema = nullptr;
|
|
ObSchemaGetterGuard schema_guard;
|
|
enum ObMVAvailableFlag mv_available_flag = arg.mv_available_flag_;
|
|
const uint64_t mview_table_id = arg.mview_table_id_;
|
|
const uint64_t tenant_id = arg.exec_tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_INVALID_ID == mview_table_id
|
|
|| (ObMVAvailableFlag::IS_MV_UNAVAILABLE != mv_available_flag
|
|
&& ObMVAvailableFlag::IS_MV_AVAILABLE != mv_available_flag)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid ObUpdateMViewStatusArg", KR(ret), K(arg));
|
|
} 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", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, mview_table_id, mview_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(tenant_id), KT(mview_table_id));
|
|
} else if (OB_ISNULL(mview_schema)) {
|
|
// maybe table has already been deleted, do-nothing
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (schema_service == nullptr) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("schema_service is null", KR(ret), KP(schema_service));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id));
|
|
} else {
|
|
SMART_VAR(ObTableSchema, new_mview_schema) {
|
|
if (OB_FAIL(new_mview_schema.assign(*mview_schema))) {
|
|
LOG_WARN("fail to assign mview schema", KR(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
new_mview_schema.set_table_id(mview_table_id);
|
|
new_mview_schema.set_mv_available(mv_available_flag);
|
|
new_mview_schema.set_schema_version(new_schema_version);
|
|
new_mview_schema.set_in_offline_ddl_white_list(arg.in_offline_ddl_white_list_);
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(schema_service->get_table_sql_service().update_mview_status(
|
|
new_mview_schema, trans))) {
|
|
LOG_WARN("failed to update mview status", KR(ret), K(mview_table_id), K(mv_available_flag));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("end transaction failed", KR(ret), KR(tmp_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", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::upgrade_table_schema(const obrpc::ObUpgradeTableSchemaArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
FLOG_INFO("[UPGRADE] begin upgrade system table", K(arg));
|
|
const uint64_t tenant_id = arg.get_tenant_id();
|
|
const uint64_t table_id = arg.get_table_id();
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", KR(ret), K(arg));
|
|
} else {
|
|
HEAP_VAR(ObTableSchema, hard_code_schema) {
|
|
ObSchemaGetterGuard schema_guard;
|
|
bool exist = false;
|
|
if (OB_FAIL(get_hard_code_system_table_schema_(
|
|
tenant_id, table_id, hard_code_schema))) {
|
|
LOG_WARN("fail to get hard code table schema", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("get_schema_guard with version in inner table failed", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.check_table_exist(tenant_id, table_id, exist))) {
|
|
LOG_WARN("fail to check table exist", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (!exist) {
|
|
if (OB_FAIL(create_system_table_(schema_guard, hard_code_schema))) {
|
|
LOG_WARN("fail to create system table", KR(ret), K(tenant_id), K(table_id));
|
|
}
|
|
} else if (OB_FAIL(alter_system_table_column_(schema_guard, hard_code_schema))) {
|
|
LOG_WARN("fail to alter system table's column", KR(ret), K(tenant_id), K(table_id));
|
|
}
|
|
}
|
|
}
|
|
FLOG_INFO("[UPGRADE] end upgrade system table",
|
|
KR(ret), K(tenant_id), K(table_id),
|
|
"cost", ObTimeUtility::current_time() - start_time);
|
|
return ret;
|
|
|
|
}
|
|
|
|
int ObDDLService::get_hard_code_system_table_schema_(
|
|
const uint64_t tenant_id,
|
|
const uint64_t table_id,
|
|
ObTableSchema &hard_code_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_INVALID_TENANT_ID == tenant_id
|
|
&& !is_system_table(table_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id or table_id", KR(ret), K(tenant_id), K(table_id));
|
|
} else {
|
|
bool finded = false;
|
|
const schema_create_func *creator_ptr_array[] = {
|
|
share::core_table_schema_creators,
|
|
share::sys_table_schema_creators, NULL };
|
|
for (const schema_create_func **creator_ptr_ptr = creator_ptr_array;
|
|
OB_SUCC(ret) && !finded && OB_NOT_NULL(*creator_ptr_ptr); ++creator_ptr_ptr) {
|
|
for (const schema_create_func *creator_ptr = *creator_ptr_ptr;
|
|
OB_SUCC(ret) && !finded && OB_NOT_NULL(*creator_ptr); ++creator_ptr) {
|
|
hard_code_schema.reset();
|
|
bool exist = false;
|
|
if (OB_FAIL((*creator_ptr)(hard_code_schema))) {
|
|
LOG_WARN("create table schema failed", KR(ret));
|
|
} else if (!is_sys_tenant(tenant_id)
|
|
&& OB_FAIL(ObSchemaUtils::construct_tenant_space_full_table(
|
|
tenant_id, hard_code_schema))) {
|
|
LOG_WARN("fail to construct tenant space table", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ObSysTableChecker::is_inner_table_exist(
|
|
tenant_id, hard_code_schema, exist))) {
|
|
LOG_WARN("fail to check inner table exist",
|
|
KR(ret), K(tenant_id), K(hard_code_schema));
|
|
} else if (!exist) {
|
|
// skip
|
|
} else if (hard_code_schema.get_table_id() == table_id) {
|
|
finded = true;
|
|
}
|
|
} // end for
|
|
} // end for
|
|
|
|
if (OB_SUCC(ret) && !finded) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("hard code table schema not exist", KR(ret), K(tenant_id), K(table_id));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_system_table_(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &hard_code_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool if_not_exist = true;
|
|
ObArray<ObTableSchema> table_schemas;
|
|
// the following variable is not used
|
|
ObString ddl_stmt_str;
|
|
ObErrorInfo error_info;
|
|
obrpc::ObSequenceDDLArg sequence_ddl_arg;
|
|
uint64_t last_replay_log_id = 0;
|
|
ObArray<ObDependencyInfo> dep_infos;
|
|
ObArray<ObMockFKParentTableSchema> mock_fk_parent_table_schema_array;
|
|
int64_t ddl_task_id = 0;
|
|
// sys index、sys lob table will be added in create_user_tables()
|
|
if (OB_FAIL(table_schemas.push_back(hard_code_schema))) {
|
|
LOG_WARN("fail to push back new table schema", KR(ret));
|
|
} else if (OB_FAIL(create_user_tables(if_not_exist, ddl_stmt_str,
|
|
error_info, table_schemas, schema_guard, sequence_ddl_arg,
|
|
last_replay_log_id, &dep_infos, mock_fk_parent_table_schema_array,
|
|
ddl_task_id))) {
|
|
LOG_WARN("fail to create system table", KR(ret), K(hard_code_schema));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_add_system_table_column_(const ObColumnSchemaV2 &column, bool &can_add)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
can_add = false;
|
|
// the newly added column of system table should satisfy one of the following requirements
|
|
// 1. the column is nullable
|
|
// 2. the column is not nullable and the default value is not null
|
|
if (!column.is_valid()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column is invalid", KR(ret), K(column));
|
|
} else if (column.is_nullable()) {
|
|
can_add = true;
|
|
} else if (!column.is_nullable() && !column.get_cur_default_value().is_null()) {
|
|
can_add = true;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_system_table_column_(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &hard_code_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = hard_code_schema.get_tenant_id();
|
|
const uint64_t table_id = hard_code_schema.get_table_id();
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
ObArray<uint64_t> add_column_ids;
|
|
ObArray<uint64_t> alter_column_ids;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_id, orig_table_schema))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table not exist", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_FAIL(ObRootInspection::check_and_get_system_table_column_diff(
|
|
*orig_table_schema, hard_code_schema, add_column_ids, alter_column_ids))) {
|
|
LOG_WARN("fail to check system table's column schemas", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (0 == add_column_ids.count() && 0 == alter_column_ids.count()) {
|
|
LOG_INFO("system table's column schemas not changed, just skip", KR(ret), K(tenant_id), K(table_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
HEAP_VAR(ObTableSchema, new_table_schema) {
|
|
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(new_table_schema.assign(*orig_table_schema))) {
|
|
LOG_WARN("fail to assign table schema", KR(ret), K(tenant_id), K(table_id));
|
|
} else {
|
|
const ObColumnSchemaV2 *hard_code_column = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < add_column_ids.count(); i++) {
|
|
const uint64_t column_id = add_column_ids.at(i);
|
|
bool can_add = false;
|
|
if (OB_ISNULL(hard_code_column = hard_code_schema.get_column_schema(column_id))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get column schema", KR(ret), K(tenant_id), K(table_id), K(column_id));
|
|
} else if (OB_FAIL(check_add_system_table_column_(*hard_code_column, can_add))) {
|
|
LOG_WARN("fail check add system table column", KR(ret), K(tenant_id), K(table_id), K(column_id));
|
|
} else if (!can_add) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("the column of system table is not allowed to add", KR(ret),
|
|
"table_name", hard_code_schema.get_table_name(),
|
|
"column_name", hard_code_column->get_column_name(),
|
|
KPC(hard_code_column));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "Adding column not satisfied with requirements to inner table is");
|
|
} else if (OB_FAIL(new_table_schema.add_column(*hard_code_column))) {
|
|
LOG_WARN("fail to add column", KR(ret), KPC(hard_code_column));
|
|
}
|
|
} // end for
|
|
|
|
ObColumnSchemaV2 new_column;
|
|
bool for_view = false;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < alter_column_ids.count(); i++) {
|
|
const uint64_t column_id = alter_column_ids.at(i);
|
|
if (OB_ISNULL(hard_code_column = hard_code_schema.get_column_schema(column_id))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get column schema", KR(ret), K(tenant_id), K(table_id), K(column_id));
|
|
} else if (OB_FAIL(new_column.assign(*hard_code_column))) {
|
|
LOG_WARN("fail to assign column", KR(ret), KPC(hard_code_column));
|
|
} else if (OB_FAIL(new_table_schema.alter_column(new_column, share::schema::ObTableSchema::CHECK_MODE_ONLINE, for_view))) {
|
|
LOG_WARN("fail to alter column", KR(ret), K(new_column));
|
|
}
|
|
} // end for
|
|
|
|
// increase progressive_merge_round to update schema in sstable
|
|
new_table_schema.set_progressive_merge_round(orig_table_schema->get_progressive_merge_round() + 1);
|
|
|
|
if (FAILEDx(ddl_operator.batch_update_system_table_columns(trans,
|
|
*orig_table_schema, new_table_schema, add_column_ids, alter_column_ids, NULL))) {
|
|
LOG_WARN("fail to batch update columns", KR(ret), K(new_table_schema));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN_RET(tmp_ret, "trans end failed", "is_commit", OB_SUCCESS == ret, K(tmp_ret));
|
|
ret = (OB_SUCC(ret)) ? tmp_ret : ret;
|
|
}
|
|
}
|
|
if (FAILEDx(publish_schema(tenant_id))) {
|
|
LOG_WARN("fail to publish schema", KR(ret), K(tenant_id));
|
|
}
|
|
|
|
} // end HEAP_VAR
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::add_table_schema(
|
|
ObTableSchema &table_schema,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(table_schema.get_tenant_id(), tenant_data_version))) {
|
|
LOG_WARN("get min data version failed", K(ret), K(table_schema));
|
|
} else if (OB_FAIL(create_index_or_mlog_table_in_trans(table_schema, NULL, NULL, schema_guard,
|
|
false/*need_check_tablet_cnt*/, tenant_data_version))) {
|
|
LOG_WARN("create_table_in_trans failed", KR(ret), K(table_schema));
|
|
}
|
|
LOG_INFO("[UPGRADE] add inner table", KR(ret),
|
|
"tenant_id", table_schema.get_tenant_id(),
|
|
"table_id", table_schema.get_table_id(),
|
|
"table_name", table_schema.get_table_name(),
|
|
"cost", ObTimeUtility::current_time() - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_inner_table(const share::schema::ObTableSchema &table_schema, const bool delete_priv)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
ObString *stmt = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
const uint64_t table_id = table_schema.get_table_id();
|
|
const ObSimpleTableSchemaV2 * table = NULL;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (!is_inner_table(table_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table not inner table", KR(ret), K(tenant_id), K(table_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(schema_guard.get_simple_table_schema(tenant_id, table_id, table))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), K(table_id));
|
|
} else if (OB_ISNULL(table)) {
|
|
// bugfix:
|
|
// virtual table index may be dropped with virtual table, so here we ignore OB_TABLE_NOT_EXIST failure.
|
|
LOG_INFO("table has already been dropped, just ignore",
|
|
K(tenant_id), K(table_id), "table_name", table_schema.get_table_name());
|
|
} else if (OB_FAIL(drop_table_in_trans(schema_guard,
|
|
table_schema,
|
|
false,
|
|
table_schema.is_index_table(),
|
|
false, /* to recyclebin*/
|
|
stmt,
|
|
NULL, NULL, NULL, delete_priv))) {
|
|
LOG_WARN("drop table in transaction failed", KR(ret), K(tenant_id), K(table_id));
|
|
}
|
|
LOG_INFO("[UPGRADE] drop inner table", KR(ret),
|
|
K(tenant_id), K(table_id),
|
|
"table_name", table_schema.get_table_name(),
|
|
"cost", ObTimeUtility::current_time() - start_time);
|
|
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_, true, false, false, false);
|
|
ObSchemaService *schema_service = NULL;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
schema_service = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service must not null", K(ret));
|
|
} else {
|
|
ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_;
|
|
ObRefreshSchemaStatus tenant_status(OB_SYS_TENANT_ID,
|
|
OB_INVALID_TIMESTAMP, OB_INVALID_VERSION);
|
|
ObSysVariableSchema sys_variable;
|
|
tenant_schema.set_tenant_id(OB_SYS_TENANT_ID);
|
|
const ObSchemaOperationType operation_type = OB_DDL_MAX_OP;
|
|
// When the system tenant is created, the log_operation of the system variable is not recorded separately
|
|
// The update of __all_core_table must be a single-partition transaction.
|
|
// Failure to create a tenant will result in garbage data, but it will not affect
|
|
int64_t refreshed_schema_version = 0; // won't lock
|
|
common::ObConfigPairs config;
|
|
common::ObSEArray<common::ObConfigPairs, 1> init_configs;
|
|
if (OB_ISNULL(schema_status_proxy)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_status_proxy is null", K(ret));
|
|
} else if (OB_FAIL(ObDDLService::gen_tenant_init_config(
|
|
OB_SYS_TENANT_ID, DATA_CURRENT_VERSION, config))) {
|
|
} else if (OB_FAIL(init_configs.push_back(config))) {
|
|
LOG_WARN("fail to push back config", KR(ret), K(config));
|
|
} else if (OB_FAIL(schema_status_proxy->set_tenant_schema_status(tenant_status))) {
|
|
LOG_WARN("init tenant create partition status failed", K(ret), K(tenant_status));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret));
|
|
} else if (OB_FAIL(init_system_variables(arg, tenant_schema, sys_variable))) {
|
|
LOG_WARN("fail to init tenant sys params", K(ret), K(tenant_schema));
|
|
} else if (OB_FAIL(ddl_operator.create_tenant(tenant_schema, OB_DDL_ADD_TENANT, trans))) {
|
|
LOG_WARN("create tenant failed", K(tenant_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.replace_sys_variable(
|
|
sys_variable, tenant_schema.get_schema_version(), trans, operation_type))) {
|
|
LOG_WARN("fail to replace sys variable", K(ret), K(sys_variable));
|
|
} else if (OB_FAIL(ddl_operator.init_tenant_env(
|
|
tenant_schema, sys_variable, share::PRIMARY_TENANT_ROLE, SCN::max_scn(), init_configs, trans))) {
|
|
LOG_WARN("init tenant env failed", K(tenant_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.insert_tenant_merge_info(OB_DDL_ADD_TENANT, tenant_schema, trans))) {
|
|
LOG_WARN("fail to insert tenant merge info", KR(ret));
|
|
} else if (OB_FAIL(ObServiceEpochProxy::init_service_epoch(
|
|
trans,
|
|
OB_SYS_TENANT_ID,
|
|
0, /*freeze_service_epoch*/
|
|
0, /*arbitration_service_epoch*/
|
|
0, /*server_zone_op_service_epoch*/
|
|
0, /*heartbeat_service_epoch*/
|
|
0 /* service_name_epoch */))) {
|
|
LOG_WARN("fail to init service epoch", KR(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
LOG_INFO("end create tenant", "is_commit", OB_SUCCESS == 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_SUCCESS == ret, K(temp_ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// If tenant config version in RS is valid first and ddl trans doesn't commit,
|
|
// observer may read from empty __tenant_parameter successfully and raise its tenant config version,
|
|
// which makes some initial tenant configs are not actually updated before related observer restarts.
|
|
// To fix this problem, tenant config version in RS should be valid after ddl trans commits.
|
|
const int64_t config_version = omt::ObTenantConfig::INITIAL_TENANT_CONF_VERSION + 1;
|
|
if (OB_FAIL(OTC_MGR.set_tenant_config_version(OB_SYS_TENANT_ID, config_version))) {
|
|
LOG_WARN("failed to set tenant config version", KR(ret), "tenant_id", OB_SYS_TENANT_ID);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/* The reconstruction of zone_list follows the following rules:
|
|
* 1. When creating tenant, the zone_list specified by the user will be ignored. Use the zone list of resource_pool.
|
|
* We still retain the zone_list column in the all_tenant table and the zone_list field in the tenantSchema.
|
|
* 2. When creating table, the zone_list specified by the user will be ignored.
|
|
* If the create table does not specify locality, the locality of the table is not filled in,
|
|
* and the zone_list is not filled in.
|
|
* Both locality and zone_list are inherited from the tenant to which the table belongs.
|
|
* 3. When creating table, the user specified zone_list and locality, the user-specified zone_list will be ignored;
|
|
* a zone list will be calculated based on the user-specified locality and the zone list of the resource pool,
|
|
* and the zone_list will be filled into the zone_list column of __all_table and in the zone_list field of TableSchema.
|
|
* 4. When modifying locality, we are not allowed to modify F{3}@region_hz to F{2}@region_hz;
|
|
* because the semantics of this modification is not clear enough,
|
|
* it is impossible to clearly derive the change of zone
|
|
*/
|
|
int ObDDLService::create_tenant(
|
|
const ObCreateTenantArg &arg,
|
|
UInt64 &tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObString &tenant_name = arg.tenant_schema_.get_tenant_name_str();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_MISS_ARGUMENT;
|
|
if (tenant_name.empty()) {
|
|
LOG_USER_ERROR(OB_MISS_ARGUMENT, "tenant name");
|
|
} else if (arg.pool_list_.count() <= 0) {
|
|
LOG_USER_ERROR(OB_MISS_ARGUMENT, "resource_pool_list");
|
|
}
|
|
LOG_WARN("missing arg to create tenant", KR(ret), K(arg));
|
|
} else if (tenant_name.case_compare(OB_DIAG_TENANT_NAME) == 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_name \'diag\' is reserved for diagnose tenant", KR(ret), K(arg));
|
|
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "tenant_name (\'diag\' is reserved for diagnose tenant)");
|
|
} else if (GCONF.in_upgrade_mode()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("create tenant when cluster is upgrading not allowed", K(ret));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "create tenant when cluster is upgrading");
|
|
} else if (OB_ISNULL(sql_proxy_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", KR(ret));
|
|
} else {
|
|
// check tenant exist
|
|
bool tenant_exist = false;
|
|
if (OB_NOT_NULL(schema_guard.get_tenant_info(tenant_name))) {
|
|
tenant_exist = true;
|
|
} else {
|
|
if (!arg.is_restore_tenant()) {
|
|
if (OB_FAIL(ObRestoreUtil::check_has_physical_restore_job(*sql_proxy_, tenant_name, tenant_exist))) {
|
|
LOG_WARN("failed to check has physical restore job", KR(ret), K(tenant_name));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (tenant_exist) {
|
|
// do nothing
|
|
} else if (!arg.is_clone_tenant()) {
|
|
// check whether has clone job, if has clone job then tenant_exist should be true
|
|
if (OB_FAIL(ObTenantCloneUtil::check_clone_tenant_exist(*sql_proxy_, tenant_name, tenant_exist))) {
|
|
LOG_WARN("failed to check clone tenant exist", KR(ret), K(tenant_name));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (tenant_exist) {
|
|
if (arg.if_not_exist_) {
|
|
ret = OB_SUCCESS;
|
|
LOG_USER_NOTE(OB_TENANT_EXIST, to_cstring(tenant_name));
|
|
LOG_INFO("tenant already exists, not need to create", KR(ret), K(tenant_name));
|
|
} else {
|
|
ret = OB_TENANT_EXIST;
|
|
LOG_USER_ERROR(OB_TENANT_EXIST, to_cstring(tenant_name));
|
|
LOG_WARN("tenant already exists", KR(ret), K(tenant_name));
|
|
}
|
|
} else if (OB_FAIL(inner_create_tenant_(arg, schema_guard, tenant_id))) {
|
|
LOG_WARN("fail to create tenant", KR(ret), K(arg));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_tenant_schema(
|
|
const ObCreateTenantArg &arg,
|
|
const share::ObTenantRole &tenant_role,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObTenantSchema &user_tenant_schema,
|
|
ObSysVariableSchema &user_sys_variable,
|
|
ObTenantSchema &meta_tenant_schema,
|
|
ObSysVariableSchema &meta_sys_variable,
|
|
common::ObIArray<common::ObConfigPairs> &init_configs)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t user_tenant_id = arg.tenant_schema_.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else if (OB_ISNULL(schema_service_)
|
|
|| OB_ISNULL(schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(schema_service));
|
|
} else if (OB_FAIL(schema_service_->get_schema_service()->fetch_new_tenant_id(user_tenant_id))) {
|
|
LOG_WARN("fetch_new_tenant_id failed", KR(ret));
|
|
} else if (OB_INVALID_ID == user_tenant_id) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tenant id is invalid", KR(ret), K(user_tenant_id));
|
|
} else if (OB_FAIL(user_tenant_schema.assign(arg.tenant_schema_))) {
|
|
LOG_WARN("fail to assign user tenant schema", KR(ret), K(arg));
|
|
} else if (OB_FAIL(meta_tenant_schema.assign(user_tenant_schema))) {
|
|
LOG_WARN("fail to assign meta tenant schema", KR(ret), K(arg));
|
|
} else if (OB_FAIL(check_create_tenant_schema(
|
|
arg.pool_list_, meta_tenant_schema, schema_guard))) {
|
|
LOG_WARN("check tenant schema failed", KR(ret), K(meta_tenant_schema), K(arg));
|
|
} else if (OB_FAIL(check_create_tenant_schema(
|
|
arg.pool_list_, user_tenant_schema, schema_guard))) {
|
|
LOG_WARN("check tenant schema failed", KR(ret), K(user_tenant_schema), K(arg));
|
|
} else {
|
|
// user tenant
|
|
if (OB_SUCC(ret)) {
|
|
user_tenant_schema.set_tenant_id(user_tenant_id);
|
|
if (!tenant_role.is_primary()) {
|
|
//standby cluster and restore tenant no need init user tenant system variables
|
|
if (tenant_role.is_restore() || tenant_role.is_clone()) {
|
|
user_tenant_schema.set_status(TENANT_STATUS_RESTORE);
|
|
} else if (arg.is_creating_standby_) {
|
|
user_tenant_schema.set_status(TENANT_STATUS_CREATING_STANDBY);
|
|
}
|
|
} else if (OB_FAIL(init_system_variables(arg, user_tenant_schema, user_sys_variable))) {
|
|
LOG_WARN("fail to init tenant sys params", KR(ret), K(user_tenant_schema), K(arg));
|
|
}
|
|
}
|
|
// meta tenant
|
|
if (OB_SUCC(ret)) {
|
|
const uint64_t meta_tenant_id = gen_meta_tenant_id(user_tenant_id);
|
|
ObSqlString table_name;
|
|
if (OB_FAIL(table_name.assign_fmt("META$%ld", user_tenant_id))) {
|
|
LOG_WARN("fail to assign tenant name",KR(ret), K(user_tenant_id));
|
|
} else {
|
|
meta_tenant_schema.set_tenant_id(meta_tenant_id);
|
|
meta_tenant_schema.set_tenant_name(table_name.string());
|
|
meta_tenant_schema.set_compatibility_mode(ObCompatibilityMode::MYSQL_MODE);
|
|
meta_tenant_schema.set_charset_type(ObCharset::get_default_charset());
|
|
meta_tenant_schema.set_collation_type(ObCharset::get_default_collation(
|
|
ObCharset::get_default_charset()));
|
|
if (OB_FAIL(init_system_variables(arg, meta_tenant_schema, meta_sys_variable))) {
|
|
LOG_WARN("fail to init tenant sys params", KR(ret), K(meta_tenant_schema), K(arg));
|
|
} else if (OB_FAIL(check_tenant_primary_zone_(schema_guard, meta_tenant_schema))) {
|
|
LOG_WARN("fail to check tenant primary zone", KR(ret), K(meta_tenant_schema));
|
|
}
|
|
}
|
|
}
|
|
// init tenant configs
|
|
if (OB_SUCC(ret)) {
|
|
init_configs.reset();
|
|
common::ObConfigPairs config;
|
|
const uint64_t meta_tenant_id = gen_meta_tenant_id(user_tenant_id);
|
|
if (OB_FAIL(gen_tenant_init_config(meta_tenant_id, DATA_CURRENT_VERSION, config))) {
|
|
LOG_WARN("fail to gen tenant init config", KR(ret), K(meta_tenant_id));
|
|
} else if (OB_FAIL(init_configs.push_back(config))) {
|
|
LOG_WARN("fail to push back config", KR(ret), K(meta_tenant_id), K(config));
|
|
// } else if (!is_sys_tenant(user_tenant_id) && !arg.is_creating_standby_) {
|
|
//
|
|
// FIXME(msy164651) : Data Version scheme is not suitable for Create
|
|
// Standby Tenant. The DDL will fail because GET_MIN_DATA_VERSION will
|
|
// return OB_ENTRY_NOT_EXIST;
|
|
//
|
|
// msy164651 wil fix it.
|
|
} else if (!is_sys_tenant(user_tenant_id)) {
|
|
/**
|
|
* When the primary tenant has done upgrade and create a standby tenant for it,
|
|
* the standby tenant must also perform the upgrade process. Don't set compatible_version for
|
|
* standby tenant so that it can be upgraded from 0 to ensure that the compatible_version matches
|
|
* the internal table. and it also prevent loss of the upgrade action.
|
|
*/
|
|
uint64_t compatible_version = (arg.is_restore_tenant() || arg.is_clone_tenant())
|
|
? arg.compatible_version_
|
|
: DATA_CURRENT_VERSION;
|
|
if (OB_FAIL(gen_tenant_init_config(user_tenant_id, compatible_version, config))) {
|
|
LOG_WARN("fail to gen tenant init config", KR(ret), K(user_tenant_id), K(compatible_version));
|
|
} else if (OB_FAIL(init_configs.push_back(config))) {
|
|
LOG_WARN("fail to push back config", KR(ret), K(user_tenant_id), K(config));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::gen_tenant_init_config(
|
|
const uint64_t tenant_id,
|
|
const uint64_t compatible_version,
|
|
common::ObConfigPairs &tenant_config)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObString config_name("compatible");
|
|
ObString config_value;
|
|
char version[common::OB_CLUSTER_VERSION_LENGTH] = {'\0'};
|
|
int64_t len = ObClusterVersion::print_version_str(
|
|
version, common::OB_CLUSTER_VERSION_LENGTH, compatible_version);
|
|
tenant_config.reset();
|
|
(void) tenant_config.init(tenant_id);
|
|
if (len < 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid version", KR(ret), K(tenant_id), K(compatible_version));
|
|
} else if (FALSE_IT(config_value.assign_ptr(version, len))) {
|
|
} else if (OB_FAIL(tenant_config.add_config(config_name, config_value))) {
|
|
LOG_WARN("fail to add config", KR(ret), K(config_name), K(config_value));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::init_schema_status(
|
|
const uint64_t tenant_id,
|
|
const share::ObTenantRole &tenant_role)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else if (is_meta_tenant(tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_status_proxy)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to init schema status", KR(ret), K(tenant_id));
|
|
} else {
|
|
// user tenant
|
|
if (OB_SUCC(ret) && is_user_tenant(tenant_id)) {
|
|
ObRefreshSchemaStatus partition_status(tenant_id,
|
|
OB_INVALID_TIMESTAMP, OB_INVALID_VERSION);
|
|
if (!tenant_role.is_primary()) {
|
|
// reset tenant's schema status in standby cluster or in physical restore
|
|
partition_status.snapshot_timestamp_ = 0;
|
|
partition_status.readable_schema_version_ = 0;
|
|
}
|
|
if (FAILEDx(schema_status_proxy->set_tenant_schema_status(partition_status))) {
|
|
LOG_WARN("fail to set refreshed schema status", KR(ret), K(partition_status));
|
|
}
|
|
}
|
|
// sys or meta tenant
|
|
if (OB_SUCC(ret)) {
|
|
// sys tenant's meta tenant is itself
|
|
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
|
|
ObRefreshSchemaStatus meta_partition_status(meta_tenant_id, OB_INVALID_TIMESTAMP, OB_INVALID_VERSION);
|
|
if (OB_FAIL(schema_status_proxy->set_tenant_schema_status(meta_partition_status))) {
|
|
LOG_WARN("fail to set refreshed schema status", KR(ret), K(meta_partition_status));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::inner_create_tenant_(
|
|
const ObCreateTenantArg &arg,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
UInt64 &tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
share::ObTenantRole tenant_role = share::PRIMARY_TENANT_ROLE;
|
|
SCN recovery_until_scn = SCN::max_scn();
|
|
uint64_t user_tenant_id = OB_INVALID_TENANT_ID;
|
|
uint64_t meta_tenant_id = OB_INVALID_TENANT_ID;
|
|
palf::PalfBaseInfo user_palf_base_info;
|
|
palf::PalfBaseInfo meta_palf_base_info;
|
|
bool create_ls_with_palf = false;
|
|
int64_t paxos_replica_number = 0;
|
|
ObSEArray<ObConfigPairs, 2> init_configs;
|
|
|
|
HEAP_VARS_4((ObTenantSchema, user_tenant_schema),
|
|
(ObTenantSchema, meta_tenant_schema),
|
|
(ObSysVariableSchema, user_sys_variable),
|
|
(ObSysVariableSchema, meta_sys_variable)) {
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else if (arg.is_restore_tenant() || arg.is_clone_tenant()) {
|
|
tenant_role = arg.is_restore_tenant() ? share::RESTORE_TENANT_ROLE : share::CLONE_TENANT_ROLE;
|
|
recovery_until_scn = arg.recovery_until_scn_;
|
|
user_palf_base_info = arg.palf_base_info_;
|
|
create_ls_with_palf = true;
|
|
} else if (arg.is_standby_tenant()) {
|
|
tenant_role = share::STANDBY_TENANT_ROLE;
|
|
} else {
|
|
tenant_role = share::PRIMARY_TENANT_ROLE;
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(generate_tenant_schema(
|
|
arg, tenant_role, schema_guard,
|
|
user_tenant_schema, user_sys_variable,
|
|
meta_tenant_schema, meta_sys_variable,
|
|
init_configs))) {
|
|
LOG_WARN("fail to generate tenant schema", KR(ret), K(arg), K(tenant_role));
|
|
} else if (user_tenant_schema.get_arbitration_service_status().is_enable_like()
|
|
&& OB_FAIL(user_tenant_schema.get_paxos_replica_num(schema_guard, paxos_replica_number))) {
|
|
LOG_WARN("fail to get paxos replica number", KR(ret), K(user_tenant_schema));
|
|
} else if (user_tenant_schema.get_arbitration_service_status().is_enable_like()
|
|
&& paxos_replica_number != 2 && paxos_replica_number != 4) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "The number of paxos replicas in locality is neither 2 nor 4, create tenant with arbitration service");
|
|
LOG_WARN("can not create tenant, because tenant with arb service, locality must be 2F or 4F", KR(ret), K(user_tenant_schema), K(paxos_replica_number));
|
|
} else if (FALSE_IT(user_tenant_id = user_tenant_schema.get_tenant_id())) {
|
|
} else if (FALSE_IT(meta_tenant_id = meta_tenant_schema.get_tenant_id())) {
|
|
} else if (OB_FAIL(init_schema_status(
|
|
user_tenant_schema.get_tenant_id(), tenant_role))) {
|
|
LOG_WARN("fail to init schema status", KR(ret), K(user_tenant_id));
|
|
} else if (OB_FAIL(create_tenant_schema(
|
|
arg, schema_guard, user_tenant_schema,
|
|
meta_tenant_schema, init_configs))) {
|
|
LOG_WARN("fail to create tenant schema", KR(ret), K(arg));
|
|
} else if (OB_FAIL(add_extra_tenant_init_config_(user_tenant_id, init_configs))) {
|
|
LOG_WARN("fail to add_extra_tenant_init_config", KR(ret), K(user_tenant_id));
|
|
} else {
|
|
DEBUG_SYNC(BEFORE_CREATE_META_TENANT);
|
|
// create ls/tablet/schema in tenant space
|
|
// TODO@jingyu.cr: need to support create clone tenant from lower version, mark compatible version for unit
|
|
ObArray<ObResourcePoolName> pools;
|
|
if (OB_FAIL(get_pools(arg.pool_list_, pools))) {
|
|
LOG_WARN("get_pools failed", KR(ret), K(arg));
|
|
} else if (OB_FAIL(create_normal_tenant(meta_tenant_id, pools, meta_tenant_schema, tenant_role,
|
|
recovery_until_scn, meta_sys_variable, false/*create_ls_with_palf*/, meta_palf_base_info, init_configs,
|
|
arg.is_standby_tenant(), arg.log_restore_source_, arg.source_tenant_id_))) {
|
|
LOG_WARN("fail to create meta tenant", KR(ret), K(meta_tenant_id), K(pools), K(meta_sys_variable),
|
|
K(tenant_role), K(recovery_until_scn), K(meta_palf_base_info), K(init_configs), K(arg));
|
|
} else {
|
|
ObString empty_str;
|
|
DEBUG_SYNC(BEFORE_CREATE_USER_TENANT);
|
|
if (OB_FAIL(create_normal_tenant(user_tenant_id, pools, user_tenant_schema, tenant_role,
|
|
recovery_until_scn, user_sys_variable, create_ls_with_palf, user_palf_base_info, init_configs,
|
|
false /* is_creating_standby */, empty_str, arg.source_tenant_id_))) {
|
|
LOG_WARN("fail to create user tenant", KR(ret), K(user_tenant_id), K(pools), K(user_sys_variable),
|
|
K(tenant_role), K(recovery_until_scn), K(user_palf_base_info), K(arg));
|
|
}
|
|
}
|
|
// drop tenant if create tenant failed.
|
|
// meta tenant will be force dropped with its user tenant.
|
|
if (OB_FAIL(ret) && tenant_role.is_primary()) {
|
|
//tenant_id can not fallback, so can not drop tenant in standby cluster
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = try_force_drop_tenant(user_tenant_schema))) {
|
|
LOG_WARN("fail to force drop tenant", KR(ret), KR(temp_ret), K(user_tenant_schema));
|
|
}
|
|
}
|
|
}
|
|
} // end HEAP_VARS_4
|
|
if (FAILEDx(create_tenant_end(meta_tenant_id))) {
|
|
LOG_WARN("failed to create tenant end", KR(ret), K(meta_tenant_id));
|
|
} else if (!tenant_role.is_primary()) {
|
|
LOG_INFO("restore or standby user tenant cannot create end", K(tenant_role),
|
|
K(user_tenant_id), K(arg));
|
|
} else if (OB_FAIL(create_tenant_end(user_tenant_id))) {
|
|
LOG_WARN("failed to create tenant end", KR(ret), K(user_tenant_id));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
tenant_id = user_tenant_id;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::try_force_drop_tenant(const ObTenantSchema &tenant_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else {
|
|
obrpc::ObDropTenantArg arg;
|
|
arg.tenant_name_ = tenant_schema.get_tenant_name();
|
|
arg.tenant_id_ = tenant_schema.get_tenant_id();
|
|
arg.if_exist_ = true;
|
|
arg.delay_to_drop_ = false;
|
|
ObSqlString sql;
|
|
if (OB_FAIL(sql.append_fmt("DROP TENANT IF EXISTS %s FORCE",
|
|
tenant_schema.get_tenant_name()))) {
|
|
LOG_WARN("fail to generate sql", KR(ret), "tenant_id", tenant_schema.get_tenant_id());
|
|
} else if (FALSE_IT(arg.ddl_stmt_str_ = sql.string())) {
|
|
} else if (OB_FAIL(drop_tenant(arg))) {
|
|
LOG_WARN("fail to drop tenant", KR(ret), K(arg));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// 1. create new tenant schema
|
|
// 2. grant resource pool to new tenant
|
|
int ObDDLService::create_tenant_schema(
|
|
const ObCreateTenantArg &arg,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObTenantSchema &user_tenant_schema,
|
|
ObTenantSchema &meta_tenant_schema,
|
|
const common::ObIArray<common::ObConfigPairs> &init_configs)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 1. start create tenant schema", K(arg));
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t user_tenant_id = user_tenant_schema.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("fail to check inner stat", KR(ret));
|
|
} else if (OB_ISNULL(schema_service_)
|
|
|| OB_ISNULL(sql_proxy_)
|
|
|| OB_ISNULL(unit_mgr_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(schema_service), KP_(sql_proxy),
|
|
KP_(unit_mgr));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = OB_INVALID_VERSION;
|
|
ObArray<ObResourcePoolName> pools;
|
|
common::ObArray<uint64_t> new_ug_id_array;
|
|
if (OB_FAIL(get_pools(arg.pool_list_, pools))) {
|
|
LOG_WARN("get_pools failed", KR(ret), K(arg));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(
|
|
OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("fail to get schema version", KR(ret), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed, ", KR(ret), K(refreshed_schema_version));
|
|
}
|
|
#ifdef OB_BUILD_ARBITRATION
|
|
// check arbitration service if needed
|
|
ObArbitrationServiceTableOperator arbitration_service_table_operator;
|
|
const ObString arbitration_service_key("default");
|
|
const bool lock_line = true;
|
|
ObArbitrationServiceInfo arbitration_service_info;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (meta_tenant_schema.get_arbitration_service_status()
|
|
!= user_tenant_schema.get_arbitration_service_status()) {
|
|
ret = OB_STATE_NOT_MATCH;
|
|
LOG_WARN("tenant has different arbitration service status with its meta tenant", KR(ret),
|
|
"meta_tenant_arb_status", meta_tenant_schema.get_arbitration_service_status(),
|
|
"user_tenant_arb_status", user_tenant_schema.get_arbitration_service_status());
|
|
} else if (meta_tenant_schema.get_arbitration_service_status().is_enabling()
|
|
|| meta_tenant_schema.get_arbitration_service_status().is_enabled()) {
|
|
if (OB_FAIL(arbitration_service_table_operator.get(
|
|
trans,
|
|
arbitration_service_key,
|
|
lock_line,
|
|
arbitration_service_info))) {
|
|
if (OB_ARBITRATION_SERVICE_NOT_EXIST == ret) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "arbitration service not exist, create tenant");
|
|
}
|
|
LOG_WARN("fail to get arbitration service", KR(ret), K(arbitration_service_key),
|
|
K(lock_line), K(arbitration_service_info));
|
|
}
|
|
}
|
|
#endif
|
|
// 1. create tenant schema
|
|
if (OB_SUCC(ret)) {
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.1. start create tenant schema", K(arg));
|
|
const int64_t tmp_start_time = ObTimeUtility::fast_current_time();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.create_tenant(meta_tenant_schema,
|
|
OB_DDL_ADD_TENANT_START, trans))) {
|
|
LOG_WARN("create tenant failed", KR(ret), K(meta_tenant_schema));
|
|
} else if (OB_FAIL(ddl_operator.create_tenant(user_tenant_schema,
|
|
OB_DDL_ADD_TENANT_START, trans, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("create tenant failed", KR(ret), K(user_tenant_schema));
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.1. finish create tenant schema", KR(ret), K(arg),
|
|
"cost", ObTimeUtility::fast_current_time() - tmp_start_time);
|
|
}
|
|
|
|
#ifdef OB_BUILD_TDE_SECURITY
|
|
if (OB_SUCC(ret)) {
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.1.1. start create root key", K(user_tenant_id));
|
|
const int64_t tmp_start_time = ObTimeUtility::fast_current_time();
|
|
ObArray<ObAddr> addrs;
|
|
bool need_create = false;
|
|
if (OB_FAIL(check_need_create_root_key(arg, need_create))) {
|
|
LOG_WARN("fail to check need create root key", K(ret));
|
|
} else if (!need_create) {
|
|
// do nothing
|
|
} else if (arg.is_creating_standby_) {
|
|
if (OB_FAIL(standby_create_root_key(user_tenant_id, arg, addrs))) {
|
|
LOG_WARN("failed to create root key", KR(ret), K(user_tenant_id), K(arg));
|
|
}
|
|
} else if (OB_FAIL(create_root_key(*rpc_proxy_, user_tenant_id, addrs))) {
|
|
LOG_WARN("fail to create root key", KR(ret), K(addrs));
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.1.1. finish create root key",
|
|
KR(ret), K(user_tenant_id), "cost", ObTimeUtility::fast_current_time() - tmp_start_time);
|
|
}
|
|
#endif
|
|
|
|
// 2. grant pool
|
|
if (OB_SUCC(ret)) {
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.2. start grant pools", K(user_tenant_id));
|
|
const int64_t tmp_start_time = ObTimeUtility::fast_current_time();
|
|
lib::Worker::CompatMode compat_mode = get_worker_compat_mode(
|
|
user_tenant_schema.get_compatibility_mode());
|
|
if (OB_FAIL(unit_mgr_->grant_pools(
|
|
trans, new_ug_id_array,
|
|
compat_mode,
|
|
pools, user_tenant_id,
|
|
false/*is_bootstrap*/,
|
|
arg.source_tenant_id_,
|
|
false/*check_data_version*/))) {
|
|
LOG_WARN("grant_pools_to_tenant failed", KR(ret), K(arg), K(pools), K(user_tenant_id));
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.2. finish grant pools", KR(ret), K(user_tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - tmp_start_time);
|
|
}
|
|
|
|
// 3. persist initial tenant config
|
|
if (OB_SUCC(ret)) {
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.3. start persist tenant config", K(user_tenant_id));
|
|
const int64_t tmp_start_time = ObTimeUtility::fast_current_time();
|
|
ObArray<ObAddr> addrs;
|
|
if (OB_FAIL(unit_mgr_->get_servers_by_pools(pools, addrs))) {
|
|
LOG_WARN("fail to get tenant's servers", KR(ret), K(user_tenant_id));
|
|
} else if (OB_FAIL(notify_init_tenant_config(*rpc_proxy_, init_configs, addrs))) {
|
|
LOG_WARN("fail to notify broadcast tenant config", KR(ret), K(addrs), K(init_configs));
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.3. finish persist tenant config",
|
|
KR(ret), K(user_tenant_id), "cost", ObTimeUtility::fast_current_time() - tmp_start_time);
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
bool commit = OB_SUCC(ret);
|
|
if (OB_SUCCESS != (temp_ret = trans.end(commit))) {
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
LOG_WARN("trans end failed", K(commit), K(temp_ret));
|
|
}
|
|
}
|
|
|
|
// If the transaction returns successfully, modify the unit_mgr memory data structure
|
|
// If the transaction fails, the transaction may be submitted successfully. At this time,
|
|
// the transaction is considered to have failed, and the unit_mgr memory state is not modified at this time,
|
|
// and the transaction 1 is subsequently rolled back through drop tenant.
|
|
if (OB_SUCC(ret)) {
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.4. start reload unit_manager", K(user_tenant_id));
|
|
const int64_t tmp_start_time = ObTimeUtility::fast_current_time();
|
|
if (OB_FAIL(unit_mgr_->load())) {
|
|
LOG_WARN("unit_manager reload failed", K(ret));
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 1.4. finish reload unit_manager", KR(ret), K(user_tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - tmp_start_time);
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObArray<ObAddr> addrs;
|
|
ObZone zone; // empty means get all zone's servers
|
|
if (OB_FAIL(unit_mgr_->get_tenant_unit_servers(user_tenant_id, zone, addrs))) {
|
|
LOG_WARN("fail to get tenant's servers", KR(ret), K(user_tenant_id));
|
|
} else if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID, addrs))) {
|
|
LOG_WARN("publish schema failed", KR(ret), K(addrs));
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 1. finish create tenant schema", KR(ret), K(user_tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::notify_init_tenant_config(
|
|
obrpc::ObSrvRpcProxy &rpc_proxy,
|
|
const common::ObIArray<common::ObConfigPairs> &init_configs,
|
|
const common::ObIArray<common::ObAddr> &addrs)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTimeoutCtx ctx;
|
|
const int64_t DEFAULT_TIMEOUT = 10 * 1000 * 1000L; // 10s
|
|
if (OB_UNLIKELY(
|
|
init_configs.count() <= 0
|
|
|| addrs.count() <= 0)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("init configs count is invalid", KR(ret), K(init_configs), K(addrs));
|
|
} else if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, DEFAULT_TIMEOUT))) {
|
|
LOG_WARN("fail to set default timeout", KR(ret));
|
|
} else {
|
|
ObArenaAllocator allocator("InitTenantConf");
|
|
// 1. construct arg
|
|
obrpc::ObInitTenantConfigArg arg;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < init_configs.count(); i++) {
|
|
const common::ObConfigPairs &pairs = init_configs.at(i);
|
|
obrpc::ObTenantConfigArg config;
|
|
char *buf = NULL;
|
|
int64_t length = pairs.get_config_str_length();
|
|
if (OB_ISNULL(buf = static_cast<char *>(allocator.alloc(length)))) {
|
|
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
LOG_WARN("alloc memory failed", KR(ret), K(length));
|
|
} else {
|
|
MEMSET(buf, '\0', length);
|
|
if (OB_FAIL(pairs.get_config_str(buf, length))) {
|
|
LOG_WARN("fail to get config str", KR(ret), K(length), K(pairs));
|
|
} else {
|
|
config.tenant_id_ = pairs.get_tenant_id();
|
|
config.config_str_.assign_ptr(buf, strlen(buf));
|
|
if (OB_FAIL(arg.add_tenant_config(config))) {
|
|
LOG_WARN("fail to add config", KR(ret), K(config));
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
// 2. send rpc
|
|
rootserver::ObInitTenantConfigProxy proxy(
|
|
rpc_proxy, &obrpc::ObSrvRpcProxy::init_tenant_config);
|
|
bool call_rs = false;
|
|
ObAddr rs_addr = GCONF.self_addr_;
|
|
int64_t timeout = ctx.get_timeout();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < addrs.count(); i++) {
|
|
const ObAddr &addr = addrs.at(i);
|
|
if (OB_FAIL(proxy.call(addr, timeout, arg))) {
|
|
LOG_WARN("send rpc failed", KR(ret), K(addr), K(timeout), K(arg));
|
|
} else if (rs_addr == addr) {
|
|
call_rs = true;
|
|
}
|
|
} // end for
|
|
if (OB_FAIL(ret) || call_rs) {
|
|
} else if (OB_FAIL(proxy.call(rs_addr, timeout, arg))) {
|
|
LOG_WARN("fail to call rs", KR(ret), K(rs_addr), K(timeout), K(arg));
|
|
}
|
|
// 3. check result
|
|
ObArray<int> return_ret_array;
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(proxy.wait_all(return_ret_array))) { // ignore ret
|
|
LOG_WARN("wait batch result failed", KR(tmp_ret), KR(ret));
|
|
ret = OB_SUCC(ret) ? tmp_ret : ret;
|
|
} else if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(proxy.check_return_cnt(return_ret_array.count()))) {
|
|
LOG_WARN("return cnt not match", KR(ret), "return_cnt", return_ret_array.count());
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < return_ret_array.count(); i++) {
|
|
int return_ret = return_ret_array.at(i);
|
|
const ObAddr &addr = proxy.get_dests().at(i);
|
|
const ObInitTenantConfigRes *result = proxy.get_results().at(i);
|
|
if (OB_SUCCESS != return_ret) {
|
|
ret = return_ret;
|
|
LOG_WARN("rpc return error", KR(ret), K(addr), K(timeout));
|
|
} else if (OB_ISNULL(result)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get empty result", KR(ret), K(addr), K(timeout));
|
|
} else if (OB_SUCCESS != result->get_ret()) {
|
|
ret = result->get_ret();
|
|
LOG_WARN("persist tenant config failed", KR(ret), K(addr), K(timeout));
|
|
}
|
|
} // end for
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
#ifdef OB_BUILD_TDE_SECURITY
|
|
int ObDDLService::check_need_create_root_key(const ObCreateTenantArg &arg, bool &need_create)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
need_create = false;
|
|
if (GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_2_0_0) {
|
|
need_create = false;
|
|
} else if (arg.is_restore_tenant() || arg.is_clone_tenant()) {
|
|
need_create = false;
|
|
} else {
|
|
need_create = true;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::standby_create_root_key(
|
|
const uint64_t tenant_id,
|
|
const obrpc::ObCreateTenantArg &arg,
|
|
const common::ObIArray<common::ObAddr> &addrs)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_UNLIKELY(!is_user_tenant(tenant_id) || !arg.is_creating_standby_)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(arg));
|
|
} else {
|
|
obrpc::RootKeyType key_type = obrpc::RootKeyType::INVALID;
|
|
common::ObString root_key;
|
|
ObArenaAllocator allocator("root_key");
|
|
|
|
if (OB_FAIL(get_root_key_from_primary(arg, tenant_id, key_type, root_key, allocator))) {
|
|
LOG_WARN("failed to get root key", KR(ret), K(arg), K(tenant_id));
|
|
} else {
|
|
obrpc::ObRootKeyArg root_key_arg;
|
|
obrpc::ObRootKeyResult dummy_result;
|
|
if (OB_FAIL(root_key_arg.init(tenant_id, key_type, root_key))) {
|
|
LOG_WARN("failed to init root key arg", KR(ret), K(tenant_id), K(key_type), K(root_key));
|
|
} else if (OB_FAIL(notify_root_key(*rpc_proxy_, root_key_arg, addrs, dummy_result))) {
|
|
LOG_WARN("fail to notify root key", K(ret), K(root_key_arg));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_root_key_from_primary(const obrpc::ObCreateTenantArg &arg,
|
|
const uint64_t tenant_id, obrpc::RootKeyType &key_type,
|
|
common::ObString &key_value,
|
|
common::ObIAllocator &allocator)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_UNLIKELY(!is_user_tenant(tenant_id) || !arg.is_creating_standby_)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(arg));
|
|
} else {
|
|
uint64_t primary_tenant_id = OB_INVALID_TENANT_ID;
|
|
uint64_t cluster_id = OB_INVALID_ID;
|
|
ObArray<ObAddr> addr_list;
|
|
ObLogRestoreSourceServiceConfigParser log_restore_source(ObBackupConfigType::LOG_RESTORE_SOURCE, tenant_id);
|
|
common::ObSqlString value;
|
|
obrpc::ObRootKeyArg root_key_arg;
|
|
obrpc::ObRootKeyResult result;
|
|
if (OB_FAIL(value.assign(arg.log_restore_source_))) {
|
|
LOG_WARN("fail to assign value", KR(ret), K(log_restore_source));
|
|
} else if (OB_FAIL(log_restore_source.get_primary_server_addr(
|
|
value, primary_tenant_id, cluster_id, addr_list))) {
|
|
LOG_WARN("failed to get primary server addr", KR(ret), K(value));
|
|
} else if (OB_FAIL(root_key_arg.init_for_get(primary_tenant_id))) {
|
|
LOG_WARN("failed to init for get", KR(ret), K(primary_tenant_id));
|
|
}
|
|
if (FAILEDx(notify_root_key(*rpc_proxy_, root_key_arg,
|
|
addr_list, result, true/*enable_default*/, true/*skip_call_rs*/,
|
|
cluster_id, &allocator))) {
|
|
LOG_WARN("failed to get root key from obs", KR(ret), K(cluster_id),
|
|
K(root_key_arg), K(addr_list));
|
|
} else {
|
|
key_type = result.key_type_;
|
|
key_value = result.root_key_;
|
|
}
|
|
if (OB_INVALID_ROOT_KEY == ret) {
|
|
LOG_USER_ERROR(OB_INVALID_ROOT_KEY, "Can not get root key from primary tenant");
|
|
}
|
|
LOG_INFO("get root key from primary tenant", K(primary_tenant_id), K(tenant_id), K(value),
|
|
K(addr_list), K(key_type), K(key_value), K(cluster_id));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_root_key(
|
|
obrpc::ObSrvRpcProxy &rpc_proxy,
|
|
const uint64_t tenant_id,
|
|
const common::ObIArray<common::ObAddr> &addrs)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id), K(addrs));
|
|
} else {
|
|
char root_key[OB_ROOT_KEY_LEN] = {0};
|
|
obrpc::ObRootKeyArg arg;
|
|
obrpc::ObRootKeyResult dummy_result;
|
|
arg.tenant_id_ = tenant_id;
|
|
arg.is_set_ = true;
|
|
arg.key_type_ = obrpc::RootKeyType::NORMAL;
|
|
if (OB_FAIL(ObKeyGenerator::generate_encrypt_key(root_key, OB_ROOT_KEY_LEN))) {
|
|
LOG_WARN("fail to generate root key", K(ret));
|
|
} else if (FALSE_IT(arg.root_key_.assign_ptr(root_key, OB_ROOT_KEY_LEN))) {
|
|
} else if (OB_FAIL(notify_root_key(rpc_proxy, arg, addrs, dummy_result))) {
|
|
LOG_WARN("fail to notify root key", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::notify_root_key(
|
|
obrpc::ObSrvRpcProxy &rpc_proxy,
|
|
const obrpc::ObRootKeyArg &arg,
|
|
const common::ObIArray<common::ObAddr> &addrs,
|
|
obrpc::ObRootKeyResult &result,
|
|
const bool enable_default /*=true*/,
|
|
const bool skip_call_rs /*=false*/,
|
|
const uint64_t &cluster_id /*=OB_INVALID_CLUSTER_ID*/,
|
|
common::ObIAllocator *allocator /*=NULL*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTimeoutCtx ctx;
|
|
bool has_failed = false;
|
|
const int64_t DEFAULT_TIMEOUT = 10 * 1000 * 1000L; // 10s
|
|
if (OB_UNLIKELY(!arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(arg));
|
|
} else if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, DEFAULT_TIMEOUT))) {
|
|
LOG_WARN("fail to set default timeout", KR(ret));
|
|
} else {
|
|
// 1. send rpc
|
|
rootserver::ObSetRootKeyProxy proxy(
|
|
rpc_proxy, &obrpc::ObSrvRpcProxy::set_root_key);
|
|
int tmp_ret = OB_SUCCESS;
|
|
int return_ret = OB_SUCCESS;
|
|
// need_to_call_rs is true only if skip_call_rs is false and not notify cross-cluster
|
|
bool need_call_rs = (!skip_call_rs) && (OB_INVALID_CLUSTER_ID == cluster_id);
|
|
ObAddr rs_addr = GCONF.self_addr_;
|
|
int64_t timeout = ctx.get_timeout();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < addrs.count(); i++) {
|
|
const ObAddr &addr = addrs.at(i);
|
|
if (rs_addr == addr && !need_call_rs) {
|
|
// skip rs_addr if need_call_rs is false
|
|
} else if (OB_TMP_FAIL(proxy.call(addr, timeout, cluster_id, OB_SYS_TENANT_ID, arg))) {
|
|
has_failed = true;
|
|
return_ret= tmp_ret;
|
|
LOG_WARN("send rpc failed", KR(ret), KR(tmp_ret), K(addr), K(timeout), K(cluster_id));
|
|
} else if (rs_addr == addr) {
|
|
need_call_rs = false;
|
|
}
|
|
} // end for
|
|
if (OB_FAIL(ret) || !need_call_rs) {
|
|
} else if (OB_TMP_FAIL(proxy.call(rs_addr, timeout, cluster_id, OB_SYS_TENANT_ID, arg))) {
|
|
has_failed = true;
|
|
return_ret= tmp_ret;
|
|
LOG_WARN("fail to call rs", KR(ret), KR(tmp_ret), K(rs_addr), K(timeout), K(cluster_id));
|
|
}
|
|
// 2. check result
|
|
ObArray<int> return_ret_array;
|
|
if (OB_TMP_FAIL(proxy.wait_all(return_ret_array))) { // ignore ret
|
|
LOG_WARN("wait batch result failed", KR(tmp_ret), KR(ret));
|
|
ret = OB_SUCC(ret) ? tmp_ret : ret;
|
|
} else if (OB_FAIL(ret) || has_failed) {
|
|
} else {
|
|
// don't use arg/dest here because call() may has failure.
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < return_ret_array.count() && !has_failed; ++i) {
|
|
if (OB_TMP_FAIL(return_ret_array.at(i))) {
|
|
has_failed = true;
|
|
return_ret = tmp_ret;
|
|
LOG_WARN("rpc return error", KR(tmp_ret), K(i));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (arg.is_set_) {
|
|
if (OB_UNLIKELY(has_failed)) {
|
|
ret = return_ret;
|
|
LOG_WARN("failed to set root key", KR(ret));
|
|
}
|
|
} else {
|
|
// 1. don't use arg/dest here because call() may has failure.
|
|
// 2. result may be meanless when related return ret is not OB_SUCCESS
|
|
obrpc::RootKeyType key_type = obrpc::RootKeyType::INVALID;
|
|
ObString root_key;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < proxy.get_results().count(); ++i) {
|
|
const ObRootKeyResult *rpc_result = proxy.get_results().at(i);
|
|
if (OB_ISNULL(rpc_result)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get empty result", KR(ret), K(i), K(addrs));
|
|
} else if (obrpc::RootKeyType::INVALID == rpc_result->key_type_) {
|
|
//There may be no root_key information on some observers
|
|
} else if (rpc_result->key_type_ != key_type) {
|
|
if (OB_UNLIKELY(obrpc::RootKeyType::INVALID != key_type)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root key type is conflict", KR(ret), K(key_type), KPC(rpc_result));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
key_type = rpc_result->key_type_;
|
|
root_key = rpc_result->root_key_;
|
|
}
|
|
} else if (OB_UNLIKELY(0 != root_key.compare(rpc_result->root_key_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root key is conflict", KR(ret), K(root_key), KPC(rpc_result));
|
|
}
|
|
} // end for
|
|
if (OB_SUCC(ret) && obrpc::RootKeyType::INVALID == key_type) {
|
|
if (has_failed) {
|
|
ret = OB_INVALID_ROOT_KEY;
|
|
LOG_WARN("failed to get root key from obs", KR(ret), K(cluster_id),
|
|
K(addrs), K(key_type), K(root_key));
|
|
} else if (enable_default) {
|
|
//If the root_key cannot be obtained from all current observers,
|
|
//set default. This tenant may be an upgraded tenant.
|
|
//The addrs are obtained from the __all_virtual_log_stat in standby cluster,
|
|
//it may not be all observers, ignore this situation
|
|
key_type = obrpc::RootKeyType::DEFAULT;
|
|
LOG_INFO("can not get root key from all observer, set default", K(cluster_id),
|
|
K(addrs), K(key_type), K(root_key));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (obrpc::RootKeyType::INVALID == key_type) {
|
|
result.key_type_ = key_type;
|
|
result.root_key_.reset();
|
|
} else if (OB_INVALID_CLUSTER_ID != cluster_id) {
|
|
if (OB_ISNULL(allocator)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("allocator is null", K(ret));
|
|
} else if (OB_FAIL(deep_copy_ob_string(*allocator, root_key, result.root_key_))) {
|
|
LOG_WARN("failed to deep copy string", KR(ret));
|
|
} else {
|
|
result.key_type_ = key_type;
|
|
}
|
|
} else if (OB_FAIL(ObMasterKeyGetter::instance().set_root_key(arg.tenant_id_,
|
|
key_type, root_key))) {
|
|
LOG_WARN("failed to set root key", K(ret));
|
|
} else if (OB_FAIL(ObMasterKeyGetter::instance().get_root_key(arg.tenant_id_,
|
|
result.key_type_, result.root_key_))) {
|
|
LOG_WARN("failed to get root key", K(ret));
|
|
} else if (OB_UNLIKELY(key_type != result.key_type_ ||
|
|
0 != root_key.compare(result.root_key_))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get unexpect root key", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
#endif
|
|
|
|
// 1. create tenant's sys ls
|
|
// 2. broadcast sys table schemas
|
|
// 3. create tenant's sys tablets
|
|
// 4. init tenant's schema(sys table schemas, database schemas, ...)
|
|
int ObDDLService::create_normal_tenant(
|
|
const uint64_t tenant_id,
|
|
const ObIArray<share::ObResourcePoolName> &pool_list,
|
|
const share::schema::ObTenantSchema &tenant_schema,
|
|
const share::ObTenantRole &tenant_role,
|
|
const SCN &recovery_until_scn,
|
|
ObSysVariableSchema &sys_variable,
|
|
const bool create_ls_with_palf,
|
|
const palf::PalfBaseInfo &palf_base_info,
|
|
const common::ObIArray<common::ObConfigPairs> &init_configs,
|
|
bool is_creating_standby,
|
|
const common::ObString &log_restore_source,
|
|
const uint64_t source_tenant_id)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 2. start create tenant", K(tenant_id), K(tenant_schema), K(source_tenant_id));
|
|
int ret = OB_SUCCESS;
|
|
ObArenaAllocator arena_allocator("InnerTableSchem", OB_MALLOC_MIDDLE_BLOCK_SIZE);
|
|
ObSArray<ObTableSchema> tables;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_UNLIKELY(!recovery_until_scn.is_valid_and_not_min())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid recovery_until_scn", KR(ret), K(recovery_until_scn));
|
|
} else if (is_sys_tenant(tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(insert_restore_or_clone_tenant_job_(tenant_id, tenant_schema.get_tenant_name(), tenant_role, source_tenant_id))) {
|
|
LOG_WARN("failed to insert restore or clone tenant job", KR(ret), K(tenant_id), K(tenant_role), K(tenant_schema), K(source_tenant_id));
|
|
} else if (OB_FAIL(create_tenant_sys_ls(tenant_schema, pool_list, create_ls_with_palf, palf_base_info, source_tenant_id))) {
|
|
LOG_WARN("fail to create tenant sys log stream", KR(ret), K(tenant_schema), K(pool_list), K(palf_base_info), K(source_tenant_id));
|
|
} else if (is_user_tenant(tenant_id) && !tenant_role.is_primary()) {
|
|
//standby cluster no need create sys tablet and init tenant schema
|
|
} else if (OB_FAIL(ObSchemaUtils::construct_inner_table_schemas(tenant_id, tables, arena_allocator))) {
|
|
LOG_WARN("fail to get inner table schemas in tenant space", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(broadcast_sys_table_schemas(tenant_id, tables))) {
|
|
LOG_WARN("fail to broadcast sys table schemas", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(create_tenant_sys_tablets(tenant_id, tables))) {
|
|
LOG_WARN("fail to create tenant partitions", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(init_tenant_schema(tenant_id, tenant_schema,
|
|
tenant_role, recovery_until_scn, tables, sys_variable, init_configs,
|
|
is_creating_standby, log_restore_source))) {
|
|
LOG_WARN("fail to init tenant schema", KR(ret), K(tenant_role), K(recovery_until_scn),
|
|
K(tenant_id), K(tenant_schema), K(sys_variable), K(init_configs),
|
|
K(is_creating_standby), K(log_restore_source));
|
|
} else if (is_user_tenant(tenant_id) && OB_FAIL(create_tenant_user_ls(tenant_id))) {
|
|
//create user ls
|
|
LOG_WARN("failed to create tenant user ls", KR(ret), K(tenant_id));
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 2. finish create tenant", KR(ret), K(tenant_id), K(source_tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::insert_restore_or_clone_tenant_job_(
|
|
const uint64_t tenant_id,
|
|
const ObString &tenant_name,
|
|
const share::ObTenantRole &tenant_role,
|
|
const uint64_t source_tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (is_sys_tenant(tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(sql_proxy_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy));
|
|
} else if (is_meta_tenant(tenant_id)) {
|
|
// no need to insert job for meta tenant
|
|
} else if (!tenant_role.is_restore() && !tenant_role.is_clone()) {
|
|
// no need to insert restore/clone job
|
|
} else if (tenant_role.is_clone()) {
|
|
// insert clone job
|
|
if (OB_FAIL(ObTenantCloneUtil::insert_user_tenant_clone_job(*sql_proxy_, tenant_name,
|
|
tenant_id))) {
|
|
LOG_WARN("failed to insert user tenant clone job", KR(ret), K(tenant_name), K(tenant_id));
|
|
}
|
|
} else if (OB_FAIL(ObRestoreUtil::insert_user_tenant_restore_job(*sql_proxy_, tenant_name,
|
|
tenant_id))) {
|
|
LOG_WARN("failed to insert user tenant restore job", KR(ret), K(tenant_id), K(tenant_name));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_tenant_user_ls(const uint64_t tenant_id)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.5. start create user log stream", K(tenant_id));
|
|
int ret = OB_SUCCESS;
|
|
common::ObTimeoutCtx ctx;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (!is_user_tenant(tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(rpc_proxy_) || OB_ISNULL(GCTX.location_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret));
|
|
} else if (OB_FAIL(ObRootUtils::get_rs_default_timeout_ctx(ctx))) {
|
|
LOG_WARN("fail to get timeout ctx", KR(ret), K(ctx));
|
|
} else {
|
|
ObAddr leader;
|
|
int64_t tmp_ret = OB_SUCCESS;
|
|
//ignore failed
|
|
while(!ctx.is_timeouted() && OB_SUCC(ret)) {
|
|
const int64_t timeout = ctx.get_timeout();
|
|
if (OB_TMP_FAIL(GCTX.location_service_->get_leader(GCONF.cluster_id, tenant_id, SYS_LS, FALSE, leader))) {
|
|
LOG_WARN("failed to get leader", KR(ret), KR(tmp_ret), K(tenant_id));
|
|
} else if (OB_TMP_FAIL(rpc_proxy_->to(leader).timeout(timeout)
|
|
.notify_create_tenant_user_ls(tenant_id))) {
|
|
LOG_WARN("failed to create tenant user ls", KR(ret), KR(tmp_ret), K(tenant_id), K(leader), K(timeout));
|
|
} else {
|
|
break;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && ctx.is_timeouted()) {
|
|
ret = OB_TIMEOUT;
|
|
LOG_WARN("create user ls timeout", KR(ret));
|
|
}
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.5. finish create user log stream", KR(ret), K(tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time);
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_tenant_sys_ls(
|
|
const ObTenantSchema &tenant_schema,
|
|
const ObIArray<share::ObResourcePoolName> &pool_list,
|
|
const bool create_ls_with_palf,
|
|
const palf::PalfBaseInfo &palf_base_info,
|
|
const uint64_t source_tenant_id)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.1. start create sys log stream", K(tenant_schema), K(source_tenant_id));
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = tenant_schema.get_tenant_id();
|
|
// meta tenant do not have to reference source tenant id
|
|
const uint64_t source_tenant_id_to_use = is_user_tenant(tenant_id) ? source_tenant_id : OB_INVALID_TENANT_ID;
|
|
int64_t wait_leader = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (is_sys_tenant(tenant_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(rpc_proxy_)
|
|
|| OB_ISNULL(sql_proxy_)
|
|
|| OB_ISNULL(lst_operator_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(rpc_proxy), KP_(sql_proxy), KP_(lst_operator));
|
|
} else {
|
|
ObArray<share::ObZoneReplicaAttrSet> locality;
|
|
ObArray<ObZone> primary_zone_list;
|
|
ObSqlString zone_priority;
|
|
share::schema::ObSchemaGetterGuard schema_guard; // not used
|
|
int64_t paxos_replica_num = OB_INVALID_ID;
|
|
ObLSCreator ls_creator(*rpc_proxy_, tenant_id, SYS_LS, sql_proxy_);
|
|
if (OB_FAIL(tenant_schema.get_zone_replica_attr_array(locality))) {
|
|
LOG_WARN("fail to get tenant's locality", KR(ret), K(locality));
|
|
} else if (OB_FAIL(tenant_schema.get_paxos_replica_num(schema_guard, paxos_replica_num))) {
|
|
LOG_WARN("failed to get paxos replica num", KR(ret));
|
|
} else if (OB_FAIL(ObPrimaryZoneUtil::get_tenant_primary_zone_array(tenant_schema, primary_zone_list))) {
|
|
LOG_WARN("failed to get tenant primary zone array", KR(ret));
|
|
} else if (OB_UNLIKELY(0 == primary_zone_list.count())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("primary zone is empty", KR(ret), K(tenant_schema));
|
|
} else if (OB_FAIL(ObTenantThreadHelper::get_zone_priority(
|
|
primary_zone_list.at(0), tenant_schema, zone_priority))) {
|
|
LOG_WARN("failed to get zone priority", KR(ret), K(primary_zone_list), K(tenant_schema));
|
|
} else if (OB_FAIL(ls_creator.create_tenant_sys_ls(
|
|
primary_zone_list.at(0), locality, pool_list, paxos_replica_num,
|
|
tenant_schema.get_compatibility_mode(), zone_priority.string(),
|
|
create_ls_with_palf, palf_base_info, source_tenant_id_to_use))) {
|
|
LOG_WARN("fail to create tenant sys ls", KR(ret), K(pool_list), K(palf_base_info),
|
|
K(locality), K(paxos_replica_num), K(tenant_schema), K(zone_priority), K(source_tenant_id_to_use));
|
|
} else {
|
|
share::ObLSLeaderElectionWaiter ls_leader_waiter(*lst_operator_, stopped_);
|
|
int64_t timeout = GCONF.rpc_timeout;
|
|
if (INT64_MAX != THIS_WORKER.get_timeout_ts()) {
|
|
timeout = max(timeout, THIS_WORKER.get_timeout_remain());
|
|
}
|
|
int64_t wait_leader_start = ObTimeUtility::current_time();
|
|
if (OB_FAIL(ls_leader_waiter.wait(tenant_id, SYS_LS, timeout))) {
|
|
LOG_WARN("fail to wait election leader", KR(ret), K(tenant_id), K(SYS_LS), K(timeout));
|
|
}
|
|
int64_t wait_leader_end = ObTimeUtility::current_time();
|
|
wait_leader = wait_leader_end - wait_leader_start;
|
|
}
|
|
}
|
|
if (is_meta_tenant(tenant_id)) {
|
|
DEBUG_SYNC(AFTER_CREATE_META_TENANT_SYS_LOGSTREAM);
|
|
} else {
|
|
DEBUG_SYNC(AFTER_CREATE_USER_TENANT_SYS_LOGSTREAM);
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.1. finish create sys log stream", KR(ret), K(tenant_schema),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time, "wait leader", wait_leader);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::broadcast_sys_table_schemas(
|
|
const uint64_t tenant_id,
|
|
common::ObIArray<ObTableSchema> &tables)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.2. start broadcast sys table schemas", K(tenant_id));
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_ISNULL(lst_operator_)
|
|
|| OB_ISNULL(rpc_proxy_)
|
|
|| OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(lst_operator), KP_(rpc_proxy), KP_(schema_service));
|
|
} else {
|
|
// Ensure observer which contains rs or tenant's sys ls leader has avaliable schemas.
|
|
ObLSInfo ls_info;
|
|
ObArray<ObAddr> addrs;
|
|
const ObLSReplica *leader = NULL;
|
|
ObLSReplica::MemberList member_list;
|
|
if (OB_FAIL(lst_operator_->get(GCONF.cluster_id, tenant_id,
|
|
SYS_LS, share::ObLSTable::DEFAULT_MODE, ls_info))) {
|
|
LOG_WARN("fail to get sys ls info", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ls_info.find_leader(leader))) {
|
|
LOG_WARN("fail to get leader", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(leader)) {
|
|
ret = OB_LEADER_NOT_EXIST;
|
|
LOG_WARN("leader is null", KR(ret), K(tenant_id));
|
|
} else {
|
|
member_list = leader->get_member_list();
|
|
ARRAY_FOREACH_N(member_list, idx, cnt) {
|
|
const ObAddr &server = member_list.at(idx).get_server();
|
|
if (OB_UNLIKELY(!server.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid server", KR(ret), K(server), K(member_list));
|
|
} else if (OB_FAIL(addrs.push_back(server))) {
|
|
LOG_WARN("fail to push back server", KR(ret), K(server), K(addrs));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& !is_contain(addrs, GCONF.self_addr_)
|
|
&& OB_FAIL(addrs.push_back(GCONF.self_addr_))) {
|
|
LOG_WARN("fail to push back rs addr", KR(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObTimeoutCtx ctx;
|
|
ObBatchBroadcastSchemaProxy proxy(*rpc_proxy_,
|
|
&ObSrvRpcProxy::batch_broadcast_schema);
|
|
obrpc::ObBatchBroadcastSchemaArg arg;
|
|
int64_t sys_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, GCONF.rpc_timeout))) {
|
|
LOG_WARN("fail to set timeout ctx", KR(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(
|
|
OB_SYS_TENANT_ID, sys_schema_version))) {
|
|
} else if (OB_FAIL(arg.init(tenant_id, sys_schema_version, tables))) {
|
|
LOG_WARN("fail to init arg", KR(ret), K(tenant_id), K(sys_schema_version));
|
|
}
|
|
const int64_t timeout_ts = ctx.get_timeout(0);
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < addrs.count(); i++) {
|
|
const ObAddr &addr = addrs.at(i);
|
|
if (OB_FAIL(proxy.call(addr, timeout_ts, arg))) {
|
|
LOG_WARN("fail to send rpc", KR(ret), K(tenant_id),
|
|
K(sys_schema_version), K(addr), K(timeout_ts));
|
|
}
|
|
} // end for
|
|
|
|
ObArray<int> return_code_array;
|
|
int tmp_ret = OB_SUCCESS; // always wait all
|
|
if (OB_TMP_FAIL(proxy.wait_all(return_code_array))) {
|
|
LOG_WARN("wait batch result failed", KR(tmp_ret), KR(ret));
|
|
ret = OB_SUCC(ret) ? tmp_ret : ret;
|
|
} else if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(proxy.check_return_cnt(return_code_array.count()))) {
|
|
LOG_WARN("return cnt not match", KR(ret), "return_cnt", return_code_array.count());
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < return_code_array.count(); i++) {
|
|
int res_ret = return_code_array.at(i);
|
|
const ObAddr &addr = proxy.get_dests().at(i);
|
|
if (OB_SUCCESS != res_ret
|
|
&& (addr == leader->get_server()
|
|
|| addr == GCONF.self_addr_)) { // leader and rs must succeed
|
|
ret = res_ret;
|
|
LOG_WARN("broadcast schema failed", KR(ret), K(addr), K(tenant_id));
|
|
}
|
|
} // end for
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.2. finish broadcast sys table schemas", KR(ret), K(tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_tenant_sys_tablets(
|
|
const uint64_t tenant_id,
|
|
common::ObIArray<ObTableSchema> &tables)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.3. start create sys table tablets", K(tenant_id));
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_ISNULL(rpc_proxy_)
|
|
|| OB_ISNULL(lst_operator_)
|
|
|| OB_ISNULL(sql_proxy_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(rpc_proxy), KP_(lst_operator));
|
|
} else {
|
|
// FIXME: (yanmu.ztl) use actual trans later
|
|
ObMySQLTransaction trans;
|
|
share::schema::ObSchemaGetterGuard dummy_guard;
|
|
SCN frozen_scn = SCN::base_scn();
|
|
ObTableCreator table_creator(tenant_id,
|
|
frozen_scn,
|
|
trans);
|
|
ObNewTableTabletAllocator new_table_tablet_allocator(
|
|
tenant_id,
|
|
dummy_guard,
|
|
sql_proxy_);
|
|
common::ObArray<share::ObLSID> ls_id_array;
|
|
const ObTablegroupSchema *dummy_tablegroup_schema = NULL;
|
|
ObArray<const share::schema::ObTableSchema*> table_schemas;
|
|
ObArray<uint64_t> index_tids;
|
|
ObArray<bool> need_create_empty_majors;
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id))) {
|
|
LOG_WARN("fail to start trans", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(table_creator.init(false/*need_tablet_cnt_check*/))) {
|
|
LOG_WARN("fail to init tablet creator", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(new_table_tablet_allocator.init())) {
|
|
LOG_WARN("fail to init new table tablet allocator", KR(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < tables.count(); i++) {
|
|
const ObTableSchema &data_table = tables.at(i);
|
|
const uint64_t data_table_id = data_table.get_table_id();
|
|
if (data_table.has_partition()) {
|
|
table_schemas.reset();
|
|
need_create_empty_majors.reset();
|
|
if (OB_FAIL(table_schemas.push_back(&data_table)) || OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("fail to push back data table ptr", KR(ret), K(data_table_id));
|
|
} else if (ObSysTableChecker::is_sys_table_has_index(data_table_id)) {
|
|
if (OB_FAIL(ObSysTableChecker::get_sys_table_index_tids(data_table_id, index_tids))) {
|
|
LOG_WARN("fail to get sys index tids", KR(ret), K(data_table_id));
|
|
} else if (i + index_tids.count() >= tables.count()
|
|
|| index_tids.count() <= 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys table's index should be next to its data table",
|
|
KR(ret), K(i), "index_cnt", index_tids.count());
|
|
} else {
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < index_tids.count(); j++) {
|
|
const ObTableSchema &index_schema = tables.at(i + j + 1);
|
|
const int64_t index_id = index_schema.get_table_id();
|
|
if (index_id != index_tids.at(j)
|
|
|| data_table_id != index_schema.get_data_table_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys index schema order is not match", KR(ret), K(data_table_id), K(j), K(index_schema));
|
|
} else if (OB_FAIL(table_schemas.push_back(&index_schema))) {
|
|
LOG_WARN("fail to push back index schema", KR(ret), K(index_id), K(data_table_id));
|
|
} else if (OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("fail to push back need create empty major", KR(ret), K(index_id), K(data_table_id));
|
|
}
|
|
} // end for
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && is_system_table(data_table_id)) {
|
|
uint64_t lob_meta_table_id = OB_INVALID_ID;
|
|
uint64_t lob_piece_table_id = OB_INVALID_ID;
|
|
if (OB_ALL_CORE_TABLE_TID == data_table_id) {
|
|
// do nothing
|
|
} else if (!get_sys_table_lob_aux_table_id(data_table_id, lob_meta_table_id, lob_piece_table_id)) {
|
|
ret = OB_ENTRY_NOT_EXIST;
|
|
LOG_WARN("fail to get_sys_table_lob_aux_table_id", KR(ret), K(data_table_id));
|
|
} else {
|
|
int64_t meta_idx = -1;
|
|
int64_t piece_idx = -1;
|
|
for (int64_t k = i + 1; OB_SUCC(ret) && k < tables.count(); k++) {
|
|
if (tables.at(k).get_table_id() == lob_meta_table_id) {
|
|
meta_idx = k;
|
|
}
|
|
if (tables.at(k).get_table_id() == lob_piece_table_id) {
|
|
piece_idx = k;
|
|
}
|
|
if (meta_idx != -1 && piece_idx != -1) {
|
|
break;
|
|
}
|
|
}
|
|
if (meta_idx == -1 || piece_idx == -1) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys table's lob table not matched", KR(ret), K(meta_idx), K(piece_idx),
|
|
K(lob_piece_table_id), K(lob_meta_table_id), K(data_table_id));
|
|
} else {
|
|
if (OB_FAIL(table_schemas.push_back(&tables.at(meta_idx))) || OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("fail to push back lob meta aux table ptr", KR(ret), K(meta_idx), K(data_table_id));
|
|
} else if (OB_FAIL(table_schemas.push_back(&tables.at(piece_idx))) || OB_FAIL(need_create_empty_majors.push_back(true))) {
|
|
LOG_WARN("fail to push back lob piece aux table ptr", KR(ret), K(piece_idx), K(data_table_id));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
// failed, bypass
|
|
} else if (OB_FAIL(new_table_tablet_allocator.prepare(trans, data_table, dummy_tablegroup_schema))) {
|
|
LOG_WARN("fail to prepare ls for sys table tablets");
|
|
} else if (OB_FAIL(new_table_tablet_allocator.get_ls_id_array(
|
|
ls_id_array))) {
|
|
LOG_WARN("fail to get ls id array", KR(ret));
|
|
} else if (OB_FAIL(table_creator.add_create_tablets_of_tables_arg(
|
|
table_schemas,
|
|
ls_id_array,
|
|
DATA_CURRENT_VERSION,
|
|
need_create_empty_majors/*need_create_empty_major_sstable*/))) {
|
|
LOG_WARN("fail to add create tablets of table", KR(ret), K(data_table), K(table_schemas), K(need_create_empty_majors));
|
|
}
|
|
}
|
|
} // end for
|
|
if (FAILEDx(table_creator.execute())) {
|
|
LOG_WARN("fail to execute creator", KR(ret), K(tenant_id));
|
|
} else {
|
|
ALLOW_NEXT_LOG();
|
|
LOG_INFO("create tenant sys tables tablet", KR(ret), K(tenant_id));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
bool commit = OB_SUCC(ret);
|
|
if (OB_SUCCESS != (temp_ret = trans.end(commit))) {
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
LOG_WARN("trans end failed", K(commit), K(temp_ret));
|
|
}
|
|
}
|
|
|
|
// finishing is always invoked for new table tablet allocator
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = new_table_tablet_allocator.finish(OB_SUCCESS == ret))) {
|
|
LOG_WARN("fail to finish new table tablet allocator", KR(tmp_ret));
|
|
}
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.3. finish create sys table tablets", KR(ret), K(tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::init_tenant_schema(
|
|
const uint64_t tenant_id,
|
|
const ObTenantSchema &tenant_schema,
|
|
const share::ObTenantRole &tenant_role,
|
|
const SCN &recovery_until_scn,
|
|
common::ObIArray<ObTableSchema> &tables,
|
|
ObSysVariableSchema &sys_variable,
|
|
const common::ObIArray<common::ObConfigPairs> &init_configs,
|
|
bool is_creating_standby,
|
|
const common::ObString &log_restore_source)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.4. start init tenant schemas", K(tenant_id));
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_UNLIKELY(!recovery_until_scn.is_valid_and_not_min()
|
|
|| (is_creating_standby && log_restore_source.empty()))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(recovery_until_scn), K(is_creating_standby), K(log_restore_source));
|
|
} else if (OB_ISNULL(sql_proxy_)
|
|
|| OB_ISNULL(schema_service_)
|
|
|| OB_ISNULL(schema_service_->get_schema_service())
|
|
|| OB_ISNULL(GCTX.lst_operator_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy), KP_(schema_service), KP(GCTX.lst_operator_));
|
|
} else {
|
|
ObSchemaService *schema_service_impl = schema_service_->get_schema_service();
|
|
// 1. init tenant global stat
|
|
if (OB_SUCC(ret)) {
|
|
const int64_t core_schema_version = OB_CORE_SCHEMA_VERSION + 1;
|
|
const int64_t baseline_schema_version = OB_INVALID_VERSION;
|
|
const int64_t ddl_epoch = 0;
|
|
const SCN snapshot_gc_scn = SCN::min_scn();
|
|
// find compatible version
|
|
uint64_t data_version = 0;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < init_configs.count(); i++) {
|
|
const ObConfigPairs &config = init_configs.at(i);
|
|
if (tenant_id == config.get_tenant_id()) {
|
|
for (int64_t j = 0; data_version == 0 && OB_SUCC(ret) && j < config.get_configs().count(); j++) {
|
|
const ObConfigPairs::ObConfigPair &pair = config.get_configs().at(j);
|
|
if (0 != pair.key_.case_compare("compatible")) {
|
|
} else if (OB_FAIL(ObClusterVersion::get_version(pair.value_.ptr(), data_version))) {
|
|
LOG_WARN("fail to get compatible version", KR(ret), K(tenant_id), K(pair));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
common::ObMySQLTransaction trans;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id))) {
|
|
LOG_WARN("failed to start trans", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObGlobalStatProxy global_stat_proxy(trans, tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
} else if (0 == data_version) {
|
|
ret = OB_ENTRY_NOT_EXIST;
|
|
LOG_WARN("compatible version not defined", KR(ret), K(tenant_id), K(init_configs));
|
|
} else if (OB_FAIL(global_stat_proxy.set_tenant_init_global_stat(
|
|
core_schema_version, baseline_schema_version,
|
|
snapshot_gc_scn, ddl_epoch, data_version, data_version))) {
|
|
LOG_WARN("fail to set tenant init global stat", KR(ret), K(tenant_id),
|
|
K(core_schema_version), K(baseline_schema_version),
|
|
K(snapshot_gc_scn), K(ddl_epoch), K(data_version));
|
|
} else if (is_user_tenant(tenant_id) && OB_FAIL(OB_STANDBY_SERVICE.write_upgrade_barrier_log(
|
|
trans, tenant_id, data_version))) {
|
|
LOG_WARN("fail to write_upgrade_barrier_log", KR(ret), K(tenant_id), K(data_version));
|
|
} else if (is_user_tenant(tenant_id) &&
|
|
OB_FAIL(OB_STANDBY_SERVICE.write_upgrade_data_version_barrier_log(
|
|
trans, tenant_id, data_version))) {
|
|
LOG_WARN("fail to write_upgrade_data_version_barrier_log", KR(ret),
|
|
K(tenant_id), K(data_version));
|
|
}
|
|
}
|
|
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", KR(ret), KR(tmp_ret));
|
|
ret = OB_SUCC(ret) ? tmp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
// 2. init tenant schema
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_, true, true, false, false);
|
|
const int64_t init_schema_version = tenant_schema.get_schema_version();
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
//FIXME:(yanmu.ztl) lock tenant's __all_core_table
|
|
const int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to start trans", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(create_sys_table_schemas(ddl_operator, trans, tables))) {
|
|
LOG_WARN("fail to create sys tables", KR(ret), K(tenant_id));
|
|
} else if (is_user_tenant(tenant_id) && OB_FAIL(set_sys_ls_status(tenant_id))) {
|
|
LOG_WARN("failed to set sys ls status", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_service_impl->gen_new_schema_version(
|
|
tenant_id, init_schema_version, new_schema_version))) {
|
|
} else if (OB_FAIL(ddl_operator.replace_sys_variable(
|
|
sys_variable, new_schema_version, trans, OB_DDL_ALTER_SYS_VAR))) {
|
|
LOG_WARN("fail to replace sys variable", KR(ret), K(sys_variable));
|
|
} else if (OB_FAIL(ddl_operator.init_tenant_env(tenant_schema, sys_variable, tenant_role,
|
|
recovery_until_scn, init_configs, trans))) {
|
|
LOG_WARN("init tenant env failed", KR(ret), K(tenant_role), K(recovery_until_scn), K(tenant_schema));
|
|
} else if (OB_FAIL(ddl_operator.insert_tenant_merge_info(OB_DDL_ADD_TENANT_START, tenant_schema, trans))) {
|
|
LOG_WARN("fail to insert tenant merge info", KR(ret), K(tenant_schema));
|
|
} else if (is_meta_tenant(tenant_id) && OB_FAIL(ObServiceEpochProxy::init_service_epoch(
|
|
trans,
|
|
tenant_id,
|
|
0, /*freeze_service_epoch*/
|
|
0, /*arbitration_service_epoch*/
|
|
0, /*server_zone_op_service_epoch*/
|
|
0, /*heartbeat_service_epoch*/
|
|
0 /* service_name_epoch */))) {
|
|
LOG_WARN("fail to init service epoch", KR(ret));
|
|
} else if (is_creating_standby && OB_FAIL(set_log_restore_source(gen_user_tenant_id(tenant_id), log_restore_source, trans))) {
|
|
LOG_WARN("fail to set_log_restore_source", KR(ret), K(tenant_id), K(log_restore_source));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
bool commit = OB_SUCC(ret);
|
|
if (OB_SUCCESS != (temp_ret = trans.end(commit))) {
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
LOG_WARN("trans end failed", K(commit), K(temp_ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && is_meta_tenant(tenant_id)) {
|
|
// If tenant config version in RS is valid first and ddl trans doesn't commit,
|
|
// observer may read from empty __tenant_parameter successfully and raise its tenant config version,
|
|
// which makes some initial tenant configs are not actually updated before related observer restarts.
|
|
// To fix this problem, tenant config version in RS should be valid after ddl trans commits.
|
|
const int64_t config_version = omt::ObTenantConfig::INITIAL_TENANT_CONF_VERSION + 1;
|
|
const uint64_t user_tenant_id = gen_user_tenant_id(tenant_id);
|
|
if (OB_FAIL(OTC_MGR.set_tenant_config_version(tenant_id, config_version))) {
|
|
LOG_WARN("failed to set tenant config version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(OTC_MGR.set_tenant_config_version(user_tenant_id, config_version))) {
|
|
LOG_WARN("failed to set tenant config version", KR(ret), K(user_tenant_id));
|
|
}
|
|
}
|
|
|
|
ObLSInfo sys_ls_info;
|
|
ObAddrArray addrs;
|
|
if (FAILEDx(GCTX.lst_operator_->get(
|
|
GCONF.cluster_id,
|
|
tenant_id,
|
|
SYS_LS,
|
|
share::ObLSTable::DEFAULT_MODE,
|
|
sys_ls_info))) {
|
|
LOG_WARN("fail to get sys ls info by operator", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(sys_ls_info.get_paxos_member_addrs(addrs))) {
|
|
LOG_WARN("fail to get paxos member addrs", K(ret), K(tenant_id), K(sys_ls_info));
|
|
} else if (OB_FAIL(publish_schema(tenant_id, addrs))) {
|
|
LOG_WARN("fail to publish schema", KR(ret), K(tenant_id), K(addrs));
|
|
}
|
|
}
|
|
|
|
// 3. set baseline schema version
|
|
if (OB_SUCC(ret)) {
|
|
ObGlobalStatProxy global_stat_proxy(*sql_proxy_, tenant_id);
|
|
ObRefreshSchemaStatus schema_status;
|
|
schema_status.tenant_id_ = tenant_id;
|
|
int64_t baseline_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(schema_service_->get_schema_version_in_inner_table(
|
|
*sql_proxy_, schema_status, baseline_schema_version))) {
|
|
LOG_WARN("fail to gen new schema version", KR(ret), K(schema_status));
|
|
} else if (OB_FAIL(global_stat_proxy.set_baseline_schema_version(baseline_schema_version))) {
|
|
LOG_WARN("fail to set baseline schema version",
|
|
KR(ret), K(tenant_id), K(baseline_schema_version));
|
|
}
|
|
}
|
|
}
|
|
|
|
LOG_INFO("[CREATE_TENANT] STEP 2.4. finish init tenant schemas", KR(ret), K(tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::set_log_restore_source(
|
|
const uint64_t tenant_id,
|
|
const common::ObString &log_restore_source,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
share::ObBackupConfigParserMgr config_parser_mgr;
|
|
common::ObSqlString name;
|
|
common::ObSqlString value;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_UNLIKELY(!is_user_tenant(tenant_id) || log_restore_source.empty())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(log_restore_source));
|
|
} else if (OB_FAIL(name.assign("log_restore_source"))) {
|
|
LOG_WARN("assign sql failed", KR(ret));
|
|
} else if (OB_FAIL(value.assign(log_restore_source))) {
|
|
LOG_WARN("fail to assign value", KR(ret), K(log_restore_source));
|
|
} else if (OB_FAIL(config_parser_mgr.init(name, value, gen_user_tenant_id(tenant_id)))) {
|
|
LOG_WARN("fail to init backup config parser mgr", KR(ret), K(name), K(value), K(tenant_id));
|
|
// TODO use the interface without rpc_proxy_
|
|
} else if (OB_FAIL(config_parser_mgr.update_inner_config_table(*rpc_proxy_, trans))) {
|
|
LOG_WARN("fail to update inner config table", KR(ret), K(name), K(value));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_sys_table_schemas(
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
common::ObIArray<ObTableSchema> &tables)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_ISNULL(sql_proxy_)
|
|
|| OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(sql_proxy), KP_(schema_service));
|
|
} else {
|
|
// persist __all_core_table's schema in inner table, which is only used for sys views.
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < tables.count(); i++) {
|
|
ObTableSchema &table = tables.at(i);
|
|
const int64_t table_id = table.get_table_id();
|
|
const ObString &table_name = table.get_table_name();
|
|
const ObString *ddl_stmt = NULL;
|
|
bool need_sync_schema_version = !(ObSysTableChecker::is_sys_table_index_tid(table_id) ||
|
|
is_sys_lob_table(table_id));
|
|
if (FAILEDx(ddl_operator.create_table(table, trans, ddl_stmt,
|
|
need_sync_schema_version,
|
|
false /*is_truncate_table*/))) {
|
|
LOG_WARN("add table schema failed", KR(ret), K(table_id), K(table_name));
|
|
} else {
|
|
LOG_INFO("add table schema succeed", K(i), K(table_id), K(table_name));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::set_sys_ls_status(const uint64_t tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id
|
|
|| !is_user_tenant(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id));
|
|
} else {
|
|
share::ObLSAttr new_ls;
|
|
share::ObLSFlag flag(share::ObLSFlag::NORMAL_FLAG);
|
|
uint64_t ls_group_id = 0;
|
|
SCN create_scn = SCN::base_scn();
|
|
share::ObLSAttrOperator ls_operator(tenant_id, sql_proxy_);
|
|
if (OB_FAIL(new_ls.init(SYS_LS, ls_group_id, flag,
|
|
share::OB_LS_NORMAL, share::OB_LS_OP_CREATE_END, create_scn))) {
|
|
LOG_WARN("failed to init new operation", KR(ret), K(flag), K(create_scn));
|
|
} else if (OB_FAIL(ls_operator.insert_ls(new_ls, share::NORMAL_SWITCHOVER_STATUS))) {
|
|
LOG_WARN("failed to insert new ls", KR(ret), K(new_ls), K(ls_group_id));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_tenant_end(const uint64_t tenant_id)
|
|
{
|
|
const int64_t start_time = ObTimeUtility::fast_current_time();
|
|
LOG_INFO("[CREATE_TENANT] STEP 3. start create tenant end", K(tenant_id));
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
ObAllTenantInfo tenant_info;
|
|
int64_t sys_schema_version = OB_INVALID_VERSION;
|
|
ObDDLSQLTransaction trans(schema_service_, true, false, false, false);
|
|
DEBUG_SYNC(BEFORE_CREATE_TENANT_END);
|
|
ObTenantSchema new_tenant_schema;
|
|
ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_;
|
|
ObRefreshSchemaStatus schema_status;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_ISNULL(schema_status_proxy)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("get invalid schema status proxy", KR(ret));
|
|
} else if (OB_FAIL(ObAllTenantInfoProxy::load_tenant_info(
|
|
tenant_id, sql_proxy_, false, tenant_info))) {
|
|
LOG_WARN("failed to load tenant info", KR(ret), K(tenant_id));
|
|
} else if (OB_INVALID_TENANT_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is null", K(ret), KP_(schema_service));
|
|
/*
|
|
After the inner-table is synchronized by the network standby tenant, the schema refresh switch
|
|
is turned on, but standby tenant may not be in the same observer with RS, causing RS to use the
|
|
old cache when creating tenant end, which may cause create tenant end to fail.
|
|
So here, force trigger schema refresh refresh cache
|
|
*/
|
|
} else if (OB_FAIL(schema_status_proxy->load_refresh_schema_status(tenant_id, schema_status))) {
|
|
LOG_WARN("fail to load refresh schema status", KR(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 with version in inner table", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, sys_schema_version))) {
|
|
LOG_WARN("fail to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, sys_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(sys_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("fail to get tenant schema", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", K(ret), K(tenant_id));
|
|
} else if (tenant_schema->is_normal()) {
|
|
// skip, Guaranteed reentrant
|
|
} else if (!tenant_schema->is_creating()
|
|
&& !tenant_schema->is_restore()) {
|
|
ret = OB_STATE_NOT_MATCH;
|
|
LOG_WARN("state not match", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(new_tenant_schema.assign(*tenant_schema))) {
|
|
LOG_WARN("fail to assign tenant schema", KR(ret));
|
|
} else {
|
|
ObDDLSQLTransaction tenant_trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = OB_INVALID_VERSION;
|
|
if (!tenant_info.is_standby()) {
|
|
// Push the system tenant schema_version, and the standalone cluster may fail due to unsynchronized heartbeat.
|
|
// The standalone cluster uses the synchronized schema_version,
|
|
// and there is no need to increase the system tenant schema_version.
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
ObSchemaService *schema_service_impl = schema_service_->get_schema_service();
|
|
// Ensure that the schema_version monotonically increases among tenants' cross-tenant transactions
|
|
//
|
|
if (OB_ISNULL(schema_service_impl)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service_impl is null", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(tenant_trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
refreshed_schema_version = sys_schema_version > refreshed_schema_version ? sys_schema_version : refreshed_schema_version;
|
|
if (OB_FAIL(schema_service_impl->gen_new_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
const ObString *ddl_stmt_str_ptr = NULL;
|
|
const int64_t DDL_STR_BUF_SIZE = 128;
|
|
char ddl_str_buf[DDL_STR_BUF_SIZE];
|
|
MEMSET(ddl_str_buf, 0, DDL_STR_BUF_SIZE);
|
|
ObString ddl_stmt_str;
|
|
if (tenant_schema->is_restore()) {
|
|
SCN gts;
|
|
int64_t pos = 0;
|
|
if (OB_FAIL(get_tenant_external_consistent_ts(tenant_id, gts))) {
|
|
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=%lu",
|
|
refreshed_schema_version, gts.get_val_for_inner_table_field()))) {
|
|
SERVER_LOG(WARN, "failed to construct ddl_stmt_str", KR(ret), K(tenant_id), K(refreshed_schema_version), K(gts));
|
|
} else {
|
|
ddl_stmt_str.assign_ptr(ddl_str_buf, pos);
|
|
ddl_stmt_str_ptr = &ddl_stmt_str;
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ddl_operator.create_tenant(new_tenant_schema, OB_DDL_ADD_TENANT_END, trans, ddl_stmt_str_ptr))) {
|
|
LOG_WARN("create tenant failed", K(new_tenant_schema), K(ret));
|
|
} else {/*do nothing*/}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ret = OB_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_SUCCESS == 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_SUCCESS == ret, K(temp_ret));
|
|
}
|
|
}
|
|
if (tenant_trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
const bool is_commit = false;//no need commit, only for check and lock
|
|
if (OB_SUCCESS != (temp_ret = tenant_trans.end(is_commit))) {
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
LOG_WARN("trans end failed", KR(ret), KR(temp_ret), K(is_commit));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_SUCCESS != (temp_ret = publish_schema(OB_SYS_TENANT_ID))) {
|
|
LOG_WARN("publish schema failed", K(temp_ret));
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("[CREATE_TENANT] STEP 3. finish create tenant end", KR(ret), K(tenant_id),
|
|
"cost", ObTimeUtility::fast_current_time() - start_time);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::commit_alter_tenant_locality(
|
|
const rootserver::ObCommitAlterTenantLocalityArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTenantSchema *orig_tenant_schema = NULL;
|
|
const ObTenantSchema *orig_meta_tenant_schema = NULL;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_UNLIKELY(!arg.is_valid())) {
|
|
LOG_WARN("invalid argument", KR(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(arg.tenant_id_, orig_tenant_schema))) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", KR(ret), "tenant_id", arg.tenant_id_);
|
|
} else if (OB_UNLIKELY(NULL == orig_tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", KR(ret), "tenant_id", arg.tenant_id_);
|
|
} else if (OB_UNLIKELY(orig_tenant_schema->get_locality_str().empty())
|
|
|| OB_UNLIKELY(orig_tenant_schema->get_previous_locality_str().empty())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tenant locality status error", KR(ret),
|
|
"tenant_id", orig_tenant_schema->get_tenant_id(),
|
|
"tenant locality", orig_tenant_schema->get_locality_str(),
|
|
"tenant previous locality", orig_tenant_schema->get_previous_locality_str());
|
|
} else {
|
|
// deal with meta tenant related to certain user tenant
|
|
if (is_user_tenant(arg.tenant_id_)) {
|
|
if (OB_FAIL(schema_guard.get_tenant_info(gen_meta_tenant_id(arg.tenant_id_), orig_meta_tenant_schema))) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("meta tenant not exist", KR(ret), "meta_tenant_id", gen_meta_tenant_id(arg.tenant_id_));
|
|
} else if (OB_UNLIKELY(NULL == orig_meta_tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("meta tenant not exist", KR(ret), "meta_tenant_id", gen_meta_tenant_id(arg.tenant_id_));
|
|
} else if (OB_UNLIKELY(orig_meta_tenant_schema->get_locality_str().empty())
|
|
|| OB_UNLIKELY(orig_meta_tenant_schema->get_previous_locality_str().empty())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("meta tenant locality status error", KR(ret),
|
|
"meta tenant_id", orig_meta_tenant_schema->get_tenant_id(),
|
|
"meta tenant locality", orig_meta_tenant_schema->get_locality_str(),
|
|
"meta tenant previous locality", orig_meta_tenant_schema->get_previous_locality_str());
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("fail to start transaction", KR(ret), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObTenantSchema new_tenant_schema;
|
|
ObTenantSchema new_meta_tenant_schema;
|
|
// refresh sys/user tenant schema
|
|
if (OB_FAIL(new_tenant_schema.assign(*orig_tenant_schema))) {
|
|
LOG_WARN("fail to assign tenant schema", KR(ret), KPC(orig_tenant_schema));
|
|
} else if (OB_FAIL(new_tenant_schema.set_previous_locality(ObString::make_string("")))) {
|
|
LOG_WARN("fail to set previous locality", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant_schema, trans))) {
|
|
LOG_WARN("fail to alter tenant", KR(ret), K(new_tenant_schema));
|
|
} else {
|
|
// refresh meta tenant schema
|
|
if (is_user_tenant(new_tenant_schema.get_tenant_id())) {
|
|
if (OB_FAIL(new_meta_tenant_schema.assign(*orig_meta_tenant_schema))) {
|
|
LOG_WARN("fail to assign meta tenant schema", KR(ret), KPC(orig_meta_tenant_schema));
|
|
} else if (OB_FAIL(new_meta_tenant_schema.set_previous_locality(ObString::make_string("")))) {
|
|
LOG_WARN("fail to set meta tenant previous locality", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_tenant(new_meta_tenant_schema, trans))) {
|
|
LOG_WARN("fail to alter meta tenant", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
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_SUCCESS == ret);
|
|
ret = (OB_SUCCESS == ret ? temp_ret : ret);
|
|
} else {} // ok
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) {// force return success
|
|
LOG_WARN("fail to publish schema", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_tenant_locality_type(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObTenantSchema &orig_tenant_schema,
|
|
const share::schema::ObTenantSchema &new_tenant_schema,
|
|
AlterLocalityType &alter_locality_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
alter_locality_type = ALTER_LOCALITY_INVALID;
|
|
const uint64_t tenant_id = orig_tenant_schema.get_tenant_id();
|
|
const common::ObString &locality = orig_tenant_schema.get_locality_str();
|
|
const common::ObString &previous_locality = orig_tenant_schema.get_previous_locality_str();
|
|
const bool is_restore = new_tenant_schema.is_restore();
|
|
if (OB_UNLIKELY(locality.empty())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, tenant locality is empty", K(ret), K(locality), K(tenant_id));
|
|
} else if (previous_locality.empty()) {
|
|
// previous locality is null, need check locality of tablegroup in tenant
|
|
alter_locality_type = TO_NEW_LOCALITY;
|
|
} else {
|
|
// The previous locality is not empty, the tenant is undergoing locality changes
|
|
// Currently, it is allowed to roll back the locality that is being changed.
|
|
// Rollback is currently defined as being set to be exactly the same as the original locality.
|
|
if (previous_locality != new_tenant_schema.get_locality_str()) {
|
|
alter_locality_type = ALTER_LOCALITY_INVALID;
|
|
} else {
|
|
// locality from 1->2, then rollback 2->1, The bottom layer does not support it, i
|
|
// it should be rejected
|
|
// Check in try_rollback_modify_tenant_locality
|
|
alter_locality_type = ROLLBACK_LOCALITY;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/*
|
|
* The locality of tenant is changed in the following function. At present,
|
|
* the locality settings of tenant and table have the following forms:
|
|
* # describe
|
|
* 1. The locality of the tenant must not be empty. The tenant locality upgraded from the version before 1.3 is empty
|
|
* in the internal table, but when the schema is refreshed, the locality of the tenant will be filled in
|
|
* as a full-featured replication of each zone.
|
|
* 2. The locality of the table can be empty, which means that the locality of the tenant is inherited.
|
|
* When the locality of the table is not empty, it means that it does not completely match the locality of the tenant;
|
|
* # locality change semantics
|
|
* 1. When the locality of a tenant changes, the distribution of replications of all tables whose locality is empty
|
|
* under that tenant will change accordingly. When the locality of the tenant is changed for a table
|
|
* whose locality is not empty, the distribution of the corresponding replication will not change.
|
|
* 2. Alter table can change the distribution of replications of a table whose locality is not empty.
|
|
* # Mutual restriction of tenant and table locality changes
|
|
* 1. When the old round of tenant locality has not been changed,
|
|
* the new round of tenant locality changes are not allowed to be executed.
|
|
* 2. When the change of the table whose locality is not empty under tenant is not completed,
|
|
* the change of tenant locality is not allowed to be executed.
|
|
* 3. When the locality change of tenant is not initiated, the locality change of the table
|
|
* whose locality is not empty is not allowed to be executed.
|
|
* # Change rules
|
|
* 1. One locality change is only allowed to do one of the operations of adding paxos,
|
|
* subtracting paxos and paxos type conversion (paxos->paxos), paxos->non_paxos is regarded as subtracting paxos,
|
|
* non_paxos->paxos is regarded as adding paxos;
|
|
* 2. In a locality change:
|
|
* 2.1. For adding paxos operation, orig_locality's paxos num >= majority(new_locality's paxos num);
|
|
* 2.2. For subtracting paxos operation, new_locality's paxos num >= majority(orig_locality's paxos num);
|
|
* 2.3. For converting paxos type operation, only one paxos type conversion is allowed for one locality change;
|
|
* 3. For replication type conversion, the following constraints need to be met:
|
|
* 3.1. For L-type replications, the replications other than F are not allowed to be converted to L,
|
|
* and L is not allowed to be converted to other replication types;
|
|
* 3.2. There will be no restrictions for the rest of the situation
|
|
* 4. In particular, in a scenario where only one replication of paxos is added,
|
|
* paxos num is allowed to go from 1 -> 2, but paxos num is not allowed to go from 2-> 1;
|
|
* 5. Non_paxos replications can occur together with the above changes, and there is no limit to the number.
|
|
* # after 1.4.7.1, the locality form of @region is no longer supported
|
|
*/
|
|
int ObDDLService::set_new_tenant_options(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObModifyTenantArg &arg,
|
|
share::schema::ObTenantSchema &new_tenant_schema,
|
|
const share::schema::ObTenantSchema &orig_tenant_schema,
|
|
AlterLocalityOp &alter_locality_op)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
common::ObArray<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_locality_compatible_(new_tenant_schema, false /*for_create_tenant*/))) {
|
|
LOG_WARN("fail to check locality with data version", KR(ret), K(new_tenant_schema));
|
|
} 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 (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::FORCE_LOCALITY)) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("only locality rollback can be forced", KR(ret), K(arg));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "only locality rollback can be forced, "
|
|
"forcing to be in a new locality is"); // forcing to be in a new locality is not allowed
|
|
}
|
|
if (FAILEDx(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_alter_meta_tenant_schema(
|
|
ObDDLOperator &ddl_operator,
|
|
const obrpc::ObModifyTenantArg &arg,
|
|
common::ObMySQLTransaction &trans,
|
|
share::schema::ObSchemaGetterGuard &sys_schema_guard,
|
|
const share::schema::ObTenantSchema &user_tenant_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = user_tenant_schema.get_tenant_id();
|
|
// only locality and primary_zone can be modified in meta_tenant
|
|
bool meta_tenant_has_option_changed = arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)
|
|
|| arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::PRIMARY_ZONE)
|
|
|| arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::ENABLE_ARBITRATION_SERVICE);
|
|
if (is_meta_tenant(tenant_id) || is_sys_tenant(tenant_id)) {
|
|
/* bypass, when this is a meta tenant,
|
|
* alter meta tenant shall be invoked in the upper layer
|
|
*/
|
|
} else if (!meta_tenant_has_option_changed) {
|
|
// do nothing
|
|
LOG_INFO("nothing changed to this tenant", KR(ret), K(arg), K(tenant_id));
|
|
} else {
|
|
const share::schema::ObTenantSchema *meta_tenant_schema = nullptr;
|
|
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
|
|
share::schema::ObTenantSchema new_meta_tenant_schema;
|
|
if (OB_FAIL(sys_schema_guard.get_tenant_info(
|
|
meta_tenant_id,
|
|
meta_tenant_schema))) {
|
|
LOG_WARN("fail to get tenant schema", KR(ret), K(meta_tenant_id));
|
|
} else if (OB_UNLIKELY(nullptr == meta_tenant_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("meta_tenant_schema ptr is null", KR(ret),
|
|
K(meta_tenant_id), KP(meta_tenant_schema));
|
|
} else if (OB_FAIL(new_meta_tenant_schema.assign(
|
|
*meta_tenant_schema))) {
|
|
LOG_WARN("fail to assign new meta tenant schema", KR(ret));
|
|
} else {
|
|
if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) {
|
|
common::ObArray<common::ObZone> user_zone_list;
|
|
if (OB_FAIL(new_meta_tenant_schema.set_previous_locality(
|
|
user_tenant_schema.get_previous_locality()))) {
|
|
LOG_WARN("fail to set previous locality", KR(ret));
|
|
} else if (OB_FAIL(new_meta_tenant_schema.set_locality(
|
|
user_tenant_schema.get_locality()))) {
|
|
LOG_WARN("fail to set locality", KR(ret));
|
|
} else if (OB_FAIL(user_tenant_schema.get_zone_list(user_zone_list))) {
|
|
LOG_WARN("fail to get zone list from user schema", KR(ret), K(user_zone_list));
|
|
} else if (OB_FAIL(new_meta_tenant_schema.set_zone_list(user_zone_list))) {
|
|
LOG_WARN("fail to set zone list", KR(ret));
|
|
} else if (OB_FAIL(new_meta_tenant_schema.set_primary_zone(user_tenant_schema.get_primary_zone()))) {
|
|
LOG_WARN("fail to set primary zone", KR(ret), "primary_zone", user_tenant_schema.get_primary_zone());
|
|
}
|
|
} else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::PRIMARY_ZONE)) {
|
|
if (OB_FAIL(new_meta_tenant_schema.set_primary_zone(user_tenant_schema.get_primary_zone()))) {
|
|
LOG_WARN("fail to set primary zone", KR(ret), "primary_zone", user_tenant_schema.get_primary_zone());
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::ENABLE_ARBITRATION_SERVICE)) {
|
|
new_meta_tenant_schema.set_arbitration_service_status(user_tenant_schema.get_arbitration_service_status());
|
|
}
|
|
|
|
if (FAILEDx(ddl_operator.alter_tenant(
|
|
new_meta_tenant_schema,
|
|
trans,
|
|
nullptr /* do not record ddl stmt str */))) {
|
|
LOG_WARN("fail to alter meta tenant locality", KR(ret), K(meta_tenant_id));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::try_rollback_modify_tenant_locality(
|
|
const obrpc::ObModifyTenantArg &arg,
|
|
share::schema::ObTenantSchema &new_schema,
|
|
const share::schema::ObTenantSchema &orig_schema,
|
|
const common::ObIArray<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, new_schema.get_arbitration_service_status()))) {
|
|
LOG_WARN("fail to check and get paxos replica task", K(ret), K(pre_zone_locality), K(cur_zone_locality),
|
|
"arbitration service status", new_schema.get_arbitration_service_status());
|
|
} 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", KR(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)) {
|
|
lib::ob_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, new_tenant_schema.get_arbitration_service_status()))) {
|
|
LOG_WARN("fail to check and get paxos replica task",
|
|
K(ret), K(pre_zone_locality), K(cur_zone_locality), "arbitration service status",
|
|
new_tenant_schema.get_arbitration_service_status());
|
|
} 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 {
|
|
lib::ob_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::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;
|
|
}
|
|
case ObModifyTenantArg::ENABLE_ARBITRATION_SERVICE: {
|
|
// 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::ObMySQLTransaction &trans,
|
|
common::ObIArray<uint64_t> &new_ug_id_array,
|
|
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;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::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)) {
|
|
lib::ob_sort(new_pool_name_list.begin(), new_pool_name_list.end());
|
|
lib::ob_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(
|
|
trans, new_ug_id_array, compat_mode, diff_pools, tenant_id,
|
|
false/*is_bootstrap*/, OB_INVALID_TENANT_ID/*source_tenant_id*/,
|
|
true/*check_data_version*/))) {
|
|
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(
|
|
trans, new_ug_id_array, diff_pools, tenant_id))) {
|
|
LOG_WARN("fail to revoke pools", K(ret));
|
|
} else {} // no more to do
|
|
} else if (new_pool_name_list.count() == old_pool_name_list.count()) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_pool_name_list.count(); ++i) {
|
|
if (new_pool_name_list.at(i) != old_pool_name_list.at(i)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(new_pool_name_list), K(old_pool_name_list));
|
|
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list");
|
|
}
|
|
}
|
|
} else {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(new_pool_name_list), K(old_pool_name_list));
|
|
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "resource pool list");
|
|
}
|
|
}
|
|
LOG_INFO("cal resource pool list result",
|
|
K(new_pool_name_list),
|
|
K(old_pool_name_list),
|
|
K(diff_pools),
|
|
K(grant));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_grant_pools_permitted(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const common::ObIArray<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_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_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_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;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_UNLIKELY(NULL == unit_mgr_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unit_mgr_ ptr is null", K(ret));
|
|
} else if (OB_FAIL(tenant_schema.get_zone_list(zone_list))) {
|
|
LOG_WARN("fail to get zones of pools", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < zone_list.count() && is_permitted; ++i) {
|
|
const common::ObZone &zone = zone_list.at(i);
|
|
int64_t total_unit_num = 0;
|
|
int64_t full_unit_num = 0;
|
|
int64_t logonly_unit_num = 0;
|
|
bool enough = false;
|
|
if (OB_FAIL(unit_mgr_->get_zone_pools_unit_num(
|
|
zone, new_pool_name_list, total_unit_num, full_unit_num, logonly_unit_num))) {
|
|
LOG_WARN("fail to get pools unit num", K(ret));
|
|
} else if (total_unit_num != full_unit_num + logonly_unit_num) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unit num value not match", K(ret),
|
|
K(total_unit_num), K(full_unit_num), K(logonly_unit_num));
|
|
} else if (!tenant_schema.get_previous_locality_str().empty()) {
|
|
is_permitted = false;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW,
|
|
"revoking resource pools when tenant in locality modification");
|
|
} else if (OB_FAIL(unit_mgr_->check_schema_zone_unit_enough(
|
|
zone, total_unit_num, full_unit_num, logonly_unit_num,
|
|
tenant_schema, schema_guard, enough))) {
|
|
LOG_WARN("fail to check schema zone unit enough", K(ret));
|
|
} else if (!enough) {
|
|
is_permitted = false;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW,
|
|
"revoking resource pools with tenant locality on");
|
|
} else { /* good */ }
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/* long_pool_name_list and short_pool_name_list has sorted
|
|
* in parameter condition:
|
|
* The length of long_pool_name_list is 1 larger than the length of short_pool_name_list
|
|
* This function has two functions:
|
|
* 1 check whether long_pool_name_list is only one more resource_pool_name than short_pool_name_list
|
|
* 2 Put this extra resource_pool_name into the diff_pools array.
|
|
*/
|
|
int ObDDLService::cal_resource_pool_list_diff(
|
|
const common::ObIArray<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;
|
|
}
|
|
|
|
/*
|
|
* 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;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (0 != arg.sys_var_list_.count() &&
|
|
!arg.alter_option_bitset_.is_empty()) {
|
|
// After the schema is split, because __all_sys_variable is split under the tenant, in order to avoid
|
|
// cross-tenant transactions, it is forbidden to modify the tenant option and the system variable at the same time.
|
|
// For this reason, the read only column of the tenant option is no longer maintained,
|
|
// and it is subject to system variables.
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("modify tenant option and system variable at the same time", K(ret), K(arg));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify tenant option and system variable at the same time");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_name, orig_tenant_schema))) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr());
|
|
LOG_WARN("tenant not exists", K(arg), K(ret));
|
|
} else if (OB_UNLIKELY(NULL == orig_tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr());
|
|
LOG_WARN("tenant not exists", K(arg), K(ret));
|
|
} else if (FALSE_IT(is_restore = orig_tenant_schema->is_restore())) {
|
|
} else if (!is_restore) {
|
|
// The physical recovery may be in the system table recovery stage, and it is necessary to avoid
|
|
// the situation where SQL cannot be executed and hang
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(
|
|
orig_tenant_schema->get_tenant_id(), schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table",
|
|
K(ret), "tenant_id", orig_tenant_schema->get_tenant_id());
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_name, orig_tenant_schema))) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr());
|
|
LOG_WARN("tenant not exists", K(arg), K(ret));
|
|
} else if (OB_UNLIKELY(NULL == orig_tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TENANT_NOT_EXIST, tenant_name.length(), tenant_name.ptr());
|
|
LOG_WARN("tenant not exists", K(arg), K(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(modify_tenant_inner_phase(arg, orig_tenant_schema, schema_guard, is_restore))) {
|
|
LOG_WARN("modify_tenant_inner_phase fail", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::modify_tenant_inner_phase(const ObModifyTenantArg &arg, const ObTenantSchema *orig_tenant_schema, ObSchemaGetterGuard &schema_guard, bool is_restore)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (0 != arg.sys_var_list_.count()) {
|
|
// modify system variable
|
|
const ObSysVariableSchema *orig_sys_variable = NULL;
|
|
const uint64_t tenant_id = orig_tenant_schema->get_tenant_id();
|
|
int64_t schema_version = OB_INVALID_VERSION;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
bool value_changed = false;
|
|
if (is_restore && is_user_tenant(tenant_id)) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("ddl operation is not allowed in standby cluster", K(ret));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "ddl operation in standby cluster");
|
|
} else if (OB_FAIL(schema_guard.get_sys_variable_schema(
|
|
orig_tenant_schema->get_tenant_id(), orig_sys_variable))) {
|
|
LOG_WARN("get sys variable schema failed", K(ret));
|
|
} else if (OB_ISNULL(orig_sys_variable)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys variable schema is null", K(ret));
|
|
} else {
|
|
ObSysVariableSchema new_sys_variable;
|
|
if (OB_FAIL(new_sys_variable.assign(*orig_sys_variable))) {
|
|
LOG_WARN("fail to assign sys variable schema", KR(ret), KPC(orig_sys_variable));
|
|
} else if (FALSE_IT(new_sys_variable.reset_sysvars())) {
|
|
} else if (OB_FAIL(update_sys_variables(arg.sys_var_list_, *orig_sys_variable, new_sys_variable, value_changed))) {
|
|
LOG_WARN("failed to update_sys_variables", K(ret));
|
|
} else if (value_changed == true) {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
|
|
} else {
|
|
const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR;
|
|
if (OB_FAIL(ddl_operator.replace_sys_variable(new_sys_variable, schema_version, trans, operation_type, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to replace sys variable", K(ret), K(new_sys_variable));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
|
|
if (OB_FAIL(new_tenant_schema.assign(*orig_tenant_schema))) {
|
|
LOG_WARN("fail to assign tenant schema", KR(ret));
|
|
} else if (is_meta_tenant(tenant_id)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not allowed to modify meta tenant's options manually", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(set_new_tenant_options(schema_guard, arg, new_tenant_schema,
|
|
*orig_tenant_schema, alter_locality_op))) {
|
|
LOG_WARN("failed to set new tenant options", K(ret));
|
|
} else if (OB_FAIL(check_alter_tenant_replica_options(
|
|
arg, new_tenant_schema, *orig_tenant_schema, schema_guard))) {
|
|
LOG_WARN("check tenant replica options failed", K(new_tenant_schema), K(ret));
|
|
}
|
|
// modify tenant option
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
common::ObArray<uint64_t> new_ug_id_array;
|
|
if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version));
|
|
} else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST)
|
|
&& OB_FAIL(modify_and_cal_resource_pool_diff(
|
|
trans, new_ug_id_array, schema_guard,
|
|
new_tenant_schema, arg.pool_list_, grant, diff_pools))) {
|
|
LOG_WARN("fail to grant_pools", K(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && !is_restore) {
|
|
if (OB_FAIL(check_tenant_primary_zone_(schema_guard, new_tenant_schema))) {
|
|
LOG_WARN("fail to check tenant primary zone", KR(ret), K(new_tenant_schema));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::ENABLE_ARBITRATION_SERVICE)) {
|
|
#ifndef OB_BUILD_ARBITRATION
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("modify tenant arbitration service status in CE version not supprted", KR(ret));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify tenant arbitration service status in CE version");
|
|
#else
|
|
const ObArbitrationServiceStatus old_status = orig_tenant_schema->get_arbitration_service_status();
|
|
const ObArbitrationServiceStatus new_status = arg.tenant_schema_.get_arbitration_service_status();
|
|
if ((new_status.is_enabling() && old_status.is_enable_like())
|
|
|| ((new_status.is_disabling() && old_status.is_disable_like()))) {
|
|
// do nothing
|
|
} else if (OB_FAIL(check_tenant_arbitration_service_status_(
|
|
trans,
|
|
tenant_id,
|
|
old_status,
|
|
new_status))) {
|
|
LOG_WARN("fail to check tenant arbitration service", KR(ret), K(tenant_id), K(old_status), K(new_status));
|
|
} else {
|
|
new_tenant_schema.set_arbitration_service_status(new_status);
|
|
}
|
|
#endif
|
|
}
|
|
|
|
if (FAILEDx(ObAlterPrimaryZoneChecker::create_alter_tenant_primary_zone_rs_job_if_needed(
|
|
arg,
|
|
tenant_id,
|
|
*orig_tenant_schema,
|
|
new_tenant_schema,
|
|
trans))) {
|
|
// if the command is alter tenant primary zone, we need to check whether first priority zone
|
|
// has been changed. if so, the number of ls will be changed as well.
|
|
// when the change occurs, we need to create a rs job ALTER_TENANT_PRIMARY_ZONE to
|
|
// track if the number of ls matches the number of first primary zone
|
|
// otherwise, the rs job is completed immediately
|
|
LOG_WARN("fail to execute create_alter_tenant_primary_zone_rs_job_if_needed", KR(ret),
|
|
K(arg), K(tenant_id), KPC(orig_tenant_schema), K(new_tenant_schema));
|
|
} else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant_schema, trans, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to alter tenant", K(ret));
|
|
} else if (OB_FAIL(try_alter_meta_tenant_schema(
|
|
ddl_operator, arg, trans, schema_guard, new_tenant_schema))) {
|
|
LOG_WARN("failed to try alter meta tenant schema", KR(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret)
|
|
&& arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::LOCALITY)) {
|
|
if (OB_FAIL(record_tenant_locality_event_history(alter_locality_op, arg, new_tenant_schema, trans))) {
|
|
LOG_WARN("fail to record tenant locality event history", K(ret));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
// publish schema
|
|
if (OB_SUCC(ret) && OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) {
|
|
LOG_WARN("publish schema failed, ", K(ret));
|
|
}
|
|
|
|
// When the new and old resource pool lists are consistent, no diff is generated, diff_pools is empty,
|
|
// and there is no need to call the following function
|
|
if (OB_SUCC(ret)
|
|
&& arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::RESOURCE_POOL_LIST)
|
|
&& diff_pools.count() > 0) {
|
|
if (OB_FAIL(unit_mgr_->load())) {
|
|
LOG_WARN("unit_manager reload failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
} else if (!arg.new_tenant_name_.empty()) {
|
|
// rename tenant
|
|
const uint64_t tenant_id = orig_tenant_schema->get_tenant_id();
|
|
const ObString new_tenant_name = arg.new_tenant_name_;
|
|
ObTenantSchema new_tenant_schema;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(new_tenant_schema.assign(*orig_tenant_schema))) {
|
|
LOG_WARN("fail to assign tenant schema", KR(ret));
|
|
} else if (is_meta_tenant(tenant_id)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not allowed to modify meta tenant's options manually", KR(ret), K(tenant_id));
|
|
} else if (orig_tenant_schema->is_restore()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("rename tenant while tenant is in physical restore status is not allowed",
|
|
KR(ret), KPC(orig_tenant_schema));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "rename tenant while tenant is in physical restore status is");
|
|
} else if (orig_tenant_schema->get_tenant_id() <= OB_MAX_RESERVED_TENANT_ID) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("rename special tenant not supported",
|
|
K(ret), K(orig_tenant_schema->get_tenant_id()));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "rename special tenant");
|
|
} else if (NULL != schema_guard.get_tenant_info(new_tenant_name)) {
|
|
ret = OB_TENANT_EXIST;
|
|
LOG_USER_ERROR(OB_TENANT_EXIST, to_cstring(new_tenant_name));
|
|
LOG_WARN("tenant already exists", K(ret), K(new_tenant_name));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(new_tenant_schema.set_tenant_name(new_tenant_name))) {
|
|
LOG_WARN("failed to rename tenant", K(ret),
|
|
K(new_tenant_name),
|
|
K(new_tenant_schema));
|
|
} else if (OB_FAIL(ddl_operator.rename_tenant(new_tenant_schema,
|
|
trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to rename tenant", K(ret), K(new_tenant_schema));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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;
|
|
if (OB_FAIL(new_sysvar.assign(*sysvar))) {
|
|
LOG_WARN("fail to assign sys var schema", KR(ret), KPC(sysvar));
|
|
} else if (SYS_VAR_OB_COMPATIBILITY_MODE
|
|
== ObSysVarFactory::find_sys_var_id_by_name(new_sysvar.get_name())) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "change tenant compatibility mode");
|
|
} else if (new_sysvar.is_read_only()) {
|
|
ret = OB_ERR_INCORRECT_GLOBAL_LOCAL_VAR;
|
|
LOG_USER_ERROR(OB_ERR_INCORRECT_GLOBAL_LOCAL_VAR,
|
|
new_sysvar.get_name().length(),
|
|
new_sysvar.get_name().ptr(),
|
|
(int)strlen("read only"),
|
|
"read only");
|
|
} else if (new_sysvar.get_value() != sysvar_value.value_) {
|
|
value_changed = true;
|
|
if (OB_FAIL(new_sysvar.set_value(sysvar_value.value_))) {
|
|
LOG_WARN("failed to set_value", K(ret));
|
|
} else if (OB_FAIL(new_sys_variable.add_sysvar_schema(new_sysvar))) {
|
|
LOG_WARN("failed to add sysvar schema", K(ret));
|
|
} else {
|
|
LOG_DEBUG("succ to update sys value", K(sysvar_value));
|
|
sysvar = NULL;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_rename_object_type(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const uint64_t database_id,
|
|
const ObString &object_name,
|
|
const ObTableSchema *&table_schema,
|
|
const ObSynonymInfo *&synonym_info,
|
|
const ObSequenceSchema *&sequence_schema,
|
|
RenameOracleObjectType &obj_type)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t sequence_id = 0;
|
|
bool sequence_exist = false;
|
|
bool is_system_generated = false;
|
|
obj_type = RENAME_TYPE_INVALID;
|
|
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
database_id,
|
|
object_name,
|
|
false,
|
|
table_schema))) {
|
|
LOG_WARN("fail to get table schema", K(ret));
|
|
} else if (NULL != table_schema) {
|
|
obj_type = RENAME_TYPE_TABLE_OR_VIEW;
|
|
}
|
|
if (FAILEDx(schema_guard.get_synonym_info(tenant_id,
|
|
database_id,
|
|
object_name,
|
|
synonym_info))) {
|
|
LOG_WARN("fail to get synonym info", K(ret));
|
|
} else if (NULL != synonym_info) {
|
|
if (RENAME_TYPE_INVALID != obj_type) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("multi types of objects have the same object name", K(ret), K(object_name), K(obj_type));
|
|
} else {
|
|
obj_type = RENAME_TYPE_SYNONYM;
|
|
}
|
|
}
|
|
if (FAILEDx(schema_guard.check_sequence_exist_with_name(tenant_id,
|
|
database_id,
|
|
object_name,
|
|
sequence_exist,
|
|
sequence_id,
|
|
is_system_generated))) {
|
|
LOG_WARN("fail to check sequence exist", K(ret));
|
|
} else if (sequence_exist) {
|
|
if (is_system_generated) {
|
|
ret = OB_ERR_CANNOT_RENAME_SYSTEM_GENERATED_SEQUENCE;
|
|
LOG_WARN("cannot rename system generated sequence", K(sequence_id), K(ret));
|
|
LOG_USER_ERROR(OB_ERR_CANNOT_RENAME_SYSTEM_GENERATED_SEQUENCE);
|
|
} else if (OB_FAIL(schema_guard.get_sequence_schema(tenant_id, sequence_id, sequence_schema))) {
|
|
LOG_WARN("fail get sequence schema", K(ret));
|
|
} else if (RENAME_TYPE_INVALID != obj_type) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("multi types of objects have the same object name", K(ret), K(object_name), K(obj_type));
|
|
} else {
|
|
obj_type = RENAME_TYPE_SEQUENCE;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && RENAME_TYPE_INVALID == obj_type) {
|
|
ret = OB_OBJECT_NAME_NOT_EXIST;
|
|
LOG_WARN("object does not exist", K(ret));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::record_tenant_locality_event_history(
|
|
const AlterLocalityOp &alter_locality_op,
|
|
const obrpc::ObModifyTenantArg &arg,
|
|
const share::schema::ObTenantSchema &tenant_schema,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_data_version = 0;
|
|
int64_t job_id = 0;
|
|
ObRsJobType job_type = JOB_TYPE_INVALID;
|
|
if (ALTER_LOCALITY_OP_INVALID == alter_locality_op) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid alter locality op", K(ret), K(alter_locality_op));
|
|
} else {
|
|
int64_t job_id = 0;
|
|
if (OB_FAIL(ObAlterLocalityFinishChecker::find_rs_job(tenant_schema.get_tenant_id(), job_id, trans))) {
|
|
if (OB_ENTRY_NOT_EXIST == ret) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("fail to find rs job", K(ret), "tenant_id", tenant_schema.get_tenant_id());
|
|
}
|
|
} else {
|
|
ret = RS_JOB_COMPLETE(job_id, OB_CANCELED, trans);
|
|
FLOG_INFO("[ALTER_TENANT_LOCALITY NOTICE] cancel an old inprogress rs job due to rollback", KR(ret),
|
|
"tenant_id", tenant_schema.get_tenant_id(), K(job_id));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && ROLLBACK_ALTER_LOCALITY == alter_locality_op) {
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(OB_SYS_TENANT_ID, tenant_data_version))) {
|
|
LOG_WARN("fail to get sys tenant's min data version", KR(ret));
|
|
} else if (tenant_data_version < DATA_VERSION_4_2_1_0) {
|
|
job_type = ObRsJobType::JOB_TYPE_ROLLBACK_ALTER_TENANT_LOCALITY;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// ALTER_LOCALITY, ROLLBACK_ALTER_LOCALITY(only 4.2), NOP_LOCALITY_OP
|
|
job_type = ObRsJobType::JOB_TYPE_INVALID == job_type ?
|
|
ObRsJobType::JOB_TYPE_ALTER_TENANT_LOCALITY : job_type;
|
|
const int64_t extra_info_len = common::MAX_ROOTSERVICE_EVENT_EXTRA_INFO_LENGTH;
|
|
HEAP_VAR(char[extra_info_len], extra_info) {
|
|
memset(extra_info, 0, extra_info_len);
|
|
int64_t pos = 0;
|
|
if (OB_FAIL(databuff_printf(extra_info, extra_info_len, pos,
|
|
"FROM: '%.*s', TO: '%.*s'", tenant_schema.get_previous_locality_str().length(),
|
|
tenant_schema.get_previous_locality_str().ptr(), tenant_schema.get_locality_str().length(),
|
|
tenant_schema.get_locality_str().ptr()))) {
|
|
LOG_WARN("format extra_info failed", KR(ret), K(tenant_schema));
|
|
} else if (OB_FAIL(RS_JOB_CREATE_WITH_RET(job_id, job_type, trans,
|
|
"tenant_name", tenant_schema.get_tenant_name(),
|
|
"tenant_id", tenant_schema.get_tenant_id(),
|
|
"sql_text", ObHexEscapeSqlStr(arg.ddl_stmt_str_),
|
|
"extra_info", ObHexEscapeSqlStr(extra_info)))) {
|
|
LOG_WARN("failed to create new rs job", KR(ret), K(job_type), K(tenant_schema), K(extra_info));
|
|
}
|
|
}
|
|
FLOG_INFO("[ALTER_TENANT_LOCALITY NOTICE] create a new rs job", KR(ret),
|
|
"tenant_id", tenant_schema.get_tenant_id(), K(job_id), K(alter_locality_op));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if ((ROLLBACK_ALTER_LOCALITY == alter_locality_op
|
|
&& arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::FORCE_LOCALITY))
|
|
|| NOP_LOCALITY_OP == alter_locality_op) {
|
|
ret = RS_JOB_COMPLETE(job_id, 0, trans);
|
|
FLOG_INFO("[ALTER_TENANT_LOCALITY NOTICE] complete a new rs job immediately", KR(ret),
|
|
"tenant_id", tenant_schema.get_tenant_id(), K(job_id), K(alter_locality_op));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/*
|
|
* This interface includes 4 situations of primary and standalone cluster in total
|
|
* primary tenant
|
|
* 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
|
|
*
|
|
* standby tenant
|
|
* 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: equal to drop tneant force;
|
|
*
|
|
* meta tenant can only be force dropped with its user tenant.
|
|
*/
|
|
int ObDDLService::drop_tenant(const ObDropTenantArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const bool if_exist = arg.if_exist_;
|
|
bool drop_force = !arg.delay_to_drop_;
|
|
const bool open_recyclebin = arg.open_recyclebin_;
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObArray<ObResourcePoolName> pool_names;
|
|
ObArray<share::ObResourcePool*> pools;
|
|
ret = OB_E(EventTable::EN_DROP_TENANT_FAILED) OB_SUCCESS;
|
|
bool is_standby = false;
|
|
uint64_t user_tenant_id = common::OB_INVALID_ID;
|
|
int64_t refreshed_schema_version = 0;
|
|
common::ObArray<uint64_t> drop_ug_id_array;
|
|
bool specify_tenant_id = OB_INVALID_TENANT_ID != arg.tenant_id_;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret));
|
|
} else if (specify_tenant_id && OB_FAIL(schema_guard.get_tenant_info(arg.tenant_id_, tenant_schema))) {
|
|
LOG_WARN("fail to gt tenant info", KR(ret), K(arg));
|
|
} else if (!specify_tenant_id && OB_FAIL(schema_guard.get_tenant_info(arg.tenant_name_, tenant_schema))) {
|
|
LOG_WARN("fail to gt tenant info", KR(ret), K(arg));
|
|
} else if (OB_ISNULL(tenant_schema)) {
|
|
if (if_exist) {
|
|
LOG_USER_NOTE(OB_TENANT_NOT_EXIST, arg.tenant_name_.length(), arg.tenant_name_.ptr());
|
|
LOG_INFO("tenant not exist, no need to delete it", K(arg));
|
|
} else {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TENANT_NOT_EXIST, arg.tenant_name_.length(), arg.tenant_name_.ptr());
|
|
LOG_WARN("tenant not exist, can't delete it", K(arg), KR(ret));
|
|
}
|
|
} else if (FALSE_IT(user_tenant_id = tenant_schema->get_tenant_id())) {
|
|
} else if (!is_user_tenant(user_tenant_id)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can't drop sys or meta tenant", KR(ret), K(user_tenant_id));
|
|
} else if (drop_force) {
|
|
//is drop force, no need to check
|
|
//pay attention
|
|
} else if (tenant_schema->is_in_recyclebin()) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant in recyclebin, can't delete it", K(arg), KR(ret));
|
|
LOG_USER_ERROR(OB_TENANT_NOT_EXIST, arg.tenant_name_.length(), arg.tenant_name_.ptr());
|
|
} else 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 {
|
|
ObAllTenantInfo tenant_info;
|
|
if (OB_FAIL(ObAllTenantInfoProxy::load_tenant_info(
|
|
user_tenant_id, sql_proxy_, false, tenant_info))) {
|
|
LOG_WARN("failed to load tenant info", KR(ret), K(arg), K(user_tenant_id));
|
|
} else if (!tenant_info.is_primary() && !tenant_info.is_standby()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("drop tenant not in primary or standby role is not supported",
|
|
KR(ret), K(arg), K(tenant_info));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "should drop tenant force, drop tenant");
|
|
} else if (tenant_info.is_standby() && !open_recyclebin) {
|
|
//if standby tenant and no recyclebin, need drop force
|
|
drop_force = true;
|
|
FLOG_INFO("is standby tenant, need drop force", K(tenant_info));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
// ignore
|
|
} else if (OB_ISNULL(tenant_schema)) {
|
|
// We need to ignore the drop tenant if exists statement in the case that the tenant has already been deleted
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(user_tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
/*
|
|
* drop tenant force: delay_to_drop_ is false
|
|
* delay_to_drop_ is true in rest the situation
|
|
* drop tenant && recyclebin enable: in recyclebin
|
|
* (drop tenant && recyclebin disable) || drop tenant purge: delay delete
|
|
*/
|
|
const bool to_recyclebin = (arg.delay_to_drop_ && open_recyclebin);
|
|
const bool delay_to_drop = (arg.delay_to_drop_ && !open_recyclebin);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
//1.drop tenant force
|
|
if (drop_force) {
|
|
const uint64_t meta_tenant_id = gen_meta_tenant_id(user_tenant_id);
|
|
if (arg.drop_only_in_restore_) {
|
|
// if drop_restore_tenant is true, it demands that the tenant must be in restore status after drop tenant trans start.
|
|
if (OB_ISNULL(tenant_schema)) {
|
|
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_restore()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "Cancel tenant not in restore is");
|
|
LOG_WARN("Cancel tenant not in restore is not allowed", K(ret), K(user_tenant_id));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(drop_resource_pool_pre(
|
|
user_tenant_id, drop_ug_id_array, pool_names, trans))) {
|
|
LOG_WARN("fail to drop resource pool pre", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_tenant(user_tenant_id, trans, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("ddl_operator drop_tenant failed", K(user_tenant_id), KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_tenant(meta_tenant_id, trans))) {
|
|
LOG_WARN("ddl_operator drop_tenant failed", K(meta_tenant_id), KR(ret));
|
|
} else if (OB_FAIL(try_drop_sys_ls_(meta_tenant_id, trans))) {
|
|
LOG_WARN("failed to drop sys ls", KR(ret), K(meta_tenant_id));
|
|
} else if (tenant_schema->is_in_recyclebin()) {
|
|
// try recycle record from __all_recyclebin
|
|
ObArray<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));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(reset_parallel_cache(meta_tenant_id))) {
|
|
LOG_WARN("fail to reset parallel cache", KR(ret), K(meta_tenant_id));
|
|
} else if (OB_FAIL(reset_parallel_cache(user_tenant_id))) {
|
|
LOG_WARN("fail to reset parallel cache", KR(ret), K(user_tenant_id));
|
|
}
|
|
}
|
|
} else {// put tenant into recyclebin
|
|
ObTenantSchema new_tenant_schema;
|
|
ObSqlString new_tenant_name;
|
|
if (OB_FAIL(new_tenant_schema.assign(*tenant_schema))) {
|
|
LOG_WARN("failed to assign tenant schema", KR(ret), KPC(tenant_schema));
|
|
} else 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));
|
|
} else if (to_recyclebin) {
|
|
//2. tenant in recyclebin
|
|
if (new_tenant_name.empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tenant name is null", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_tenant_to_recyclebin(
|
|
new_tenant_name,
|
|
new_tenant_schema,
|
|
trans, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("fail to drop tenant in recyclebin", KR(ret), K(user_tenant_id));
|
|
}
|
|
} else if (delay_to_drop) {
|
|
//3. tenant delay delete
|
|
if (OB_FAIL(ddl_operator.delay_to_drop_tenant(new_tenant_schema, trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("fail to delay_to drop tenant", K(ret));
|
|
} else {
|
|
// ObLSManager will process force_drop_tenant() logic each 100ms.
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (drop_force) {
|
|
if (OB_SUCC(ret) && OB_NOT_NULL(tenant_schema)) {
|
|
if (OB_FAIL(drop_resource_pool_final(
|
|
tenant_schema->get_tenant_id(), drop_ug_id_array,
|
|
pool_names))) {
|
|
LOG_WARN("fail to drop resource pool finsl", KR(ret), KPC(tenant_schema));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) {
|
|
LOG_WARN("publish schema failed", KR(ret));
|
|
}
|
|
}
|
|
|
|
LOG_INFO("drop tenant", K(arg), KR(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::try_drop_sys_ls_(const uint64_t meta_tenant_id,
|
|
common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_UNLIKELY(!is_meta_tenant(meta_tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("not meta tenant", KR(ret), K(meta_tenant_id));
|
|
} else if (OB_ISNULL(sql_proxy_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sql proxy is null", KR(ret));
|
|
} else {
|
|
//check ls exist in status
|
|
ObLSLifeAgentManager life_agent(*sql_proxy_);
|
|
ObLSStatusOperator ls_status;
|
|
ObLSStatusInfo sys_ls_info;
|
|
if (OB_FAIL(ls_status.get_ls_status_info(meta_tenant_id, SYS_LS, sys_ls_info, trans))) {
|
|
if (OB_ENTRY_NOT_EXIST == ret) {
|
|
ret = OB_SUCCESS;
|
|
LOG_INFO("sys ls not exist, no need to drop", KR(ret), K(meta_tenant_id));
|
|
} else {
|
|
LOG_WARN("failed to get ls status info", KR(ret), K(meta_tenant_id));
|
|
}
|
|
} else if (OB_FAIL(life_agent.drop_ls_in_trans(meta_tenant_id, SYS_LS, share::NORMAL_SWITCHOVER_STATUS, trans))) {
|
|
LOG_WARN("failed to drop ls in trans", KR(ret), K(meta_tenant_id));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::drop_resource_pool_pre(const uint64_t tenant_id,
|
|
common::ObIArray<uint64_t> &drop_ug_id_array,
|
|
ObIArray<ObResourcePoolName> &pool_names,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(unit_mgr_->get_pool_names_of_tenant(tenant_id, pool_names))) {
|
|
LOG_WARN("get_pool_names_of_tenant failed", K(tenant_id), KR(ret));
|
|
} else if (OB_FAIL(unit_mgr_->revoke_pools(trans, drop_ug_id_array, pool_names, tenant_id))) {
|
|
LOG_WARN("revoke_pools failed", K(pool_names), K(tenant_id), KR(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_resource_pool_final(const uint64_t tenant_id,
|
|
common::ObIArray<uint64_t> &drop_ug_id_array,
|
|
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_->load())) {
|
|
LOG_WARN("unit_manager reload failed", K(ret));
|
|
}
|
|
|
|
// delete from __all_schema_status
|
|
// The update of __all_core_table must be guaranteed to be a single partition transaction,
|
|
// so a separate transaction is required here.
|
|
// The failure of the transaction will not affect it, but there is garbage data in __all_core_table.
|
|
if (OB_SUCC(ret)) {
|
|
int temp_ret = OB_SUCCESS;
|
|
ObSchemaStatusProxy *schema_status_proxy = GCTX.schema_status_proxy_;
|
|
if (OB_ISNULL(schema_status_proxy)) {
|
|
temp_ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_status_proxy is null", K(temp_ret));
|
|
} else if (OB_SUCCESS !=
|
|
(temp_ret = schema_status_proxy->del_tenant_schema_status(tenant_id))) {
|
|
LOG_ERROR("del tenant schema status failed", KR(temp_ret), "tenant_id", tenant_id);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_tenant_object_name_with_origin_name_in_recyclebin(
|
|
const ObString &origin_tenant_name,
|
|
ObString &object_name,
|
|
common::ObIAllocator *allocator,
|
|
const bool is_flashback)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSqlString sql;
|
|
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
|
common::sqlclient::ObMySQLResult *result = NULL;
|
|
const char *desc_or_asc = (is_flashback ? "desc" : "asc");
|
|
if (OB_FAIL(sql.append_fmt(
|
|
"select object_name from oceanbase.__all_recyclebin where "
|
|
"original_name = '%.*s' and TYPE = 7 order by gmt_create %s limit 1",
|
|
origin_tenant_name.length(),
|
|
origin_tenant_name.ptr(),
|
|
desc_or_asc))) {
|
|
LOG_WARN("failed to append sql",
|
|
K(ret), K(origin_tenant_name), K(*desc_or_asc));
|
|
} else if (OB_FAIL(sql_proxy_->read(res, OB_SYS_TENANT_ID, sql.ptr()))) {
|
|
LOG_WARN("failed to execute sql", K(sql), K(ret));
|
|
} else if (OB_ISNULL(result = res.get_result())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get result", K(ret));
|
|
} else if (OB_FAIL(result->next())) {
|
|
if (OB_ITER_END == ret) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("origin tenant_name not exist in recyclebin", K(ret), K(sql));
|
|
} else {
|
|
LOG_WARN("iterate next result fail", K(ret), K(sql));
|
|
}
|
|
} else {
|
|
ObString tmp_object_name;
|
|
EXTRACT_VARCHAR_FIELD_MYSQL(*result, "object_name", tmp_object_name);
|
|
if (OB_FAIL(deep_copy_ob_string(*allocator, tmp_object_name, object_name))) {
|
|
LOG_WARN("failed to deep copy member list", K(ret), K(object_name));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ITER_END != result->next()) {
|
|
// The result will not exceed one line
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("result failed", K(ret), K(sql));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_tenant(const obrpc::ObFlashBackTenantArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
ObArenaAllocator allocator(ObModIds::OB_TENANT_INFO);
|
|
ObString final_tenant_name;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check_inner_stat failed", K(ret));
|
|
} else if (arg.tenant_id_ != OB_SYS_TENANT_ID) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "falshback tenant must in sys tenant");
|
|
LOG_WARN("falshback tenant must in sys tenant", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(
|
|
OB_SYS_TENANT_ID, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(arg.origin_tenant_name_, tenant_schema))) {
|
|
LOG_WARN("failt to get tenant info", K(ret));
|
|
} else if (OB_ISNULL(tenant_schema)) {
|
|
const bool is_flashback = true;
|
|
ObString new_tenant_name;
|
|
if (OB_FAIL(get_tenant_object_name_with_origin_name_in_recyclebin(arg.origin_tenant_name_,
|
|
new_tenant_name, &allocator,
|
|
is_flashback))) {
|
|
LOG_WARN("fail to get tenant obfect name", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(new_tenant_name, tenant_schema))) {
|
|
LOG_WARN("fail to get tenant info", K(ret));
|
|
} else if (OB_ISNULL(tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant name is not exist", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (!tenant_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OBJECT_NOT_IN_RECYCLEBIN;
|
|
LOG_WARN("tenant schema is not in recyclebin", K(ret), K(arg), K(*tenant_schema));
|
|
} else if (!arg.new_tenant_name_.empty()) {
|
|
final_tenant_name = arg.new_tenant_name_;
|
|
} else {}//nothing todo
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(flashback_tenant_in_trans(*tenant_schema,
|
|
final_tenant_name,
|
|
schema_guard,
|
|
arg.ddl_stmt_str_))) {
|
|
LOG_WARN("flashback tenant in trans failed", K(ret));
|
|
} else if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
LOG_INFO("finish flashback tenant", K(arg), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::flashback_tenant_in_trans(const share::schema::ObTenantSchema &tenant_schema,
|
|
const ObString &new_tenant_name,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObString &ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.flashback_tenant_from_recyclebin(tenant_schema,
|
|
trans,
|
|
new_tenant_name,
|
|
schema_guard,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("flashback tenant from recyclebin failed", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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;
|
|
ObAllTenantInfo tenant_info;
|
|
bool is_standby_tenant = false;
|
|
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 (FAILEDx(ObAllTenantInfoProxy::load_tenant_info(
|
|
tenant_schema->get_tenant_id(), sql_proxy_, false, tenant_info))) {
|
|
LOG_WARN("failed to load tenant info", KR(ret), K(arg), KPC(tenant_schema));
|
|
} else if (FALSE_IT(is_standby_tenant = tenant_info.is_standby())) {
|
|
//can not be there
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_standby_tenant) {
|
|
//drop tenant force
|
|
if (OB_FAIL(try_force_drop_tenant(*tenant_schema))) {
|
|
LOG_WARN("failed to try drop tenant force", KR(ret), KPC(tenant_schema));
|
|
}
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const uint64_t tenant_id = tenant_schema->get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("finish purge tenant", K(arg), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_tenant(const ObString &tenant_name, const bool is_lock)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObTenantSchema new_tenant_schema;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (tenant_name.length() <= 0) {
|
|
ret = OB_INVALID_TENANT_NAME;
|
|
LOG_WARN("invalid tenant name", K(tenant_name), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(OB_SYS_TENANT_ID, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_name, tenant_schema)) ||
|
|
NULL == tenant_schema) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist, can't lock it", K(tenant_name), K(ret));
|
|
} else if (tenant_schema->get_locked() == is_lock) {
|
|
ret = OB_SUCCESS;
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(new_tenant_schema.assign(*tenant_schema))) {
|
|
LOG_WARN("fail to assign tenant schema", KR(ret));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
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_SUCCESS == 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)
|
|
{
|
|
FLOG_INFO("[UPGRADE] begin upgrade system variable", K(arg));
|
|
int64_t start_time = ObTimeUtility::current_time();
|
|
DEBUG_SYNC(BEFORE_UPRADE_SYSTEM_VARIABLE);
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObSysVarSchema *old_schema = NULL;
|
|
const ObTenantSchema *tenant_info = NULL;
|
|
const ObSysVariableSchema *sys_variable_schema = NULL;
|
|
const ObString var_name = arg.get_sysvar().get_name();
|
|
const uint64_t tenant_id = arg.get_sysvar().get_tenant_id();
|
|
bool execute = true;
|
|
ObSysVarSchema new_sys_var;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", KR(ret), K(arg));
|
|
} else if (OB_FAIL(new_sys_var.assign(arg.get_sysvar()))) {
|
|
LOG_WARN("fail to assign sysvar", KR(ret), K(arg));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_info))) {
|
|
LOG_WARN("get tenant info failed", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) {
|
|
LOG_WARN("get sys variable schema failed", KR(ret));
|
|
} else if (OB_ISNULL(sys_variable_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys variable schema is null", KR(ret));
|
|
}
|
|
// check sys var schema
|
|
if (FAILEDx(sys_variable_schema->get_sysvar_schema(var_name, old_schema))) {
|
|
if (!arg.get_update_sys_var() && OB_ERR_SYS_VARIABLE_UNKNOWN == ret) {
|
|
// add sys var, sys var should not exist
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("get tenant system variable failed", KR(ret), K(arg));
|
|
}
|
|
} else if (OB_ISNULL(old_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys var schema is null", KR(ret), K(arg));
|
|
} else if (!arg.get_update_sys_var()) {
|
|
// case 1. add sys var, and sys var exist
|
|
if (new_sys_var.is_equal_for_add(*old_schema)) {
|
|
// new sys var will be mocked by schema when upgrade,
|
|
// only persist new sys var when sys var is equal with old sys var.
|
|
} else {
|
|
ret = OB_SCHEMA_ERROR;
|
|
LOG_WARN("system variable duplicated", KR(ret), K(var_name));
|
|
}
|
|
} else {
|
|
// update sys var
|
|
if (new_sys_var.is_equal_except_value(*old_schema)) {
|
|
// case 2. new sys var is same with existed schema(except value), do nothing
|
|
execute = false;
|
|
} else if (OB_FAIL(new_sys_var.set_value(old_schema->get_value()))) {
|
|
LOG_WARN("fail to set old value", KR(ret), K(new_sys_var), KPC(old_schema));
|
|
}
|
|
}
|
|
// execute add/update sys var
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!execute) {
|
|
// Reentrant update, do nothing in the following scenarios:
|
|
// case 1. add sys var, sys var exist
|
|
// case 2. update sys var, sys var is same with existed schema(except value)
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t schema_version = OB_INVALID_VERSION;
|
|
ObSysVariableSchema new_sys_variable_schema;
|
|
if (OB_UNLIKELY(!arg.is_valid())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("add sysvar argument is invalid", K(arg));
|
|
} else if (OB_FAIL(new_sys_variable_schema.assign(*sys_variable_schema))) {
|
|
LOG_WARN("fail to ass new sys variable schema", KR(ret), KPC(sys_variable_schema));
|
|
} else if (FALSE_IT(new_sys_variable_schema.reset_sysvars())) {
|
|
} else if (OB_FAIL(new_sys_variable_schema.add_sysvar_schema(new_sys_var))) {
|
|
LOG_WARN("add sysvar schema to new tenant info failed", KR(ret));
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id));
|
|
} else {
|
|
const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR;
|
|
if (OB_FAIL(ddl_operator.replace_sys_variable(new_sys_variable_schema, schema_version, trans, operation_type, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("alter tenant info failed", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", KR(ret));
|
|
}
|
|
}
|
|
FLOG_INFO("[UPGRADE] end upgrade system variable",
|
|
KR(ret), K(arg), "cost", ObTimeUtility::current_time() - start_time);
|
|
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;
|
|
if (OB_FAIL(new_sys_variable_schema.assign(*sys_variable_schema))) {
|
|
LOG_WARN("fail to assign sys variable schema", KR(ret));
|
|
} else 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 if (OB_ISNULL(old_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys var schema is null", KR(ret), K(modify_var));
|
|
} else if (OB_FAIL(new_schema.assign(*old_schema))) {
|
|
LOG_WARN("fail to assign sys var schema", KR(ret));
|
|
} else if (OB_UNLIKELY(!new_schema.is_valid())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("new schema is invalid", K(new_schema));
|
|
} else if(new_schema.get_value() != modify_var.get_value()) {
|
|
value_changed = true;
|
|
if(OB_FAIL(new_schema.set_value(modify_var.get_value()))) {
|
|
LOG_WARN("set new schema value failed", K(ret));
|
|
} else if (OB_FAIL(new_sys_variable_schema.add_sysvar_schema(new_schema))) {
|
|
LOG_WARN("add sysvar schema to new sys variable schema failed", K(ret));
|
|
}
|
|
} else {
|
|
//new value == old value, no need to update sys var schema
|
|
//do nothing
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && value_changed == true) {
|
|
bool is_oracle_mode = true;
|
|
int64_t schema_version = OB_INVALID_VERSION;
|
|
const ObSchemaOperationType operation_type = OB_DDL_ALTER_SYS_VAR;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ddl_operator.replace_sys_variable(new_sys_variable_schema, schema_version, trans, operation_type, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("alter tenant info failed", K(ret));
|
|
} else if (OB_FAIL(sys_variable_schema->get_oracle_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get oracle mode", K(ret));
|
|
#ifdef OB_BUILD_ORACLE_PL
|
|
} else if (!is_oracle_mode && OB_FAIL(pl::ObDbmsAuditMgmt::handle_audit_param_mysql(new_sys_variable_schema, trans))) {
|
|
LOG_WARN("failed to refresh audit log trail jobs", K(ret), K(new_sys_variable_schema));
|
|
#endif
|
|
}
|
|
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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && value_changed) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_database(const bool if_not_exist,
|
|
share::schema::ObDatabaseSchema &database_schema,
|
|
const ObString *ddl_stmt_str,
|
|
ObMySQLTransaction *ora_user_trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool is_exist = false;
|
|
uint64_t database_id = OB_INVALID_ID;
|
|
const uint64_t tenant_id = database_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObCollationType tenant_collation_type = CS_TYPE_INVALID;
|
|
ObCharsetType tenant_charset_type = CHARSET_INVALID;
|
|
int64_t refreshed_schema_version = 0;
|
|
LOG_INFO("database schema received", K(database_schema));
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_service_->check_database_exist(database_schema.get_tenant_id(),
|
|
database_schema.get_database_name_str(), database_id, is_exist))) {
|
|
LOG_WARN("check database exist failed", "database", database_schema.get_database_name(), K(ret));
|
|
} else {
|
|
const ObString database_name = database_schema.get_database_name_str();
|
|
if (is_exist) {
|
|
if (if_not_exist) {
|
|
ret = OB_SUCCESS;
|
|
database_schema.set_database_id(database_id);
|
|
LOG_USER_NOTE(OB_DATABASE_EXIST, database_name.length(), database_name.ptr());
|
|
LOG_WARN("database already exists, not need to create", "tenant_id",
|
|
database_schema.get_tenant_id(), "database_name",
|
|
database_schema.get_database_name());
|
|
} else {
|
|
ret = OB_DATABASE_EXIST;
|
|
LOG_USER_ERROR(OB_DATABASE_EXIST, database_name.length(), database_name.ptr());
|
|
LOG_WARN("database already exists, ", "tenant_id", database_schema.get_tenant_id(),
|
|
"database_name", database_schema.get_database_name(), K(ret));
|
|
}
|
|
} else if (OB_FAIL(check_create_with_db_id(database_schema))) {
|
|
LOG_WARN("not supported operator", K(ret), K(database_schema));
|
|
} else if (OB_FAIL(set_default_tablegroup_id(database_schema))) {
|
|
LOG_WARN("set_tablegroup_id failed", "tablegroup name",
|
|
database_schema.get_default_tablegroup_name(), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(ora_user_trans)
|
|
&& OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
|
|
if (OB_SUCC(ret)) {
|
|
const ObSysVarSchema *collation_var = NULL;
|
|
ObObj collation_obj;
|
|
ObMalloc alloc(ObModIds::OB_TEMP_VARIABLES);
|
|
if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id,
|
|
SYS_VAR_COLLATION_SERVER,
|
|
collation_var))) {
|
|
LOG_WARN("fail to get tenant var schema", K(ret));
|
|
} else if (OB_ISNULL(collation_var)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get charset_var or collation_var", K(ret));
|
|
} else if (OB_FAIL(collation_var->get_value(&alloc, NULL, collation_obj))) {
|
|
LOG_WARN("fail to get charset var value", K(ret));
|
|
} else {
|
|
tenant_collation_type = static_cast<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(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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
// Specify ora_user_trans, create user ddl, publish schema when create_user is completed
|
|
if (NULL == ora_user_trans) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed, ", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
//set new database options to new database schema
|
|
int ObDDLService::set_new_database_options(const ObAlterDatabaseArg &arg,
|
|
ObDatabaseSchema &new_database_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObDatabaseSchema &alter_database_schema = arg.database_schema_;
|
|
//replace alter options
|
|
for (int32_t i = ObAlterDatabaseArg::REPLICA_NUM;
|
|
ret == OB_SUCCESS && i < ObAlterDatabaseArg::MAX_OPTION; ++i) {
|
|
if (arg.alter_option_bitset_.has_member(i)) {
|
|
switch (i) {
|
|
case ObAlterDatabaseArg::REPLICA_NUM: {
|
|
// ignore alter replica num
|
|
break;
|
|
}
|
|
case ObAlterDatabaseArg::CHARSET_TYPE:
|
|
case ObAlterDatabaseArg::COLLATION_TYPE: {
|
|
new_database_schema.set_charset_type(alter_database_schema.get_charset_type());
|
|
new_database_schema.set_collation_type(alter_database_schema.get_collation_type());
|
|
break;
|
|
}
|
|
case ObAlterDatabaseArg::PRIMARY_ZONE: {
|
|
LOG_INFO("changing database's primary_zone does not take effect");
|
|
ret = OB_SUCCESS; // do nothing
|
|
break;
|
|
}
|
|
case ObAlterDatabaseArg::READ_ONLY: {
|
|
new_database_schema.set_read_only(alter_database_schema.is_read_only());
|
|
break;
|
|
}
|
|
case ObAlterDatabaseArg::DEFAULT_TABLEGROUP: {
|
|
if (OB_FAIL(new_database_schema.set_default_tablegroup_name(
|
|
alter_database_schema.get_default_tablegroup_name()))) {
|
|
LOG_WARN("failed to set default tablegroup name", K(ret));
|
|
} else if (OB_FAIL(set_default_tablegroup_id(new_database_schema))) {
|
|
LOG_WARN("failed to set default tablegroup id", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("Unknown option!", K(i));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_database(const ObAlterDatabaseArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
LOG_DEBUG("zixu alter database arg", K(arg));
|
|
const ObDatabaseSchema *origin_database_schema = NULL;
|
|
ObSchemaGetterGuard schema_guard;
|
|
uint64_t tenant_id = arg.database_schema_.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
if (OB_FAIL(schema_guard.get_database_schema(
|
|
tenant_id, arg.database_schema_.get_database_name_str(), origin_database_schema)) ||
|
|
NULL == origin_database_schema) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("database not exists", K(arg), K(ret));
|
|
} else if (origin_database_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("Can not perform alter operation on object in recyclebin", K(ret),
|
|
K(*origin_database_schema));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDatabaseSchema new_database_schema;
|
|
if (OB_FAIL(new_database_schema.assign(*origin_database_schema))) {
|
|
LOG_WARN("fail to assign database schema", KR(ret));
|
|
} else if (OB_FAIL(set_new_database_options(arg, new_database_schema))) {
|
|
LOG_WARN("failed to set new database options", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ret = ddl_operator.alter_database(new_database_schema, trans,
|
|
OB_DDL_ALTER_DATABASE,
|
|
&arg.ddl_stmt_str_);
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret) && OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed, ", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_database(const ObDropDatabaseArg &arg,
|
|
obrpc::ObDropDatabaseRes &res,
|
|
ObDDLSQLTransaction *ora_user_trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool if_exist = arg.if_exist_;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const ObString &database_name = arg.database_name_;
|
|
UInt64 &affected_row = res.affected_row_;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
uint64_t table_count = 0;
|
|
ObArray<uint64_t> table_ids;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObDatabaseSchema *db_schema = NULL;
|
|
ObTabletAutoincCacheCleaner tablet_autoinc_cleaner(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_INVALID_ID == tenant_id || database_name.length() <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(tenant_id), K(database_name), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id, database_name, db_schema))) {
|
|
LOG_WARN("get_database schema failed", K(ret));
|
|
} else if (OB_ISNULL(db_schema)) {
|
|
if (if_exist) {
|
|
LOG_USER_NOTE(OB_ERR_DB_DROP_EXISTS, database_name.length(), database_name.ptr());
|
|
LOG_INFO("database not exist, no need to delete it", K(tenant_id), K(database_name));
|
|
} else {
|
|
ret = OB_ERR_DB_DROP_EXISTS;
|
|
LOG_USER_ERROR(OB_ERR_DB_DROP_EXISTS, database_name.length(), database_name.ptr());
|
|
LOG_WARN("database not exist, can't delete it",K(tenant_id), K(database_name), K(ret));
|
|
}
|
|
} else if (db_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("can not drop database in recyclebin", K(ret), K(*db_schema));
|
|
} else {
|
|
uint64_t database_id = db_schema->get_database_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(schema_guard.get_table_ids_in_database(tenant_id,
|
|
database_id,
|
|
table_ids))) {
|
|
LOG_WARN("fail to get table ids in database", K(tenant_id),
|
|
K(database_id), K(ret));
|
|
} else if (FALSE_IT(table_count = table_ids.count())) {
|
|
//nothing to do
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(ora_user_trans)
|
|
&& OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLSQLTransaction &actual_trans = OB_ISNULL(ora_user_trans) ? trans : *ora_user_trans;
|
|
// lock table when drop data table
|
|
if (OB_FAIL(lock_tables_of_database(*db_schema, actual_trans))) {
|
|
LOG_WARN("lock tables of database", K(ret));
|
|
} else if (!arg.to_recyclebin_ && OB_FAIL(lock_tables_in_recyclebin(*db_schema, actual_trans))) {
|
|
LOG_WARN("failed to lock tables in recyclebin", K(ret));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && arg.to_recyclebin_ && !is_inner_db(db_schema->get_database_id())) {
|
|
if (OB_FAIL(ddl_operator.drop_database_to_recyclebin(*db_schema,
|
|
actual_trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("drop database to recyclebin failed", K(arg), K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ret)) {
|
|
// FAIL
|
|
} else if (OB_FAIL(ddl_operator.drop_database(*db_schema,
|
|
actual_trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("ddl_operator drop_database failed", K(tenant_id), KT(database_id), K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (arg.task_id_ > 0 && OB_FAIL(ObDDLRetryTask::update_task_status_wait_child_task_finish(
|
|
actual_trans, tenant_id, arg.task_id_))) {
|
|
LOG_WARN("update ddl task status to success failed", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(tablet_autoinc_cleaner.add_database(*db_schema))) {
|
|
LOG_WARN("failed to add database to tablet autoinc cleaner", K(tmp_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_SUCCESS == 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));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(tablet_autoinc_cleaner.commit())) {
|
|
LOG_WARN("failed to commit tablet autoinc cleaner", K(tmp_ret));
|
|
}
|
|
}
|
|
|
|
LOG_INFO("finish drop database", K(tenant_id), K(arg), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_tablegroup(const bool if_not_exist,
|
|
share::schema::ObTablegroupSchema &tablegroup_schema,
|
|
const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
RS_TRACE(create_tablegroup_begin);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool is_exist = false;
|
|
uint64_t tablegroup_id = OB_INVALID_ID;
|
|
const uint64_t tenant_id = tablegroup_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(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 {
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (compat_version < DATA_VERSION_4_2_0_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can not create tablegroup while observer is upgrading", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (is_exist) {
|
|
if (if_not_exist) {
|
|
ret = OB_SUCCESS;
|
|
tablegroup_schema.set_tablegroup_id(tablegroup_id);
|
|
LOG_USER_NOTE(OB_TABLEGROUP_EXIST);
|
|
LOG_INFO("tablegroup already exists, not need to create", "tenant_id",
|
|
tablegroup_schema.get_tenant_id(), "tablegroup_name",
|
|
tablegroup_schema.get_tablegroup_name_str());
|
|
} else {
|
|
ret = OB_TABLEGROUP_EXIST;
|
|
LOG_WARN("tablegroup already exists", "tenant_id", tablegroup_schema.get_tenant_id(),
|
|
"tablegroup_name", tablegroup_schema.get_tablegroup_name_str(), K(ret));
|
|
}
|
|
} else {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
// do nothing
|
|
}
|
|
RS_TRACE(write_tablegroup_schema_begin);
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.create_tablegroup(tablegroup_schema, trans, ddl_stmt_str))) {
|
|
LOG_WARN("ddl operator failed to create tablegroup", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
RS_TRACE(publish_tablegroup_schema_begin);
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("fail to public schema", KR(ret), K(tenant_id));
|
|
} else {
|
|
RS_TRACE(publish_tablegroup_schema_end);
|
|
}
|
|
}
|
|
FORCE_PRINT_TRACE(THE_RS_TRACE, "[create tablegroup]");
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_tablegroup(const ObDropTablegroupArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const bool if_exist = arg.if_exist_;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const ObString &tablegroup_name = arg.tablegroup_name_;
|
|
uint64_t tablegroup_id = OB_INVALID_ID;
|
|
bool is_exist = false;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id || tablegroup_name.length() <= 0) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(tenant_id), K(tablegroup_name), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const ObTablegroupSchema *tablegroup_schema = NULL;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_service_->check_tablegroup_exist(tenant_id, tablegroup_name,
|
|
tablegroup_id, is_exist))) {
|
|
LOG_WARN("check_tablegroup_exist failed", K(tenant_id), K(tablegroup_name), K(ret));
|
|
} else if (!is_exist) {
|
|
if (if_exist) {
|
|
LOG_USER_NOTE(OB_TABLEGROUP_NOT_EXIST);
|
|
LOG_INFO("tablegroup not exist, no need to delete it", K(tenant_id), K(tablegroup_name));
|
|
} else {
|
|
ret = OB_TABLEGROUP_NOT_EXIST;
|
|
LOG_WARN("tablegroup not exist, can't delete it", K(tenant_id), K(tablegroup_name), K(ret));
|
|
}
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_tablegroup_schema(tenant_id, tablegroup_id, tablegroup_schema))) {
|
|
LOG_WARN("fail to get tablegroup schema", K(ret), K(tenant_id), KT(tablegroup_id));
|
|
} else if (OB_ISNULL(tablegroup_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tablegroup schema is null", K(ret), KT(tablegroup_id));
|
|
} else {
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ddl_operator.drop_tablegroup(*tablegroup_schema,
|
|
trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("ddl_operator drop_tablegroup failed", K(tenant_id), KT(tablegroup_id), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
|
|
LOG_INFO("finish drop tablegroup", K(tenant_id), KT(tablegroup_id), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
// alter tablegroup usage:
|
|
// 1. Add the specified table_list to the tablegroup, it need to check whether the newly added table
|
|
// meets the constraints of the tablegroup;
|
|
// 2. Modify the attributes of the tablegroup: including partition;
|
|
// it can be regarded as an atomic operation to modify the attributes of all tables in the tablegroup;
|
|
// when all the table information is modified successfully, the user will return to success,
|
|
// otherwise the rollback operation will fail;
|
|
int ObDDLService::alter_tablegroup(const ObAlterTablegroupArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const ObString &tablegroup_name = arg.tablegroup_name_;
|
|
uint64_t tablegroup_id = OB_INVALID_ID;
|
|
const ObTablegroupSchema *tablegroup_schema = NULL;
|
|
// after publish_schema, timeout will be chenged to max value
|
|
const int64_t trans_timeout = THIS_WORKER.get_timeout_remain();
|
|
const int64_t abs_timeout = trans_timeout + ObTimeUtility::current_time();
|
|
ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (arg.alter_option_bitset_.has_member(obrpc::ObAlterTablegroupArg::DROP_PARTITION)
|
|
|| arg.alter_option_bitset_.has_member(obrpc::ObAlterTablegroupArg::ADD_PARTITION)) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("modify partition for tablegroup in 4.0 not allowed", K(ret), K(tablegroup_id));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "modify partition for tablegroup in 4.0");
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(tenant_id), K(ret));
|
|
} else if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service should not be null", K(ret));
|
|
} else {
|
|
uint64_t compat_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("get min data_version failed", K(ret), K(tenant_id));
|
|
} else if (compat_version < DATA_VERSION_4_2_0_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("can not alter tablegroup while observer is upgrading", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
if (FAILEDx(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_tablegroup_id(tenant_id, tablegroup_name, tablegroup_id))) {
|
|
LOG_WARN("fail to get tablegroup id", K(ret), K(tenant_id), K(arg));
|
|
} else if (OB_INVALID_ID == tablegroup_id) {
|
|
ret = OB_TABLEGROUP_NOT_EXIST;
|
|
LOG_WARN("tablegroup not exist", K(ret), K(ret), K(tablegroup_name));
|
|
} else if (OB_FAIL(schema_guard.get_tablegroup_schema(tenant_id, tablegroup_id, tablegroup_schema))) {
|
|
LOG_WARN("fail to get tablegroup schema", K(ret), K(tenant_id), K(tablegroup_id));
|
|
} else {
|
|
ObSchemaGetterGuard new_schema_guard;
|
|
const ObTablegroupSchema *orig_tablegroup = NULL;
|
|
ObTablegroupSchema new_tablegroup;
|
|
// Since tablegroup involves the update of all the following tables, it takes a long time
|
|
// Set the transaction timeout time to the remaining time of the query
|
|
// abs_timeout = timeout + current_time
|
|
ObTimeoutCtx timeout_ctx;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, new_schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(timeout_ctx.set_trx_timeout_us(trans_timeout))) {
|
|
LOG_WARN("failed to set trans timeout", K(ret), K(trans_timeout));
|
|
} else if (OB_FAIL(timeout_ctx.set_abs_timeout(abs_timeout))) {
|
|
LOG_WARN("failed to set stmt timeout", K(ret), K(abs_timeout));
|
|
} else if (OB_FAIL(new_schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(new_schema_guard.get_tablegroup_schema(tenant_id, tablegroup_id, orig_tablegroup))) {
|
|
LOG_WARN("fail to get tablegroup schema", K(ret), K(tenant_id), K(tablegroup_id));
|
|
} else if (OB_ISNULL(orig_tablegroup)) {
|
|
ret = OB_TABLEGROUP_NOT_EXIST;
|
|
LOG_WARN("get invalid tablegroup schema", K(ret), K(orig_tablegroup));
|
|
} else if (OB_FAIL(helper.add_tables_to_tablegroup(trans, new_schema_guard, *orig_tablegroup, arg))) {
|
|
LOG_WARN("fail to add tables to tablegroup", K(ret));
|
|
} else if (OB_FAIL(helper.modify_sharding_type(arg, *orig_tablegroup, trans, new_schema_guard))) {
|
|
LOG_WARN("fail to modify partition option", K(ret), K(*orig_tablegroup), K(arg));
|
|
}
|
|
// however, end the trans
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("fail to publish schema", KR(ret), K(tenant_id));
|
|
}
|
|
LOG_INFO("finish alter tablegroup", K(tenant_id), KT(tablegroup_id), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::refresh_schema(uint64_t tenant_id, int64_t *publish_schema_version /*NULL*/)
|
|
{
|
|
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", KR(ret), K(tenant_id));
|
|
}
|
|
while (!stopped_) {
|
|
// reset ctx to retry to die
|
|
common::ObTimeoutCtx ctx;
|
|
ctx.reset_timeout_us();
|
|
ret = schema_service_->refresh_and_add_schema(tenant_ids);
|
|
|
|
if (OB_SUCC(ret)) {
|
|
break;
|
|
} else {
|
|
int tmp_ret = OB_SUCCESS;
|
|
bool is_dropped = false;
|
|
if (OB_TMP_FAIL(check_tenant_has_been_dropped_(tenant_id, is_dropped))) {
|
|
LOG_WARN("fail to check if tenant has been dropped", KR(ret), K(tmp_ret), K(tenant_id));
|
|
} else if (is_dropped) {
|
|
LOG_WARN("tenant has been dropped, just exit", KR(ret), K(tenant_id));
|
|
break;
|
|
}
|
|
++refresh_count;
|
|
LOG_WARN("refresh schema failed", KR(ret), K(tenant_id), K(refresh_count),
|
|
"refresh_schema_interval", static_cast<int64_t>(REFRESH_SCHEMA_INTERVAL_US));
|
|
if (refresh_count > 2 && REACH_TIME_INTERVAL_NO_INSTANT(10 * 60 * 1000 * 1000L)) { // 10 min
|
|
LOG_DBA_ERROR(OB_ERR_REFRESH_SCHEMA_TOO_LONG,
|
|
"msg", "refresh schema failed", KR(ret), K(refresh_count));
|
|
}
|
|
ob_usleep(REFRESH_SCHEMA_INTERVAL_US);
|
|
}
|
|
} // end while
|
|
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", KR(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", KR(ret), K(schema_info));
|
|
} else if (OB_NOT_NULL(publish_schema_version)) {
|
|
*publish_schema_version = schema_version;
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret) && stopped_) {
|
|
ret = OB_CANCELED;
|
|
LOG_WARN("rs is stopped", KR(ret), K(tenant_id));
|
|
}
|
|
THIS_WORKER.set_timeout_ts(original_timeout_us);
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_tenant_has_been_dropped_(
|
|
const uint64_t tenant_id,
|
|
bool &is_dropped)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_dropped = false;
|
|
ObSchemaGetterGuard guard;
|
|
const ObSimpleTenantSchema *tenant = NULL;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("ddl_service is not init", KR(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(OB_SYS_TENANT_ID, guard))) {
|
|
LOG_WARN("fail to get schema guard", KR(ret));
|
|
} else if (OB_FAIL(guard.check_if_tenant_has_been_dropped(tenant_id, is_dropped))) {
|
|
LOG_WARN("fail to check if tenant has been dropped", KR(ret), K(tenant_id));
|
|
} else if (is_dropped) {
|
|
LOG_WARN("tenant has been dropped, just exit", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(guard.get_tenant_info(tenant_id, tenant))) {
|
|
LOG_WARN("fail to get tenant info", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(tenant)) {
|
|
// ignore ret
|
|
LOG_WARN("tenant not exist, maybe schema is fall behind", KR(ret), K(tenant_id));
|
|
} else if (tenant->is_dropping()) {
|
|
ObLSStatusOperator ls_status;
|
|
ObLSStatusInfo sys_ls_info;
|
|
if (OB_FAIL(ls_status.get_ls_status_info(tenant_id, SYS_LS, sys_ls_info, *sql_proxy_))) {
|
|
if (OB_ENTRY_NOT_EXIST == ret) {
|
|
LOG_WARN("sys ls not exist, consider that tenant has been dropped", KR(ret), K(tenant_id));
|
|
ret = OB_SUCCESS;
|
|
is_dropped = true;
|
|
} else {
|
|
LOG_WARN("fail to get ls status info", KR(ret), K(tenant_id));
|
|
}
|
|
} else if (sys_ls_info.ls_is_tenant_dropping()
|
|
|| sys_ls_info.ls_is_wait_offline()) {
|
|
is_dropped = true;
|
|
LOG_INFO("sys ls is not readable, consider that tenant has been dropped", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::notify_refresh_schema(const ObAddrIArray &addrs)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObZone zone;
|
|
ObArray<ObAddr> server_list;
|
|
ObSwitchSchemaProxy proxy(*rpc_proxy_, &ObSrvRpcProxy::switch_schema);
|
|
ObSwitchSchemaArg arg;
|
|
ObRefreshSchemaInfo local_schema_info;
|
|
ObRefreshSchemaInfo &schema_info = arg.schema_info_;
|
|
int64_t schema_version = OB_INVALID_VERSION;
|
|
ObAddr rs_addr;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_ISNULL(GCTX.rs_mgr_) || OB_ISNULL(schema_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("GCTX.rs_mgr_ or schema_service_ is null", KR(ret), KP(GCTX.rs_mgr_), KP(schema_service_));
|
|
} else if (OB_FAIL(GCTX.rs_mgr_->get_master_root_server(rs_addr))) {
|
|
LOG_WARN("fail to get master root servcer", KR(ret));
|
|
} else if (OB_UNLIKELY(!rs_addr.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("rs_addr is invalid", KR(ret), K(rs_addr));
|
|
} else if (OB_FAIL(SVR_TRACER.get_alive_and_not_stopped_servers(zone, server_list))) {
|
|
LOG_WARN("get alive server failed", KR(ret), K(zone));
|
|
} else if (OB_FAIL(schema_service_->get_refresh_schema_info(local_schema_info))) {
|
|
LOG_WARN("fail to get schema info", KR(ret));
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_version(OB_SYS_TENANT_ID, schema_version))) {
|
|
LOG_WARN("fail to get sys schema version", KR(ret));
|
|
} else {
|
|
// For compatibility reasons, local_schema_info is invalid before the schema is split,
|
|
// and sequence_id is not broadcast
|
|
if (local_schema_info.is_valid()) {
|
|
if (OB_FAIL(schema_info.assign(local_schema_info))) {
|
|
LOG_WARN("fail to assign schema_info", KR(ret), K(local_schema_info));
|
|
}
|
|
} else {
|
|
schema_info.set_schema_version(schema_version);
|
|
}
|
|
|
|
bool is_async = false;
|
|
if (OB_INVALID_TENANT_ID != schema_info.get_tenant_id()) {
|
|
omt::ObTenantConfigGuard tenant_config(OTC_MGR.get_tenant_config_with_lock(
|
|
schema_info.get_tenant_id()));
|
|
if (tenant_config.is_valid()) {
|
|
is_async = (0 == tenant_config->_publish_schema_mode.case_compare(PUBLISH_SCHEMA_MODE_ASYNC));
|
|
}
|
|
}
|
|
LOG_INFO("try to notify refresh schema", K(is_async), K(schema_version), K(local_schema_info), K(schema_info));
|
|
const int64_t rpc_timeout = GCONF.rpc_timeout;
|
|
int64_t timeout = 0;
|
|
bool report_failure = false;
|
|
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 (rs_addr == *s) {
|
|
continue;
|
|
} else {
|
|
arg.force_refresh_ = has_exist_in_array(addrs, *s);
|
|
if (arg.force_refresh_) {
|
|
// refresh schema sync and report error
|
|
if (!report_failure) {
|
|
report_failure = true;
|
|
}
|
|
if (OB_FAIL(ObShareUtil::get_ctx_timeout(rpc_timeout, timeout))) {
|
|
LOG_WARN("fail to get timeout", KR(ret));
|
|
}
|
|
arg.is_async_ = false;
|
|
} else {
|
|
// refresh schema async and ignore error
|
|
timeout = std::min(THIS_WORKER.get_timeout_remain(), rpc_timeout);
|
|
arg.is_async_ = is_async;
|
|
}
|
|
// overwrite ret
|
|
if (FAILEDx(proxy.call(*s, timeout, arg))) {
|
|
LOG_WARN("send switch schema rpc failed", KR(ret),
|
|
K(timeout), K(schema_version), K(schema_info), K(arg), "server", *s);
|
|
if (!arg.force_refresh_) { // ignore servers that are not in addrs
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
ObArray<int> return_code_array;
|
|
int tmp_ret = OB_SUCCESS; // always wait all
|
|
if (OB_TMP_FAIL(proxy.wait_all(return_code_array))) {
|
|
LOG_WARN("wait result failed", KR(tmp_ret), KR(ret));
|
|
ret = OB_SUCC(ret) ? tmp_ret : ret;
|
|
} else if (OB_FAIL(ret)) {
|
|
} else if (report_failure) {
|
|
// check result only when report_failure = true
|
|
if (OB_FAIL(proxy.check_return_cnt(return_code_array.count()))) {
|
|
LOG_WARN("fail to check return count",
|
|
KR(ret), "return_cnt", return_code_array.count());
|
|
} else {
|
|
ARRAY_FOREACH_N(return_code_array, i, cnt) {
|
|
int res_ret = return_code_array.at(i);
|
|
const ObAddr &addr = proxy.get_dests().at(i);
|
|
const obrpc::ObSwitchSchemaArg &tmp_arg = proxy.get_args().at(i);
|
|
if (OB_SUCCESS != res_ret && tmp_arg.force_refresh_) {
|
|
ret = res_ret;
|
|
LOG_WARN("switch schema failed", KR(ret), K(addr));
|
|
}
|
|
} // end for
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("notify switch schema finished", KR(ret),
|
|
K(schema_version), K(schema_info), K(arg), K(addrs), K(server_list));
|
|
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::publish_schema_and_get_schema_version(uint64_t tenant_id,
|
|
const ObAddrIArray &addrs,
|
|
int64_t *schema_version /*NULL*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTimeoutCtx ctx;
|
|
const int64_t TIMEOUT = 10L * 1000L * 1000L; // 10s
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(refresh_schema(tenant_id, schema_version))) {
|
|
LOG_WARN("refresh schema failed", KR(ret));
|
|
} else if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, TIMEOUT))) {// 10s for notify_refresh_schema
|
|
LOG_WARN("fail to set default timeout ctx", KR(ret));
|
|
} else if (OB_FAIL(notify_refresh_schema(addrs))) {
|
|
LOG_WARN("notify refresh schema failed", KR(ret));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_create_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", KR(ret));
|
|
} else if (tenant_schema.get_tenant_name_str().length() > OB_MAX_TENANT_NAME_LENGTH) {
|
|
ret = OB_INVALID_TENANT_NAME;
|
|
LOG_USER_ERROR(OB_INVALID_TENANT_NAME,
|
|
to_cstring(tenant_schema.get_tenant_name_str()), OB_MAX_TENANT_NAME_LENGTH);
|
|
LOG_WARN("tenant name can't over max_tenant_name_length", KR(ret), K(OB_MAX_TENANT_NAME_LENGTH));
|
|
} else if (OB_FAIL(check_create_tenant_locality(pool_list, tenant_schema, schema_guard))) {
|
|
LOG_WARN("fail to check create tenant locality", KR(ret), K(pool_list), K(tenant_schema));
|
|
} else if (OB_FAIL(check_create_tenant_replica_options(tenant_schema, schema_guard))) {
|
|
LOG_WARN("check replica options failed", KR(ret), K(tenant_schema));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_create_tenant_locality(
|
|
const ObIArray<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
|
|
lib::ob_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_locality_compatible_(tenant_schema, true /*for_create_tenant*/))) {
|
|
LOG_WARN("fail to check locality with data version", KR(ret), K(tenant_schema));
|
|
} 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::handle_security_audit_in_trans(const ObSAuditSchema &audit_schema,
|
|
const ObSAuditModifyType modify_type, const bool need_update, const ObString &ddl_str,
|
|
common::ObSqlString &public_sql_string,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const uint64_t tenant_id = audit_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.handle_audit_metainfo(audit_schema,
|
|
modify_type,
|
|
need_update,
|
|
&ddl_str,
|
|
trans,
|
|
public_sql_string))) {
|
|
LOG_WARN("Failed to handle audit meta info", K(ret));
|
|
} else {
|
|
LOG_INFO("Succeed in inserting audit in audit table", K(audit_schema), K(ddl_str));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int tmp_ret = ret;
|
|
if (OB_UNLIKELY(OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret))))) {
|
|
LOG_WARN("Failed to commit/rollback trans", K(tmp_ret), K(ret));
|
|
ret = (OB_LIKELY(OB_SUCCESS == tmp_ret) ? ret : tmp_ret);
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::handle_security_audit(const obrpc::ObSecurityAuditArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_UNLIKELY(!arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("arg is invalid", K(arg), K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObSAuditOperTimingType in_success = AUDIT_OT_INVALID;//no operate
|
|
ObSAuditOperTimingType in_failure = AUDIT_OT_INVALID;//no operate
|
|
if (AUDIT_MT_ADD == arg.modify_type_) {
|
|
if (AUDIT_WHEN_NOT_SET == arg.when_type_) {
|
|
in_success = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION);
|
|
in_failure = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION);
|
|
} else if (AUDIT_WHEN_SUCCESS == arg.when_type_) {
|
|
in_success = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION);
|
|
} else {
|
|
in_failure = (AUDIT_BY_ACCESS == arg.by_type_ ? AUDIT_OT_ACCESS : AUDIT_OT_SESSION);
|
|
}
|
|
} else {
|
|
if (AUDIT_WHEN_NOT_SET == arg.when_type_) {
|
|
in_success = AUDIT_OT_NOT_SET;
|
|
in_failure = AUDIT_OT_NOT_SET;
|
|
} else if (AUDIT_WHEN_SUCCESS == arg.when_type_) {
|
|
in_success = AUDIT_OT_NOT_SET;
|
|
} else {
|
|
in_failure = AUDIT_OT_NOT_SET;
|
|
}
|
|
}
|
|
|
|
ObSAuditSchema audit_schema;
|
|
audit_schema.set_tenant_id(arg.tenant_id_);
|
|
audit_schema.set_audit_type(arg.audit_type_);
|
|
audit_schema.set_in_success(in_success);
|
|
audit_schema.set_in_failure(in_failure);
|
|
|
|
if (AUDIT_STMT_ALL_USER == arg.audit_type_ || AUDIT_STMT == arg.audit_type_) {//stmt
|
|
if (OB_FAIL(handle_security_audit_for_stmt(arg, audit_schema))) {
|
|
LOG_WARN("failed to handle_security_audit_for_object", K(arg), K(ret));
|
|
}
|
|
} else if (AUDIT_OBJ_DEFAULT <= arg.audit_type_ && arg.audit_type_ <= AUDIT_PROCEDURE) {//object
|
|
if (OB_FAIL(handle_security_audit_for_object(arg, audit_schema))) {
|
|
LOG_WARN("failed to handle_security_audit_for_object", K(arg), K(ret));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("audit_type is unexpected", K(arg.audit_type_), K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_security_audit_for_stmt(const obrpc::ObSecurityAuditArg &arg,
|
|
ObSAuditSchema &audit_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSqlString ddl_stmt_str;
|
|
ObSqlString ddl_operation_sql_string;
|
|
share::schema::ObSchemaService *schema_service_impl = NULL;
|
|
if (OB_UNLIKELY(AUDIT_STMT_ALL_USER != arg.audit_type_ && AUDIT_STMT != arg.audit_type_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected audit_type", K(arg.audit_type_), K(ret));
|
|
} else if (OB_ISNULL(schema_service_)
|
|
|| OB_ISNULL(schema_service_impl = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service_impl must not be null", K(ret));
|
|
} else {
|
|
uint64_t mock_user_id[1] = { OB_AUDIT_MOCK_USER_ID };
|
|
ObArrayHelper<uint64_t> mock_user_ids(1, mock_user_id, 1);
|
|
const common::ObIArray<uint64_t> *stmt_user_ids = &arg.stmt_user_ids_;
|
|
if (AUDIT_STMT_ALL_USER == arg.audit_type_) {
|
|
stmt_user_ids = &mock_user_ids;
|
|
}
|
|
const ObSAuditOperTimingType target_in_success = audit_schema.get_in_success();
|
|
const ObSAuditOperTimingType target_in_failure = audit_schema.get_in_failure();
|
|
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(arg.tenant_id_,
|
|
schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(arg));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < stmt_user_ids->count(); ++i) {
|
|
audit_schema.set_owner_id(stmt_user_ids->at(i));
|
|
|
|
ObString username;
|
|
const ObUserInfo *user_info = NULL;
|
|
if (AUDIT_STMT == arg.audit_type_) {
|
|
if (OB_FAIL(schema_guard.get_user_info(arg.tenant_id_,stmt_user_ids->at(i),user_info))) {
|
|
LOG_WARN("fail to get get_user_info", K(ret), K(arg), "user_id", stmt_user_ids->at(i));
|
|
} else if (OB_ISNULL(user_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("user_info is null", K(ret), K(arg), "user_id", stmt_user_ids->at(i));
|
|
} else {
|
|
username = user_info->get_user_name_str();
|
|
}
|
|
}
|
|
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < arg.operation_types_.count(); ++j) {
|
|
audit_schema.set_operation_type(arg.operation_types_.at(j));
|
|
audit_schema.set_audit_id(OB_INVALID_ID);
|
|
audit_schema.set_in_success(target_in_success);
|
|
audit_schema.set_in_failure(target_in_failure);
|
|
|
|
bool need_update = false;
|
|
bool need_continue = false;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(
|
|
arg.tenant_id_, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table",
|
|
KR(ret), K(arg));
|
|
} else if (OB_FAIL(check_and_update_audit_schema(schema_guard,
|
|
schema_service_impl,
|
|
audit_schema,
|
|
arg.modify_type_,
|
|
need_update,
|
|
need_continue))) {
|
|
LOG_WARN("failed to check_and_update_audit_schema", K(arg), K(ret));
|
|
} else if (need_continue) {
|
|
continue;
|
|
} else {
|
|
LOG_DEBUG("before gen_audit_stmt_sql", K(arg), K(audit_schema));
|
|
|
|
ddl_stmt_str.reuse();
|
|
ddl_operation_sql_string.reuse();
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_audit_stmt_sql(username,
|
|
arg.modify_type_,
|
|
audit_schema,
|
|
arg.by_type_,
|
|
arg.when_type_,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail to gen_audit_stmt_sql", K(audit_schema), K(ret));
|
|
} else if (OB_FAIL(handle_security_audit_in_trans(audit_schema,
|
|
arg.modify_type_,
|
|
need_update,
|
|
ddl_stmt_str.string(),
|
|
ddl_operation_sql_string,
|
|
schema_guard))) {
|
|
LOG_WARN("failed to handle_security_audit_in_trans", K(ret));
|
|
} else if (OB_FAIL(publish_schema(audit_schema.get_tenant_id()))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}//end of for j
|
|
}//end of for i
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_and_get_object_name(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObSAuditType audit_type,
|
|
const uint64_t tenant_id,
|
|
const uint64_t obj_object_id,
|
|
common::ObString &schema_name,
|
|
common::ObString &object_name)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
switch (audit_type) {
|
|
case AUDIT_OBJ_DEFAULT: {
|
|
//do nothing
|
|
break;
|
|
}
|
|
case AUDIT_TABLE: {
|
|
const ObSimpleTableSchemaV2 *table_schema = NULL;
|
|
const ObSimpleDatabaseSchema *database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_simple_table_schema(
|
|
tenant_id, obj_object_id, table_schema))) {
|
|
LOG_WARN("failed to get_table_schema", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("table_schema is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id,
|
|
table_schema->get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else {
|
|
schema_name = database_schema->get_database_name_str();
|
|
object_name = table_schema->get_table_name_str();
|
|
}
|
|
break;
|
|
}
|
|
case AUDIT_SEQUENCE: {
|
|
const ObSequenceSchema *sequence_schema = NULL;
|
|
const ObSimpleDatabaseSchema *database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_sequence_schema(tenant_id,
|
|
obj_object_id,
|
|
sequence_schema))) {
|
|
LOG_WARN("failed to get_sequence_schema", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(sequence_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sequence_schema is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id,
|
|
sequence_schema->get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else {
|
|
schema_name = database_schema->get_database_name_str();
|
|
object_name = sequence_schema->get_sequence_name();
|
|
}
|
|
break;
|
|
}
|
|
case AUDIT_PACKAGE: {
|
|
const ObPackageInfo *package_info = NULL;
|
|
const ObSimpleDatabaseSchema *database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_package_info(tenant_id, obj_object_id, package_info))) {
|
|
LOG_WARN("failed to get_package_info", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(package_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("package_info is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id,
|
|
package_info->get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else {
|
|
schema_name = database_schema->get_database_name_str();
|
|
object_name = package_info->get_package_name();
|
|
}
|
|
break;
|
|
}
|
|
case AUDIT_PROCEDURE: {
|
|
const ObRoutineInfo *routine_info = NULL;
|
|
const ObSimpleDatabaseSchema *database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_routine_info(tenant_id, obj_object_id, routine_info))) {
|
|
LOG_WARN("failed to get_routine_schema", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(routine_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("routine_info is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_database_schema(tenant_id,
|
|
routine_info->get_database_id(), database_schema))) {
|
|
LOG_WARN("failed to get_database_schema", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else if (OB_ISNULL(database_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database_schema is null", K(tenant_id), K(obj_object_id), K(ret));
|
|
} else {
|
|
schema_name = database_schema->get_database_name_str();
|
|
object_name = routine_info->get_routine_name();
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected type", K(audit_type), K(obj_object_id), K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_security_audit_for_object(const obrpc::ObSecurityAuditArg &arg,
|
|
ObSAuditSchema &audit_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
share::schema::ObSchemaService *schema_service_impl = schema_service_->get_schema_service();
|
|
if (OB_UNLIKELY(arg.audit_type_ < AUDIT_OBJ_DEFAULT || arg.audit_type_ > AUDIT_PROCEDURE)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected audit_type", K(arg.audit_type_), K(ret));
|
|
} else if (OB_ISNULL(schema_service_impl)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service_impl must not be null", K(ret));
|
|
} else {
|
|
ObSqlString ddl_stmt_str;
|
|
ObSqlString ddl_operation_sql_string;
|
|
ObString schema_name;
|
|
ObString object_name;
|
|
const ObSAuditOperTimingType target_in_success = audit_schema.get_in_success();
|
|
const ObSAuditOperTimingType target_in_failure = audit_schema.get_in_failure();
|
|
|
|
const uint64_t object_id = (AUDIT_OBJ_DEFAULT == arg.audit_type_
|
|
? OB_AUDIT_MOCK_USER_ID
|
|
: arg.obj_object_id_);
|
|
audit_schema.set_owner_id(object_id);
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(arg.tenant_id_,
|
|
schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(arg.tenant_id_));
|
|
} else if (OB_FAIL(check_and_get_object_name(schema_guard, arg.audit_type_, arg.tenant_id_,
|
|
arg.obj_object_id_, schema_name, object_name))) {
|
|
LOG_WARN("fail to check_and_get_object_name", K(arg), K(ret));
|
|
}
|
|
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < arg.operation_types_.count(); ++j) {
|
|
audit_schema.set_operation_type(arg.operation_types_.at(j));
|
|
audit_schema.set_audit_id(OB_INVALID_ID);
|
|
audit_schema.set_in_success(target_in_success);
|
|
audit_schema.set_in_failure(target_in_failure);
|
|
|
|
bool need_update = false;
|
|
bool need_continue = false;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(
|
|
arg.tenant_id_, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table",
|
|
KR(ret), K(arg.tenant_id_));
|
|
} else if (OB_FAIL(check_and_update_audit_schema(schema_guard,
|
|
schema_service_impl,
|
|
audit_schema,
|
|
arg.modify_type_,
|
|
need_update,
|
|
need_continue))) {
|
|
LOG_WARN("failed to check_and_update_audit_schema", K(arg), K(ret));
|
|
} else if (need_continue) {
|
|
continue;
|
|
} else {
|
|
ddl_stmt_str.reuse();
|
|
ddl_operation_sql_string.reuse();
|
|
LOG_DEBUG("before gen_audit_object_sql", K(arg), K(audit_schema));
|
|
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_audit_object_sql(schema_name,
|
|
object_name,
|
|
arg.modify_type_,
|
|
audit_schema,
|
|
arg.by_type_,
|
|
arg.when_type_,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail to gen_audit_stmt_sql", K(schema_name), K(object_name), K(ret));
|
|
} else if (OB_FAIL(handle_security_audit_in_trans(audit_schema,
|
|
arg.modify_type_,
|
|
need_update,
|
|
ddl_stmt_str.string(),
|
|
ddl_operation_sql_string,
|
|
schema_guard))) {
|
|
LOG_WARN("failed to handle_security_audit_in_trans", KR(ret), K(arg));
|
|
} else if (OB_FAIL(publish_schema(audit_schema.get_tenant_id()))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::check_and_update_audit_schema(ObSchemaGetterGuard &schema_guard,
|
|
share::schema::ObSchemaService *ss_impl,
|
|
ObSAuditSchema &audit_schema, const ObSAuditModifyType modify_type,
|
|
bool &need_update, bool &need_continue)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObSAuditSchema *tmp_audit_schema = NULL;
|
|
need_update = false;
|
|
need_continue = false;
|
|
if (OB_FAIL(schema_guard.get_audit_schema_in_tenant(audit_schema.get_tenant_id(),
|
|
audit_schema.get_audit_type(),
|
|
audit_schema.get_owner_id(),
|
|
audit_schema.get_operation_type(),
|
|
tmp_audit_schema))) {
|
|
LOG_WARN("failed to get_audit_schema", K(audit_schema), K(ret));
|
|
} else if (NULL == tmp_audit_schema) {
|
|
if (AUDIT_MT_ADD == modify_type) {
|
|
//add new audit
|
|
uint64_t new_audit_id = common::OB_INVALID_ID;
|
|
if (OB_FAIL(ss_impl->fetch_new_audit_id(audit_schema.get_tenant_id(), new_audit_id))) {
|
|
LOG_WARN("Failed to fetch new_audit_id", K(ret));
|
|
} else {
|
|
audit_schema.set_audit_id(new_audit_id);
|
|
if (AUDIT_OT_INVALID == audit_schema.get_in_success()) {
|
|
audit_schema.set_in_success(AUDIT_OT_NOT_SET);
|
|
}
|
|
if (AUDIT_OT_INVALID == audit_schema.get_in_failure()) {
|
|
audit_schema.set_in_failure(AUDIT_OT_NOT_SET);
|
|
}
|
|
}
|
|
} else {
|
|
//mark do nothing about schema_table
|
|
need_continue = true;
|
|
}
|
|
} else {
|
|
audit_schema.set_audit_id(tmp_audit_schema->get_audit_id());
|
|
if (AUDIT_MT_ADD == modify_type) {
|
|
need_update = true;
|
|
//use orig one
|
|
if (AUDIT_OT_INVALID == audit_schema.get_in_success()) {
|
|
audit_schema.set_in_success(tmp_audit_schema->get_in_success());
|
|
}
|
|
if (AUDIT_OT_INVALID == audit_schema.get_in_failure()) {
|
|
audit_schema.set_in_failure(tmp_audit_schema->get_in_failure());
|
|
}
|
|
} else {
|
|
const int64_t MAX_ATTR_COUNT = 2;//in_success, in_failure
|
|
const bool effect_in_succ = (tmp_audit_schema->get_in_success() != AUDIT_OT_NOT_SET);
|
|
const bool effect_in_fail = (tmp_audit_schema->get_in_failure() != AUDIT_OT_NOT_SET);
|
|
const bool del_in_succ = (audit_schema.get_in_success() != AUDIT_OT_INVALID);
|
|
const bool del_in_fail = (audit_schema.get_in_failure() != AUDIT_OT_INVALID);
|
|
if ((!effect_in_succ || del_in_succ) + (!effect_in_fail || del_in_fail) != MAX_ATTR_COUNT) {
|
|
need_update = true;
|
|
//use orig one
|
|
if (effect_in_succ && !del_in_succ) {
|
|
audit_schema.set_in_success(tmp_audit_schema->get_in_success());
|
|
}
|
|
if (effect_in_fail && !del_in_fail) {
|
|
audit_schema.set_in_failure(tmp_audit_schema->get_in_failure());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
LOG_DEBUG("finish check_and_update_audit_schema", KPC(tmp_audit_schema), K(audit_schema),
|
|
K(need_update), K(need_continue));
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::create_user(ObCreateUserArg &arg,
|
|
ObIArray<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));
|
|
} else if (!is_oracle_mode && arg.is_create_role_) {
|
|
if (OB_FAIL(ObSQLUtils::compatibility_check_for_mysql_role_and_column_priv(arg.tenant_id_))) {
|
|
LOG_WARN("not support role while upgrading", K(ret));
|
|
} else if (OB_FAIL(create_mysql_roles_in_trans(arg.tenant_id_, arg.if_not_exist_, arg.user_infos_))) {
|
|
LOG_WARN("fail to create mysql roles", K(ret));
|
|
}
|
|
} else {
|
|
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))) {
|
|
const ObString &user_name = user_info.get_user_name_str();
|
|
const ObString &host_name = user_info.get_host_name_str();
|
|
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));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} //end if
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_user(ObUserInfo &user_info,
|
|
uint64_t creator_id,
|
|
uint64_t &user_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_service_impl = NULL;
|
|
const uint64_t tenant_id = user_info.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == user_info.get_tenant_id()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id is valid", K(user_info), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
schema_service_impl = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service_impl)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service_impl must not be null", K(schema_service_impl), K(ret));
|
|
} else if (OB_FAIL(check_user_exist(user_info))) {
|
|
if (OB_ERR_USER_EXIST == ret) {
|
|
LOG_WARN("User is exist, cannot create it twice,",
|
|
"tenant_id", user_info.get_tenant_id(),
|
|
"user_id", user_info.get_user_id(),
|
|
"user_name", user_info.get_user_name_str(), K(ret));
|
|
} else {
|
|
LOG_WARN("check_user_exist failed", K(user_info), K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
// if user_id is valid, don't fetch new table id, in case of create user with user_id
|
|
uint64_t new_user_id = user_info.get_user_id();
|
|
if (OB_FAIL(schema_service_impl->fetch_new_user_id(
|
|
user_info.get_tenant_id(), new_user_id))) {
|
|
LOG_WARN("Failed to fetch new_user_id", K(ret));
|
|
} else {
|
|
user_info.set_user_id(new_user_id);
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(create_user_in_trans(user_info, creator_id, user_id, schema_guard))) {
|
|
LOG_WARN("create_user_in_trans failed", K(user_info), K(ret), K(creator_id));
|
|
}
|
|
}
|
|
} else if (OB_ERR_USER_EXIST == ret) {
|
|
}
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_user(const ObDropUserArg &arg,
|
|
ObIArray<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;
|
|
ObAccountArg account;
|
|
ObString ddl_sql;
|
|
bool is_oracle_mode = false;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObArray<uint64_t> user_ids;
|
|
|
|
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(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode))) {
|
|
LOG_WARN("fail to check compat mode", K(ret));
|
|
} else if (arg.is_role_ && !is_oracle_mode) {
|
|
//mysql drop roles in one trans
|
|
//either succeeds for all named roles or rolls back and has no effect if any error occurs
|
|
bool has_any_role_not_exist = false;
|
|
if (OB_FAIL(ObSQLUtils::compatibility_check_for_mysql_role_and_column_priv(arg.tenant_id_))) {
|
|
LOG_WARN("not support role while upgrading", K(ret));
|
|
} else if (OB_FAIL(ddl_stmt_str.append("DROP ROLE "))) {
|
|
LOG_WARN("fail to append str", K(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < arg.users_.count(); ++i) {
|
|
const ObUserInfo *user_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.users_.at(i), arg.hosts_.at(i), user_info))) {
|
|
LOG_WARN("get_user_id failed", K(ret));
|
|
} else if (NULL == user_info) {
|
|
has_any_role_not_exist = true;
|
|
if (OB_FAIL(failed_index.push_back(i))) {
|
|
LOG_WARN("fail to push back", K(ret));
|
|
}
|
|
LOG_WARN("drop non-exist user or role", K(ret), K(arg.users_.at(i)), K(arg.hosts_.at(i)));
|
|
} else if (OB_FAIL(user_ids.push_back(user_info->get_user_id()))) {
|
|
LOG_WARN("fail to push back", K(ret));
|
|
} else if (OB_FAIL(ddl_stmt_str.append_fmt("`%.*s`@`%.*s`,",
|
|
arg.users_.at(i).length(), arg.users_.at(i).ptr(),
|
|
arg.hosts_.at(i).length(), arg.hosts_.at(i).ptr()))) {
|
|
LOG_WARN("fail to apend format", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && arg.users_.count() > 0 && !has_any_role_not_exist) {
|
|
ddl_sql = ObString(ddl_stmt_str.string().length() - 1, ddl_stmt_str.string().ptr());
|
|
if (OB_FAIL(drop_user_in_trans(tenant_id, user_ids, &ddl_sql))) {
|
|
LOG_WARN("Drop one user failed", K(account), K(tenant_id), K(user_id), K(ret));
|
|
}
|
|
}
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < arg.users_.count(); ++i) {
|
|
user_ids.reuse();
|
|
ddl_stmt_str.reuse();
|
|
ddl_sql.reset();
|
|
account.user_name_ = arg.users_.at(i);
|
|
account.host_name_ = arg.hosts_.at(i);
|
|
const bool is_role = arg.is_role_;
|
|
account.is_role_ = is_role;
|
|
const ObUserInfo *user_info = NULL;
|
|
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_oracle_mode && 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(user_ids.push_back(user_info->get_user_id()))) {
|
|
LOG_WARN("fail to push back", K(ret));
|
|
} else if (OB_FAIL(drop_user_in_trans(tenant_id, user_ids, &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));
|
|
}
|
|
}
|
|
}
|
|
} //end if
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_user_in_trans(const uint64_t tenant_id,
|
|
const common::ObIArray<uint64_t> &user_ids,
|
|
const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant id is invalid", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
|
|
for (int i = 0; OB_SUCC(ret) && i < user_ids.count(); i++) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
uint64_t user_id = user_ids.at(i);
|
|
if (OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("User id is invalid", K(ret), K(user_id));
|
|
} else if (OB_FAIL(ddl_operator.drop_user(tenant_id, user_id, (0 == i) ? ddl_stmt_str : NULL, trans))) {
|
|
LOG_WARN("failed to drop user", K(ret), K(tenant_id), K(user_id));
|
|
} else {
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
const ObSysVariableSchema *sys_variable_schema = NULL;
|
|
bool is_oracle_mode = false;
|
|
const ObUserInfo *user_info = NULL;
|
|
if (OB_FAIL(ret)) {
|
|
// do-nothing
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("tenant not exists", K(ret), K(tenant_id));
|
|
} else if (OB_UNLIKELY(NULL == tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) {
|
|
LOG_WARN("get sys variable schema failed", K(ret));
|
|
} else if (OB_ISNULL(sys_variable_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys variable schema is null", K(ret));
|
|
} else if (OB_FAIL(sys_variable_schema->get_oracle_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get oracle mode", K(ret), K(tenant_id));
|
|
} else if (is_oracle_mode) {
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, user_id, user_info))) {
|
|
LOG_WARN("failed to get user info", K(ret), K(tenant_id), K(user_id));
|
|
} else if (NULL == user_info) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get user info", K(ret), K(tenant_id), K(user_id));
|
|
} else if (!user_info->is_role()) {
|
|
// For Oracle mode, drop user will drop the same name db
|
|
// role does not contain the same name db
|
|
obrpc::ObDropDatabaseArg arg;
|
|
arg.tenant_id_ = tenant_id;
|
|
arg.database_name_ = user_info->get_user_name();
|
|
arg.if_exist_ = false;
|
|
arg.to_recyclebin_ = false;
|
|
obrpc::ObDropDatabaseRes res;
|
|
if (OB_FAIL(drop_database(arg, res, &trans))) {
|
|
LOG_WARN("failed to create oracle user database", K(ret), K(tenant_id), K(user_info));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_role(const ObAlterRoleArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t role_id = OB_INVALID_ID;
|
|
const ObString &role_name = arg.role_name_;
|
|
const ObString &host_name = arg.host_name_;
|
|
const ObString &passwd = arg.pwd_enc_;
|
|
const ObUserInfo *role_info = NULL;
|
|
|
|
ObSchemaGetterGuard schema_guard;
|
|
if(OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id,
|
|
schema_guard))) {
|
|
LOG_WARN("failed to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_user_info(tenant_id, role_name, host_name, role_info))) {
|
|
LOG_WARN("get_user_info failed", K(ret), K(tenant_id), K(role_name), K(host_name));
|
|
} else if (OB_ISNULL(role_info) || OB_UNLIKELY(false == role_info->is_role())) {
|
|
ret = OB_ROLE_NOT_EXIST;
|
|
LOG_WARN("alter non-exist role", K(ret), K(tenant_id), K(role_name), K(host_name));
|
|
LOG_USER_ERROR(OB_ROLE_NOT_EXIST, role_name.length(), role_name.ptr());
|
|
} else {
|
|
ObSqlString ddl_stmt_str;
|
|
ObString ddl_sql;
|
|
role_id = role_info->get_user_id();
|
|
if (OB_UNLIKELY(OB_INVALID_ID == role_id)) {
|
|
ret = OB_ROLE_NOT_EXIST;
|
|
LOG_WARN("Try to alter a not-exist role", K(tenant_id), K(role_name), K(host_name), K(ret));
|
|
LOG_USER_ERROR(OB_ROLE_NOT_EXIST, role_name.length(), role_name.ptr());
|
|
} else if (OB_FAIL(ObDDLSqlGenerator::gen_alter_role_sql(ObAccountArg(role_name, host_name),
|
|
passwd,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("gen_alter_role_sql failed", K(ret));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(alter_role_in_trans(tenant_id, role_id, passwd, &ddl_sql, schema_guard))) {
|
|
LOG_WARN("alter_role_in_trans failed", K(tenant_id), K(role_id), K(arg), K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_role_in_trans(const uint64_t tenant_id,
|
|
const uint64_t role_id,
|
|
const common::ObString &new_passwd,
|
|
const common::ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variale is not init", K(ret));
|
|
} else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id) || OB_UNLIKELY(OB_INVALID_ID == role_id)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("tenant_id or role_id is invalid", K(ret), K(tenant_id), K(role_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else {
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.alter_role(tenant_id,
|
|
role_id,
|
|
new_passwd,
|
|
ddl_stmt_str,
|
|
trans))) {
|
|
LOG_WARN("fail to alter role", K(ret), K(tenant_id), K(role_id), K(new_passwd));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(tmp_ret));
|
|
ret = (OB_SUCC(ret)) ? tmp_ret : ret;
|
|
}
|
|
}
|
|
|
|
}
|
|
// publish schema
|
|
if(OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
//FIXME: For oracle user, database should be renamed too.
|
|
int ObDDLService::rename_user(const ObRenameUserArg &arg,
|
|
ObIArray<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, schema_guard))) {
|
|
LOG_WARN("Rename one user failed", K(tenant_id), K(user_id), K(old_account),
|
|
K(new_account), K(ret));
|
|
ret = OB_SUCCESS; //rename fail, try next, recover
|
|
if (OB_FAIL(failed_index.push_back(i))) {
|
|
LOG_WARN("push_back failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rename_user_in_trans(
|
|
const uint64_t tenant_id,
|
|
const uint64_t user_id,
|
|
const ObAccountArg &new_account,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant_id or user_id is invalid", K(ret));
|
|
} else {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.rename_user(tenant_id, user_id, new_account, ddl_stmt_str, trans))) {
|
|
LOG_WARN("fail to rename user", K(ret), K(tenant_id), K(user_id), K(new_account));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::set_passwd(const ObSetPasswdArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t user_id = OB_INVALID_ID;
|
|
const ObString &user_name = arg.user_;
|
|
const ObString &host_name = arg.host_;
|
|
const ObString &passwd = arg.passwd_;
|
|
const bool modify_max_connections = arg.modify_max_connections_;
|
|
const uint64_t max_connections_per_hour = arg.max_connections_per_hour_;
|
|
const uint64_t max_user_connections = arg.max_user_connections_;
|
|
const share::schema::ObSSLType ssl_type = arg.ssl_type_;
|
|
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat faile", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else {
|
|
ObSqlString ddl_stmt_str;
|
|
ObString ddl_sql;
|
|
if (OB_FAIL(schema_guard.get_user_id(tenant_id, user_name, host_name, user_id))) {
|
|
LOG_WARN("get_user_id failed", K(tenant_id), K(user_name), K(host_name), K(ret));
|
|
} else if (OB_INVALID_ID == user_id) {
|
|
ret = OB_USER_NOT_EXIST; //no such user
|
|
LOG_WARN("Try to set password for non-exist user", K(tenant_id), K(user_name), K(host_name), K(ret));
|
|
} else if (share::schema::ObSSLType::SSL_TYPE_NOT_SPECIFIED == ssl_type) {
|
|
if (modify_max_connections) {
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_set_max_connections_sql(
|
|
ObAccountArg(user_name, host_name), max_connections_per_hour, max_user_connections,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("gen_set_passwd_sql failed", K(ret), K(arg));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(set_max_connection_in_trans(tenant_id, user_id,
|
|
max_connections_per_hour, max_user_connections,
|
|
&ddl_sql, schema_guard))) {
|
|
LOG_WARN("Set passwd failed", K(tenant_id), K(user_id), K(passwd), K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_set_passwd_sql(ObAccountArg(user_name, host_name),
|
|
passwd, ddl_stmt_str))) {
|
|
LOG_WARN("gen_set_passwd_sql failed", K(ret), K(arg));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(set_passwd_in_trans(tenant_id, user_id, passwd,
|
|
&ddl_sql, schema_guard))) {
|
|
LOG_WARN("Set passwd failed", K(tenant_id), K(user_id), K(passwd), K(ret));
|
|
}
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_alter_user_require_sql(
|
|
ObAccountArg(user_name, host_name), arg, ddl_stmt_str))) {
|
|
LOG_WARN("gen_alter_user_require_sql failed", K(ret), K(arg));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(alter_user_require_in_trans(tenant_id, user_id, arg, &ddl_sql, schema_guard))) {
|
|
LOG_WARN("alter_user_require failed", K(tenant_id), K(user_id), K(arg), K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::set_passwd_in_trans(
|
|
const uint64_t tenant_id,
|
|
const uint64_t user_id,
|
|
const common::ObString &new_passwd,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant_id or user_id is invalid", K(tenant_id), K(user_id), K(ret));
|
|
} else {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.set_passwd(tenant_id,
|
|
user_id,
|
|
new_passwd,
|
|
ddl_stmt_str,
|
|
trans))) {
|
|
LOG_WARN("fail to set password", K(ret), K(tenant_id), K(user_id), K(new_passwd));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("pubish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::set_max_connection_in_trans(
|
|
const uint64_t tenant_id,
|
|
const uint64_t user_id,
|
|
const uint64_t max_connections_per_hour,
|
|
const uint64_t max_user_connections,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant_id or user_id is invalid", K(tenant_id), K(user_id), K(ret));
|
|
} else {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.set_max_connections(tenant_id,
|
|
user_id,
|
|
max_connections_per_hour,
|
|
max_user_connections,
|
|
ddl_stmt_str,
|
|
trans))) {
|
|
LOG_WARN("fail to set max connections", K(ret), K(tenant_id), K(user_id));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("pubish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_user_require_in_trans(
|
|
const uint64_t tenant_id,
|
|
const uint64_t user_id,
|
|
const obrpc::ObSetPasswdArg &arg,
|
|
const common::ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant_id or user_id is invalid", K(tenant_id), K(user_id), K(ret));
|
|
} else {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.alter_user_require(tenant_id, user_id, arg, ddl_stmt_str, trans))) {
|
|
LOG_WARN("fail to alter user require", K(ret), K(tenant_id), K(user_id));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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_);
|
|
int64_t refreshed_schema_version = 0;
|
|
const ObUserInfo *user_info = NULL;
|
|
ObSysPriv *sys_priv = NULL;
|
|
uint64_t grantee_id = 0;
|
|
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)));
|
|
}
|
|
|
|
if (FAILEDx(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
}
|
|
|
|
OZ (trans.start(sql_proxy_, tenant_id, refreshed_schema_version));
|
|
for (int i = 0; OB_SUCC(ret) && i < users_name.count(); ++i) {
|
|
const ObString &user_name = users_name.at(i);
|
|
const ObString &host_name = hosts_name.at(i);
|
|
/* check grantee info */
|
|
OZ (schema_guard.get_user_info(tenant_id,
|
|
user_name/*user_name*/,
|
|
host_name/*host_name*/,
|
|
user_info));
|
|
if (OB_SUCC(ret)) {
|
|
if (NULL == user_info) {
|
|
ret = OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST, user_name.length(), user_name.ptr());
|
|
LOG_WARN("user doesn't exist", K(ret), K(user_name));
|
|
} else {
|
|
grantee_id = user_info->get_user_id();
|
|
}
|
|
}
|
|
OZ (schema_guard.get_sys_priv_with_grantee_id(tenant_id,
|
|
grantee_id,
|
|
sys_priv));
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
OZ (ddl_operator.grant_sys_priv_to_ur(tenant_id,
|
|
grantee_id,
|
|
sys_priv,
|
|
arg.option_,
|
|
arg.sys_priv_array_,
|
|
trans,
|
|
true /*is_grant*/,
|
|
&arg.ddl_stmt_str_,
|
|
schema_guard));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
// publish schema
|
|
if (FAILEDx(publish_schema(tenant_id))) {
|
|
LOG_WARN("pubish schema failed", K(ret));
|
|
}
|
|
|
|
LOG_INFO("finish grant_sys_priv_to_ur", K(ret));
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::grant_table_and_col_privs_to_user(
|
|
const ObGrantArg &arg,
|
|
uint64_t grantee_id,
|
|
ObString &user_name,
|
|
ObString &host_name,
|
|
ObNeedPriv &need_priv,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
/* 1. deal with table privs first if has */
|
|
ObObjPrivSortKey obj_priv_key(arg.tenant_id_,
|
|
arg.object_id_,
|
|
static_cast<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_,
|
|
false,
|
|
obj_priv_key,
|
|
schema_guard));
|
|
/* 2. deal with cols privs */
|
|
/* 2.1 reorg privs according to colid */
|
|
uint64_t colid = 0;
|
|
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_,
|
|
false,
|
|
obj_priv_key,
|
|
schema_guard));
|
|
}
|
|
for (int i = 0; i < arg.upd_col_ids_.count() && OB_SUCC(ret); i++) {
|
|
obj_priv_array.reset();
|
|
obj_priv_array.push_back(OBJ_PRIV_ID_UPDATE);
|
|
colid = arg.upd_col_ids_.at(i);
|
|
ObObjPrivSortKey obj_priv_key(arg.tenant_id_,
|
|
arg.object_id_,
|
|
static_cast<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_,
|
|
false,
|
|
obj_priv_key,
|
|
schema_guard));
|
|
}
|
|
for (int i = 0; i < arg.ref_col_ids_.count() && OB_SUCC(ret); i++) {
|
|
obj_priv_array.reset();
|
|
obj_priv_array.push_back(OBJ_PRIV_ID_REFERENCES);
|
|
colid = arg.ref_col_ids_.at(i);
|
|
ObObjPrivSortKey obj_priv_key(arg.tenant_id_,
|
|
arg.object_id_,
|
|
static_cast<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_,
|
|
false,
|
|
obj_priv_key,
|
|
schema_guard));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/* Check whether there is role_id information in user_info, recursively */
|
|
int ObDDLService::exists_role_grant_cycle(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const ObUserInfo &role_info,
|
|
const ObUserInfo *user_info,
|
|
const bool is_oracle_mode)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool found = false;
|
|
CK (user_info != NULL);
|
|
uint64_t role_id = role_info.get_user_id();
|
|
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_info, tmp_role_info, is_oracle_mode));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && found) {
|
|
ret = OB_ERR_CIRCULAR_ROLE_GRANT_DETECTED;
|
|
if (!is_oracle_mode) {
|
|
LOG_USER_ERROR(OB_ERR_CIRCULAR_ROLE_GRANT_DETECTED,
|
|
role_info.get_user_name_str().length(), role_info.get_user_name_str().ptr(),
|
|
role_info.get_host_name_str().length(), role_info.get_host_name_str().ptr(),
|
|
user_info->get_user_name_str().length(), user_info->get_user_name_str().ptr(),
|
|
user_info->get_host_name_str().length(), user_info->get_host_name_str().ptr());
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::grant(const ObGrantArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_UNLIKELY(!arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("arg is invalid", K(arg), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_parallel_ddl_conflict(schema_guard, arg))) {
|
|
LOG_WARN("check parallel ddl conflict failed", 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 {
|
|
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_or_role = roles.count() >= GRANT_SYS_ROLE_NUM;
|
|
const bool is_grant_role = roles.count() > GRANT_SYS_ROLE_NUM;
|
|
if (is_grant_sys_or_role) {
|
|
// grant role and sys_privs to user_list
|
|
ObSArray<ObString> users_name;
|
|
ObSArray<ObString> hosts_name;
|
|
ObArray<ObUserInfo> users_info;
|
|
// Start transaction at beginning
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
// Save all user names in array
|
|
const ObUserInfo *user_info = NULL;
|
|
OZ (users_name.push_back(roles.at(0)));
|
|
OZ (hosts_name.push_back(roles.at(1)));
|
|
CK (OB_LIKELY(remain_roles.count() % 2 == 0));
|
|
for (int i = 0; OB_SUCC(ret) && i < remain_roles.count(); i += 2) {
|
|
OZ (users_name.push_back(remain_roles.at(i)));
|
|
OZ (hosts_name.push_back(remain_roles.at(i + 1)));
|
|
}
|
|
CK (OB_LIKELY(users_name.count() == hosts_name.count()));
|
|
// Save all user infos
|
|
for (int i = 0; OB_SUCC(ret) && i < users_name.count(); ++i) {
|
|
const ObString &user_name = users_name.at(i);
|
|
const ObString &host_name = hosts_name.at(i);
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id,
|
|
user_name,
|
|
host_name,
|
|
user_info))) {
|
|
LOG_WARN("Failed to get role info", K(tenant_id), K(user_name), K(host_name), K(ret));
|
|
} else if (OB_UNLIKELY(NULL == user_info)) {
|
|
ret = OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST, user_name.length(), user_name.ptr());
|
|
LOG_WARN("user doesn't exist", K(ret), K(user_name));
|
|
} else if (OB_FAIL(users_info.push_back(*user_info))) {
|
|
LOG_WARN("Failed to push back user info", K(ret), K(*user_info));
|
|
}
|
|
}
|
|
}
|
|
// Grant role
|
|
if (OB_SUCC(ret)) {
|
|
if (is_grant_role) {
|
|
// grant roles to user
|
|
// 1. Get the specified user granted
|
|
// 2. Get all permissions owned by each role (three levels: user/db/table)
|
|
// 3. All permissions of compatible role are appended to the specified user
|
|
/* resolve grantee, priv info stored in roles[0], roles[1],
|
|
grantee info stored in roles[2], roles[3], ... */
|
|
ObArray<uint64_t> role_ids;
|
|
ObArray<ObUserInfo> roles_info;
|
|
// Resolve each role id and role info
|
|
bool is_oracle_mode = lib::Worker::CompatMode::ORACLE == compat_mode;
|
|
int64_t step = is_oracle_mode ? 1 : 2;
|
|
if (!is_oracle_mode) {
|
|
OZ (ObSQLUtils::compatibility_check_for_mysql_role_and_column_priv(tenant_id));
|
|
}
|
|
for (int64_t i = GRANT_ROLE_MIN_ROLE_NUM - 1; OB_SUCC(ret) && i + step <= roles.count(); i+=step) {
|
|
// Oracle currently does not support specifying hostname to create a role
|
|
const ObString host_name = is_oracle_mode ? ObString(OB_DEFAULT_HOST_NAME) : roles.at(i+1);
|
|
const ObString role = roles.at(i);
|
|
const ObUserInfo *role_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, role, host_name, role_info))) {
|
|
LOG_WARN("Failed to get role info", K(tenant_id), K(role), K(host_name), K(ret));
|
|
} else if (NULL == role_info) {
|
|
ret = OB_ROLE_NOT_EXIST;
|
|
LOG_WARN("role doesn't exist", K(ret), K(role));
|
|
LOG_USER_ERROR(OB_ROLE_NOT_EXIST, role.length(), role.ptr());
|
|
} else if (OB_FAIL(role_ids.push_back(role_info->get_user_id()))) {
|
|
LOG_WARN("Failed to push back role_id", K(ret), K(tenant_id), K(*role_info));
|
|
} else if (OB_FAIL(roles_info.push_back(*role_info))) {
|
|
LOG_WARN("Failed to push back role_info", K(ret), K(tenant_id), K(*role_info));
|
|
}
|
|
}
|
|
// Operate on each user_name
|
|
for (int i = 0; OB_SUCC(ret) && i < users_info.count(); ++i) {
|
|
const ObUserInfo &user_info = users_info.at(i);
|
|
if (!user_info.is_role() && is_oracle_mode) {
|
|
//skip check
|
|
} else {
|
|
// 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,
|
|
&roles_info.at(j),
|
|
is_oracle_mode))) {
|
|
LOG_WARN("role cycle exists", K(ret), K(roles_info.at(j)));
|
|
}
|
|
}
|
|
}
|
|
// Do the operation
|
|
if (OB_SUCC(ret) && role_ids.count() > 0) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
bool log_operation = true;
|
|
if (true == arg.sys_priv_array_.empty()) {
|
|
// No sys_priv granted toghther, log grant role operation
|
|
log_operation = true;
|
|
} else {
|
|
// "grant role" will be logged with "grant sys_priv" later
|
|
log_operation = false;
|
|
}
|
|
if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id,
|
|
user_info,
|
|
role_ids,
|
|
NULL, /*Single specified role info*/
|
|
trans,
|
|
log_operation,
|
|
true /*is_grant*/,
|
|
arg.option_))) {
|
|
LOG_WARN("falied to grant_revoke_role", K(ret), K(tenant_id), K(user_info));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// Grant sys_privs
|
|
if (OB_SUCC(ret)) {
|
|
if (false == arg.sys_priv_array_.empty()) {
|
|
ObSysPriv *sys_priv = NULL;
|
|
for (int i = 0; OB_SUCC(ret) && i < users_info.count(); ++i) {
|
|
const ObUserInfo &user_info = users_info.at(i);
|
|
uint64_t grantee_id = user_info.get_user_id();
|
|
OZ (schema_guard.get_sys_priv_with_grantee_id(tenant_id,
|
|
grantee_id,
|
|
sys_priv));
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
OZ (ddl_operator.grant_sys_priv_to_ur(tenant_id,
|
|
grantee_id,
|
|
sys_priv,
|
|
arg.option_,
|
|
arg.sys_priv_array_,
|
|
trans,
|
|
true /*is_grant*/,
|
|
&arg.ddl_stmt_str_,
|
|
schema_guard));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
// End transaction
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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, schema_guard))) {
|
|
LOG_WARN("Set password error", KR(ret), K(arg), K(user_id), K(pwd), K(ddl_sql));
|
|
}
|
|
}
|
|
} else if (!is_user_exist) {
|
|
if (arg.need_create_user_ || !pwd.empty()) {
|
|
if (!arg.has_create_user_priv_) {
|
|
tmp_ret = OB_ERR_CREATE_USER_WITH_GRANT;
|
|
} else {
|
|
ObUserInfo user_info;
|
|
user_info.set_tenant_id(arg.tenant_id_);
|
|
user_info.set_passwd(pwd);
|
|
if (OB_FAIL(user_info.set_user_name(user_name))) {
|
|
LOG_WARN("set_user_name error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret));
|
|
} else if (OB_FAIL(user_info.set_host(host_name))) {
|
|
LOG_WARN("set_host error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret));
|
|
} else if (OB_FAIL(user_info.set_passwd(pwd))) {
|
|
LOG_WARN("set_passwd error", "tenant_id", arg.tenant_id_, K(user_name), K(host_name), K(ret));
|
|
} else if (OB_FAIL(create_user(user_info,
|
|
OB_INVALID_ID,
|
|
user_id))) {
|
|
LOG_WARN("Create user error", "tenant_id", arg.tenant_id_,
|
|
K(user_name), K(host_name), K(ret));
|
|
} else {
|
|
is_user_exist = true;
|
|
}
|
|
}
|
|
} else {
|
|
if (lib::Worker::CompatMode::ORACLE == compat_mode) {
|
|
tmp_ret = OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_USER_OR_ROLE_DOES_NOT_EXIST, user_name.length(), user_name.ptr());
|
|
} else {
|
|
tmp_ret = OB_PASSWORD_WRONG;
|
|
LOG_USER_ERROR(OB_PASSWORD_WRONG,
|
|
user_name.length(), user_name.ptr(), host_name.length(), host_name.ptr(), "NO");
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && is_user_exist) {
|
|
ObNeedPriv need_priv(arg.db_, arg.table_, arg.priv_level_, arg.priv_set_, false, arg.object_type_);
|
|
bool is_owner = false;
|
|
// In oracle mode, if it is oracle syntax, it need to determine grantee is obj owner,
|
|
// if yes, return success directly
|
|
if (lib::Worker::CompatMode::ORACLE == compat_mode
|
|
&& (arg.priv_level_ == OB_PRIV_SYS_ORACLE_LEVEL
|
|
|| (arg.ins_col_ids_.count() +
|
|
arg.upd_col_ids_.count() +
|
|
arg.ref_col_ids_.count() +
|
|
arg.obj_priv_array_.count() > 0))) {
|
|
is_owner = share::ObOraPrivCheck::user_is_owner(user_name, arg.db_);
|
|
}
|
|
if (!is_owner) {
|
|
/* No column level permissions */
|
|
if (arg.column_names_priv_.count() == 0 &&
|
|
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_,
|
|
arg.is_inner_,
|
|
obj_priv_key,
|
|
schema_guard,
|
|
arg.grantor_,
|
|
arg.grantor_host_))) {
|
|
LOG_WARN("Grant priv to user failed", K(ret));
|
|
}
|
|
} else if (lib::Worker::CompatMode::MYSQL == compat_mode) {
|
|
OZ (grant_table_and_column_mysql(arg,
|
|
user_id,
|
|
user_name,
|
|
host_name,
|
|
need_priv,
|
|
schema_guard));
|
|
} else {
|
|
OZ (grant_table_and_col_privs_to_user(arg, user_id, user_name,
|
|
host_name, need_priv, schema_guard));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && tmp_ret != OB_SUCCESS) {
|
|
ret = tmp_ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::revoke(const ObRevokeUserArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObUserInfo *user_info = NULL;
|
|
const bool grant = false;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const bool is_revoke_role = arg.role_ids_.count() > 0;
|
|
uint64_t user_id = OB_INVALID_ID;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_INVALID_ID == arg.tenant_id_
|
|
|| OB_INVALID_ID == arg.user_id_) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(arg));
|
|
} else if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.user_id_, user_info))) {
|
|
LOG_WARN("get_user_info failed", K(arg), K(ret));
|
|
} else if (OB_ISNULL(user_info)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("user not exist", K(ret), K(arg));
|
|
} else if (is_revoke_role) {
|
|
// this process include revoke role
|
|
user_id = user_info->get_user_id();
|
|
ObArray<uint64_t> role_ids;
|
|
bool is_oracle_mode = false;
|
|
OZ (ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode));
|
|
if (OB_SUCC(ret) && !is_oracle_mode) {
|
|
OZ (ObSQLUtils::compatibility_check_for_mysql_role_and_column_priv(tenant_id));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < arg.role_ids_.count(); ++i) {
|
|
const uint64_t role_id = arg.role_ids_.at(i);
|
|
const ObUserInfo *role_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, role_id, role_info))) {
|
|
LOG_WARN("Failed to get role info", K(ret), K(tenant_id), K(role_id));
|
|
} else if (NULL == role_info) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("role doesn't exist", K(ret), K(role_id));
|
|
} else {
|
|
// Determine whether the current role is granted to the user; otherwise, it will not be processed
|
|
ObSEArray<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;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(user.assign(*user_info))) {
|
|
LOG_WARN("assign user failed", K(ret), KPC(user_info));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id,
|
|
user,
|
|
role_ids,
|
|
NULL /*Single specified role info*/,
|
|
trans,
|
|
true /*log_operation*/,
|
|
false /*is_grant*/,
|
|
NO_OPTION))) {
|
|
LOG_WARN("falied to grant_revoke_role", K(ret), K(tenant_id), K(user), K(*user_info));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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,
|
|
false,
|
|
&ddl_sql,
|
|
schema_guard))) {
|
|
LOG_WARN("Revoke user failed", K(arg), K(grant), K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(revoke_all(arg.tenant_id_, user_info->get_user_name_str(),
|
|
user_info->get_host_name_str(), arg.user_id_,
|
|
schema_guard))) {
|
|
LOG_WARN("Revoke all failed", K(arg), K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::grant_priv_to_user(const uint64_t tenant_id,
|
|
const uint64_t user_id,
|
|
const ObString &user_name,
|
|
const ObString &host_name,
|
|
const ObNeedPriv &need_priv,
|
|
const share::ObRawObjPrivArray &obj_priv_array,
|
|
const uint64_t option,
|
|
const bool is_from_inner_sql,
|
|
ObObjPrivSortKey &obj_priv_key,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const common::ObString &grantor,
|
|
const common::ObString &grantor_host)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Input arguments error", K(tenant_id), K(user_id), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObSqlString ddl_stmt_str;
|
|
ObString ddl_sql;
|
|
switch (need_priv.priv_level_) {
|
|
case OB_PRIV_USER_LEVEL: {
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_user_priv_sql(ObAccountArg(user_name, host_name), need_priv,
|
|
true, ddl_stmt_str))) {
|
|
LOG_WARN("gen user_priv sql failed", K(ret), K(user_name), K(host_name));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(grant_revoke_user(tenant_id, user_id, need_priv.priv_set_,
|
|
true, is_from_inner_sql, &ddl_sql, schema_guard))) {
|
|
LOG_WARN("Grant user error", KR(ret), K(tenant_id), K(user_id), K(ddl_sql), K(need_priv));
|
|
}
|
|
break;
|
|
}
|
|
case OB_PRIV_DB_LEVEL: {
|
|
ObOriginalDBKey db_key(tenant_id, user_id, need_priv.db_);
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_db_priv_sql(ObAccountArg(user_name, host_name), need_priv,
|
|
true, ddl_stmt_str))) {
|
|
LOG_WARN("gen_db_priv sql failed", K(need_priv), K(user_name), K(ret));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(grant_database(db_key, need_priv.priv_set_, &ddl_sql, schema_guard))) {
|
|
LOG_WARN("Grant database error", KR(ret), K(db_key), K(ddl_sql), K(need_priv));
|
|
}
|
|
break;
|
|
}
|
|
case OB_PRIV_TABLE_LEVEL: {
|
|
ObTablePrivSortKey table_key(tenant_id, user_id, need_priv.db_, need_priv.table_);
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_table_priv_sql(ObAccountArg(user_name, host_name), need_priv, true, ddl_stmt_str))) {
|
|
LOG_WARN("gen_table_priv sql failed", K(need_priv), K(ret));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(grant_table(table_key,
|
|
need_priv.priv_set_,
|
|
&ddl_sql,
|
|
obj_priv_array,
|
|
option,
|
|
obj_priv_key,
|
|
schema_guard,
|
|
grantor,
|
|
grantor_host))) {
|
|
LOG_WARN("Grant table error", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case OB_PRIV_ROUTINE_LEVEL: {
|
|
ObRoutinePrivSortKey routine_key(tenant_id, user_id, need_priv.db_, need_priv.table_,
|
|
obj_priv_key.obj_type_ == (int)ObObjectType::PROCEDURE ? ObRoutineType::ROUTINE_PROCEDURE_TYPE :
|
|
obj_priv_key.obj_type_ == (int)ObObjectType::FUNCTION ? ObRoutineType::ROUTINE_FUNCTION_TYPE :
|
|
ObRoutineType::INVALID_ROUTINE_TYPE);
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_routine_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_routine(routine_key,
|
|
need_priv.priv_set_,
|
|
&ddl_sql,
|
|
option,
|
|
schema_guard,
|
|
grantor,
|
|
grantor_host))) {
|
|
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::grant_table_and_column_mysql(const obrpc::ObGrantArg &arg,
|
|
uint64_t user_id,
|
|
const ObString &user_name,
|
|
const ObString &host_name,
|
|
const ObNeedPriv &need_priv,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table_key is invalid", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(arg.tenant_id_, user_id)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", arg.tenant_id_,
|
|
"user_id", user_id,
|
|
K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObTablePrivSortKey table_key(tenant_id, user_id, need_priv.db_, need_priv.table_);
|
|
const share::ObRawObjPrivArray obj_priv_array;
|
|
const share::schema::ObObjPrivSortKey obj_key;
|
|
uint64_t option = 0;
|
|
ObSqlString ddl_stmt_str;
|
|
ObString ddl_stmt;
|
|
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 (OB_FALSE_IT(ddl_stmt = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(ddl_operator.grant_table(table_key,
|
|
arg.priv_set_,
|
|
&ddl_stmt,
|
|
trans,
|
|
obj_priv_array,
|
|
option,
|
|
obj_key,
|
|
arg.grantor_,
|
|
arg.grantor_host_))) {
|
|
LOG_WARN("fail to grant table", K(ret));
|
|
} else if (OB_FAIL(grant_or_revoke_column_priv_mysql(tenant_id, arg.object_id_, user_id,
|
|
user_name, host_name, need_priv.db_,
|
|
need_priv.table_, arg.column_names_priv_,
|
|
ddl_operator, trans, schema_guard, true))) {
|
|
LOG_WARN("grant or revoke column priv mysql 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_SUCCESS == 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_or_revoke_column_priv_mysql(const uint64_t tenant_id,
|
|
const uint64_t table_id,
|
|
const uint64_t user_id,
|
|
const ObString& user_name,
|
|
const ObString& host_name,
|
|
const ObString& db,
|
|
const ObString& table,
|
|
const ObIArray<std::pair<ObString, ObPrivType>> &column_names_priv,
|
|
ObDDLOperator &ddl_operator,
|
|
ObDDLSQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const bool is_grant)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
common::hash::ObHashMap<ObColumnSchemaHashWrapper, ObPrivSet> col_privs;
|
|
if (OB_FAIL(col_privs.create(OB_MAX_COLUMN_NUMBER, lib::ObLabel("ColPriv")))) {
|
|
LOG_WARN("failed to create column id map", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < column_names_priv.count(); i++) {
|
|
ObColumnSchemaHashWrapper column_key(column_names_priv.at(i).first);
|
|
ObPrivSet priv_set = 0;
|
|
int hash_ret = col_privs.get_refactored(column_key, priv_set);
|
|
if (hash_ret == OB_HASH_NOT_EXIST || hash_ret == OB_SUCCESS) {
|
|
priv_set |= column_names_priv.at(i).second;
|
|
if (OB_FAIL(col_privs.set_refactored(column_key, priv_set, 1))) {
|
|
LOG_WARN("set hash refactored failed", K(ret));
|
|
}
|
|
} else {
|
|
ret = hash_ret;
|
|
LOG_WARN("get hash obj failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
for (common::hash::ObHashMap<ObColumnSchemaHashWrapper, ObPrivSet>::iterator it = col_privs.begin();
|
|
OB_SUCC(ret) && it != col_privs.end(); it++) {
|
|
ObColumnPrivSortKey column_key(tenant_id, user_id, db, table, it->first.column_name_);
|
|
ObSqlString ddl_stmt_str;
|
|
ObString ddl_stmt;
|
|
ObNeedPriv need_priv;
|
|
need_priv.priv_level_ = OB_PRIV_TABLE_LEVEL;
|
|
need_priv.priv_set_ = it->second;
|
|
need_priv.db_ = db;
|
|
need_priv.table_ = table;
|
|
if (OB_FAIL(need_priv.columns_.push_back(it->first.column_name_))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
} else if (OB_FAIL(ObDDLSqlGenerator::gen_column_priv_sql(ObAccountArg(user_name, host_name),
|
|
need_priv, is_grant, ddl_stmt_str))) {
|
|
LOG_WARN("gen_table_priv sql failed", K(need_priv), K(ret));
|
|
} else if (OB_FALSE_IT(ddl_stmt = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(ddl_operator.grant_column(schema_guard,
|
|
column_key,
|
|
it->second,
|
|
&ddl_stmt,
|
|
trans,
|
|
is_grant))) {
|
|
LOG_WARN("fail to grant table", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::revoke_all(
|
|
const uint64_t tenant_id,
|
|
const ObString &user_name,
|
|
const ObString &host_name,
|
|
const uint64_t user_id,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id
|
|
|| user_name.empty()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant id or user_id is invalid",
|
|
K(tenant_id), K(user_id), K(user_name), K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObSqlString ddl_stmt_str;
|
|
ObString ddl_sql;
|
|
if (OB_FAIL(ObDDLSqlGenerator::gen_revoke_all_sql(ObAccountArg(user_name, host_name), ddl_stmt_str))) {
|
|
LOG_WARN("gen revoke all sql failed", K(ret));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(ddl_operator.grant_revoke_user(tenant_id, user_id,
|
|
OB_PRIV_ALL|OB_PRIV_GRANT,
|
|
false, 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_SUCCESS == 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 bool is_from_inner_sql,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_ora_mode = false;
|
|
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 if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_ora_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
uint64_t compat_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("fail to get data version", K(ret), K(tenant_id));
|
|
} else if (!ObSQLUtils::is_data_version_ge_422_or_431(compat_version) && !is_ora_mode
|
|
&& (0 != (priv_set & OB_PRIV_EXECUTE) ||
|
|
0 != (priv_set & OB_PRIV_ALTER_ROUTINE) ||
|
|
0 != (priv_set & OB_PRIV_CREATE_ROUTINE))) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("some column of user info is not empty when MIN_DATA_VERSION is below DATA_VERSION_4_3_1_0 or DATA_VERSION_4_2_2_0", K(ret), K(priv_set));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "grant or revoke execute/alter routine/create routine privilege");
|
|
} else if (!ObSQLUtils::is_data_version_ge_423_or_432(compat_version) && !is_ora_mode
|
|
&& (0 != (priv_set & OB_PRIV_CREATE_TABLESPACE) ||
|
|
0 != (priv_set & OB_PRIV_SHUTDOWN) ||
|
|
0 != (priv_set & OB_PRIV_RELOAD))) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("some column of user info is not empty when MIN_DATA_VERSION is below DATA_VERSION_4_2_3_0 or DATA_VERSION_4_3_2_0", K(ret), K(priv_set));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "grant or revoke create tablespace/shutdown/reload privilege");
|
|
} else if (!ObSQLUtils::is_data_version_ge_424_or_433(compat_version) && !is_ora_mode
|
|
&& (0 != (priv_set & OB_PRIV_REFERENCES) ||
|
|
0 != (priv_set & OB_PRIV_CREATE_ROLE) ||
|
|
0 != (priv_set & OB_PRIV_DROP_ROLE) ||
|
|
0 != (priv_set & OB_PRIV_TRIGGER))) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("some column of user info is not empty when MIN_DATA_VERSION is below DATA_VERSION_4_2_4_0 or DATA_VERSION_4_3_3_0", K(ret), K(priv_set));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "grant or revoke references/create role/drop role/trigger");
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.grant_revoke_user(tenant_id, user_id, priv_set, grant, is_from_inner_sql, 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_all_users_in_tenant_with_profile(const uint64_t tenant_id,
|
|
const uint64_t profile_id,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObIArray<uint64_t> &user_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_sql_service = NULL;
|
|
ObSEArray<const ObUserInfo*, 10> user_infos;
|
|
|
|
if (OB_UNLIKELY(!is_valid_id(profile_id)
|
|
|| !is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(ret), K(profile_id), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_sql_service = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_sql_service must not null", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_user_infos_with_tenant_id(tenant_id, user_infos))) {
|
|
LOG_WARN("fail to get all user in tenant", K(ret));
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < user_infos.count(); ++i) {
|
|
const ObUserInfo *user_info = NULL;
|
|
if (OB_ISNULL(user_info = user_infos.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema is NULL", K(ret));
|
|
} else if (user_info->get_profile_id() == profile_id) {
|
|
if (OB_FAIL(user_ids.push_back(user_info->get_user_id()))) {
|
|
LOG_WARN("fail to push back", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
#define PUSH_BACK_ROLE_ID_INFO(target_disable_flag) \
|
|
OZ (role_id_array.push_back(role_id)); \
|
|
OZ (disable_flag_array.push_back(target_disable_flag)); \
|
|
OX (need_flush = true);
|
|
|
|
int ObDDLService::build_need_flush_role_array(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
uint64_t tenant_id,
|
|
const ObUserInfo *user_info,
|
|
const ObAlterUserProfileArg &arg,
|
|
bool &need_flush,
|
|
ObIArray<uint64_t> &role_id_array,
|
|
ObIArray<uint64_t> &disable_flag_array)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t role_id = 0;
|
|
uint64_t target_disable_flag = 0;
|
|
uint64_t org_disable_flag = 0;
|
|
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;
|
|
bool is_oracle_mode = false;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (!is_oracle_mode && OB_FAIL(ObSQLUtils::compatibility_check_for_mysql_role_and_column_priv(tenant_id))) {
|
|
LOG_WARN("not support set role while upgrading", 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_);
|
|
|
|
for (int i = 0; OB_SUCC(ret) && i < MAX(1, arg.user_ids_.count()); i++) {
|
|
/* 1. check user exists */
|
|
uint64_t user_id = arg.user_ids_.empty() ? arg.user_id_ : arg.user_ids_.at(i);
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, user_id, user_info))) {
|
|
LOG_WARN("get user info fail", K(tenant_id), K(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 */
|
|
role_id_array.reuse();
|
|
disable_flag_array.reuse();
|
|
OZ (build_need_flush_role_array(schema_guard, tenant_id, user_info, arg,
|
|
need_flush, role_id_array, disable_flag_array));
|
|
LOG_DEBUG("check role id array", K(need_flush), K(role_id_array), K(disable_flag_array), K(arg.default_role_flag_));
|
|
}
|
|
|
|
if (OB_SUCC(ret) && need_flush) {
|
|
if (!trans.is_started()) {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
}
|
|
if (OB_FAIL(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_SUCCESS == 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;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(user_info.assign(*user))) {
|
|
LOG_WARN("assign failed", K(ret));
|
|
} else if (OB_FALSE_IT(user_info.set_profile_id(profile_id))) {
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (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_SUCCESS == 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@xiyu: inside for ?
|
|
ddl_stmt_str.reuse();
|
|
ddl_sql.reset();
|
|
user_id = OB_INVALID_ID;
|
|
account.user_name_ = arg.users_.at(i);
|
|
account.host_name_ = arg.hosts_.at(i);;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
if (OB_FAIL(schema_guard.get_user_id(tenant_id, account.user_name_, account.host_name_, user_id))) {
|
|
LOG_WARN("get_user_id failed", K(ret), K(ret), K(account));
|
|
} else if (OB_INVALID_ID == user_id) {
|
|
ret = OB_SUCCESS; //no such user, recover
|
|
LOG_WARN("Try to lock/unlock non-exist user", K(tenant_id), K(account));
|
|
if (OB_FAIL(failed_index.push_back(i))) {
|
|
LOG_WARN("push_back failed", K(ret));
|
|
}
|
|
} else if (OB_FAIL(ObDDLSqlGenerator::gen_lock_user_sql(account, arg.locked_, ddl_stmt_str))) {
|
|
LOG_WARN("gen lock_user sql failed", K(ret), K(account), K(ret));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(lock_user_in_trans(tenant_id, user_id, arg.locked_, &ddl_sql, schema_guard))) {
|
|
LOG_WARN("Lock one user failed", K(tenant_id), K(user_id), K(account),
|
|
"locked", arg.locked_, K(ret));
|
|
ret = OB_SUCCESS; //lock fail, try next, recover
|
|
if (OB_FAIL(failed_index.push_back(i))) {
|
|
LOG_WARN("push_back failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::lock_user_in_trans(const uint64_t tenant_id,
|
|
const uint64_t user_id,
|
|
const bool locked,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant id is invalid", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.lock_user(tenant_id, user_id, locked, ddl_stmt_str, trans))) {
|
|
LOG_WARN("fail to lock user", K(ret), K(user_id), K(locked));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::standby_grant(const ObStandbyGrantArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObNeedPriv need_priv(arg.db_, arg.table_, arg.priv_level_, arg.priv_set_, false);
|
|
ObString ddl_sql;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(arg.tenant_id_, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(arg));
|
|
} else if (OB_PRIV_DB_LEVEL == arg.priv_level_) {
|
|
ObOriginalDBKey db_key(arg.tenant_id_, arg.user_id_, need_priv.db_);
|
|
if (OB_FAIL(grant_database(db_key, need_priv.priv_set_, &ddl_sql, schema_guard))) {
|
|
LOG_WARN("fail to grant database", KR(ret));
|
|
}
|
|
} else if (OB_PRIV_TABLE_LEVEL == arg.priv_level_) {
|
|
ObTablePrivSortKey table_key(arg.tenant_id_, arg.user_id_, need_priv.db_, need_priv.table_);
|
|
if (OB_FAIL(grant_table(table_key,
|
|
need_priv.priv_set_,
|
|
&ddl_sql,
|
|
schema_guard))) {
|
|
LOG_WARN("Grant table error", KR(ret), K(ddl_sql), K(table_key));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid priv level", KR(ret), K(arg));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::grant_database(
|
|
const share::schema::ObOriginalDBKey &db_key,
|
|
const ObPrivSet priv_set,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = db_key.tenant_id_;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if(!db_key.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("db_key is invalid", K(db_key), K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (!is_user_exist(db_key.tenant_id_, db_key.user_id_)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", db_key.tenant_id_,
|
|
"user_id", db_key.user_id_,
|
|
K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.grant_database(db_key, priv_set, ddl_stmt_str, trans))) {
|
|
LOG_WARN("fail to grant database", K(ret), K(db_key), K(priv_set));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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::revoke_database(
|
|
const share::schema::ObOriginalDBKey &db_key,
|
|
const ObPrivSet priv_set)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = db_key.tenant_id_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
uint64_t compat_version = 0;
|
|
bool is_ora_mode = false;
|
|
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(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_ora_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("fail to get data version", K(ret), K(tenant_id));
|
|
} else if (!ObSQLUtils::is_data_version_ge_422_or_431(compat_version) && !is_ora_mode
|
|
&& (0 != (priv_set & OB_PRIV_EXECUTE) ||
|
|
0 != (priv_set & OB_PRIV_ALTER_ROUTINE) ||
|
|
0 != (priv_set & OB_PRIV_CREATE_ROUTINE))) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("some column of user info is not empty when MIN_DATA_VERSION is below DATA_VERSION_4_3_1_0 or DATA_VERSION_4_2_2_0", K(ret), K(priv_set));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "revoke execute/alter routine/create routine privilege");
|
|
} else if (!ObSQLUtils::is_data_version_ge_424_or_433(compat_version) && !is_ora_mode
|
|
&& (0 != (priv_set & OB_PRIV_REFERENCES) ||
|
|
0 != (priv_set & OB_PRIV_TRIGGER))) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("some column of user info is not empty when MIN_DATA_VERSION is below DATA_VERSION_4_2_4_0 or DATA_VERSION_4_3_3_0", K(ret), K(priv_set));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "revoke references/trigger");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(db_key.tenant_id_, db_key.user_id_)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", db_key.tenant_id_,
|
|
"user_id", db_key.user_id_,
|
|
K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.revoke_database(db_key, priv_set, trans))) {
|
|
LOG_WARN("fail to revoke database", K(ret), K(db_key), K(priv_set));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::revoke_syspriv(
|
|
const uint64_t tenant_id,
|
|
const uint64_t grantee_id,
|
|
const share::ObRawPrivArray &sys_priv_array,
|
|
const common::ObSArray<uint64_t> &role_ids,
|
|
const common::ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObSysPriv *sys_priv = NULL;
|
|
int64_t refreshed_schema_version = 0;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id,
|
|
schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(tenant_id, grantee_id)) {
|
|
/* error because: grantee schema error */
|
|
ret = OB_ERR_USER_NOT_EXIST;
|
|
LOG_WARN("User or Role is not exist", "tenant_id", tenant_id,
|
|
"user_id", grantee_id,
|
|
K(ret));
|
|
}
|
|
OZ (schema_guard.get_sys_priv_with_grantee_id(tenant_id,
|
|
grantee_id,
|
|
sys_priv),
|
|
tenant_id, grantee_id);
|
|
OZ (trans.start(sql_proxy_, tenant_id, refreshed_schema_version));
|
|
// revoke sys_priv
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_SUCC(ret) && false == sys_priv_array.empty()) {
|
|
OZ (ddl_operator.grant_sys_priv_to_ur(tenant_id,
|
|
grantee_id,
|
|
sys_priv,
|
|
NO_OPTION,
|
|
sys_priv_array,
|
|
trans,
|
|
false /*is_grant*/,
|
|
ddl_stmt_str,
|
|
schema_guard),
|
|
tenant_id, grantee_id, sys_priv_array);
|
|
}
|
|
// revoke role
|
|
if (OB_SUCC(ret) && false == role_ids.empty()) {
|
|
bool log_operation = true;
|
|
if (true == sys_priv_array.empty()) {
|
|
// ddl_stmt_str was logged in grant_sys_priv_to_ur before
|
|
log_operation = true;
|
|
} else {
|
|
// no sys_priv, sql_string shall be logged in revoke_role_inner_trans
|
|
log_operation = false;
|
|
}
|
|
if (OB_FAIL(revoke_role_inner_trans(ddl_operator,
|
|
trans,
|
|
schema_guard,
|
|
tenant_id,
|
|
grantee_id,
|
|
role_ids,
|
|
log_operation))) {
|
|
LOG_WARN("revoke_role_inner_trans failed", K(ret),
|
|
K(tenant_id), K(grantee_id), K(role_ids));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::revoke_role_inner_trans(
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const uint64_t user_id,
|
|
const common::ObSArray<uint64_t> &role_ids,
|
|
const bool log_operation)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObUserInfo *user_info = NULL;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id
|
|
|| OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(tenant_id), K(user_id));
|
|
} else if (OB_FAIL(schema_guard.get_user_info(tenant_id, user_id, user_info))) {
|
|
LOG_WARN("get_user_info failed", K(tenant_id), K(user_id), K(ret));
|
|
} else if (OB_ISNULL(user_info)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("user not exist", K(ret), K(user_id));
|
|
} else if (role_ids.count() > 0) {
|
|
ObArray<uint64_t> role_id_array;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < role_ids.count(); ++i) {
|
|
const uint64_t role_id = role_ids.at(i);
|
|
const ObUserInfo *role_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, role_id, role_info))) {
|
|
LOG_WARN("Failed to get role info", K(ret), K(tenant_id), K(role_id));
|
|
} else if (NULL == role_info) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("role doesn't exist", K(ret), K(role_id));
|
|
} else {
|
|
// Determine whether the current role is granted to the user; otherwise, it will not be processed
|
|
ObSEArray<uint64_t, 8> role_id_array_from_user = user_info->get_role_id_array();
|
|
for (int j = 0, found_role = 0;
|
|
OB_SUCC(ret) && 0 == found_role && j < role_id_array_from_user.count(); ++j) {
|
|
if (role_info->get_user_id() == role_id_array_from_user.at(j)) {
|
|
if (OB_FAIL(role_id_array.push_back(role_id_array_from_user.at(j)))) {
|
|
LOG_WARN("Failed to push back role_id", K(ret), K(tenant_id), K(*role_info));
|
|
}
|
|
found_role = 1;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (role_id_array.count() > 0) {
|
|
if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id,
|
|
*user_info,
|
|
role_id_array,
|
|
NULL, /*Single specified role info*/
|
|
trans,
|
|
log_operation, /*log_operation*/
|
|
false, /*is_grant*/
|
|
NO_OPTION))) {
|
|
LOG_WARN("failed to grant_revoke_role", K(ret), K(tenant_id), K(*user_info));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::grant_table(
|
|
const share::schema::ObTablePrivSortKey &table_key,
|
|
const ObPrivSet priv_set,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = table_key.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (!table_key.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table_key is invalid", K(table_key), K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(table_key.tenant_id_, table_key.user_id_)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", table_key.tenant_id_,
|
|
"user_id", table_key.user_id_,
|
|
K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
|
|
if (OB_FAIL(ddl_operator.grant_table(table_key,
|
|
priv_set,
|
|
ddl_stmt_str,
|
|
trans,
|
|
share::ObRawObjPrivArray(),
|
|
0,
|
|
ObObjPrivSortKey()))) {
|
|
LOG_WARN("fail to grant table", K(ret), K(table_key), K(priv_set));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// publish schema
|
|
if (OB_SUCC(ret)) {
|
|
ret = publish_schema(tenant_id);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::grant_table(
|
|
const share::schema::ObTablePrivSortKey &table_key,
|
|
const ObPrivSet priv_set,
|
|
const ObString *ddl_stmt_str,
|
|
const share::ObRawObjPrivArray &obj_priv_array,
|
|
const uint64_t option,
|
|
const share::schema::ObObjPrivSortKey &obj_key,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const common::ObString &grantor,
|
|
const common::ObString &grantor_host)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = table_key.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (!table_key.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table_key is invalid", K(table_key), K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(table_key.tenant_id_, table_key.user_id_)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", table_key.tenant_id_,
|
|
"user_id", table_key.user_id_,
|
|
K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.grant_table(table_key,
|
|
priv_set,
|
|
ddl_stmt_str,
|
|
trans,
|
|
obj_priv_array,
|
|
option,
|
|
obj_key,
|
|
grantor,
|
|
grantor_host))) {
|
|
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_SUCCESS == 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::revoke_table_and_column_mysql(const obrpc::ObRevokeTableArg& arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
uint64_t compat_version = 0;
|
|
bool is_ora_mode = false;
|
|
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(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_ora_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("fail to get data version", K(ret), K(tenant_id));
|
|
} else if (!ObSQLUtils::is_data_version_ge_424_or_433(compat_version) && !is_ora_mode
|
|
&& (0 != (arg.priv_set_ & OB_PRIV_TRIGGER))) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("some column of user info is not empty when MIN_DATA_VERSION is below DATA_VERSION_4_2_4_0 or DATA_VERSION_4_3_3_0", K(ret), K(arg.priv_set_));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "revoke table level trigger privilege");
|
|
} 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 (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObPrivSet priv_set = arg.priv_set_;
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObTablePrivSortKey table_priv_key(arg.tenant_id_, arg.user_id_, arg.db_, arg.table_);
|
|
ObObjPrivSortKey obj_priv_key(arg.tenant_id_,
|
|
arg.obj_id_,
|
|
arg.obj_type_,
|
|
OBJ_LEVEL_FOR_TAB_PRIV,
|
|
arg.grantor_id_,
|
|
arg.user_id_);
|
|
share::ObRawObjPrivArray obj_priv_array; //useless
|
|
if (priv_set != 0 && OB_FAIL(ddl_operator.revoke_table(table_priv_key, priv_set, trans,
|
|
obj_priv_key, obj_priv_array, false,
|
|
arg.grantor_, arg.grantor_host_))) {
|
|
LOG_WARN("fail to revoke table", K(ret), K(table_priv_key), K(priv_set));
|
|
} else {
|
|
ObSEArray<std::pair<ObString, ObPrivType>, 4> column_names_priv;
|
|
if (OB_FAIL(append(column_names_priv, arg.column_names_priv_))) {
|
|
LOG_WARN("append failed", K(ret));
|
|
} else if ((priv_set & OB_PRIV_SELECT) != 0
|
|
|| (priv_set & OB_PRIV_INSERT) != 0
|
|
|| (priv_set & OB_PRIV_UPDATE) != 0
|
|
|| (priv_set & OB_PRIV_REFERENCES) != 0) {
|
|
ObSEArray<const ObColumnPriv *, 4> column_privs;
|
|
if (OB_FAIL(schema_guard.get_column_priv_in_table(tenant_id, arg.user_id_, arg.db_, arg.table_, column_privs))) {
|
|
LOG_WARN("get column priv in table failed", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < column_privs.count(); i++) {
|
|
if (OB_ISNULL(column_privs.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column priv is null", K(ret));
|
|
} else if ((priv_set & OB_PRIV_SELECT) != 0
|
|
&& (column_privs.at(i)->get_priv_set() & OB_PRIV_SELECT) != 0) {
|
|
if (OB_FAIL(column_names_priv.push_back(
|
|
std::make_pair(column_privs.at(i)->get_column_name_str(), OB_PRIV_SELECT)))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if ((priv_set & OB_PRIV_INSERT) != 0
|
|
&& (column_privs.at(i)->get_priv_set() & OB_PRIV_INSERT) != 0) {
|
|
if (OB_FAIL(column_names_priv.push_back(
|
|
std::make_pair(column_privs.at(i)->get_column_name_str(), OB_PRIV_INSERT)))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if ((priv_set & OB_PRIV_UPDATE) != 0
|
|
&& (column_privs.at(i)->get_priv_set() & OB_PRIV_UPDATE) != 0) {
|
|
if (OB_FAIL(column_names_priv.push_back(
|
|
std::make_pair(column_privs.at(i)->get_column_name_str(), OB_PRIV_UPDATE)))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if ((priv_set & OB_PRIV_REFERENCES) != 0
|
|
&& (column_privs.at(i)->get_priv_set() & OB_PRIV_REFERENCES) != 0) {
|
|
if (OB_FAIL(column_names_priv.push_back(
|
|
std::make_pair(column_privs.at(i)->get_column_name_str(), OB_PRIV_REFERENCES)))) {
|
|
LOG_WARN("push back failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
const ObUserInfo *user_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.user_id_, user_info))) {
|
|
LOG_WARN("get user info failed", K(table_priv_key), K(ret));
|
|
} else if (OB_ISNULL(user_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("user not exist", K(table_priv_key), K(ret));
|
|
} else if (OB_FAIL(grant_or_revoke_column_priv_mysql(tenant_id, arg.obj_id_, arg.user_id_,
|
|
user_info->get_user_name_str(),
|
|
user_info->get_host_name_str(),
|
|
arg.db_, arg.table_, column_names_priv,
|
|
ddl_operator, trans, schema_guard, false))) {
|
|
LOG_WARN("grant or revoke column priv mysql 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_SUCCESS == 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::revoke_table(
|
|
const obrpc::ObRevokeTableArg &arg,
|
|
const share::schema::ObTablePrivSortKey &table_key,
|
|
const ObPrivSet priv_set,
|
|
const share::schema::ObObjPrivSortKey &obj_key,
|
|
const share::ObRawObjPrivArray &obj_priv_array,
|
|
const bool revoke_all_ora)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = table_key.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (!table_key.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table_key is invalid", K(table_key), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_parallel_ddl_conflict(schema_guard, arg))) {
|
|
LOG_WARN("check parallel ddl conflict failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(table_key.tenant_id_, table_key.user_id_)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", table_key.tenant_id_,
|
|
"user_id", table_key.user_id_,
|
|
K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.revoke_table(table_key, priv_set, trans,
|
|
obj_key, obj_priv_array, revoke_all_ora))) {
|
|
LOG_WARN("fail to revoke table", K(ret), K(table_key), K(priv_set));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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::revoke_routine(
|
|
const share::schema::ObRoutinePrivSortKey &routine_key,
|
|
const ObPrivSet priv_set,
|
|
const common::ObString &grantor,
|
|
const common::ObString &grantor_host)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = routine_key.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
uint64_t compat_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("fail to get data version", K(ret), K(tenant_id));
|
|
} else if (!ObSQLUtils::is_data_version_ge_422_or_431(compat_version)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("version lower than 4.3.1 or 4.2.2 does not support this operation", K(ret));
|
|
} else if (!routine_key.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("routine_key is invalid", K(routine_key), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(routine_key.tenant_id_, routine_key.user_id_)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", routine_key.tenant_id_,
|
|
"user_id", routine_key.user_id_,
|
|
K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.revoke_routine(routine_key, priv_set, trans, true, true, grantor, grantor_host))) {
|
|
LOG_WARN("fail to revoke routine", K(ret), K(routine_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_SUCCESS == 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_routine(
|
|
const share::schema::ObRoutinePrivSortKey &routine_key,
|
|
const ObPrivSet priv_set,
|
|
const ObString *ddl_stmt_str,
|
|
const uint64_t option,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const common::ObString &grantor,
|
|
const common::ObString &grantor_host)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = routine_key.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
uint64_t compat_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, compat_version))) {
|
|
LOG_WARN("fail to get data version", K(ret), K(tenant_id));
|
|
} else if (!ObSQLUtils::is_data_version_ge_422_or_431(compat_version)) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("version lower than 4.3.1 or 4.2.2 does not support this operation", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (!routine_key.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("routine_key is invalid", K(routine_key), K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (!is_user_exist(routine_key.tenant_id_, routine_key.user_id_)) {
|
|
ret = OB_USER_NOT_EXIST;
|
|
LOG_WARN("User is not exist", "tenant_id", routine_key.tenant_id_,
|
|
"user_id", routine_key.user_id_,
|
|
K(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.grant_routine(routine_key,
|
|
priv_set,
|
|
trans,
|
|
option,
|
|
true,
|
|
grantor,
|
|
grantor_host))) {
|
|
LOG_WARN("fail to grant routine", K(ret), K(routine_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_SUCCESS == 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.is_format() && outline_info.get_signature_str().empty() &&
|
|
!ObOutlineInfo::is_sql_id_valid(outline_info.get_sql_id_str()))
|
|
|| (outline_info.is_format() && outline_info.get_format_sql_text_str().empty() &&
|
|
!ObOutlineInfo::is_sql_id_valid(outline_info.get_format_sql_id_str())))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(outline_info.is_format()), K(outline_info.get_format_sql_text_str().empty()), K(ObOutlineInfo::is_sql_id_valid(outline_info.get_format_sql_id_str())), 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,
|
|
outline_info.is_format(),
|
|
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(),
|
|
outline_info.is_format(),
|
|
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.is_format() ? outline_info.get_format_sql_id_str() : outline_info.get_sql_id_str()),
|
|
outline_info.is_format(),
|
|
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(),
|
|
outline_info.is_format(),
|
|
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) {
|
|
ObString outline_name;
|
|
outline_name = outline_info.is_format() ? outline_info.get_format_sql_text_str() : outline_info.get_sql_text_str();
|
|
ret = OB_ERR_OUTLINE_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_OUTLINE_EXIST, outline_name.length(),
|
|
outline_name.ptr());
|
|
} else {/*do nothing*/}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_outline(share::schema::ObOutlineInfo &outline_info,
|
|
const bool is_update,
|
|
const common::ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = outline_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} // else if (!outline_info.is_valid()) {
|
|
// ret = OB_INVALID_ARGUMENT;
|
|
// LOG_WARN("invalid argument", K(outline_info), K(is_update), K(ret));
|
|
// }
|
|
else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
if (!is_update) {
|
|
ret = ddl_operator.create_outline(outline_info, trans, ddl_stmt_str);
|
|
} else {
|
|
ret = ddl_operator.replace_outline(outline_info, trans, ddl_stmt_str);
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("failed to create outline", K(outline_info), K(is_update), K(ret));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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;
|
|
bool is_format = alter_outline_info.is_format();
|
|
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,
|
|
is_format,
|
|
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 (OB_FAIL(new_outline_info.get_err_ret())) {
|
|
LOG_WARN("copy assign failed", KR(ret), KPC(orig_outline_info));
|
|
} else if (alter_outline_info.get_alter_option_bitset().has_member(
|
|
obrpc::ObAlterOutlineArg::ADD_OUTLINE_CONTENT)) {
|
|
//add outline_content
|
|
const ObString &orig_outline_content = orig_outline_info->get_outline_content_str();
|
|
if (!orig_outline_content.empty()) {
|
|
ret = OB_ERR_OUTLINE_CONTENT_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_OUTLINE_CONTENT_EXIST, orig_outline_content.length(),
|
|
orig_outline_content.ptr(), outline_name.length(),
|
|
outline_name.ptr());
|
|
LOG_WARN("the outline has already has outline content", K(new_outline_info), K(ret));
|
|
} else if (alter_outline_info.get_outline_content_str().empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("the outline content should not be empty", K(alter_outline_info), K(ret));
|
|
} else {
|
|
new_outline_info.set_outline_content(alter_outline_info.get_outline_content_str());
|
|
// Only this place is set to sql_text, to ensure that if there is an outline on the sql statement,
|
|
// the content in sql_text is the sql that created the outline
|
|
new_outline_info.set_sql_text(alter_outline_info.get_sql_text_str());
|
|
}
|
|
} else if (alter_outline_info.get_alter_option_bitset().has_member(
|
|
obrpc::ObAlterOutlineArg::ADD_CONCURRENT_LIMIT)) {
|
|
//add outline_max_concurrent
|
|
const ObOutlineParamsWrapper &outline_params_wrapper = alter_outline_info.get_outline_params_wrapper();
|
|
if (OB_UNLIKELY(1 != outline_params_wrapper.get_outline_params().count())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid alter_outline_info, count of outline_params should be 1",
|
|
K(alter_outline_info), K(ret));
|
|
} else {
|
|
ObMaxConcurrentParam *concurrent_param = outline_params_wrapper.get_outline_params().at(0);
|
|
bool has_param = false;
|
|
if (OB_ISNULL(concurrent_param)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("concurrent_param is NULL", K(ret));
|
|
} else if (OB_FAIL(orig_outline_info->get_outline_params_wrapper().has_param(*concurrent_param, has_param))) {
|
|
LOG_WARN("failed to check if has param", K(ret));
|
|
} else if (has_param) {
|
|
ret = OB_ERR_OUTLINE_MAX_CONCURRENT_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_OUTLINE_MAX_CONCURRENT_EXIST, outline_name.length(), outline_name.ptr());
|
|
} else if (OB_FAIL(new_outline_info.get_outline_params_wrapper().add_param(*concurrent_param))) {
|
|
LOG_WARN("failed to add param", K(ret));
|
|
} else {/*do nothing*/}
|
|
}
|
|
} else {/*do nothing*/}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.alter_outline(new_outline_info, trans, &arg.ddl_stmt_str_))) {
|
|
LOG_WARN("failed to alter outline", K(new_outline_info), K(ret));
|
|
} else {/*do nothing*/}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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 = 0;
|
|
int64_t end_usec = 0;
|
|
int64_t cost_usec = 0;
|
|
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_MOCK_DEFAULT_DATABASE_NAME) {
|
|
database_id = OB_MOCK_DEFAULT_DATABASE_ID;
|
|
database_exist = true;
|
|
} else if (OB_FAIL(schema_service_->check_database_exist(tenant_id,
|
|
database_name,
|
|
database_id,
|
|
database_exist))) {
|
|
LOG_WARN("failed to check database exist!", K(tenant_id), K(database_name),
|
|
K(database_id), K(database_exist), K(ret));
|
|
} else if (!database_exist) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr());
|
|
LOG_WARN("database not exist!", K(arg), K(ret));
|
|
} else if (OB_INVALID_ID == database_id) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr());
|
|
LOG_WARN("database is invalid!", K(arg), K(ret));
|
|
}
|
|
}
|
|
|
|
bool outline_exist = false;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(ret)) {
|
|
//do nothing
|
|
} else if (OB_FAIL(schema_service_->check_outline_exist_with_name(tenant_id, database_id,
|
|
outline_name, outline_id,
|
|
arg.is_format_, outline_exist))) {
|
|
LOG_WARN("check_outline_exist failed", K(tenant_id), K(database_name), K(outline_name), K(ret));
|
|
} else if (!outline_exist) {
|
|
ret = OB_OUTLINE_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_OUTLINE_NOT_EXIST, database_name.length(), database_name.ptr(), outline_name.length(), outline_name.ptr());
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ddl_operator.drop_outline(tenant_id,
|
|
database_id,
|
|
outline_id,
|
|
trans,
|
|
&arg.ddl_stmt_str_))) {
|
|
LOG_WARN("ddl_operator drop_outline failed", K(tenant_id), KT(outline_id), K(ret));
|
|
} else {/*do nothing*/}
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
|
|
LOG_INFO("finish drop outline", K(tenant_id), K(database_name), K(outline_name), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_routine(ObRoutineInfo &routine_info,
|
|
const ObRoutineInfo* old_routine_info,
|
|
bool replace,
|
|
ObErrorInfo &error_info,
|
|
ObIArray<ObDependencyInfo> &dep_infos,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
CK((replace && OB_NOT_NULL(old_routine_info)) || (!replace && OB_ISNULL(old_routine_info)));
|
|
const uint64_t tenant_id = routine_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (replace) {
|
|
if (OB_FAIL(ddl_operator.replace_routine(routine_info,
|
|
old_routine_info,
|
|
trans,
|
|
error_info,
|
|
dep_infos,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("replace routine failded", K(routine_info), K(ret));
|
|
}
|
|
} else {
|
|
if (OB_FAIL(ddl_operator.create_routine(routine_info, trans,
|
|
error_info, dep_infos, ddl_stmt_str))) {
|
|
LOG_WARN("create procedure failed", K(ret), K(routine_info));
|
|
}
|
|
}
|
|
}
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
uint64_t data_version = 0;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (replace) {
|
|
} 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 (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
|
|
LOG_WARN("fail to get data version", K(tenant_id));
|
|
} else if (ObSQLUtils::is_data_version_ge_422_or_431(data_version) && lib::Worker::CompatMode::MYSQL == compat_mode) {
|
|
const ObSysVarSchema *sys_var = NULL;
|
|
ObMalloc alloc(ObModIds::OB_TEMP_VARIABLES);
|
|
ObObj val;
|
|
if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id, SYS_VAR_AUTOMATIC_SP_PRIVILEGES, sys_var))) {
|
|
LOG_WARN("fail to get tenant var schema", K(ret));
|
|
} else if (OB_ISNULL(sys_var)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys variable schema is null", KR(ret));
|
|
} else if (OB_FAIL(sys_var->get_value(&alloc, NULL, val))) {
|
|
LOG_WARN("fail to get charset var value", K(ret));
|
|
} else {
|
|
bool grant_priv = val.get_bool();
|
|
if (grant_priv) {
|
|
int64_t db_id = routine_info.get_database_id();
|
|
const ObDatabaseSchema* database_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, db_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 should not be null", K(ret));
|
|
} else {
|
|
ObRoutinePrivSortKey routine_key(tenant_id, routine_info.get_owner_id(),
|
|
database_schema->get_database_name_str(),
|
|
routine_info.get_routine_name(), routine_info.is_procedure() ?
|
|
ObRoutineType::ROUTINE_PROCEDURE_TYPE : ObRoutineType::ROUTINE_FUNCTION_TYPE);
|
|
ObPrivSet priv_set = (OB_PRIV_EXECUTE | OB_PRIV_ALTER_ROUTINE);
|
|
int64_t option = 0;
|
|
const bool gen_ddl_stmt = false;
|
|
const ObUserInfo *user_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id,
|
|
routine_info.get_owner_id(),
|
|
user_info))) {
|
|
LOG_WARN("failed to get user info", K(ret));
|
|
} else if (OB_ISNULL(user_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("user info is unexpeced null", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.grant_routine(routine_key,
|
|
priv_set,
|
|
trans,
|
|
option,
|
|
gen_ddl_stmt,
|
|
user_info->get_user_name_str(),
|
|
user_info->get_host_name_str()))) {
|
|
LOG_WARN("fail to grant routine", K(ret), K(routine_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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_routine(const ObRoutineInfo &routine_info,
|
|
ObErrorInfo &error_info,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = routine_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed!", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObDependencyInfo::modify_dep_obj_status(trans, tenant_id, routine_info.get_routine_id(),
|
|
ddl_operator, *schema_service_))) {
|
|
LOG_WARN("failed to modify obj status", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_routine(
|
|
routine_info, trans, error_info, ddl_stmt_str))) {
|
|
LOG_WARN("alter routine failed!", K(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed!", K(ret), K(temp_ret));
|
|
ret = OB_SUCCESS == ret ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed!", K(ret), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_routine(const ObRoutineInfo &routine_info,
|
|
ObErrorInfo &error_info, const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = routine_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObDependencyInfo::modify_dep_obj_status(trans, tenant_id, routine_info.get_routine_id(),
|
|
ddl_operator, *schema_service_))) {
|
|
LOG_WARN("failed to modify obj status", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_routine(routine_info, trans, error_info, ddl_stmt_str))) {
|
|
LOG_WARN("drop procedure failed", K(ret), K(routine_info));
|
|
} else {
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
uint64_t data_version = 0;
|
|
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 (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
|
|
LOG_WARN("fail to get data version", K(tenant_id));
|
|
} else if (ObSQLUtils::is_data_version_ge_422_or_431(data_version) && lib::Worker::CompatMode::MYSQL == compat_mode) {
|
|
const ObSysVarSchema *sys_var = NULL;
|
|
ObMalloc alloc(ObModIds::OB_TEMP_VARIABLES);
|
|
ObObj val;
|
|
if (OB_FAIL(schema_guard.get_tenant_system_variable(tenant_id, SYS_VAR_AUTOMATIC_SP_PRIVILEGES, sys_var))) {
|
|
LOG_WARN("fail to get tenant var schema", K(ret));
|
|
} else if (OB_ISNULL(sys_var)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys variable schema is null", KR(ret));
|
|
} else if (OB_FAIL(sys_var->get_value(&alloc, NULL, val))) {
|
|
LOG_WARN("fail to get charset var value", K(ret));
|
|
} else if (val.get_bool()) {
|
|
int64_t db_id = routine_info.get_database_id();
|
|
const ObDatabaseSchema* database_schema = NULL;
|
|
ObSEArray<const ObUserInfo*, 10> user_infos;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, db_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 (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 = user_infos.at(i);
|
|
if (OB_ISNULL(user_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", K(ret));
|
|
} else {
|
|
ObRoutinePrivSortKey routine_key(tenant_id, user_info->get_user_id(),
|
|
database_schema->get_database_name_str(),
|
|
routine_info.get_routine_name(), routine_info.is_procedure() ?
|
|
ObRoutineType::ROUTINE_PROCEDURE_TYPE : ObRoutineType::ROUTINE_FUNCTION_TYPE);
|
|
ObPrivSet priv_set = (OB_PRIV_EXECUTE | OB_PRIV_ALTER_ROUTINE);
|
|
bool gen_ddl_stmt = false;
|
|
if (OB_FAIL(ddl_operator.revoke_routine(routine_key, priv_set, trans, false, gen_ddl_stmt))) {
|
|
LOG_WARN("fail to grant routine", K(ret), K(routine_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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_udt(ObUDTTypeInfo &udt_info,
|
|
const ObUDTTypeInfo* old_udt_info,
|
|
ObIArray<ObRoutineInfo> &public_routine_infos,
|
|
ObErrorInfo &error_info,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObIArray<ObDependencyInfo> &dep_infos,
|
|
const ObString *ddl_stmt_str,
|
|
bool need_replace,
|
|
bool exist_valid_udt,
|
|
bool specify_force)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = udt_info.get_tenant_id();
|
|
if (need_replace == OB_ISNULL(old_udt_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("value contradict", K(ret), K(need_replace), K(old_udt_info));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
|
|
bool udt_dependency_feature_enabled =
|
|
GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_3_1_0 ||
|
|
(GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_2_2_0 &&
|
|
GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_3_0_0);
|
|
if (OB_FAIL(ret)) {
|
|
} else if (need_replace) {
|
|
if (udt_dependency_feature_enabled && exist_valid_udt) {
|
|
// check if the udt which is to be replaced has any type or table dependent
|
|
ObArray<CriticalDepInfo> objs;
|
|
if (OB_ISNULL(old_udt_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("value contradict", K(ret), K(need_replace), K(old_udt_info));
|
|
} else if (OB_FAIL(ObDependencyInfo::collect_all_dep_objs(tenant_id,
|
|
old_udt_info->get_type_id(),
|
|
udt_info.get_object_type(),
|
|
trans,
|
|
objs))) {
|
|
// TODO: type body id design flaw
|
|
// Here we pass `udt_info.get_object_type()` to `collect_all_dep_objs` instead of
|
|
// `old_udt_info->get_object_type()`, because the type body and type share the same id
|
|
// (which is a design flaw that needs to be fixed), and the schema cache uses the object
|
|
// id as the key for storage, which casues retrieving the wrong schema from the cache.
|
|
LOG_WARN("failed to collect all dependent objects", K(ret));
|
|
} else {
|
|
bool has_type_dep_obj = false;
|
|
bool has_table_dep_obj = false;
|
|
for (int64_t i = 0; i < objs.count(); i++) {
|
|
schema::ObObjectType dep_obj_type =
|
|
static_cast<schema::ObObjectType>(objs.at(i).element<1>());
|
|
if (schema::ObObjectType::TABLE == dep_obj_type) {
|
|
has_table_dep_obj = true;
|
|
} else if (schema::ObObjectType::TYPE == dep_obj_type) {
|
|
has_type_dep_obj = true;
|
|
}
|
|
}
|
|
if (!has_type_dep_obj && !has_table_dep_obj) {
|
|
// pass
|
|
} else if (!specify_force && (has_type_dep_obj || has_table_dep_obj)) {
|
|
ret = OB_ERR_HAS_TYPE_OR_TABLE_DEPENDENT;
|
|
LOG_WARN("cannot drop or replace a type with type or table dependents",
|
|
K(ret), K(objs));
|
|
} else if (specify_force && has_table_dep_obj) {
|
|
// create or replace type force will not replace types with table dependents,
|
|
// although ob does not support udt column in tables for now.
|
|
ret = OB_ERR_REPLACE_TYPE_WITH_TABLE_DEPENDENT;
|
|
LOG_WARN("cannot replace a type with table dependents", K(ret), K(objs));
|
|
} else if (specify_force && has_type_dep_obj) {
|
|
// pass
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& OB_FAIL(ObDependencyInfo::batch_invalidate_dependents(
|
|
objs, trans, tenant_id, old_udt_info->get_type_id()))) {
|
|
LOG_WARN("invalidate dependents failed");
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ddl_operator.replace_udt(udt_info,
|
|
old_udt_info,
|
|
trans,
|
|
error_info,
|
|
public_routine_infos,
|
|
schema_guard,
|
|
dep_infos,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("replace udt failded", K(udt_info), K(ret));
|
|
}
|
|
} else {
|
|
if (udt_dependency_feature_enabled && exist_valid_udt) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("value contradict", K(ret), K(need_replace), K(exist_valid_udt));
|
|
} else if (OB_FAIL(ddl_operator.create_udt(udt_info,
|
|
trans,
|
|
error_info,
|
|
public_routine_infos,
|
|
schema_guard,
|
|
dep_infos,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("create udt failed", K(ret), K(udt_info));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_udt(const ObUDTTypeInfo &udt_info,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObString *ddl_stmt_str,
|
|
bool specify_force,
|
|
bool exist_valid_udt)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = udt_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
|
|
if (!(GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_3_1_0 ||
|
|
(GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_2_2_0 &&
|
|
GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_3_0_0)
|
|
/*udt dependency feature enabled*/) ||
|
|
!exist_valid_udt) {
|
|
// 1) compatible with version without udt dependency feature
|
|
// 2) otherwise depednency should have been invalid already
|
|
} else {
|
|
// check if the udt which is to be drop has any type or table dependent
|
|
bool has_type_or_table_dep_obj = false;
|
|
ObArray<CriticalDepInfo> objs;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ObDependencyInfo::collect_all_dep_objs(tenant_id,
|
|
udt_info.get_type_id(),
|
|
udt_info.get_object_type(),
|
|
trans,
|
|
objs))) {
|
|
LOG_WARN("failed to collect all dependent objects");
|
|
} else {
|
|
for (int64_t i = 0; i < objs.count(); i++) {
|
|
schema::ObObjectType dep_obj_type =
|
|
static_cast<schema::ObObjectType>(objs.at(i).element<1>());
|
|
if (schema::ObObjectType::TABLE == dep_obj_type
|
|
|| schema::ObObjectType::TYPE == dep_obj_type) {
|
|
has_type_or_table_dep_obj = true;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (!specify_force && has_type_or_table_dep_obj) {
|
|
ret = OB_ERR_HAS_TYPE_OR_TABLE_DEPENDENT;
|
|
LOG_WARN("cannot drop or replace a type with type or table dependents", K(ret));
|
|
} else if ((specify_force || !has_type_or_table_dep_obj)
|
|
&& OB_FAIL(ObDependencyInfo::batch_invalidate_dependents(
|
|
objs, trans, tenant_id, udt_info.get_type_id()))) {
|
|
LOG_WARN("invalidate dependents failed");
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ObDependencyInfo::modify_dep_obj_status(trans,
|
|
tenant_id,
|
|
udt_info.get_type_id(),
|
|
ddl_operator,
|
|
*schema_service_))) {
|
|
LOG_WARN("failed to modify obj status", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_udt(udt_info, trans, schema_guard, ddl_stmt_str))) {
|
|
LOG_WARN("drop procedure failed", K(ret), K(udt_info));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
//----Functions for managing dblinks----
|
|
int ObDDLService::create_dblink(const obrpc::ObCreateDbLinkArg &arg, const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDbLinkInfo dblink_info = arg.dblink_info_;
|
|
uint64_t tenant_id = dblink_info.get_tenant_id();
|
|
const ObString &dblink_name = dblink_info.get_dblink_name();
|
|
ObSchemaGetterGuard schema_guard;
|
|
bool is_exist = false;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.check_dblink_exist(tenant_id, dblink_name, is_exist))) {
|
|
LOG_WARN("failed to check dblink exist", K(ret), K(dblink_info));
|
|
} else if (is_exist) {
|
|
if (arg.dblink_info_.get_if_not_exist()) {
|
|
// do nothing
|
|
} else {
|
|
ret = OB_OBJ_ALREADY_EXIST;
|
|
LOG_WARN("dblink already exist", K(ret), K(dblink_info.get_dblink_name()));
|
|
}
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.create_dblink(dblink_info, trans, ddl_stmt_str))) {
|
|
LOG_WARN("failed to create dblink", K(ret), K(dblink_info));
|
|
}
|
|
if (trans.is_started()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("failed to end trans", K(tmp_ret), "is_commit", OB_SUCCESS == 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)) {
|
|
if (arg.if_exist_) {
|
|
// do nothing
|
|
LOG_WARN("loglcq dblink not exist", K(ret), K(tenant_id), K(dblink_name));
|
|
} else {
|
|
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_);
|
|
int64_t refreshed_schema_version = 0;
|
|
ObDbLinkSchema dblink_info;
|
|
if (OB_FAIL(dblink_info.assign(*dblink_schema))) {
|
|
LOG_WARN("fail to assign ObDbLinkSchema", KR(ret));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.drop_dblink(dblink_info, trans, ddl_stmt_str))) {
|
|
LOG_WARN("failed to drop dblink", K(ret), K(dblink_info));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("failed to end trans", K(temp_ret), "is_commit", OB_SUCCESS == 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 ObDependencyInfo &dep_info,
|
|
const common::ObString *ddl_stmt_str,
|
|
bool is_update,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = synonym_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
if (!is_update) {
|
|
ret = ddl_operator.create_synonym(synonym_info, trans, ddl_stmt_str);
|
|
} else {
|
|
ret = ddl_operator.replace_synonym(synonym_info, trans, ddl_stmt_str);
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("failed to create synonym", K(synonym_info), K(is_update), K(ret));
|
|
} else if (OB_INVALID_ID != dep_info.get_ref_obj_id()) {
|
|
ObDependencyInfo dep;
|
|
if (OB_FAIL(dep.assign(dep_info))) {
|
|
LOG_WARN("failed to assign dependency info", K(ret));
|
|
} else {
|
|
dep.set_tenant_id(tenant_id);
|
|
dep.set_dep_obj_id(synonym_info.get_synonym_id());
|
|
dep.set_dep_obj_owner_id(synonym_info.get_synonym_id());
|
|
dep.set_schema_version(synonym_info.get_schema_version());
|
|
OZ (dep.insert_schema_object_dependency(trans, is_update));
|
|
}
|
|
}
|
|
}
|
|
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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
LOG_INFO("finish create synonym", K(synonym_info), K(is_update), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_synonym(const obrpc::ObDropSynonymArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const ObString &database_name = arg.db_name_;
|
|
const ObString &synonym_name = arg.synonym_name_;
|
|
uint64_t database_id = OB_INVALID_ID;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool database_exist = false;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_UNLIKELY(false == arg.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", K(arg), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_service_->check_database_exist(tenant_id,
|
|
database_name,
|
|
database_id,
|
|
database_exist))) {
|
|
LOG_WARN("failed to check database exist!", K(tenant_id), K(database_name),
|
|
K(database_id), K(database_exist), K(ret));
|
|
} else if (OB_UNLIKELY(!database_exist)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr());
|
|
LOG_WARN("database not exist!", K(arg), K(ret));
|
|
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id)) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr());
|
|
LOG_WARN("database is invalid!", K(arg), K(ret));
|
|
}
|
|
|
|
//check synonym exist & drop synonym
|
|
if (OB_SUCC(ret)) {
|
|
bool is_exist = false;
|
|
uint64_t synonym_id = OB_INVALID_ID;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_service_->check_synonym_exist(tenant_id, database_id, synonym_name,
|
|
is_exist, synonym_id))) {
|
|
LOG_WARN("check_synonym_exist failed", K(tenant_id), K(database_name), K(synonym_name), K(ret));
|
|
} else if (!is_exist) {
|
|
ret = OB_SYNONYM_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_SYNONYM_NOT_EXIST, database_name.length(), database_name.ptr(), synonym_name.length(), synonym_name.ptr());
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ObDependencyInfo::modify_dep_obj_status(trans, tenant_id, synonym_id,
|
|
ddl_operator, *schema_service_))) {
|
|
LOG_WARN("failed to modify obj status", K(ret));
|
|
} else 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 if (OB_FAIL(ObDependencyInfo::delete_schema_object_dependency(trans,
|
|
tenant_id,
|
|
synonym_id,
|
|
OB_INVALID_SCHEMA_VERSION,
|
|
ObObjectType::SYNONYM))) {
|
|
LOG_WARN("failed to delete_schema_object_dependency", K(ret), K(tenant_id), K(synonym_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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
|
|
LOG_INFO("finish drop synonym", K(tenant_id), K(database_name), K(synonym_name), K(ret));
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_package(ObSchemaGetterGuard &schema_guard,
|
|
const ObPackageInfo *old_package_info,
|
|
ObPackageInfo &new_package_info,
|
|
ObIArray<ObRoutineInfo> &public_routine_infos,
|
|
ObErrorInfo &error_info,
|
|
ObIArray<ObDependencyInfo> &dep_infos,
|
|
const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
const uint64_t tenant_id = new_package_info.get_tenant_id();
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.create_package(old_package_info,
|
|
new_package_info,
|
|
trans,
|
|
schema_guard,
|
|
public_routine_infos,
|
|
error_info,
|
|
dep_infos,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("create package failed", K(ret), K(new_package_info));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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_package(ObSchemaGetterGuard &schema_guard,
|
|
ObPackageInfo &package_info,
|
|
ObIArray<ObRoutineInfo> &public_routine_infos,
|
|
share::schema::ObErrorInfo &error_info,
|
|
const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = package_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", KR(ret));
|
|
} else {
|
|
const uint64_t tenant_id = package_info.get_tenant_id();
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.alter_package(package_info, schema_guard, trans, public_routine_infos,
|
|
error_info, ddl_stmt_str))) {
|
|
LOG_WARN("alter package failed", K(package_info), 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_package(const ObPackageInfo &package_info,
|
|
ObErrorInfo &error_info,
|
|
const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = package_info.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.drop_package(package_info,
|
|
trans,
|
|
schema_guard,
|
|
error_info,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("drop procedure failed", K(ret), K(package_info));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_trigger(const ObCreateTriggerArg &arg,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObCreateTriggerRes *res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTriggerInfo new_trigger_info;
|
|
//in_second_stage_ is false, Indicates that the trigger is created normally
|
|
//true Indicates that the error message is inserted into the system table after the trigger is created
|
|
//So the following steps can be skipped
|
|
uint64_t tenant_id = OB_INVALID_ID;
|
|
uint64_t trigger_database_id = OB_INVALID_ID;
|
|
uint64_t base_object_id = OB_INVALID_ID;
|
|
ObSchemaType base_object_type = OB_MAX_SCHEMA;
|
|
const ObString &trigger_database = arg.trigger_database_;
|
|
const ObString &base_object_database = arg.base_object_database_;
|
|
const ObString &base_object_name = arg.base_object_name_;
|
|
if (OB_FAIL(new_trigger_info.assign(arg.trigger_info_))) {
|
|
LOG_WARN("assign trigger_info failed", K(ret));
|
|
} else if (FALSE_IT(tenant_id = new_trigger_info.get_tenant_id())) {
|
|
} else {
|
|
const ObTriggerInfo *old_trigger_info = NULL;
|
|
if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(arg), K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_database_id(schema_guard, tenant_id, trigger_database, trigger_database_id))) {
|
|
LOG_WARN("get database id failed", K(ret));
|
|
} else if (OB_FAIL(get_object_info(schema_guard, tenant_id, base_object_database,
|
|
base_object_name, base_object_type, base_object_id))) {
|
|
LOG_WARN("get base object info failed", K(ret));
|
|
} else if (FALSE_IT(new_trigger_info.set_database_id(trigger_database_id))) {
|
|
} else if (FALSE_IT(new_trigger_info.set_base_object_type(base_object_type))) {
|
|
} else if (FALSE_IT(new_trigger_info.set_base_object_id(base_object_id))) {
|
|
} else if (OB_FAIL(try_get_exist_trigger(schema_guard, new_trigger_info, old_trigger_info, arg.with_replace_))) {
|
|
LOG_WARN("check trigger exist failed", K(ret));
|
|
} else {
|
|
if (NULL != old_trigger_info) {
|
|
new_trigger_info.set_trigger_id(old_trigger_info->get_trigger_id());
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
bool with_res = (GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_2_1_2);
|
|
int64_t table_schema_version = OB_INVALID_VERSION;
|
|
if (with_res && OB_ISNULL(res)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("res is NULL", K(ret));
|
|
} else if (OB_FAIL(create_trigger_in_trans(new_trigger_info,
|
|
const_cast<ObErrorInfo &>(arg.error_info_),
|
|
const_cast<ObSArray<ObDependencyInfo> &>(arg.dependency_infos_),
|
|
&arg.ddl_stmt_str_,
|
|
arg.in_second_stage_,
|
|
schema_guard,
|
|
table_schema_version))) {
|
|
LOG_WARN("create trigger in trans failed", K(ret));
|
|
} else if (with_res) {
|
|
res->table_schema_version_ = table_schema_version;
|
|
res->trigger_schema_version_ = new_trigger_info.get_schema_version();
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_trigger_in_trans(ObTriggerInfo &trigger_info,
|
|
ObErrorInfo &error_info,
|
|
ObIArray<ObDependencyInfo> &dep_infos,
|
|
const ObString *ddl_stmt_str,
|
|
bool in_second_stage,
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
int64_t &table_schema_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = trigger_info.get_tenant_id();
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
if (OB_SUCC(ret) && !in_second_stage) {
|
|
OZ (adjust_trigger_action_order(schema_guard, trans, ddl_operator, trigger_info, true));
|
|
}
|
|
OZ (ddl_operator.create_trigger(trigger_info, trans, error_info, dep_infos, table_schema_version, 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_trigger(const ObDropTriggerArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
uint64_t tenant_id = arg.tenant_id_;
|
|
uint64_t trigger_database_id = OB_INVALID_ID;
|
|
const ObString &trigger_database = arg.trigger_database_;
|
|
const ObString &trigger_name = arg.trigger_name_;
|
|
const ObTriggerInfo *trigger_info = NULL;
|
|
bool is_ora_mode = false;
|
|
if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(arg), K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_ora_mode))) {
|
|
LOG_WARN("fail to check is oracle mode", K(ret));
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("get schema guard in inner table failed", K(ret));
|
|
} else if (OB_FAIL(get_database_id(schema_guard, tenant_id,
|
|
trigger_database, trigger_database_id))) {
|
|
LOG_WARN("get database id failed", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_trigger_info(tenant_id, trigger_database_id,
|
|
trigger_name, trigger_info))) {
|
|
LOG_WARN("get trigger info failed", K(ret), K(trigger_database), K(trigger_name));
|
|
} else if (OB_ISNULL(trigger_info)) {
|
|
ret = OB_ERR_TRIGGER_NOT_EXIST;
|
|
if (is_ora_mode) {
|
|
LOG_ORACLE_USER_ERROR(OB_ERR_TRIGGER_NOT_EXIST, trigger_name.length(), trigger_name.ptr());
|
|
}
|
|
} else if (trigger_info->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("trigger is in recyclebin", K(ret),
|
|
K(trigger_info->get_trigger_id()), K(trigger_info->get_trigger_name()));
|
|
} else if (OB_FAIL(drop_trigger_in_trans(*trigger_info, &arg.ddl_stmt_str_, schema_guard))) {
|
|
LOG_WARN("drop trigger in trans failed", K(ret), K(trigger_database), K(trigger_name));
|
|
}
|
|
if (!is_ora_mode && (OB_ERR_TRIGGER_NOT_EXIST == ret || OB_ERR_BAD_DATABASE == ret)) {
|
|
ret = OB_ERR_TRIGGER_NOT_EXIST;
|
|
if (arg.if_exist_) {
|
|
ret = OB_SUCCESS;
|
|
LOG_MYSQL_USER_NOTE(OB_ERR_TRIGGER_NOT_EXIST);
|
|
} else {
|
|
LOG_MYSQL_USER_ERROR(OB_ERR_TRIGGER_NOT_EXIST);
|
|
}
|
|
LOG_WARN("trigger not exist", K(arg.trigger_database_), K(arg.trigger_name_), K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_trigger_in_trans(const ObTriggerInfo &trigger_info,
|
|
const ObString *ddl_stmt_str,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = trigger_info.get_tenant_id();
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
OZ (adjust_trigger_action_order(schema_guard, trans, ddl_operator, const_cast<ObTriggerInfo &>(trigger_info), false));
|
|
OZ (ddl_operator.drop_trigger(trigger_info, trans, 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::alter_trigger(const ObAlterTriggerArg &arg,
|
|
obrpc::ObRoutineDDLRes *res)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
uint64_t tenant_id = OB_INVALID_ID;
|
|
bool is_enable = false;
|
|
int64_t refreshed_schema_version = 0;
|
|
CK (OB_NOT_NULL(schema_service_) && OB_NOT_NULL(sql_proxy_));
|
|
CK (arg.is_valid(), OB_INVALID_ARGUMENT);
|
|
OX (is_enable = arg.trigger_infos_.at(0).is_enable());
|
|
OZ (check_inner_stat());
|
|
OX (tenant_id = arg.exec_tenant_id_);
|
|
OZ (get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard));
|
|
OZ (schema_guard.get_schema_version(tenant_id, refreshed_schema_version));
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
OZ (trans.start(sql_proxy_, tenant_id, refreshed_schema_version), refreshed_schema_version);
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < arg.trigger_infos_.count(); ++i) {
|
|
const ObTriggerInfo *old_tg_info = NULL;
|
|
ObTriggerInfo new_tg_info;
|
|
OZ (new_tg_info.assign(arg.trigger_infos_.at(i)));
|
|
OZ (schema_guard.get_trigger_info(tenant_id, new_tg_info.get_trigger_id(), old_tg_info));
|
|
CK (OB_NOT_NULL(old_tg_info), OB_ERR_TRIGGER_NOT_EXIST);
|
|
if (OB_SUCC(ret)) {
|
|
if (!arg.is_set_status_) {
|
|
const ObTriggerInfo *other_trg_info = NULL;
|
|
ObString new_trg_name = new_tg_info.get_trigger_name();
|
|
ObString new_trg_body = new_tg_info.get_trigger_body();
|
|
OZ (schema_guard.get_trigger_info(new_tg_info.get_tenant_id(), new_tg_info.get_database_id(),
|
|
new_trg_name, other_trg_info));
|
|
OV (OB_ISNULL(other_trg_info), OB_OBJ_ALREADY_EXIST, new_tg_info);
|
|
OZ (new_tg_info.deep_copy(*old_tg_info));
|
|
OZ (new_tg_info.set_trigger_name(new_trg_name));
|
|
OZ (new_tg_info.set_trigger_body(new_trg_body));
|
|
} else {
|
|
OZ (new_tg_info.deep_copy(*old_tg_info));
|
|
OX (new_tg_info.set_is_enable(is_enable));
|
|
}
|
|
OZ (ddl_operator.alter_trigger(new_tg_info, trans, &arg.ddl_stmt_str_));
|
|
}
|
|
// alter trigger scenes
|
|
if (OB_SUCC(ret) && 1 == arg.trigger_infos_.count() && OB_NOT_NULL(res)) {
|
|
res->store_routine_schema_version_ = new_tg_info.get_schema_version();
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::try_get_exist_trigger(ObSchemaGetterGuard &schema_guard,
|
|
const ObTriggerInfo &new_trigger_info,
|
|
const ObTriggerInfo *&old_trigger_info,
|
|
bool with_replace)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObString &trigger_name = new_trigger_info.get_trigger_name();
|
|
if (OB_FAIL(schema_guard.get_trigger_info(new_trigger_info.get_tenant_id(),
|
|
new_trigger_info.get_database_id(),
|
|
trigger_name, old_trigger_info))) {
|
|
LOG_WARN("failed to get old trigger info", K(ret));
|
|
} else if (NULL != old_trigger_info) {
|
|
if (new_trigger_info.get_base_object_id() != old_trigger_info->get_base_object_id()) {
|
|
ret = OB_ERR_TRIGGER_EXIST_ON_OTHER_TABLE;
|
|
LOG_USER_ERROR(OB_ERR_TRIGGER_EXIST_ON_OTHER_TABLE, trigger_name.length(), trigger_name.ptr());
|
|
} else if (!with_replace) {
|
|
ret = OB_ERR_TRIGGER_ALREADY_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_TRIGGER_ALREADY_EXIST, trigger_name.length(), trigger_name.ptr());
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_trigger_package(ObSchemaGetterGuard &schema_guard,
|
|
const ObTableSchema &table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObDatabaseSchema *database_schema = NULL;
|
|
const ObString *database_name = NULL;
|
|
const ObString &table_name = table_schema.get_table_name_str();
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
OZ (schema_guard.get_database_schema(tenant_id, table_schema.get_database_id(), database_schema),
|
|
table_schema.get_database_id());
|
|
OV (OB_NOT_NULL(database_schema), OB_ERR_UNEXPECTED, table_schema.get_database_id());
|
|
OX (database_name = &database_schema->get_database_name_str());
|
|
OZ (rebuild_trigger_package(schema_guard, tenant_id, table_schema.get_trigger_list(),
|
|
*database_name, table_name, ddl_operator, trans));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::rebuild_trigger_package(ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const ObIArray<uint64_t> &trigger_list,
|
|
const ObString &database_name,
|
|
const ObString &table_name,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObTriggerInfo *trigger_info = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < trigger_list.count(); i++) {
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trigger_list.at(i), trigger_info), trigger_list.at(i));
|
|
OV (OB_NOT_NULL(trigger_info), OB_ERR_UNEXPECTED, trigger_list.at(i));
|
|
OZ (ddl_operator.rebuild_trigger_package(*trigger_info, database_name, table_name, trans));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_trigger_for_truncate_table(ObSchemaGetterGuard &schema_guard,
|
|
const ObIArray<uint64_t> &origin_trigger_list,
|
|
ObTableSchema &new_table_schema,
|
|
ObDDLOperator &ddl_operator,
|
|
ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObTriggerInfo *origin_trigger_info = NULL;
|
|
ObTriggerInfo new_trigger_info;
|
|
ObString spec_source;
|
|
ObString body_source;
|
|
ObErrorInfo error_info;
|
|
ObArenaAllocator inner_alloc;
|
|
new_table_schema.get_trigger_list().reset();
|
|
bool is_update_table_schema_version = false;
|
|
const ObDatabaseSchema *db_schema = NULL;
|
|
const uint64_t tenant_id = new_table_schema.get_tenant_id();
|
|
OZ (schema_guard.get_database_schema(tenant_id,
|
|
new_table_schema.get_database_id(),
|
|
db_schema));
|
|
CK (db_schema != NULL);
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < origin_trigger_list.count(); i++) {
|
|
is_update_table_schema_version = i == origin_trigger_list.count() - 1 ? true : false;
|
|
uint64_t new_trigger_id = OB_INVALID_ID;
|
|
OZ (schema_guard.get_trigger_info(tenant_id, origin_trigger_list.at(i), origin_trigger_info),
|
|
origin_trigger_list.at(i));
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_trigger_info.deep_copy(*origin_trigger_info))) {
|
|
LOG_WARN("failed to create trigger for truncate table", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_schema_service()->fetch_new_trigger_id(
|
|
origin_trigger_info->get_tenant_id(), new_trigger_id))) {
|
|
LOG_WARN("failed to fetch_new_trigger_id", K(ret));
|
|
} else {
|
|
new_trigger_info.set_trigger_id(new_trigger_id);
|
|
new_trigger_info.set_base_object_id(new_table_schema.get_table_id());
|
|
new_table_schema.get_trigger_list().push_back(new_trigger_id);
|
|
if (OB_SUCC(ret)) {
|
|
ObSEArray<ObDependencyInfo, 1> dep_infos;
|
|
int64_t table_schema_version = OB_INVALID_VERSION;
|
|
if (OB_FAIL(ddl_operator.create_trigger(new_trigger_info, trans, error_info, dep_infos,
|
|
table_schema_version,
|
|
&origin_trigger_info->get_trigger_body(),
|
|
is_update_table_schema_version,
|
|
true))) {
|
|
LOG_WARN("failed to create trigger for truncate table", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::do_sequence_ddl(const obrpc::ObSequenceDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObString *ddl_stmt_str = &arg.ddl_stmt_str_;
|
|
ObSequenceSchema seq_schema;
|
|
const ObBitSet<> &opt_bitset = arg.get_option_bitset();
|
|
ObSchemaGetterGuard schema_guard;
|
|
uint64_t database_id = OB_INVALID_ID;
|
|
const uint64_t tenant_id = arg.seq_schema_.get_tenant_id();
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(arg), K(ret));
|
|
} else if (OB_FAIL(seq_schema.assign(arg.seq_schema_))) {
|
|
LOG_WARN("fail assign sequence schema", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(get_database_id(schema_guard,
|
|
tenant_id,
|
|
arg.database_name_,
|
|
database_id))) {
|
|
LOG_WARN("fail get database id", K(ret));
|
|
} else {
|
|
seq_schema.set_database_id(database_id);
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObSequenceDDLProxy ddl_operator(*schema_service_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
switch(arg.get_stmt_type()) {
|
|
case sql::stmt::T_CREATE_SEQUENCE: {
|
|
if (OB_FAIL(ddl_operator.create_sequence(seq_schema,
|
|
opt_bitset,
|
|
trans,
|
|
schema_guard,
|
|
ddl_stmt_str))) {
|
|
if (OB_OBJECT_NAME_EXIST == ret && arg.ignore_exists_error()) {
|
|
LOG_USER_NOTE(OB_OBJECT_NAME_EXIST);
|
|
LOG_WARN("sequence is exist, no need to create again", K(arg), K(ret));
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("fail create sequence", K(arg), K(ret));
|
|
}
|
|
}
|
|
break;
|
|
}
|
|
case sql::stmt::T_ALTER_SEQUENCE: {
|
|
if (OB_FAIL(ddl_operator.alter_sequence(seq_schema,
|
|
opt_bitset,
|
|
trans,
|
|
schema_guard,
|
|
ddl_stmt_str,
|
|
FROM_SEQUENCE_DDL))) {
|
|
LOG_WARN("fail alter sequence", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case sql::stmt::T_DROP_SEQUENCE: {
|
|
if (OB_FAIL(ddl_operator.drop_sequence(seq_schema,
|
|
trans,
|
|
schema_guard,
|
|
ddl_stmt_str,
|
|
FROM_SEQUENCE_DDL))) {
|
|
if (OB_OBJECT_NAME_NOT_EXIST == ret && arg.ignore_exists_error()) {
|
|
LOG_USER_NOTE(OB_OBJECT_NAME_NOT_EXIST, "sequence");
|
|
LOG_WARN("sequence is not exist, no need to drop", K(arg), K(ret));
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
LOG_INFO("finish do sequence ddl", K(arg), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::do_context_ddl(const obrpc::ObContextDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObString *ddl_stmt_str = &arg.ddl_stmt_str_;
|
|
ObContextSchema context_schema;
|
|
bool or_replace = arg.or_replace_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
const uint64_t tenant_id = arg.ctx_schema_.get_tenant_id();
|
|
bool need_clean_global_ctx = false;
|
|
int64_t refreshed_schema_version = 0;
|
|
CK (OB_NOT_NULL(schema_service_));
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(arg), K(ret));
|
|
} else if (OB_FAIL(context_schema.assign(arg.ctx_schema_))) {
|
|
LOG_WARN("fail assign context schema", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id,
|
|
schema_guard))) {
|
|
LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
bool obj_exist = false;
|
|
const ObContextSchema *old_schema = nullptr;
|
|
ObContextDDLProxy ddl_operator(*schema_service_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.check_context_exist_with_name(context_schema.get_tenant_id(),
|
|
context_schema.get_namespace(),
|
|
old_schema,
|
|
obj_exist))) {
|
|
LOG_WARN("fail to check context exist", K(ret), K(context_schema));
|
|
} else {
|
|
switch(arg.get_stmt_type()) {
|
|
case sql::stmt::T_CREATE_CONTEXT: {
|
|
if (OB_FAIL(ddl_operator.create_context(context_schema,
|
|
trans,
|
|
schema_guard,
|
|
or_replace,
|
|
obj_exist,
|
|
old_schema,
|
|
need_clean_global_ctx,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail create context", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case sql::stmt::T_DROP_CONTEXT: {
|
|
if (!obj_exist) {
|
|
ret = OB_ERR_OBJECT_STRING_DOES_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_ERR_OBJECT_STRING_DOES_NOT_EXIST,
|
|
static_cast<int>(context_schema.get_namespace().length()),
|
|
context_schema.get_namespace().ptr());
|
|
} else if (OB_FAIL(ddl_operator.drop_context(context_schema,
|
|
trans,
|
|
schema_guard,
|
|
old_schema,
|
|
need_clean_global_ctx,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail drop context", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default:
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected stmt type",
|
|
K(arg), K(ret));
|
|
break;
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && need_clean_global_ctx) {
|
|
if (OB_FAIL(clean_global_context(context_schema))) {
|
|
LOG_ERROR("succ to drop but failed to clean global context value", K(context_schema), K(ret));
|
|
ret = OB_SUCCESS;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
LOG_INFO("finish do context ddl", K(arg), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::clean_global_context(const ObContextSchema &context_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObGlobalContextOperator ctx_operator;
|
|
CK (OB_NOT_NULL(sql_proxy_));
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(ctx_operator.delete_global_contexts_by_id(context_schema.get_tenant_id(),
|
|
context_schema.get_context_id(),
|
|
*sql_proxy_))) {
|
|
LOG_WARN("fail to delete global context", K(context_schema.get_tenant_id()), K(ret));
|
|
} else {
|
|
LOG_INFO("success delete global context value",
|
|
K(context_schema.get_tenant_id()),
|
|
K(context_schema.get_context_id()),
|
|
K(context_schema.get_namespace()));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::recompile_view(const ObTableSchema &view_schema, const bool reset_view_column_infos, ObDDLSQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = view_schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaService *schema_service = nullptr;
|
|
uint64_t data_version = 0;
|
|
CK (OB_NOT_NULL(schema_service_) && OB_NOT_NULL(schema_service = schema_service_->get_schema_service()));
|
|
LOG_TRACE("recompile view", K(view_schema.get_table_id()), K(view_schema.get_table_name()), K(ret), K(reset_view_column_infos));
|
|
ObSchemaGetterGuard schema_guard;
|
|
const ObSimpleTableSchemaV2 *view_schema_in_guard = nullptr;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
|
|
LOG_WARN("failed to get data version", K(ret));
|
|
} else if (data_version < DATA_VERSION_4_1_0_0) {
|
|
// do nothing
|
|
} else if (!view_schema.is_view_table()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("wrong schema get", K(ret), K(view_schema));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_simple_table_schema(tenant_id, view_schema.get_table_id(), view_schema_in_guard))) {
|
|
LOG_WARN("failed to get simpile table schema", K(ret));
|
|
} else if (nullptr == view_schema_in_guard
|
|
|| view_schema_in_guard->get_schema_version() > view_schema.get_schema_version()) {
|
|
// view is deleted or modified before, ignore this recompile task
|
|
LOG_TRACE("task is ignore", K(view_schema_in_guard->get_schema_version()), K(view_schema.get_schema_version()));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObTableSchema new_view_schema;
|
|
ObObjectStatus new_status = reset_view_column_infos ? ObObjectStatus::INVALID : ObObjectStatus::VALID;
|
|
const bool update_object_status_ignore_version = false;
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(new_view_schema.assign(view_schema))) {
|
|
LOG_WARN("failed to copy schema", K(ret));
|
|
} else if (FALSE_IT(new_view_schema.set_schema_version(refreshed_schema_version))) {
|
|
} else if (view_schema.get_column_count() > 0
|
|
&& OB_FAIL(ddl_operator.update_view_columns(new_view_schema, trans))) {
|
|
LOG_WARN("failed to add view column info", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_table_status(new_view_schema, refreshed_schema_version,
|
|
new_status, update_object_status_ignore_version,
|
|
trans))) {
|
|
LOG_WARN("failed to update table status", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_label_se_policy_ddl(const ObLabelSePolicyDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObLabelSePolicySchema schema = arg.schema_; //make a copy
|
|
uint64_t tenant_id = schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.handle_label_se_policy_function(
|
|
arg.ddl_type_,
|
|
arg.ddl_stmt_str_,
|
|
schema_guard,
|
|
schema,
|
|
trans))) {
|
|
LOG_WARN("handle label security policy failed", K(ret), K(arg));
|
|
}
|
|
if (OB_SUCC(ret) && arg.ddl_type_ == OB_DDL_DROP_LABEL_SE_POLICY) {
|
|
//When drop policy, cascade delete policy-related components, labels and user levels, as well as security columns
|
|
if (OB_FAIL(ddl_operator.drop_all_label_se_user_components(
|
|
tenant_id,
|
|
OB_INVALID_ID,
|
|
schema.get_label_se_policy_id(),
|
|
trans,
|
|
ObString(),
|
|
schema_guard))) {
|
|
LOG_WARN("fail to drop user label components cascaded", K(ret), K(schema));
|
|
} else if (OB_FAIL(ddl_operator.drop_all_label_se_labels_in_policy(
|
|
tenant_id,
|
|
schema.get_label_se_policy_id(),
|
|
trans,
|
|
ObString(),
|
|
schema_guard))) {
|
|
LOG_WARN("fail to drop label se labels", K(ret), K(schema));
|
|
} else if (OB_FAIL(ddl_operator.drop_all_label_se_components_in_policy(
|
|
tenant_id, schema.get_label_se_policy_id(),
|
|
trans, ObString(), schema_guard))) {
|
|
LOG_WARN("fail to drop label se components", K(ret), K(schema));
|
|
} else if (OB_FAIL(ddl_operator.drop_all_label_se_table_column(
|
|
tenant_id, schema.get_label_se_policy_id(), trans, schema_guard))) {
|
|
LOG_WARN("fail to drop all label se table column", K(ret));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_label_se_component_ddl(const ObLabelSeComponentDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObLabelSeComponentSchema schema = arg.schema_; //make a copy
|
|
uint64_t tenant_id = schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.handle_label_se_component_function(arg.ddl_type_,
|
|
arg.ddl_stmt_str_,
|
|
arg.policy_name_,
|
|
schema_guard,
|
|
schema,
|
|
trans))) {
|
|
LOG_WARN("handle label security policy failed", K(ret), K(arg));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
|
|
int ObDDLService::handle_label_se_label_ddl(const ObLabelSeLabelDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObLabelSeLabelSchema schema = arg.schema_; //make a copy
|
|
uint64_t tenant_id = schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.handle_label_se_label_function(arg.ddl_type_,
|
|
arg.ddl_stmt_str_,
|
|
arg.policy_name_,
|
|
schema_guard,
|
|
schema,
|
|
trans))) {
|
|
LOG_WARN("handle label security policy failed", K(ret), K(arg));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_label_se_user_level_ddl(const ObLabelSeUserLevelDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObLabelSeUserLevelSchema schema = arg.level_schema_; //make a copy
|
|
uint64_t tenant_id = schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.handle_label_se_user_level_function(arg.ddl_type_,
|
|
arg.ddl_stmt_str_,
|
|
arg.policy_name_,
|
|
schema_guard,
|
|
schema,
|
|
trans))) {
|
|
LOG_WARN("handle label security policy failed", K(ret), K(arg));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_user_exist(const share::schema::ObUserInfo &user_info) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_user_name_exist = false;
|
|
bool is_user_id_exist = false;
|
|
bool is_oracle_mode = 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 (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(user_info.get_tenant_id(), is_oracle_mode))) {
|
|
LOG_WARN("fail to check compat mode", K(ret));
|
|
} else if (is_user_name_exist || is_user_id_exist) {
|
|
ret = user_info.is_role() && is_oracle_mode ? OB_ROLE_EXIST : OB_ERR_USER_EXIST;
|
|
LOG_WARN("User/role is exist, cannot create it twice,",
|
|
"tenant_id", user_info.get_tenant_id(),
|
|
"user_id", user_info.get_user_id(),
|
|
"user/role_name", user_info.get_user_name_str(),
|
|
"host_name", user_info.get_host_name_str(),
|
|
K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
bool ObDDLService::is_user_exist(const uint64_t tenant_id, const uint64_t user_id) const
|
|
{
|
|
bool is_user_id_exist = false;
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == user_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("Tenant ID and user ID should not be invalid", K(ret));
|
|
} else if (OB_FAIL(schema_service_->check_user_exist(tenant_id, user_id, is_user_id_exist))) {
|
|
LOG_WARN("Failed to check whether user exist", K(tenant_id), K(user_id), K(ret));
|
|
}
|
|
return is_user_id_exist;
|
|
}
|
|
|
|
int ObDDLService::replay_alter_user(const share::schema::ObUserInfo &user_info,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const uint64_t tenant_id = user_info.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_SYS;
|
|
LOG_ERROR("schema_service must exist", K(ret));
|
|
} else if (OB_FAIL(schema_service->get_user_sql_service().alter_user(
|
|
user_info, new_schema_version, NULL, trans))) {
|
|
LOG_WARN("alter user failed", K(ret), K(user_info));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("Failed to commit trans", K(ret), K(tmp_ret));
|
|
ret = OB_SUCC(ret) ? tmp_ret : ret;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_mysql_roles_in_trans(const uint64_t tenant_id,
|
|
const bool if_not_exist,
|
|
ObIArray<share::schema::ObUserInfo> &user_infos)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaGetterGuard schema_guard;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSqlString ddl_stmt_str;
|
|
ObSqlString exist_user_str;
|
|
bool role_exist_error = false;
|
|
ObSchemaService *schema_service_impl = NULL;
|
|
ObArray<int> skip_index;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_ISNULL(schema_service_impl = schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service impl is null", KR(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));
|
|
}
|
|
|
|
for (int i = 0; OB_SUCC(ret) && i < user_infos.count(); i++) {
|
|
share::schema::ObUserInfo &user_info = user_infos.at(i);
|
|
if (!user_info.is_role()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret));
|
|
} else if (OB_FAIL(check_user_exist(user_info))) {
|
|
if (OB_ERR_USER_EXIST == ret) {
|
|
ret = OB_SUCCESS;
|
|
if (if_not_exist) {
|
|
LOG_USER_NOTE(OB_ERR_USER_ALREADY_EXISTS,
|
|
user_infos.at(i).get_user_name_str().length(),
|
|
user_infos.at(i).get_user_name_str().ptr(),
|
|
user_infos.at(i).get_host_name_str().length(),
|
|
user_infos.at(i).get_host_name_str().ptr());
|
|
if (OB_FAIL(skip_index.push_back(i))) {
|
|
LOG_WARN("fail to push back", K(ret));
|
|
}
|
|
} else {
|
|
role_exist_error = true;
|
|
if (OB_FAIL(exist_user_str.append_fmt("'%.*s'@'%.*s',",
|
|
user_info.get_user_name_str().length(),
|
|
user_info.get_user_name_str().ptr(),
|
|
user_info.get_host_name_str().length(),
|
|
user_info.get_host_name_str().ptr()))) {
|
|
LOG_WARN("fail to append str", K(ret));
|
|
}
|
|
}
|
|
} else {
|
|
LOG_WARN("fail to check user exist", K(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret) && role_exist_error) {
|
|
ret = OB_CANNOT_USER;
|
|
if (exist_user_str.length() > 0) {
|
|
exist_user_str.set_length(exist_user_str.length() - 1);
|
|
}
|
|
LOG_USER_ERROR(OB_CANNOT_USER, (int)strlen("CREATE ROLE"), "CREATE ROLE",
|
|
exist_user_str.string().length(), exist_user_str.string().ptr());
|
|
}
|
|
|
|
if (OB_SUCC(ret) && skip_index.count() < user_infos.count()) {
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ddl_stmt_str.append("CREATE ROLE "))) {
|
|
LOG_WARN("fail to append", K(ret));
|
|
}
|
|
for (int i = 0; OB_SUCC(ret) && i < user_infos.count(); i++) {
|
|
share::schema::ObUserInfo &user_info = user_infos.at(i);
|
|
if (has_exist_in_array(skip_index, i)) {
|
|
//skip
|
|
} else if (OB_FAIL(ddl_stmt_str.append_fmt("`%.*s`@`%.*s`,",
|
|
user_info.get_user_name_str().length(),
|
|
user_info.get_user_name_str().ptr(),
|
|
user_info.get_host_name_str().length(),
|
|
user_info.get_host_name_str().ptr()))) {
|
|
LOG_WARN("fail to append str", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && ddl_stmt_str.length() > 0) {
|
|
ddl_stmt_str.set_length(ddl_stmt_str.length() - 1);
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
}
|
|
|
|
for (int i = 0; OB_SUCC(ret) && i < user_infos.count(); i++) {
|
|
share::schema::ObUserInfo &user_info = user_infos.at(i);
|
|
if (has_exist_in_array(skip_index, i)) {
|
|
//skip
|
|
} else {
|
|
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)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObString ddl_sql = ddl_stmt_str.string();
|
|
if (OB_FAIL(ddl_operator.create_user(user_info, (0 == i) ? &ddl_sql : NULL, trans))) {
|
|
LOG_WARN("Failed to create user", 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_SUCCESS == 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::create_user_in_trans(share::schema::ObUserInfo &user_info,
|
|
uint64_t creator_id,
|
|
uint64_t &user_id,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const uint64_t tenant_id = user_info.get_tenant_id();
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObSqlString ddl_stmt_str;
|
|
ObString ddl_sql;
|
|
const bool is_role = user_info.is_role();
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("Failed to start trans", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ObDDLSqlGenerator::gen_create_user_sql(
|
|
ObAccountArg(user_info.get_user_name_str(), user_info.get_host_name_str(), is_role),
|
|
user_info.get_passwd_str(),
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("gen create user sql failed", K(ret));
|
|
} else if (OB_FAIL(ObDDLSqlGenerator::append_ssl_info_sql(user_info.get_ssl_type(),
|
|
user_info.get_ssl_cipher_str(), user_info.get_x509_issuer_str(),
|
|
user_info.get_x509_subject_str(), ddl_stmt_str))) {
|
|
LOG_WARN("gen append_ssl_info_sql failed", K(ret));
|
|
} else if (FALSE_IT(ddl_sql = ddl_stmt_str.string())) {
|
|
} else if (OB_FAIL(ddl_operator.create_user(user_info, &ddl_sql, trans))) {
|
|
LOG_WARN("Failed to create user", K(ret));
|
|
} else {
|
|
LOG_INFO("Succeed in inserting user in user table",
|
|
"tenant_id", user_info.get_tenant_id(),
|
|
"user_id", user_info.get_user_id(),
|
|
"user_name", user_info.get_user_name_str(),
|
|
K(user_info));
|
|
user_id = user_info.get_user_id();
|
|
}
|
|
}
|
|
|
|
const ObTenantSchema *tenant_schema = NULL;
|
|
const ObSysVariableSchema *sys_variable_schema = NULL;
|
|
bool is_oracle_mode = false;
|
|
if (OB_FAIL(ret)) {
|
|
// do-nothing
|
|
} else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) {
|
|
LOG_WARN("tenant not exists", K(ret), K(tenant_id));
|
|
} else if (OB_UNLIKELY(NULL == tenant_schema)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("tenant not exist", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_sys_variable_schema(tenant_id, sys_variable_schema))) {
|
|
LOG_WARN("get sys variable schema failed", K(ret));
|
|
} else if (OB_ISNULL(sys_variable_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("sys variable schema is null", K(ret));
|
|
} else if (OB_FAIL(sys_variable_schema->get_oracle_mode(is_oracle_mode))) {
|
|
LOG_WARN("failed to get oracle mode", K(ret), K(tenant_id));
|
|
} else if (is_oracle_mode && !user_info.is_role()) {
|
|
// For Oracle mode, Creating a user will also create a db with the same name
|
|
// role does not need to create a db with the same name
|
|
ObDatabaseSchema db_schema;
|
|
db_schema.set_tenant_id(user_info.get_tenant_id());
|
|
if (OB_FAIL(db_schema.set_database_name(user_info.get_user_name()))) {
|
|
LOG_WARN("failed to set database name", K(ret), K(user_info.get_user_name_str()));
|
|
} else if (OB_FAIL(db_schema.set_comment("oracle user database"))) {
|
|
LOG_WARN("failed to set database comment", K(ret), K(user_info.get_user_name_str()));
|
|
} else if (OB_FAIL(create_database(false, db_schema, NULL, &trans))) {
|
|
LOG_WARN("failed to create oracle user database", K(ret), K(tenant_id));
|
|
}
|
|
} else if (is_oracle_mode && user_info.is_role() && OB_INVALID_ID != creator_id) {
|
|
// For Oracle mode, role shall be granted to creator after creation with admin option
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const ObUserInfo *creator_info = NULL;
|
|
ObArray<uint64_t> role_ids;
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("failed to grant role to creator", K(ret));
|
|
} else if (is_root_user(creator_id)) {
|
|
LOG_WARN("creator_id is OB_SYS_USER_ID, refuse to grant role to it",
|
|
K(user_id), K(creator_id));
|
|
} else if (OB_FAIL(schema_guard.get_user_info(tenant_id, creator_id, creator_info))) {
|
|
LOG_WARN("get_user_info failed", K(ret), K(tenant_id), K(creator_id));
|
|
} else if (NULL == creator_info) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get creator_info", K(ret));
|
|
} else if (OB_FAIL(role_ids.push_back(user_id))) {
|
|
LOG_WARN("failed to add to role_ids", K(ret), K(tenant_id), K(user_id),
|
|
K(user_info.get_user_name_str()));
|
|
} else if (OB_FAIL(ddl_operator.grant_revoke_role(tenant_id,
|
|
*creator_info,
|
|
role_ids,
|
|
&user_info,
|
|
trans,
|
|
true /*log_operation*/,
|
|
true /*is grant*/,
|
|
ADMIN_OPTION))) {
|
|
LOG_WARN("failed to grant_revoke_role", K(ret), K(tenant_id), K(*creator_info));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
//----End of functions for managing privileges----
|
|
|
|
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;
|
|
ObArenaAllocator allocator("PrimaryZone");
|
|
ObPrimaryZone primary_zone_schema(allocator);
|
|
if (OB_FAIL(schema.get_primary_zone_inherit(schema_guard, primary_zone_schema))) {
|
|
LOG_WARN("fail to get primary zone inherit", K(ret));
|
|
} else if (ObString(OB_RANDOM_PRIMARY_ZONE) == primary_zone_schema.primary_zone_str_
|
|
|| primary_zone_schema.primary_zone_str_.empty()) {
|
|
common::ObArray<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;
|
|
}
|
|
|
|
#ifdef OB_BUILD_ARBITRATION
|
|
int ObDDLService::check_tenant_arbitration_service_status_(
|
|
ObMySQLTransaction &trans,
|
|
const uint64_t tenant_id,
|
|
const share::ObArbitrationServiceStatus &old_status,
|
|
const share::ObArbitrationServiceStatus &new_status)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_compatible = false;
|
|
bool can_promote = false;
|
|
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id
|
|
|| !old_status.is_valid()
|
|
|| !new_status.is_valid())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(old_status), K(new_status));
|
|
} else if (OB_FAIL(ObShareUtil::check_compat_version_for_arbitration_service(tenant_id, is_compatible))) {
|
|
LOG_WARN("fail to check data version", KR(ret), K(tenant_id));
|
|
} else if (!is_compatible) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("can not change arbitration service status with data version below 4.1", KR(ret), K(tenant_id));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "data version is below 4.1, change tenant arbitration service status");
|
|
} else if ((new_status.is_enabled() && !old_status.is_enabling())
|
|
|| (new_status.is_disabled() && !old_status.is_disabling())) {
|
|
ret = OB_STATE_NOT_MATCH;
|
|
LOG_WARN("unexpected status", KR(ret), K(new_status), K(old_status));
|
|
} else if (OB_FAIL(ObArbitrationServiceUtils::check_can_promote_arbitration_service_status(trans, tenant_id, old_status, new_status, can_promote))) {
|
|
LOG_WARN("fail to check whether can enable arb service", KR(ret), K(tenant_id), K(old_status), K(new_status));
|
|
} else if (!can_promote) {
|
|
// LOG_USER_ERROR will raise inside check_can_promote_arbitration_service_status
|
|
ret = OB_STATE_NOT_MATCH;
|
|
LOG_WARN("promote conditions not satisfied", KR(ret), K(tenant_id), K(old_status), K(new_status), K(can_promote));
|
|
}
|
|
return ret;
|
|
}
|
|
#endif
|
|
|
|
int ObDDLService::check_tenant_primary_zone_(
|
|
share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const share::schema::ObTenantSchema &new_tenant_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
common::ObArray<common::ObRegion> tenant_primary_regions;
|
|
if (OB_FAIL(get_schema_primary_regions(
|
|
new_tenant_schema, schema_guard, tenant_primary_regions))) {
|
|
LOG_WARN("fail to get tenant primary regions", K(ret));
|
|
} else if (tenant_primary_regions.count() <= 0) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("primary regions unexpected", K(ret));
|
|
} else if (tenant_primary_regions.count() > 1) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("tenant primary zone span regions not supported", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "tenant primary zone span regions");
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_alter_tenant_replica_options(
|
|
const obrpc::ObModifyTenantArg &arg,
|
|
share::schema::ObTenantSchema &new_tenant_schema,
|
|
const share::schema::ObTenantSchema &orig_tenant_schema,
|
|
share::schema::ObSchemaGetterGuard &schema_guard)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArray<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
|
|
if (OB_OP_NOT_ALLOW == ret
|
|
&& arg.alter_option_bitset_.has_member(obrpc::ObModifyTenantArg::FORCE_LOCALITY)) {
|
|
ret = OB_SUCCESS;
|
|
LOG_WARN("FORCE ROLLBACK LOCALITY should skip all checks", KR(ret), K(arg),
|
|
K(orig_tenant_schema), K(new_tenant_schema));
|
|
}
|
|
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_schema_zone_list(
|
|
common::ObArray<common::ObZone> &zone_list)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
lib::ob_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 {
|
|
SMART_VARS_2((char[MAX_ZONE_LIST_LENGTH], primary_zone_str),
|
|
(ObPrimaryZoneUtil, primary_zone_util,
|
|
schema.get_primary_zone(), &zone_region_list)) {
|
|
int64_t pos = 0;
|
|
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_LIST_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
|
|
} // end smart var
|
|
}
|
|
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;
|
|
ObArenaAllocator allocator("PrimaryZone");
|
|
ObPrimaryZone primary_zone_schema(allocator);
|
|
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;
|
|
if (OB_FAIL(schema.get_paxos_replica_num(schema_guard, paxos_num))) {
|
|
LOG_WARN("fail to get paxos replica num", K(ret));
|
|
} else if (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;
|
|
}
|
|
|
|
int ObDDLService::check_alter_schema_replica_options(
|
|
const bool alter_primary_zone,
|
|
share::schema::ObTenantSchema &new_schema,
|
|
const share::schema::ObTenantSchema &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));
|
|
}
|
|
}
|
|
}
|
|
|
|
// retrun OB_OP_NOT_ALLOW if first_primary_zone changed when tenant rebalance is disabled.
|
|
if (FAILEDx(check_alter_tenant_when_rebalance_is_disabled_(orig_schema, new_schema))) {
|
|
LOG_WARN("failed to check alter tenant when rebalance is disabled", KR(ret), K(orig_schema), K(new_schema));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
int64_t paxos_num = 0;
|
|
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 (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 {
|
|
lib::ob_sort(new_zone_score_array.begin(), new_zone_score_array.end());
|
|
SMART_VAR(char[MAX_ZONE_LIST_LENGTH], primary_zone_str) {
|
|
if (OB_FAIL(format_primary_zone_from_zone_score_array(
|
|
new_zone_score_array, primary_zone_str, MAX_ZONE_LIST_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
|
|
} // end smart var
|
|
}
|
|
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> ®ion_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 ®ion)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool find = false;
|
|
for (int64_t i = 0; !find && i < zone_region_list.count(); ++i) {
|
|
const share::schema::ObZoneRegion &zone_region = zone_region_list.at(i);
|
|
if (zone_region.zone_ == zone) {
|
|
region = zone_region.region_;
|
|
find = true;
|
|
} else {} // go on check next;
|
|
}
|
|
if (!find) {
|
|
ret = OB_ENTRY_NOT_EXIST;
|
|
LOG_WARN("entry not exist", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::construct_zone_region_list(
|
|
common::ObIArray<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 (OB_FAIL(zone_region.set_check_zone_type(zone_info.zone_type_.value_))) {
|
|
LOG_WARN("fail to set check zone type", KR(ret));
|
|
} else if (!has_exist_in_array(zone_list, zone_region.zone_)) {
|
|
// this zone do not exist in my zone list, ignore it
|
|
} else if (OB_FAIL(zone_region_list.push_back(zone_region))) {
|
|
LOG_WARN("fail to push back", K(ret));
|
|
} else {} // no more to do
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
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, should have at least one paxos replica");
|
|
LOG_WARN("full replica num is zero", K(ret), K(full_replica_num), K(schema));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
for (int64_t i = 0; i < zone_replica_attr_array.count() && OB_SUCC(ret); ++i) {
|
|
ObZoneReplicaAttrSet &zone_replica_set = zone_replica_attr_array.at(i);
|
|
if (zone_replica_set.zone_set_.count() > 1) {
|
|
if (zone_replica_set.zone_set_.count() != zone_replica_set.get_paxos_replica_num()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality, too many paxos replicas in multiple zones");
|
|
LOG_WARN("too many paxos replicas in multi zone", K(ret));
|
|
}
|
|
} else if (zone_replica_set.get_full_replica_num() > 1
|
|
|| zone_replica_set.get_logonly_replica_num() > 1
|
|
|| zone_replica_set.get_encryption_logonly_replica_num() > 1) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality");
|
|
LOG_WARN("one zone should only have one paxos replica", K(ret), K(zone_replica_set));
|
|
} else if (zone_replica_set.get_full_replica_num() == 1
|
|
&& (zone_replica_set.get_logonly_replica_num() == 1
|
|
|| zone_replica_set.get_encryption_logonly_replica_num() == 1)) {
|
|
bool find = false;
|
|
for (int64_t j = 0; j < unit_infos.count() && OB_SUCC(ret); j++) {
|
|
if (unit_infos.at(j).unit_.zone_ == zone_replica_set.zone_
|
|
&& REPLICA_TYPE_LOGONLY == unit_infos.at(j).unit_.replica_type_) {
|
|
find = true;
|
|
break;
|
|
}
|
|
} //end for unit_infos
|
|
if (!find) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_USER_ERROR(OB_INVALID_ARGUMENT, "locality");
|
|
LOG_WARN("no logonly unit exist", K(ret), K(zone_replica_set));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_pools(const ObIArray<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;
|
|
}
|
|
|
|
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;
|
|
int64_t refreshed_schema_version = 0;
|
|
const uint64_t tenant_id = table_schema.get_tenant_id();
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, table_schema.get_data_table_id(), data_table_schema))) {
|
|
LOG_WARN("get table schema failed", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(data_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("data table schema is null", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_index_drop_options(table_schema, data_table_schema->get_table_name_str(), new_index_table_schema, trans))) {
|
|
LOG_WARN("alter index invisible failed", K(table_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_table_to_recyclebin(new_index_table_schema,
|
|
schema_guard,
|
|
trans,
|
|
NULL))) {
|
|
LOG_WARN("drop index to recyclebin failed", K(ret));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is commit", OB_SUCCESS == 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 (OB_INVALID_ID != tablegroup_id) {
|
|
//user table cannot add to sys tablegroup
|
|
if (is_sys_tablegroup_id(tablegroup_id)
|
|
&& !is_inner_table(schema.get_table_id())) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("user table cannot add to sys tablegroup", KR(ret), K(schema.get_table_id()));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "user table add to sys tablegroup");
|
|
} else if (!is_sys_tablegroup_id(tablegroup_id)
|
|
&& is_inner_table(schema.get_table_id())) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("inner table cannot add to user tablegroup", KR(ret), K(schema.get_table_id()));
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, "inner table add to user tablegroup");
|
|
} else if (!is_sys_tablegroup_id(tablegroup_id)) {
|
|
ObTableGroupHelp helper(*this, *schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(helper.check_table_partition_in_tablegroup(NULL, schema, schema_guard))) {
|
|
LOG_WARN("fail to check tablegroup partition", KR(ret), K(schema.get_table_id()));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/*
|
|
* this fuction is used to broadcast new schema version in tenant space.
|
|
* [intput] :
|
|
* - arg.schema_operation_ : info that should be recorded in __all_ddl_operation.
|
|
* arg.schema_operation_.tenant_id_ is used to specific the namespace.
|
|
*/
|
|
int ObDDLService::log_nop_operation(const obrpc::ObDDLNopOpreatorArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("inner stat error", KR(ret));
|
|
} else {
|
|
const uint64_t tenant_id = arg.schema_operation_.tenant_id_;
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
share::schema::ObSchemaService *tmp_schema_service = schema_service_->get_schema_service();
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_ISNULL(tmp_schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", KR(ret), KP(tmp_schema_service), KP(schema_service_));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get tenant schema guard", KR(ret), K(tenant_id));
|
|
} else {
|
|
share::schema::ObDDLSqlService ddl_sql_service(*tmp_schema_service);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans", KR(ret),
|
|
K(tenant_id), K(arg), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ddl_sql_service.log_nop_operation(arg.schema_operation_,
|
|
new_schema_version,
|
|
arg.ddl_stmt_str_,
|
|
trans))) {
|
|
LOG_WARN("log non ddl operation failed", KR(ret), K(arg));
|
|
} else {
|
|
LOG_INFO("log nop opereation success", K(arg), K(new_schema_version));
|
|
}
|
|
if (trans.is_started()) {
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (tmp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("failed to end trans", KR(ret), K(tmp_ret));
|
|
ret = (OB_SUCCESS == ret ? tmp_ret : ret);
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
// publish_schema() will modify THIS_WORKER's timeout.
|
|
// we should recovery the timeout ctx after publish schema.
|
|
const int64_t timeout_us = THIS_WORKER.get_timeout_ts();
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", KR(ret));
|
|
}
|
|
THIS_WORKER.set_timeout_ts(timeout_us);
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_user_defined_function(share::schema::ObUDF &udf_info,
|
|
const common::ObString &ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = udf_info.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
ret = ddl_operator.create_user_defined_function(udf_info, trans, &ddl_stmt_str);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("failed to create udf", K(udf_info), K(ddl_stmt_str), K(ret));
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == 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;
|
|
uint64_t udf_id = OB_INVALID_ID;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_service_->check_udf_exist(tenant_id, name,
|
|
is_exist, udf_id))) {
|
|
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, "FUNCTION", name.length(), name.ptr());
|
|
LOG_INFO("function not exist, no need to delete it", K(tenant_id), K(name));
|
|
} else {
|
|
ret = OB_ERR_FUNCTION_UNKNOWN;
|
|
LOG_WARN("function not exist, can't delete it", K(tenant_id), K(name), K(ret));
|
|
}
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(ObDependencyInfo::modify_dep_obj_status(trans, tenant_id, udf_id,
|
|
ddl_operator, *schema_service_))) {
|
|
LOG_WARN("failed to modify obj status", K(ret));
|
|
} else 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_SUCCESS == 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, uint64_t &udf_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_FAIL(schema_service_->check_udf_exist(tenant_id, name, is_exist, udf_id))) {
|
|
LOG_WARN("failed to check if udf_name exists", K(name), K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::reconstruct_table_schema_from_recyclebin(ObTableSchema &index_table_schema,
|
|
const ObRecycleObject &recycle_obj,
|
|
ObSchemaGetterGuard &guard) {
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = index_table_schema.get_tenant_id();
|
|
index_table_schema.set_tablegroup_id(recycle_obj.get_tablegroup_id());
|
|
const ObDatabaseSchema *db_schema = NULL;
|
|
if (OB_INVALID_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invaild tenant id", K(tenant_id), K(ret));
|
|
} else if (OB_FAIL(guard.get_database_schema(tenant_id,
|
|
recycle_obj.get_database_id(),
|
|
db_schema))) {
|
|
LOG_WARN("get database schema failed", K(ret));
|
|
} else if (NULL == db_schema) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("database not exist", K(recycle_obj), K(ret));
|
|
} else if (db_schema->is_in_recyclebin()) {
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("database in recyclebin is not allowed",
|
|
K(recycle_obj), K(*db_schema), K(ret));
|
|
} else if (OB_FAIL(index_table_schema.set_table_name(recycle_obj.get_original_name()))) {
|
|
LOG_WARN("set table name failed", K(ret), K(recycle_obj));
|
|
} else {
|
|
index_table_schema.set_database_id(recycle_obj.get_database_id());
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_INVALID_ID != recycle_obj.get_tablegroup_id()) {
|
|
bool is_tablegroup_exist = false;
|
|
if (OB_FAIL(guard.check_tablegroup_exist(recycle_obj.get_tenant_id(),
|
|
recycle_obj.get_tablegroup_id(), is_tablegroup_exist))) {
|
|
LOG_WARN("check tablegroup exist failed", K(recycle_obj), K(ret));
|
|
} else if (!is_tablegroup_exist) {
|
|
//replace tablegroup id with invalid id
|
|
index_table_schema.set_tablegroup_id(OB_INVALID_ID);
|
|
LOG_WARN("tablegroup is not exist", K(recycle_obj), K(ret));
|
|
} else {
|
|
index_table_schema.set_tablegroup_id(recycle_obj.get_tablegroup_id());
|
|
}
|
|
} else {
|
|
index_table_schema.set_tablegroup_id(OB_INVALID_ID);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_database_id(ObSchemaGetterGuard &schema_guard,
|
|
uint64_t tenant_id,
|
|
const ObString &database_name,
|
|
uint64_t &database_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObDatabaseSchema *db_schema = NULL;
|
|
if (OB_FAIL(schema_guard.get_database_schema(tenant_id, database_name, db_schema))) {
|
|
LOG_WARN("get database schema failed", K(ret));
|
|
} else if (NULL == db_schema) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr());
|
|
} else if (db_schema->is_or_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("Can't not operate db in recyclebin",
|
|
K(tenant_id), K(database_name), K(database_id), K(*db_schema), K(ret));
|
|
} else if (OB_INVALID_ID == (database_id = db_schema->get_database_id())) {
|
|
ret = OB_ERR_BAD_DATABASE;
|
|
LOG_WARN("database id is invalid",
|
|
K(tenant_id), K(database_name), K(database_id), K(*db_schema), K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_object_info(ObSchemaGetterGuard &schema_guard,
|
|
const uint64_t tenant_id,
|
|
const ObString &object_database,
|
|
const ObString &object_name,
|
|
ObSchemaType &object_type,
|
|
uint64_t &object_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t database_id = OB_INVALID_ID;
|
|
const ObTableSchema *table_schema = NULL;
|
|
if (OB_FAIL(get_database_id(schema_guard, tenant_id, object_database, database_id))) {
|
|
LOG_WARN("failed to get database id", K(ret));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, database_id,
|
|
object_name, false, table_schema))) {
|
|
LOG_WARN("failed to get table schema", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_ERR_BAD_TABLE;
|
|
LOG_WARN("table schema is invalid", K(ret), K(object_name), K(object_name));
|
|
} else if (table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("table is in recyclebin", K(ret), K(object_name), K(object_name));
|
|
} else if (!table_schema->is_user_table() && !table_schema->is_user_view()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("trigger only support create on user table or user view now", K(ret));
|
|
} else {
|
|
object_type = table_schema->is_user_table() ? TABLE_SCHEMA : VIEW_SCHEMA;
|
|
object_id = table_schema->get_table_id();
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_mysql_tenant_sys_var(
|
|
const ObTenantSchema &tenant_schema,
|
|
const ObSysVariableSchema &sys_variable_schema,
|
|
ObSysParam *sys_params,
|
|
int64_t params_capacity)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = sys_variable_schema.get_tenant_id();
|
|
if (OB_ISNULL(sys_params) || OB_UNLIKELY(params_capacity < ObSysVarFactory::ALL_SYS_VARS_COUNT)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity));
|
|
} else if (tenant_schema.is_mysql_tenant()) {
|
|
HEAP_VAR(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf) {
|
|
// If it is a tenant in mysql mode, you need to consider setting the charset and collation
|
|
// corresponding to the tenant to sys var
|
|
VAR_INT_TO_STRING(val_buf, tenant_schema.get_collation_type());
|
|
// set collation and char set
|
|
SET_TENANT_VARIABLE(SYS_VAR_COLLATION_DATABASE, val_buf);
|
|
SET_TENANT_VARIABLE(SYS_VAR_COLLATION_SERVER, val_buf);
|
|
SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_DATABASE, val_buf);
|
|
SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_SERVER, val_buf);
|
|
} // end HEAP_VAR
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::update_oracle_tenant_sys_var(
|
|
const ObTenantSchema &tenant_schema,
|
|
const ObSysVariableSchema &sys_variable_schema,
|
|
ObSysParam *sys_params,
|
|
int64_t params_capacity)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = sys_variable_schema.get_tenant_id();
|
|
if (OB_ISNULL(sys_params) || OB_UNLIKELY(params_capacity < ObSysVarFactory::ALL_SYS_VARS_COUNT)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity));
|
|
} else if (tenant_schema.is_oracle_tenant()) {
|
|
HEAP_VAR(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf) {
|
|
// For oracle tenants, the collation of sys variable and tenant_option is set to binary by default.
|
|
// set group_concat_max_len = 4000
|
|
// set autocommit = off
|
|
// When setting oracle variables, try to keep the format consistent
|
|
VAR_INT_TO_STRING(val_buf, OB_DEFAULT_GROUP_CONCAT_MAX_LEN_FOR_ORACLE);
|
|
SET_TENANT_VARIABLE(SYS_VAR_GROUP_CONCAT_MAX_LEN, val_buf);
|
|
|
|
SET_TENANT_VARIABLE(SYS_VAR_AUTOCOMMIT, "0");
|
|
|
|
VAR_INT_TO_STRING(val_buf, tenant_schema.get_collation_type());
|
|
SET_TENANT_VARIABLE(SYS_VAR_COLLATION_DATABASE, val_buf);
|
|
SET_TENANT_VARIABLE(SYS_VAR_COLLATION_SERVER, val_buf);
|
|
SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_DATABASE, val_buf);
|
|
SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_SERVER, val_buf);
|
|
|
|
// Here is the collation of the connection, OB currently only supports the client as utf8mb4
|
|
VAR_INT_TO_STRING(val_buf, CS_TYPE_UTF8MB4_BIN);
|
|
SET_TENANT_VARIABLE(SYS_VAR_COLLATION_CONNECTION, val_buf);
|
|
SET_TENANT_VARIABLE(SYS_VAR_CHARACTER_SET_CONNECTION, val_buf);
|
|
|
|
/*
|
|
* In Oracle mode, we are only compatible with binary mode, so collate can only end with _bin
|
|
*/
|
|
if (ObCharset::is_bin_sort(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);
|
|
ObCharsetType charset_type = ObCharset::charset_type_by_coll(tenant_schema.get_collation_type());
|
|
OZ(databuff_printf(val_buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s",
|
|
ObCharset::get_oracle_charset_name_by_charset_type(charset_type)));
|
|
SET_TENANT_VARIABLE(SYS_VAR_NLS_CHARACTERSET, val_buf);
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tenant collation set error", K(ret), K(tenant_schema.get_collation_type()));
|
|
}
|
|
|
|
// update oracle tenant schema
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(databuff_printf(sys_params[SYS_VAR_SQL_MODE].value_,
|
|
sizeof(sys_params[SYS_VAR_SQL_MODE].value_), "%llu", DEFAULT_ORACLE_MODE))) {
|
|
ret = OB_BUF_NOT_ENOUGH;
|
|
LOG_WARN("set oracle tenant default sql mode failed", K(ret));
|
|
}
|
|
}
|
|
} // end HEAP_VAR
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// The value of certain system variables of the system/meta tenant
|
|
int ObDDLService::update_special_tenant_sys_var(
|
|
const ObSysVariableSchema &sys_variable_schema,
|
|
ObSysParam *sys_params,
|
|
int64_t params_capacity)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = sys_variable_schema.get_tenant_id();
|
|
if (OB_ISNULL(sys_params) || OB_UNLIKELY(params_capacity < ObSysVarFactory::ALL_SYS_VARS_COUNT)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity));
|
|
} else {
|
|
HEAP_VAR(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf) {
|
|
if (is_sys_tenant(tenant_id)) {
|
|
VAR_INT_TO_STRING(val_buf, sys_variable_schema.get_name_case_mode());
|
|
SET_TENANT_VARIABLE(SYS_VAR_LOWER_CASE_TABLE_NAMES, val_buf);
|
|
|
|
OZ(databuff_printf(val_buf, OB_MAX_SYS_PARAM_VALUE_LENGTH, "%s", OB_SYS_HOST_NAME));
|
|
SET_TENANT_VARIABLE(SYS_VAR_OB_TCP_INVITED_NODES, val_buf);
|
|
} else if (is_meta_tenant(tenant_id)) {
|
|
ObString compatibility_mode("0");
|
|
SET_TENANT_VARIABLE(SYS_VAR_OB_COMPATIBILITY_MODE, compatibility_mode);
|
|
}
|
|
} // end HEAP_VAR
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::init_system_variables(
|
|
const ObCreateTenantArg &arg,
|
|
ObTenantSchema &tenant_schema,
|
|
ObSysVariableSchema &sys_variable_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
//MAX_SYS_PARAM_NUM is 500. When param num needed greater than 500,
|
|
//you need to change OB_MAX_SYS_PARAM_NUM in "ob_define.h".
|
|
const int64_t params_capacity = OB_MAX_SYS_PARAM_NUM;
|
|
int64_t var_amount = ObSysVariables::get_amount();
|
|
const uint64_t tenant_id = tenant_schema.get_tenant_id();
|
|
ObMalloc alloc(ObModIds::OB_TEMP_VARIABLES);
|
|
ObPtrGuard<ObSysParam, OB_MAX_SYS_PARAM_NUM> sys_params_guard(alloc);
|
|
sys_variable_schema.reset();
|
|
sys_variable_schema.set_tenant_id(tenant_id);
|
|
ObSysParam *sys_params = NULL;
|
|
if (OB_INVALID_TENANT_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_service_)
|
|
|| OB_ISNULL(sql_proxy_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("ptr is null", KR(ret), KP_(schema_service), KP_(sql_proxy));
|
|
} else if (OB_FAIL(sys_params_guard.init())) {
|
|
LOG_WARN("alloc sys parameters failed", KR(ret));
|
|
} else if (FALSE_IT(sys_params = sys_params_guard.ptr())) {
|
|
} else if (OB_ISNULL(sys_params) || OB_UNLIKELY(var_amount > params_capacity)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arguments", KR(ret), K(sys_params), K(params_capacity), K(var_amount));
|
|
} else {
|
|
HEAP_VARS_2((char[OB_MAX_SYS_PARAM_VALUE_LENGTH], val_buf),
|
|
(char[OB_MAX_SYS_PARAM_VALUE_LENGTH], version_buf)) {
|
|
// name_case_mode
|
|
if (is_meta_tenant(tenant_id)) {
|
|
sys_variable_schema.set_name_case_mode(OB_ORIGIN_AND_INSENSITIVE);
|
|
} else if (OB_NAME_CASE_INVALID == arg.name_case_mode_) {
|
|
sys_variable_schema.set_name_case_mode(OB_LOWERCASE_AND_INSENSITIVE);
|
|
} else {
|
|
sys_variable_schema.set_name_case_mode(arg.name_case_mode_);
|
|
}
|
|
|
|
// init default values
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < var_amount; ++i) {
|
|
if (OB_FAIL(sys_params[i].init(tenant_id,
|
|
ObSysVariables::get_name(i),
|
|
ObSysVariables::get_type(i),
|
|
ObSysVariables::get_value(i),
|
|
ObSysVariables::get_min(i),
|
|
ObSysVariables::get_max(i),
|
|
ObSysVariables::get_info(i),
|
|
ObSysVariables::get_flags(i)))) {
|
|
LOG_WARN("fail to init param", KR(ret), K(tenant_id), K(i));
|
|
}
|
|
}
|
|
|
|
int64_t set_sys_var_count = arg.sys_var_list_.count();
|
|
bool use_default_parallel_servers_target = true;
|
|
bool explicit_set_compatibility_version = false;
|
|
bool explicit_set_security_version = false;
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < set_sys_var_count; ++j) {
|
|
ObSysVarIdValue sys_var;
|
|
if (OB_FAIL(arg.sys_var_list_.at(j, sys_var))) {
|
|
LOG_WARN("failed to get sys var", K(j), K(ret));
|
|
} else {
|
|
const ObString &new_value = sys_var.value_;
|
|
SET_TENANT_VARIABLE(sys_var.sys_id_, new_value);
|
|
// sync tenant schema
|
|
if (SYS_VAR_READ_ONLY == sys_var.sys_id_) {
|
|
bool read_only_value = false;
|
|
if (is_user_tenant(tenant_id)) {
|
|
read_only_value = (0 == sys_var.value_.compare("1"));
|
|
tenant_schema.set_read_only(read_only_value);
|
|
} else {
|
|
tenant_schema.set_read_only(read_only_value);
|
|
}
|
|
} else if (SYS_VAR_OB_COMPATIBILITY_MODE == sys_var.sys_id_) {
|
|
if (is_user_tenant(tenant_id) && 0 == sys_var.value_.compare("1")) {
|
|
tenant_schema.set_compatibility_mode(ObCompatibilityMode::ORACLE_MODE);
|
|
} else {
|
|
tenant_schema.set_compatibility_mode(ObCompatibilityMode::MYSQL_MODE);
|
|
}
|
|
} else if (SYS_VAR_PARALLEL_SERVERS_TARGET == sys_var.sys_id_) {
|
|
use_default_parallel_servers_target = false;
|
|
} else if (SYS_VAR_OB_COMPATIBILITY_VERSION == sys_var.sys_id_) {
|
|
explicit_set_compatibility_version = true;
|
|
} else if (SYS_VAR_OB_SECURITY_VERSION == sys_var.sys_id_) {
|
|
explicit_set_security_version = true;
|
|
}
|
|
}
|
|
} // end for
|
|
|
|
// For read_only, its priority: sys variable > tenant option.
|
|
if (OB_SUCC(ret)) {
|
|
ObString read_only_value = tenant_schema.is_read_only() ? "1" : "0";
|
|
SET_TENANT_VARIABLE(SYS_VAR_READ_ONLY, read_only_value);
|
|
}
|
|
|
|
// For compatibility_mode, its priority: sys variable > tenant option.
|
|
if (OB_SUCC(ret)) {
|
|
if (is_meta_tenant(tenant_id) || !tenant_schema.is_oracle_tenant()) {
|
|
tenant_schema.set_compatibility_mode(ObCompatibilityMode::MYSQL_MODE);
|
|
}
|
|
ObString compat_mode_value = tenant_schema.is_oracle_tenant() ? "1" : "0";
|
|
SET_TENANT_VARIABLE(SYS_VAR_OB_COMPATIBILITY_MODE, compat_mode_value);
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
char version[common::OB_CLUSTER_VERSION_LENGTH] = {0};
|
|
int64_t len = ObClusterVersion::print_version_str(
|
|
version, common::OB_CLUSTER_VERSION_LENGTH, DATA_CURRENT_VERSION);
|
|
SET_TENANT_VARIABLE(SYS_VAR_PRIVILEGE_FEATURES_ENABLE, ObString(len, version));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObString enable = "1";
|
|
SET_TENANT_VARIABLE(SYS_VAR__ENABLE_MYSQL_PL_PRIV_CHECK, enable);
|
|
}
|
|
|
|
// If the user does not specify parallel_servers_target when creating tenant,
|
|
// then calculate a default value based on cpu_count.
|
|
// Considering that a tenant may have multiple resource pools, it is currently rudely considered
|
|
// that the units in the pool are of the same structure, and directly take the unit config of the first resource pool
|
|
// WARNING: If the unit is not structured, the number of threads allocated by default may be too large/too small
|
|
int64_t default_px_thread_count = 0;
|
|
if (OB_SUCC(ret) && (use_default_parallel_servers_target)) {
|
|
HEAP_VAR(ObUnitConfig, unit_config) {
|
|
if (OB_SYS_TENANT_ID == sys_variable_schema.get_tenant_id()) {
|
|
// When creating a system tenant, the default value of px_thread_count is related to
|
|
// default sys tenant min cpu
|
|
const int64_t sys_default_min_cpu =
|
|
static_cast<int64_t>(GCONF.get_sys_tenant_default_min_cpu());
|
|
default_px_thread_count = ObTenantCpuShare::calc_px_pool_share(
|
|
sys_variable_schema.get_tenant_id(), sys_default_min_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.unit_resource().min_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));
|
|
VAR_INT_TO_STRING(val_buf, default_px_servers_target);
|
|
SET_TENANT_VARIABLE(SYS_VAR_PARALLEL_SERVERS_TARGET, val_buf);
|
|
}
|
|
|
|
VAR_UINT_TO_STRING(version_buf, CLUSTER_CURRENT_VERSION);
|
|
if (OB_SUCC(ret) && !(is_user_tenant(tenant_id) && explicit_set_compatibility_version)) {
|
|
SET_TENANT_VARIABLE(SYS_VAR_OB_COMPATIBILITY_VERSION, version_buf);
|
|
}
|
|
|
|
if (OB_SUCC(ret) && !(is_user_tenant(tenant_id) && explicit_set_security_version)) {
|
|
SET_TENANT_VARIABLE(SYS_VAR_OB_SECURITY_VERSION, version_buf);
|
|
}
|
|
|
|
if (FAILEDx(update_mysql_tenant_sys_var(
|
|
tenant_schema, sys_variable_schema, sys_params, params_capacity))) {
|
|
LOG_WARN("failed to update_mysql_tenant_sys_var",
|
|
KR(ret), K(tenant_schema), K(sys_variable_schema));
|
|
} else if (OB_FAIL(update_oracle_tenant_sys_var(
|
|
tenant_schema, sys_variable_schema, sys_params, params_capacity))) {
|
|
LOG_WARN("failed to update_oracle_tenant_sys_var",
|
|
KR(ret), K(tenant_schema), K(sys_variable_schema));
|
|
} else if (OB_FAIL(update_special_tenant_sys_var(
|
|
sys_variable_schema, sys_params, params_capacity))) {
|
|
LOG_WARN("failed to update_special_tenant_sys_var", K(ret), K(sys_variable_schema));
|
|
}
|
|
|
|
// set sys_variable
|
|
if (OB_SUCC(ret)) {
|
|
ObSysVarSchema sysvar_schema;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < var_amount; i++) {
|
|
sysvar_schema.reset();
|
|
if (OB_FAIL(ObSchemaUtils::convert_sys_param_to_sysvar_schema(sys_params[i], sysvar_schema))) {
|
|
LOG_WARN("convert to sysvar schema failed", K(ret));
|
|
} else if (OB_FAIL(sys_variable_schema.add_sysvar_schema(sysvar_schema))) {
|
|
LOG_WARN("add system variable failed", K(ret));
|
|
}
|
|
} //end for
|
|
}
|
|
} // end HEAP_VAR
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
ObDDLSQLTransaction::~ObDDLSQLTransaction()
|
|
{
|
|
if (is_started()) {
|
|
int ret = end(false);
|
|
if (OB_FAIL(ret)) {
|
|
LOG_WARN("fail end DDL trans", KR(ret));
|
|
}
|
|
}
|
|
}
|
|
|
|
int ObDDLService::reset_parallel_cache(const uint64_t tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int tmp_ret = OB_SUCCESS;
|
|
if (OB_TMP_FAIL(index_name_checker_.reset_cache(tenant_id))) {
|
|
ret = OB_FAIL(ret) ? ret : tmp_ret;
|
|
LOG_ERROR("reset cache failed", KR(tmp_ret), KR(ret), K(tenant_id));
|
|
}
|
|
|
|
if (OB_TMP_FAIL(non_partitioned_tablet_allocator_.reset_cache(tenant_id))) {
|
|
ret = OB_FAIL(ret) ? ret : tmp_ret;
|
|
LOG_ERROR("reset cache failed", KR(tmp_ret), KR(ret), K(tenant_id));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::set_dbms_job_exec_env(const obrpc::ObCreateIndexArg &create_index_arg,
|
|
ObTableSchema& vidx_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_LIKELY(!vidx_table_schema.is_vec_delta_buffer_type())) {
|
|
// do nothing
|
|
} else if (OB_UNLIKELY(create_index_arg.index_type_ != INDEX_TYPE_VEC_DELTA_BUFFER_LOCAL)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("index arg type is not INDEX_TYPE_VEC_DELTA_BUFFER_LOCAL", K(ret), K(create_index_arg.index_type_));
|
|
} else if (OB_FAIL(vidx_table_schema.set_exec_env(create_index_arg.vidx_refresh_info_.exec_env_))) {
|
|
LOG_WARN("fail to set exec env", K(ret));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
/*
|
|
* @description:
|
|
* start transaction for DDL, lock and check schema has refreshed
|
|
* @param[in] proxy
|
|
* @param[in] tenant_id : the tenant of DDL
|
|
* @param[in] tenant_refreshed_schema_version : the schema of tenant refreshed, if is 0, no need to lock and check.
|
|
* @param[in] with_snapshot
|
|
* */
|
|
int ObDDLSQLTransaction::start(ObISQLClient *proxy,
|
|
const uint64_t &tenant_id,
|
|
const int64_t &tenant_refreshed_schema_version,
|
|
bool with_snapshot /*= false*/)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_ISNULL(proxy)
|
|
|| OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id
|
|
|| OB_INVALID_VERSION == tenant_refreshed_schema_version)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid argument", KR(ret), K(tenant_id), KP(proxy),
|
|
K(tenant_refreshed_schema_version));
|
|
} else if (OB_ISNULL(schema_service_)
|
|
|| OB_ISNULL(schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", KR(ret),
|
|
KP(schema_service_), KP(schema_service_->get_schema_service()));
|
|
} else {
|
|
tenant_id_ = tenant_id;
|
|
auto *tsi_oper = GET_TSI(share::schema::TSILastOper);
|
|
if (OB_ISNULL(tsi_oper)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("tsi_oper is null", KR(ret));
|
|
} else {
|
|
start_operation_schema_version_ = tsi_oper->last_operation_schema_version_;
|
|
start_operation_tenant_id_ = tsi_oper->last_operation_tenant_id_;
|
|
}
|
|
|
|
if (enable_ddl_parallel_) {
|
|
enable_check_newest_schema_ = false;
|
|
}
|
|
|
|
if (FAILEDx(common::ObMySQLTransaction::start(proxy, tenant_id, with_snapshot))) {
|
|
LOG_WARN("fail to start trans", KR(ret), K(with_snapshot), K(tenant_id_));
|
|
} else if (0 == tenant_refreshed_schema_version) {
|
|
//maybe in bootstrap or second transaction of create tenant
|
|
LOG_INFO("no need to lock and check schema is newest", KR(ret),
|
|
K(tenant_id), K(tenant_refreshed_schema_version));
|
|
} else if (OB_FAIL(lock_all_ddl_operation(*this, tenant_id_, enable_ddl_parallel_))) {
|
|
LOG_WARN("fail to lock all ddl operation", K(ret), K(tenant_id_));
|
|
} else if (enable_check_newest_schema_) {
|
|
//double check, after lock success, check schema_version is newest before lock
|
|
ObRefreshSchemaStatus schema_status;
|
|
schema_status.tenant_id_ = tenant_id;
|
|
int64_t version_in_inner_table = 0;
|
|
if (OB_FAIL(schema_service_->get_schema_version_in_inner_table(*proxy, schema_status, version_in_inner_table))) {
|
|
LOG_WARN("failed to get version in inner table", KR(ret), K(schema_status));
|
|
} else if (tenant_refreshed_schema_version != version_in_inner_table) {
|
|
ret = OB_EAGAIN;
|
|
LOG_WARN("RS not refresh the newest schema version, try again", KR(ret),
|
|
K(tenant_id), K(tenant_refreshed_schema_version), K(version_in_inner_table));
|
|
} else {
|
|
trans_start_schema_version_ = tenant_refreshed_schema_version;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLSQLTransaction::start(
|
|
ObISQLClient *proxy,
|
|
const uint64_t tenant_id,
|
|
bool with_snapshot /*= false*/,
|
|
const int32_t group_id /* = 0*/)
|
|
{
|
|
int ret = OB_NOT_SUPPORTED;
|
|
UNUSEDx(proxy, with_snapshot, tenant_id, group_id);
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLSQLTransaction::lock_ddl_epoch_(common::ObMySQLTransaction &trans)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("schema service is null", KR(ret));
|
|
} else if (OB_FAIL(schema_service_->get_ddl_epoch_mgr().check_and_lock_ddl_epoch(
|
|
trans, tenant_id_, trans_start_ddl_epoch_))) {
|
|
LOG_WARN("fail to check and lock ddl epoch", KR(ret), K_(tenant_id), K_(trans_start_ddl_epoch));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLSQLTransaction::end(const bool commit)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
// always reset index_name_checker_ before non parallell ddl commits.
|
|
if (commit && !enable_ddl_parallel_) {
|
|
if (OB_ISNULL(GCTX.root_service_)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("root_service is null", KR(ret));
|
|
} else if (OB_FAIL(GCTX.root_service_->get_ddl_service()
|
|
.reset_parallel_cache(tenant_id_))) {
|
|
LOG_WARN("fail to reset parallel cache", KR(ret), K_(tenant_id));
|
|
}
|
|
}
|
|
|
|
int tmp_ret = OB_SUCCESS;
|
|
auto *tsi_oper = GET_TSI(share::schema::TSILastOper);
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(tsi_oper)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get tsi oper", KR(ret));
|
|
} else if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("schema service is null", K(ret));
|
|
} else if (commit && enable_check_ddl_epoch_ && OB_FAIL(lock_ddl_epoch_(*this))) {
|
|
// compare ddl_epoch promise execute on master
|
|
LOG_WARN("lock_ddl_epoch fail", K(ret));
|
|
} else if (commit && need_end_signal_) {
|
|
share::schema::ObSchemaService *schema_service_impl = schema_service_->get_schema_service();
|
|
if (OB_ISNULL(schema_service_impl)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema service is null", K(ret), KP(schema_service_impl), KP(schema_service_));
|
|
} else if (start_operation_schema_version_ == tsi_oper->last_operation_schema_version_
|
|
&& start_operation_tenant_id_ == tsi_oper->last_operation_tenant_id_) {
|
|
LOG_INFO("ddl operation is same, just skip", K(ret),
|
|
K_(start_operation_schema_version), K_(start_operation_tenant_id));
|
|
} else {
|
|
int64_t new_schema_version = OB_INVALID_VERSION;
|
|
obrpc::ObDDLNopOpreatorArg arg;
|
|
arg.schema_operation_.op_type_ = OB_DDL_END_SIGN;
|
|
share::schema::ObDDLSqlService ddl_sql_service(*schema_service_impl);
|
|
arg.schema_operation_.tenant_id_ = tenant_id_;
|
|
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id_, new_schema_version))) {
|
|
LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id_));
|
|
} 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));
|
|
}
|
|
}
|
|
}
|
|
|
|
// new truncate table implement will pass an unusable start schema version,
|
|
// it needs record increment table schemas alone.
|
|
if (OB_SUCC(ret) && !enable_ddl_parallel_ && commit) {
|
|
ObArenaAllocator allocator;
|
|
ObSEArray<const ObTenantSchema*, 2> tenant_schemas;
|
|
ObSEArray<const ObDatabaseSchema*, 2> database_schemas;
|
|
ObSEArray<const ObTableSchema*, 8> table_schemas;
|
|
uint64_t data_version = 0;
|
|
if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id_, data_version))) {
|
|
LOG_WARN("fail to get data version", KR(ret), K_(tenant_id));
|
|
} else if (data_version >= DATA_VERSION_4_1_0_0
|
|
&& trans_start_schema_version_ > 0) {
|
|
if (OB_FAIL(schema_service_->get_increment_schemas_for_data_dict(
|
|
*this, tenant_id_, trans_start_schema_version_,
|
|
allocator, tenant_schemas, database_schemas, table_schemas))) {
|
|
LOG_WARN("fail to get increment schemas for data dict",
|
|
KR(ret), K_(tenant_id), K_(trans_start_schema_version));
|
|
}
|
|
} else {
|
|
// won't record increment schemas in the following cases:
|
|
// 1. tenant not upgrade to 4.1.0.0.
|
|
// 2. bootstrap/create tenant.
|
|
// 3. schema dropped.
|
|
// 4. inner tables changed.
|
|
// tenant_schemas/database_schemas/table_schemas are empty will record log like 4.0 for compatibility.
|
|
}
|
|
if (FAILEDx(serialize_inc_schemas_(allocator, tenant_schemas, database_schemas, table_schemas))) {
|
|
LOG_WARN("serialize_inc_schemas_ fail", KR(ret));
|
|
}
|
|
}
|
|
|
|
if (OB_SUCCESS != (tmp_ret = common::ObMySQLTransaction::end(commit && OB_SUCC(ret)))) {
|
|
LOG_WARN("failed to end transaction", K(ret), K(tmp_ret), K(commit));
|
|
}
|
|
|
|
ret = OB_SUCC(ret) ? tmp_ret : ret;
|
|
// Clear tenant_id_ for success or failure
|
|
tenant_id_ = OB_INVALID_ID;
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLSQLTransaction::serialize_inc_schemas(const int64_t start_schema_version)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObArenaAllocator allocator;
|
|
ObSEArray<const ObTenantSchema*, 2> tenant_schemas;
|
|
ObSEArray<const ObDatabaseSchema*, 2> database_schemas;
|
|
ObSEArray<const ObTableSchema*, 8> table_schemas;
|
|
uint64_t data_version = 0;
|
|
if (OB_ISNULL(schema_service_)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("schema service is null", K(ret));
|
|
} else if (OB_UNLIKELY(!is_started()
|
|
|| tenant_id_ == OB_INVALID_TENANT_ID
|
|
|| start_schema_version <= 0)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", KR(ret), K_(in_trans), K_(tenant_id), K(start_schema_version));
|
|
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id_, data_version))) {
|
|
LOG_WARN("fail to get data version", KR(ret), K_(tenant_id));
|
|
} else if (data_version < DATA_VERSION_4_1_0_0) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("data version is less than 4.1", KR(ret), K_(tenant_id), K(data_version));
|
|
} else if (OB_FAIL(schema_service_->get_increment_schemas_for_data_dict(
|
|
*this, tenant_id_, start_schema_version, allocator,
|
|
tenant_schemas, database_schemas, table_schemas))) {
|
|
LOG_WARN("fail to get increment schemas for data dict",
|
|
KR(ret), K_(tenant_id), K(start_schema_version));
|
|
} else if (OB_FAIL(serialize_inc_schemas_(allocator, tenant_schemas, database_schemas, table_schemas))) {
|
|
LOG_WARN("serialize_inc_schemas_ fail", KR(ret), K_(tenant_id), K(start_schema_version));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLSQLTransaction::serialize_inc_schemas_(
|
|
ObIAllocator &allocator,
|
|
const ObIArray<const ObTenantSchema*> &tenant_schemas,
|
|
const ObIArray<const ObDatabaseSchema*> &database_schemas,
|
|
const ObIArray<const ObTableSchema*> &table_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
char *buf = NULL;
|
|
int64_t buf_len = 0;
|
|
int64_t pos = 0;
|
|
|
|
if (OB_FAIL(datadict::ObDataDictStorage::gen_and_serialize_dict_metas(
|
|
allocator,
|
|
tenant_schemas,
|
|
database_schemas,
|
|
table_schemas,
|
|
buf,
|
|
buf_len,
|
|
pos))) {
|
|
LOG_WARN("serialize_schema into dict metas failed", KR(ret), K_(tenant_id), KP(buf), K(buf_len), K(pos));
|
|
} else if (OB_ISNULL(buf)
|
|
|| OB_UNLIKELY(pos > buf_len)
|
|
|| OB_UNLIKELY(pos < 0)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("expect valid buf", KR(ret), KP(buf), K(buf_len), K(pos));
|
|
} else if (OB_FAIL(register_tx_data(
|
|
tenant_id_,
|
|
SYS_LS,
|
|
transaction::ObTxDataSourceType::DDL_TRANS,
|
|
buf,
|
|
pos))) {
|
|
LOG_WARN("register_msd_into_tx for DDL_TRANS failed", KR(ret), K_(tenant_id));
|
|
} else {
|
|
allocator.free(buf);
|
|
buf = NULL;
|
|
buf_len = 0;
|
|
pos = 0;
|
|
// success
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::do_schema_revise(const obrpc::ObSchemaReviseArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(ret), K(arg));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
switch(arg.type_) {
|
|
case obrpc::ObSchemaReviseArg::SchemaReviseType::REVISE_CONSTRAINT_COLUMN_INFO: {
|
|
if (OB_FAIL(ddl_operator.revise_constraint_column_info(arg, trans))) {
|
|
LOG_WARN("fail to create keystore", K(ret), K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case obrpc::ObSchemaReviseArg::SchemaReviseType::REVISE_NOT_NULL_CONSTRAINT: {
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
LOG_WARN("get schema guard failed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.revise_not_null_constraint_info(arg, schema_guard, trans))) {
|
|
LOG_WARN("fail to revise not null constraint info", K(ret), K(arg));
|
|
}
|
|
break;
|
|
}
|
|
default:
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected schema revise type", K(ret), K(arg.type_));
|
|
break;
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
} else {
|
|
LOG_INFO("finish do schema revise", K(ret), K(arg));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::do_keystore_ddl(const obrpc::ObKeystoreDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObKeystoreSchema &keystore_schema = const_cast<ObKeystoreSchema&>(arg.schema_);
|
|
const ObString *ddl_stmt_str = &arg.ddl_stmt_str_;
|
|
const uint64_t tenant_id = keystore_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(arg), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
bool is_exist = false;
|
|
bool is_set_key = false;
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
switch(arg.type_) {
|
|
case obrpc::ObKeystoreDDLArg::DDLType::CREATE_KEYSTORE: {
|
|
if (OB_FAIL(schema_guard.check_keystore_exist(tenant_id,
|
|
is_exist))) {
|
|
LOG_WARN("fail get keystore from schema guard", K(ret), K(arg));
|
|
} else if (is_exist) {
|
|
ret = OB_KEYSTORE_EXIST;
|
|
LOG_WARN("keystore already exist", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.create_keystore(keystore_schema,
|
|
trans,
|
|
schema_guard,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail create keystore", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_SET_KEY:
|
|
is_set_key = true;
|
|
case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_PASSWORD:
|
|
case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_CLOSE:
|
|
case obrpc::ObKeystoreDDLArg::DDLType::ALTER_KEYSTORE_OPEN: {
|
|
if (OB_FAIL(schema_guard.check_keystore_exist(tenant_id,
|
|
is_exist))) {
|
|
LOG_WARN("fail get keystore from schema guard", K(ret), K(arg));
|
|
} else if (!is_exist) {
|
|
ret = OB_KEYSTORE_NOT_EXIST;
|
|
LOG_WARN("keystore is not exist", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_keystore(keystore_schema,
|
|
trans,
|
|
schema_guard,
|
|
ddl_stmt_str,
|
|
is_set_key,
|
|
arg.is_kms_))) {
|
|
LOG_WARN("fail alter keystore", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default:
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected stmt type", K(arg), K(ret));
|
|
break;
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
LOG_INFO("finish do keystore ddl", K(arg), K(ret));
|
|
return ret;
|
|
}
|
|
|
|
// bugfix:
|
|
// In order to avoid the problem that log_id and schema_version cannot maintain the partial order relationship
|
|
// caused by concurrent submission of DDL transactions of different sessions of inner_sql.
|
|
// Before the DDL transaction is submitted, lock the same row of __all_ddl_operation (non-partitioned table)
|
|
// to provide a table lock-like function to keep RS serially submitting DDL transactions.
|
|
//
|
|
// insert requires special support) and to avoid modifying the internal table during upgrade, lock the min(schema_version)
|
|
// corresponding line of __all_ddl_operation.
|
|
// 1. __all_ddl_operation is currently unclear about history,
|
|
// and it can ensure that min (schema_version) corresponding lines are not deleted.
|
|
// 2. Only bootstrap, and when building tenant transaction 2, __all_ddl_operation is empty,
|
|
// and the DDL of other tenants can guarantee that the corresponding row of min (schema_version) can be locked.
|
|
// However, the two scenarios either cause bootstrap to fail, or tenant creation fails.
|
|
// You can restart bootstrap or rebuild the tenant to avoid this problem.
|
|
int ObDDLSQLTransaction::lock_all_ddl_operation(
|
|
ObMySQLTransaction &trans,
|
|
const uint64_t tenant_id)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (OB_INVALID_TENANT_ID == tenant_id
|
|
|| OB_INVALID_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObSqlString sql;
|
|
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
|
common::sqlclient::ObMySQLResult *result = NULL;
|
|
if (OB_FAIL(sql.append_fmt(
|
|
"select schema_version from %s where schema_version in ("
|
|
"select min(schema_version) as schema_version from %s) for update",
|
|
OB_ALL_DDL_OPERATION_TNAME, OB_ALL_DDL_OPERATION_TNAME))) {
|
|
LOG_WARN("failed to append sql", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.read(res, tenant_id, sql.ptr()))) {
|
|
LOG_WARN("failed to execute sql", KR(ret), K(tenant_id), K(sql));
|
|
} else if (OB_ISNULL(result = res.get_result())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get result", K(ret));
|
|
} else if (OB_FAIL(result->next())) {
|
|
LOG_WARN("fail to get result", KR(ret), K(tenant_id));
|
|
} else {
|
|
int64_t schema_version = OB_INVALID_VERSION;
|
|
EXTRACT_INT_FIELD_MYSQL(*result, "schema_version", schema_version, int64_t);
|
|
LOG_DEBUG("lock __all_ddl_operation by schema_version",
|
|
KR(ret), K(tenant_id), K(schema_version));
|
|
if (OB_SUCC(ret)) {
|
|
int64_t tmp_ret = result->next();
|
|
if (OB_SUCCESS == tmp_ret) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("more than one row", KR(ret), K(tenant_id));
|
|
} else if (OB_ITER_END == tmp_ret) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
ret = tmp_ret;
|
|
LOG_WARN("fail to get next row", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLSQLTransaction::lock_all_ddl_operation(
|
|
ObMySQLTransaction &trans,
|
|
const uint64_t tenant_id,
|
|
const bool enable_parallel)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
|
|
ObTimeoutCtx ctx;
|
|
if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, GCONF.rpc_timeout))) {
|
|
LOG_WARN("fail to set timeout ctx", KR(ret));
|
|
} else {
|
|
// ddl service init after RS master start service
|
|
// we don't want ddl service grab lock block RS master start
|
|
// so trigger ddl epoch load delay
|
|
int64_t ddl_epoch_tmp = 0;
|
|
if (enable_check_ddl_epoch_) {
|
|
if (OB_FAIL(schema_service_->get_ddl_epoch_mgr().get_ddl_epoch(tenant_id, ddl_epoch_tmp))) {
|
|
if (OB_ENTRY_NOT_EXIST == ret) {
|
|
ret = OB_SUCCESS;
|
|
if (!GCTX.root_service_->in_service()) {
|
|
ret = OB_RS_NOT_MASTER;
|
|
LOG_WARN("rs not in service", K(ret));
|
|
} else if (OB_FAIL(schema_service_->get_ddl_epoch_mgr().promote_ddl_epoch(tenant_id, ctx.get_timeout(), ddl_epoch_tmp))) {
|
|
LOG_WARN("promote epoch fail", K(ret), K(tenant_id));
|
|
} else {
|
|
trans_start_ddl_epoch_ = ddl_epoch_tmp;
|
|
}
|
|
} else {
|
|
LOG_WARN("get_ddl_epoch", K(ret), K(tenant_id));
|
|
}
|
|
} else {
|
|
trans_start_ddl_epoch_ = ddl_epoch_tmp;
|
|
}
|
|
}
|
|
}
|
|
/*
|
|
* for compat
|
|
* enable_ddl_trans_new_lock default false use old all_ddl_operation row lock
|
|
* if MIN_DATA_VERSION >= 4.1 use new lock and set flag [enable_ddl_trans_new_lock]
|
|
*
|
|
*/
|
|
|
|
if (OB_SUCC(ret)) {
|
|
bool enable_ddl_trans_new_lock = false;
|
|
observer::ObInnerSQLConnection *conn = NULL;
|
|
if (OB_ISNULL(conn = dynamic_cast<observer::ObInnerSQLConnection *>(trans.get_connection()))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("conn_ is NULL", KR(ret));
|
|
} else {
|
|
ObLockObjRequest lock_arg;
|
|
lock_arg.obj_type_ = ObLockOBJType::OBJ_TYPE_TENANT;
|
|
lock_arg.obj_id_ = tenant_id;
|
|
lock_arg.owner_id_.set_default();
|
|
lock_arg.lock_mode_ = !enable_parallel ? EXCLUSIVE : SHARE;
|
|
lock_arg.op_type_ = ObTableLockOpType::IN_TRANS_COMMON_LOCK;
|
|
lock_arg.timeout_us_ = ctx.get_timeout();
|
|
if (OB_FAIL(ObInnerConnectionLockUtil::lock_obj(tenant_id,
|
|
lock_arg,
|
|
conn))) {
|
|
LOG_WARN("lock table failed", KR(ret), K(tenant_id));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::do_tablespace_ddl(const obrpc::ObTablespaceDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTablespaceSchema &tablespace_schema = const_cast<ObTablespaceSchema&>(arg.schema_);
|
|
const ObString &tablespace_name = tablespace_schema.get_tablespace_name();
|
|
const ObString *ddl_stmt_str = &arg.ddl_stmt_str_;
|
|
const uint64_t tenant_id = tablespace_schema.get_tenant_id();
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (!arg.is_valid()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid arg", K(arg), K(ret));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get latest schema version in inner table", K(ret), K(tenant_id));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
const ObTablespaceSchema *ts_schema = NULL;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
switch(arg.type_) {
|
|
case obrpc::ObTablespaceDDLArg::DDLType::CREATE_TABLESPACE: {
|
|
if (OB_FAIL(schema_guard.get_tablespace_schema_with_name(tenant_id,
|
|
tablespace_name, ts_schema))) {
|
|
LOG_WARN("fail get tablespace from schema guard", K(ret), K(tablespace_name), K(arg));
|
|
} else if (ts_schema) {
|
|
ret = OB_TABLESPACE_EXIST;
|
|
LOG_USER_ERROR(OB_TABLESPACE_EXIST, tablespace_name.length(), tablespace_name.ptr());
|
|
LOG_WARN("tablespace already exist", K(ret), K(tablespace_name));
|
|
} else if (OB_FAIL(ddl_operator.create_tablespace(tablespace_schema,
|
|
trans,
|
|
schema_guard,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail create tablespace", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case obrpc::ObTablespaceDDLArg::DDLType::DROP_TABLESPACE: {
|
|
if (OB_FAIL(schema_guard.get_tablespace_schema_with_name(tenant_id,
|
|
tablespace_name, ts_schema))) {
|
|
LOG_WARN("fail get tablespace from schema guard", K(ret), K(tablespace_name), K(arg));
|
|
} else if (OB_ISNULL(ts_schema)) {
|
|
ret = OB_TABLESPACE_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TABLESPACE_NOT_EXIST, tablespace_name.length(), tablespace_name.ptr());
|
|
LOG_WARN("tablespace does not exist", K(ret), K(tablespace_name), K(arg));
|
|
} else if (OB_FAIL(tablespace_schema.assign(*ts_schema))) {
|
|
LOG_WARN("fail to assign tablespace schema", KR(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_tablespace(tablespace_schema,
|
|
trans,
|
|
schema_guard,
|
|
ddl_stmt_str))) {
|
|
LOG_WARN("fail drop tablespace", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case obrpc::ObTablespaceDDLArg::DDLType::ALTER_TABLESPACE: {
|
|
if (OB_FAIL(schema_guard.get_tablespace_schema_with_name(tenant_id,
|
|
tablespace_name, ts_schema))) {
|
|
LOG_WARN("fail get tablespace from schema guard", K(ret), K(tablespace_name), K(arg));
|
|
} else if (OB_ISNULL(ts_schema)) {
|
|
ret = OB_TABLESPACE_NOT_EXIST;
|
|
LOG_USER_ERROR(OB_TABLESPACE_NOT_EXIST,
|
|
tablespace_name.length(), tablespace_name.ptr());
|
|
LOG_WARN("tablespace does not exist", K(ret), K(tablespace_name), K(arg));
|
|
} else if (OB_FAIL(ddl_operator.alter_tablespace(tablespace_schema,
|
|
trans, schema_guard, ddl_stmt_str))) {
|
|
LOG_WARN("fail to alter tablespace", K(arg), K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default:
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected stmt type", K(arg), K(ret));
|
|
break;
|
|
}
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
LOG_INFO("finish do tablespace ddl", K(arg), K(ret));
|
|
return ret;
|
|
}
|
|
int ObDDLService::handle_profile_ddl(const ObProfileDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObProfileSchema schema = arg.schema_; //make a copy
|
|
uint64_t tenant_id = schema.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version), K(tenant_id));
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(ddl_operator.handle_profile_function(schema,
|
|
trans,
|
|
arg.ddl_type_,
|
|
arg.ddl_stmt_str_,
|
|
schema_guard))) {
|
|
LOG_WARN("handle profile function failed", K(ret), K(arg));
|
|
}
|
|
}
|
|
|
|
// Handle user cascading delete
|
|
if (OB_SUCC(ret) && OB_DDL_DROP_PROFILE == arg.ddl_type_) {
|
|
ObSEArray<uint64_t, 64> user_ids_with_profile;
|
|
if (OB_FAIL(get_all_users_in_tenant_with_profile(tenant_id,
|
|
schema.get_profile_id(),
|
|
schema_guard,
|
|
user_ids_with_profile))) {
|
|
LOG_WARN("fail to get all users with profile id", K(schema.get_profile_id()), K(ret));
|
|
} else if (user_ids_with_profile.count() > 0 && !arg.is_cascade_) {
|
|
ret = OB_ERR_PROFILE_STRING_HAS_USERS_ASSIGNED;
|
|
LOG_USER_ERROR(OB_ERR_PROFILE_STRING_HAS_USERS_ASSIGNED, schema.get_profile_name_str().length(), schema.get_profile_name_str().ptr());
|
|
} else if (arg.is_cascade_) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support feature", K(ret));
|
|
}
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::create_directory(const obrpc::ObCreateDirectoryArg &arg, const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const bool is_or_replace = arg.or_replace_;
|
|
const uint64_t tenant_id = arg.schema_.get_tenant_id();
|
|
const uint64_t user_id = arg.user_id_;
|
|
const ObString &directory_name = arg.schema_.get_directory_name();
|
|
const ObString &directory_path = arg.schema_.get_directory_path();
|
|
const ObDirectorySchema *schema_ptr = NULL;
|
|
bool is_exist = false;
|
|
bool is_oracle_mode = false;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
ObDirectorySchema new_schema;
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) {
|
|
LOG_WARN("failed to get compat mode", K(ret), K(tenant_id));
|
|
} else if (lib::Worker::CompatMode::ORACLE == compat_mode
|
|
&& FALSE_IT(is_oracle_mode = true)) {
|
|
// do nothing
|
|
} else if (!is_oracle_mode) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("create directory under non oracle mode is not supported", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "create directory under non oracle mode");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_directory_schema_by_name(tenant_id, directory_name, schema_ptr))) {
|
|
LOG_WARN("failed to get directory schema by name", K(ret), K(tenant_id), K(directory_name));
|
|
} else if (NULL != schema_ptr) {
|
|
is_exist = true;
|
|
if (OB_FAIL(new_schema.assign(*schema_ptr))) {
|
|
LOG_WARN("failed to assign new directory schema", K(ret), K(*schema_ptr));
|
|
} else if (OB_FAIL(new_schema.set_directory_path(directory_path))) {
|
|
LOG_WARN("failed to set directory path", K(ret), K(directory_path));
|
|
}
|
|
} else if (NULL == schema_ptr) {
|
|
if (OB_FAIL(new_schema.assign(arg.schema_))) {
|
|
LOG_WARN("failed to assign new directory schema", K(ret), K(arg));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
// do nothing
|
|
} else if (is_exist && !is_or_replace) {
|
|
ret = OB_ERR_EXIST_OBJECT;
|
|
LOG_WARN("directory already exists and is not replace operation", K(ret),
|
|
K(is_or_replace), K(directory_name));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (is_exist && is_or_replace
|
|
&& OB_FAIL(ddl_operator.alter_directory(*ddl_stmt_str, new_schema, trans))) {
|
|
LOG_WARN("failed to alter directory", K(ret), K(new_schema));
|
|
} else if (!is_exist && OB_FAIL(ddl_operator.create_directory(*ddl_stmt_str, user_id, new_schema, trans))) {
|
|
LOG_WARN("failed to create directory", K(ret), K(new_schema));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_directory(const obrpc::ObDropDirectoryArg &arg, const ObString *ddl_stmt_str)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = arg.tenant_id_;
|
|
const ObString &directory_name = arg.directory_name_;
|
|
const ObDirectorySchema *schema_ptr = NULL;
|
|
bool is_exist = false;
|
|
bool is_oracle_mode = false;
|
|
lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::INVALID;
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init", K(ret));
|
|
} else if (OB_FAIL(ObCompatModeGetter::get_tenant_mode(tenant_id, compat_mode))) {
|
|
LOG_WARN("failed to get compat mode", K(ret), K(tenant_id));
|
|
} else if (lib::Worker::CompatMode::ORACLE == compat_mode
|
|
&& FALSE_IT(is_oracle_mode = true)) {
|
|
// do nothing
|
|
} else if (!is_oracle_mode) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("drop directory under non oracle mode is not supported", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "drop directory under non oracle mode");
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("failed to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_directory_schema_by_name(tenant_id, directory_name, schema_ptr))) {
|
|
LOG_WARN("failed to get schema by directory name", K(ret), K(tenant_id), K(directory_name));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (NULL != schema_ptr) {
|
|
is_exist = true;
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
// do nothing
|
|
} else if (!is_exist) {
|
|
ret = OB_ERR_OBJECT_STRING_DOES_NOT_EXIST;
|
|
LOG_WARN("directory does not exist", K(ret), K(directory_name));
|
|
LOG_USER_ERROR(OB_ERR_OBJECT_STRING_DOES_NOT_EXIST,
|
|
static_cast<int>(directory_name.length()),
|
|
directory_name.ptr());
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
ObDirectorySchema schema;
|
|
if (OB_FAIL(schema.assign(*schema_ptr))) {
|
|
LOG_WARN("fail to assign directory schema", K(ret), K(*schema_ptr));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start transaction", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.drop_directory(*ddl_stmt_str, schema, trans))) {
|
|
LOG_WARN("failed to drop directory", K(ret), K(schema));
|
|
}
|
|
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::force_set_locality(
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObTenantSchema &new_tenant)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
common::ObArray<common::ObZone> zones_in_pool;
|
|
common::ObArray<share::schema::ObZoneRegion> zone_region_list;
|
|
common::ObArray<share::ObResourcePoolName> resource_pool_names;
|
|
uint64_t tenant_id = new_tenant.get_tenant_id();
|
|
const ObTenantSchema *orig_meta_tenant = NULL;
|
|
ObTenantSchema new_meta_tenant;
|
|
|
|
obrpc::ObModifyTenantArg dummy_arg;
|
|
if (OB_FAIL(get_new_tenant_pool_zone_list(dummy_arg, new_tenant, resource_pool_names,
|
|
zones_in_pool, zone_region_list))) {
|
|
LOG_WARN("fail to get new tenant pool zone list", KR(ret), K(new_tenant));
|
|
} else if (OB_FAIL(parse_and_set_create_tenant_new_locality_options(
|
|
schema_guard, new_tenant, resource_pool_names,
|
|
zones_in_pool, zone_region_list))) {
|
|
LOG_WARN("fail to parse and set new locality option", KR(ret), K(new_tenant));
|
|
} else {
|
|
// deal with meta tenant related to a certain user tenant
|
|
if (is_user_tenant(tenant_id)) {
|
|
if (OB_FAIL(schema_guard.get_tenant_info(gen_meta_tenant_id(tenant_id), orig_meta_tenant))) {
|
|
LOG_WARN("fail to get meta tenant schema", KR(ret), "meta_tenant_id", gen_meta_tenant_id(tenant_id));
|
|
} else if (OB_ISNULL(orig_meta_tenant)) {
|
|
ret = OB_TENANT_NOT_EXIST;
|
|
LOG_WARN("meta tenant not exist", KR(ret), "meta_tenant_id", gen_meta_tenant_id(tenant_id));
|
|
} else if (OB_FAIL(new_meta_tenant.assign(*orig_meta_tenant))) {
|
|
LOG_WARN("fail to assgin meta tenant schema", KR(ret), KPC(orig_meta_tenant));
|
|
} else if (OB_FAIL(new_meta_tenant.set_locality(new_tenant.get_locality_str()))) {
|
|
LOG_WARN("fail to set locality", KR(ret), "locality str", new_tenant.get_locality_str());
|
|
} else if (OB_FAIL(new_meta_tenant.set_previous_locality(ObString("")))) {
|
|
LOG_WARN("fail to reset meta tenant previous locality", KR(ret));
|
|
} else if (OB_FAIL(parse_and_set_create_tenant_new_locality_options(
|
|
schema_guard, new_meta_tenant, resource_pool_names,
|
|
zones_in_pool, zone_region_list))) {
|
|
LOG_WARN("fail to parse and set meta tenant new locality option", KR(ret), K(new_meta_tenant));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version));
|
|
} else if (OB_FAIL(ddl_operator.alter_tenant(new_tenant, trans))) {
|
|
LOG_WARN("failed to alter tenant", KR(ret));
|
|
} else if (is_user_tenant(tenant_id) && OB_FAIL(ddl_operator.alter_tenant(new_meta_tenant, trans))) {
|
|
LOG_WARN("failed to alter meta tenant", KR(ret));
|
|
}
|
|
if (trans.is_started()) {
|
|
int temp_ret = OB_SUCCESS;
|
|
if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) {
|
|
LOG_WARN("trans end failed", "is_commit", OB_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
// publish schema
|
|
if (OB_SUCC(ret) && OB_FAIL(publish_schema(OB_SYS_TENANT_ID))) {
|
|
LOG_WARN("publish schema failed, ", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::add_sys_table_lob_aux(
|
|
const int64_t tenant_id,
|
|
const uint64_t table_id,
|
|
ObTableSchema &meta_schema,
|
|
ObTableSchema &data_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
meta_schema.reset();
|
|
data_schema.reset();
|
|
if (OB_ALL_CORE_TABLE_TID == table_id) {
|
|
// do nothing
|
|
} else if (OB_FAIL(get_sys_table_lob_aux_schema(table_id, meta_schema, data_schema))) {
|
|
LOG_WARN("fail to get sys table's index schema", KR(ret), K(table_id));
|
|
} else if (OB_FAIL(ObSchemaUtils::construct_tenant_space_full_table(
|
|
tenant_id, meta_schema))) {
|
|
LOG_WARN("fail to construct tenant meta table", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(ObSchemaUtils::construct_tenant_space_full_table(
|
|
tenant_id, data_schema))) {
|
|
LOG_WARN("fail to construct tenant data schema", KR(ret), K(tenant_id));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::get_tenant_external_consistent_ts(const int64_t tenant_id, SCN &scn)
|
|
{
|
|
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, scn,
|
|
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(scn),
|
|
K(is_external_consistent));
|
|
} else {
|
|
LOG_INFO("success to get_tenant_external_consistent_ts", K(tenant_id), K(scn),
|
|
K(is_external_consistent));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_has_multi_autoinc(ObTableSchema &table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema::const_column_iterator it_begin = table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = table_schema.column_end();
|
|
ObColumnSchemaV2 *new_column_schema = nullptr;
|
|
bool has_autoinc_col = false;
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(new_column_schema = static_cast<ObColumnSchemaV2 *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else if (new_column_schema->is_autoincrement()) {
|
|
if (has_autoinc_col) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "More than one auto increment column");
|
|
LOG_WARN("Only one auto increment row is allowed", K(ret));
|
|
} else {
|
|
table_schema.set_autoinc_column_id(new_column_schema->get_column_id());
|
|
has_autoinc_col = true;
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// check whether it's modify column not null or modify constraint state, which need send two rpc.
|
|
int ObDDLService::need_modify_not_null_constraint_validate(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
bool &is_add_not_null_col,
|
|
bool &need_modify) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
need_modify = false;
|
|
is_add_not_null_col = false;
|
|
ObSchemaGetterGuard schema_guard;
|
|
schema_guard.set_session_id(alter_table_arg.session_id_);
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
const uint64_t tenant_id = alter_table_schema.get_tenant_id();
|
|
const ObString &origin_database_name = alter_table_schema.get_origin_database_name();
|
|
const ObString &origin_table_name = alter_table_schema.get_origin_table_name();
|
|
const ObTableSchema *orig_table_schema = NULL;
|
|
if (!inited_) {
|
|
ret = OB_NOT_INIT;
|
|
LOG_WARN("not init", KR(ret));
|
|
} else if (obrpc::ObAlterTableArg::ADD_CONSTRAINT != alter_table_arg.alter_constraint_type_
|
|
&& obrpc::ObAlterTableArg::ALTER_CONSTRAINT_STATE != alter_table_arg.alter_constraint_type_) {
|
|
// skip
|
|
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get tenant schema guard", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
|
origin_database_name,
|
|
origin_table_name,
|
|
false,
|
|
orig_table_schema))) {
|
|
LOG_WARN("fail to get table schema", KR(ret), K(tenant_id), K(origin_database_name),
|
|
K(origin_table_name));
|
|
} else if (OB_ISNULL(orig_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("NULL ptr", K(ret), KR(tenant_id), K(alter_table_arg), K(schema_guard.get_session_id()));
|
|
} else if (alter_table_arg.alter_table_schema_.get_constraint_count() == 1) {
|
|
ObTableSchema::const_constraint_iterator iter =
|
|
alter_table_arg.alter_table_schema_.constraint_begin();
|
|
if (OB_ISNULL(iter) || OB_ISNULL(*iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("constraint is null", K(ret));
|
|
} else if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) {
|
|
if (OB_UNLIKELY(1 != (*iter)->get_column_cnt())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected column count of not null constraint", K(ret), KPC(*iter));
|
|
} else if (!(*iter)->get_need_validate_data()) {
|
|
// don't need validate data, do nothing.
|
|
} else if (OB_INVALID_ID == *(*iter)->cst_col_begin()) {
|
|
is_add_not_null_col = true;
|
|
} else {
|
|
need_modify = true;
|
|
}
|
|
}
|
|
} else if (alter_table_arg.alter_table_schema_.get_constraint_count() > 1) {
|
|
// more than one constraint, check column_id of all not null constraint must be invalid.
|
|
// since we only support add more than one not null column in one ddl,
|
|
// not support modify more than one column not null in one ddl.
|
|
ObTableSchema::const_constraint_iterator iter =
|
|
alter_table_arg.alter_table_schema_.constraint_begin();
|
|
for(; iter != alter_table_arg.alter_table_schema_.constraint_end() && OB_SUCC(ret); iter++) {
|
|
if (OB_ISNULL(iter) || OB_ISNULL(*iter)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("constraint is null", K(ret));
|
|
} else if (CONSTRAINT_TYPE_NOT_NULL == (*iter)->get_constraint_type()) {
|
|
if (OB_UNLIKELY(1 != (*iter)->get_column_cnt())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected column count of not null constraint", K(ret), KPC(*iter));
|
|
} else if (OB_UNLIKELY(OB_INVALID_ID != *(*iter)->cst_col_begin())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("modify not null column is not allowed with other DDL", K(ret));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "Add/modify not null constraint together with other DDLs");
|
|
}
|
|
}
|
|
}
|
|
is_add_not_null_col = true;
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
bool ObDDLService::need_check_constraint_validity(const obrpc::ObAlterTableArg &alter_table_arg) const
|
|
{
|
|
bool need_check_validity = false;
|
|
ObTableSchema::const_constraint_iterator iter =
|
|
alter_table_arg.alter_table_schema_.constraint_begin();
|
|
// a ddl query add at most one constraint before.
|
|
// after support not null constraint, in order to support alter table add multiple
|
|
// not null columns, we have to support a ddl query add multiple constraint.
|
|
for (; !need_check_validity
|
|
&& iter != alter_table_arg.alter_table_schema_.constraint_end(); iter++) {
|
|
need_check_validity = (*iter)->get_need_validate_data();
|
|
}
|
|
return need_check_validity;
|
|
}
|
|
|
|
int ObDDLService::adjust_trigger_action_order(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObDDLSQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
ObTriggerInfo &trigger_info,
|
|
bool is_create_trigger)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
#define ALTER_OLD_TRIGGER(source_trg_info) \
|
|
ObTriggerInfo copy_trg_info; \
|
|
OZ (copy_trg_info.assign(*source_trg_info)); \
|
|
OX (copy_trg_info.set_action_order(new_action_order)); \
|
|
OZ (ddl_operator.alter_trigger(copy_trg_info, trans, NULL, false/*is_update_table_schema_version*/));
|
|
|
|
bool is_oracle_mode = false;
|
|
const uint64_t tenant_id = trigger_info.get_tenant_id();
|
|
const ObTableSchema *table_schema = NULL;
|
|
OZ (ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode));
|
|
OZ (schema_guard.get_table_schema(tenant_id, trigger_info.get_base_object_id(), table_schema));
|
|
OV (OB_NOT_NULL(table_schema));
|
|
if (OB_SUCC(ret)) {
|
|
const common::ObIArray<uint64_t> &trg_list = table_schema->get_trigger_list();
|
|
const ObTriggerInfo *old_trg_info = NULL;
|
|
int64_t new_action_order = 0; // the old trigger's new action order
|
|
if (is_create_trigger) {
|
|
int64_t action_order = 1; // action order for the trigger being created
|
|
const ObTriggerInfo *ref_trg_info = NULL;
|
|
if (OB_SUCC(ret)) {
|
|
if (!trigger_info.get_ref_trg_name().empty()) {
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trigger_info.get_database_id(),
|
|
trigger_info.get_ref_trg_name(), ref_trg_info));
|
|
OV (OB_NOT_NULL(ref_trg_info));
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (is_oracle_mode) {
|
|
OZ (recursive_check_trigger_ref_cyclic(schema_guard, trigger_info, trg_list,
|
|
trigger_info.get_trigger_name(), trigger_info.get_ref_trg_name()));
|
|
if (OB_SUCC(ret)) {
|
|
if (NULL != ref_trg_info) {
|
|
uint64_t ref_db_id = OB_INVALID_ID;
|
|
OZ (schema_guard.get_database_id(tenant_id, trigger_info.get_ref_trg_db_name(), ref_db_id));
|
|
OZ (schema_guard.get_trigger_info(tenant_id, ref_db_id, trigger_info.get_ref_trg_name(), ref_trg_info));
|
|
if (OB_SUCC(ret) && trigger_info.is_order_follows()) {
|
|
action_order = ref_trg_info->get_action_order() + 1;
|
|
}
|
|
}
|
|
OZ (recursive_alter_ref_trigger(schema_guard, trans, ddl_operator, trigger_info,
|
|
trg_list, trigger_info.get_trigger_name(), action_order));
|
|
}
|
|
} else {
|
|
if (NULL == ref_trg_info) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < trg_list.count(); i++) {
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trg_list.at(i), old_trg_info));
|
|
OV (OB_NOT_NULL(old_trg_info));
|
|
if (OB_SUCC(ret) && ObTriggerInfo::is_same_timing_event(trigger_info, *old_trg_info)) {
|
|
action_order++;
|
|
}
|
|
}
|
|
} else {
|
|
bool is_follows = trigger_info.is_order_follows();
|
|
action_order = is_follows ? ref_trg_info->get_action_order() + 1 : ref_trg_info->get_action_order();
|
|
// ref_trg_info need to modify
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < trg_list.count(); i++) {
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trg_list.at(i), old_trg_info));
|
|
OV (OB_NOT_NULL(old_trg_info));
|
|
if (OB_SUCC(ret) && ObTriggerInfo::is_same_timing_event(trigger_info, *old_trg_info)
|
|
&& trigger_info.get_trigger_id() != old_trg_info->get_trigger_id()
|
|
&& ref_trg_info->get_trigger_id() != old_trg_info->get_trigger_id()) {
|
|
if (ref_trg_info->get_action_order() < old_trg_info->get_action_order()) {
|
|
new_action_order = old_trg_info->get_action_order() + 1;
|
|
ALTER_OLD_TRIGGER(old_trg_info);
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && !is_follows) {
|
|
// if `PRECEDES`, the ref_trg_info action_order need to +1
|
|
new_action_order = ref_trg_info->get_action_order() + 1;
|
|
ALTER_OLD_TRIGGER(ref_trg_info);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
OX (trigger_info.set_action_order(action_order));
|
|
} else if (!is_oracle_mode) {
|
|
if (OB_SUCC(ret)) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < trg_list.count(); i++) {
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trg_list.at(i), old_trg_info));
|
|
OV (OB_NOT_NULL(old_trg_info));
|
|
if (OB_SUCC(ret) && ObTriggerInfo::is_same_timing_event(trigger_info, *old_trg_info)
|
|
&& trigger_info.get_trigger_id() != old_trg_info->get_trigger_id()
|
|
&& trigger_info.get_action_order() < old_trg_info->get_action_order()) {
|
|
new_action_order = old_trg_info->get_action_order() - 1;
|
|
ALTER_OLD_TRIGGER(old_trg_info);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
#undef ALTER_OLD_TRIGGER
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::recursive_alter_ref_trigger(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
ObDDLSQLTransaction &trans,
|
|
ObDDLOperator &ddl_operator,
|
|
const ObTriggerInfo &ref_trigger_info,
|
|
const common::ObIArray<uint64_t> &trigger_list,
|
|
const ObString &trigger_name,
|
|
int64_t action_order)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
int64_t tenant_id = ref_trigger_info.get_tenant_id();
|
|
const ObTriggerInfo *trg_info = NULL;
|
|
int64_t new_action_order = 0;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < trigger_list.count(); i++) {
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trigger_list.at(i), trg_info));
|
|
OV (OB_NOT_NULL(trg_info));
|
|
if (0 != trg_info->get_trigger_name().case_compare(trigger_name)) {
|
|
if (OB_SUCC(ret) && 0 == trg_info->get_ref_trg_name().case_compare(ref_trigger_info.get_trigger_name())) {
|
|
ObTriggerInfo copy_trg_info;
|
|
OX (new_action_order = action_order + 1);
|
|
OZ (copy_trg_info.assign(*trg_info));
|
|
OX (copy_trg_info.set_action_order(new_action_order));
|
|
OZ (ddl_operator.alter_trigger(copy_trg_info, trans, NULL, false/*is_update_table_schema_version*/));
|
|
OZ (SMART_CALL(recursive_alter_ref_trigger(schema_guard, trans, ddl_operator,
|
|
*trg_info, trigger_list, trigger_name, new_action_order)));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::recursive_check_trigger_ref_cyclic(share::schema::ObSchemaGetterGuard &schema_guard,
|
|
const ObTriggerInfo &ref_trigger_info,
|
|
const common::ObIArray<uint64_t> &trigger_list,
|
|
const ObString &create_trigger_name,
|
|
const ObString &generate_cyclic_name)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = ref_trigger_info.get_tenant_id();
|
|
const ObTriggerInfo *trg_info = NULL;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < trigger_list.count(); i++) {
|
|
OZ (schema_guard.get_trigger_info(tenant_id, trigger_list.at(i), trg_info));
|
|
OV (OB_NOT_NULL(trg_info));
|
|
if (OB_SUCC(ret)) {
|
|
if (0 != trg_info->get_trigger_name().case_compare(create_trigger_name)) {
|
|
if (0 == trg_info->get_ref_trg_name().case_compare(ref_trigger_info.get_trigger_name())) {
|
|
if (0 == trg_info->get_trigger_name().case_compare(generate_cyclic_name)) {
|
|
ret = OB_ERR_REF_CYCLIC_IN_TRG;
|
|
LOG_WARN("OBE-25023: cyclic trigger dependency is not allowed", K(ret),
|
|
K(generate_cyclic_name), KPC(trg_info));
|
|
}
|
|
OZ (SMART_CALL(recursive_check_trigger_ref_cyclic(schema_guard, *trg_info, trigger_list,
|
|
create_trigger_name, generate_cyclic_name)));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_rls_policy_ddl(const obrpc::ObRlsPolicyDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObRlsPolicySchema schema;
|
|
const ObString &policy_name = arg.schema_.get_policy_name();
|
|
const ObString &ddl_stmt_str = arg.ddl_stmt_str_;
|
|
uint64_t tenant_id = arg.schema_.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("failed to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema.assign(arg.schema_))) {
|
|
LOG_WARN("failed to assign rls policy schema", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const ObTableSchema *table_schema = NULL;
|
|
const ObRlsGroupSchema *rls_group_schema = NULL;
|
|
const ObRlsPolicySchema *old_schema = NULL;
|
|
bool is_db_in_recyclebin = false;
|
|
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, schema.get_table_id(), table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(schema));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table not exist", KR(ret), K(arg));
|
|
} else if (table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("rls table is in recyclebin", KR(ret), K(arg));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("failed to check database in recyclebin", KR(ret), KPC(table_schema));
|
|
} else if (is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("database of rls table is in recyclebin", KR(ret), KPC(table_schema));
|
|
} else if (!table_schema->is_user_table() && !table_schema->is_view_table() && !table_schema->is_external_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("only support rls on user table or user view", KR(ret), KPC(table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "policy on non-user table");
|
|
} else if (0 == schema.get_rls_group_id()) {
|
|
// the policy does not belong to any group, do nothing
|
|
} else if (OB_FAIL(schema_guard.get_rls_group_schema_by_id(tenant_id,
|
|
schema.get_rls_group_id(),
|
|
rls_group_schema))) {
|
|
LOG_WARN("failed to get rls group schema", KR(ret), K(arg));
|
|
} else if (OB_ISNULL(rls_group_schema)) {
|
|
ret = OB_ERR_POLICY_GROUP_NOT_EXIST;
|
|
LOG_WARN("rls group not exist", KR(ret), K(schema));
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_rls_policy_schema_by_name(tenant_id, schema.get_table_id(),
|
|
schema.get_rls_group_id(), policy_name, old_schema))) {
|
|
LOG_WARN("failed to get rls policy from schema guard", K(ret), K(policy_name), K(arg));
|
|
} else {
|
|
switch (arg.ddl_type_) {
|
|
case OB_DDL_CREATE_RLS_POLICY: {
|
|
if (OB_NOT_NULL(old_schema)) {
|
|
ret = OB_ERR_POLICY_EXIST;
|
|
LOG_WARN("rls policy exist", KPC(old_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.create_rls_policy(schema, trans, ddl_stmt_str,
|
|
true, table_schema))) {
|
|
LOG_WARN("failed to create rls policy", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_DROP_RLS_POLICY: {
|
|
if (OB_ISNULL(old_schema)) {
|
|
ret = OB_ERR_POLICY_NOT_EXIST;
|
|
LOG_WARN("rls policy not exist", K(ret), K(policy_name), K(arg));
|
|
} else if (OB_FAIL(schema.assign(*old_schema))) {
|
|
LOG_WARN("failed to assign rls policy schema", KPC(old_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_rls_policy(schema, trans, ddl_stmt_str,
|
|
true, table_schema))) {
|
|
LOG_WARN("failed to drop rls policy", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_ALTER_RLS_POLICY: {
|
|
if (OB_ISNULL(old_schema)) {
|
|
ret = OB_ERR_POLICY_NOT_EXIST;
|
|
LOG_WARN("rls policy not exist", K(ret), K(policy_name), K(arg));
|
|
} else if (OB_FAIL(schema.assign(*old_schema))) {
|
|
LOG_WARN("failed to assign rls policy schema", KPC(old_schema), K(ret));
|
|
} else if (arg.option_bitset_.has_member(ObRlsPolicyDDLArg::AlterOption::ENABLE)) {
|
|
schema.set_enable_flag(arg.schema_.get_enable_flag());
|
|
if (OB_FAIL(ddl_operator.alter_rls_policy(schema, trans, ddl_stmt_str))) {
|
|
LOG_WARN("failed to drop rls policy", K(ret));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected alter option", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected stmt type", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_rls_group_ddl(const obrpc::ObRlsGroupDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObRlsGroupSchema schema;
|
|
const ObString &group_name = arg.schema_.get_policy_group_name();
|
|
const ObString &ddl_stmt_str = arg.ddl_stmt_str_;
|
|
uint64_t tenant_id = arg.schema_.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("failed to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema.assign(arg.schema_))) {
|
|
LOG_WARN("failed to assign rls group schema", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const ObTableSchema *table_schema = NULL;
|
|
const ObRlsGroupSchema *old_schema = NULL;
|
|
bool is_db_in_recyclebin = false;
|
|
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, schema.get_table_id(), table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(schema));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table not exist", KR(ret), K(arg));
|
|
} else if (table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("rls table is in recyclebin", KR(ret), K(arg));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("failed to check database in recyclebin", KR(ret), KPC(table_schema));
|
|
} else if (is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("database of rls table is in recyclebin", KR(ret), KPC(table_schema));
|
|
} else if (!table_schema->is_user_table() && !table_schema->is_view_table() && !table_schema->is_external_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("only support rls on user table or user view", KR(ret), KPC(table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "policy on non-user table");
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_rls_group_schema_by_name(tenant_id, schema.get_table_id(),
|
|
group_name, old_schema))) {
|
|
LOG_WARN("failed to get rls group from schema guard", K(ret), K(group_name), K(arg));
|
|
} else {
|
|
switch (arg.ddl_type_) {
|
|
case OB_DDL_CREATE_RLS_GROUP: {
|
|
if (OB_NOT_NULL(old_schema)) {
|
|
ret = OB_ERR_POLICY_GROUP_EXIST;
|
|
LOG_WARN("rls group exist", KPC(old_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.create_rls_group(schema, trans, ddl_stmt_str,
|
|
true, table_schema))) {
|
|
LOG_WARN("failed to create rls group", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_DROP_RLS_GROUP: {
|
|
ObSEArray<const ObRlsPolicySchema *, 4> policy_schemas;
|
|
if (OB_ISNULL(old_schema)) {
|
|
ret = OB_ERR_POLICY_GROUP_NOT_EXIST;
|
|
LOG_WARN("rls group not exist", K(ret), K(group_name), K(arg));
|
|
} else if (OB_FAIL(schema_guard.get_rls_policy_schemas_in_group(tenant_id,
|
|
schema.get_table_id(), old_schema->get_rls_group_id(), policy_schemas))) {
|
|
LOG_WARN("failed to get rls policy schemas", K(ret));
|
|
} else if (OB_UNLIKELY(policy_schemas.count() != 0)) {
|
|
ret = OB_ERR_POLICY_EXIST;
|
|
LOG_WARN("rls group is not empty", K(policy_schemas), K(ret));
|
|
} else if (OB_FAIL(schema.assign(*old_schema))) {
|
|
LOG_WARN("failed to assign rls group schema", KPC(old_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_rls_group(schema, trans, ddl_stmt_str,
|
|
true, table_schema))) {
|
|
LOG_WARN("failed to drop rls group", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected stmt type", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::handle_rls_context_ddl(const obrpc::ObRlsContextDDLArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObRlsContextSchema schema;
|
|
const ObString &context_name = arg.schema_.get_context_name();
|
|
const ObString &attribute = arg.schema_.get_attribute();
|
|
const ObString &ddl_stmt_str = arg.ddl_stmt_str_;
|
|
uint64_t tenant_id = arg.schema_.get_tenant_id();
|
|
int64_t refreshed_schema_version = 0;
|
|
ObSchemaGetterGuard schema_guard;
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("failed to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(schema.assign(arg.schema_))) {
|
|
LOG_WARN("failed to assign rls context schema", K(ret));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
const ObTableSchema *table_schema = NULL;
|
|
const ObRlsContextSchema *old_schema = NULL;
|
|
bool is_db_in_recyclebin = false;
|
|
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, schema.get_table_id(), table_schema))) {
|
|
LOG_WARN("failed to get table schema", KR(ret), K(schema));
|
|
} else if (OB_ISNULL(table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("table not exist", KR(ret), K(arg));
|
|
} else if (table_schema->is_in_recyclebin()) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("rls table is in recyclebin", KR(ret), K(arg));
|
|
} else if (OB_FAIL(schema_guard.check_database_in_recyclebin(tenant_id,
|
|
table_schema->get_database_id(),
|
|
is_db_in_recyclebin))) {
|
|
LOG_WARN("failed to check database in recyclebin", KR(ret), KPC(table_schema));
|
|
} else if (is_db_in_recyclebin) {
|
|
ret = OB_ERR_OPERATION_ON_RECYCLE_OBJECT;
|
|
LOG_WARN("database of rls table is in recyclebin", KR(ret), KPC(table_schema));
|
|
} else if (!table_schema->is_user_table() && !table_schema->is_view_table() && !table_schema->is_external_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("only support rls on user table or user view", KR(ret), KPC(table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "policy on non-user table");
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(refreshed_schema_version), K(tenant_id));
|
|
} else if (OB_FAIL(schema_guard.get_rls_context_schema_by_name(tenant_id, schema.get_table_id(),
|
|
context_name, attribute, old_schema))) {
|
|
LOG_WARN("failed to get rls context from schema guard", K(ret), K(context_name), K(attribute));
|
|
} else {
|
|
switch (arg.ddl_type_) {
|
|
case OB_DDL_CREATE_RLS_CONTEXT: {
|
|
if (OB_NOT_NULL(old_schema)) {
|
|
ret = OB_ERR_DRIVING_CONTEXT_EXIST;
|
|
LOG_WARN("rls context exist", KPC(old_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.create_rls_context(schema, trans, ddl_stmt_str,
|
|
true, table_schema))) {
|
|
LOG_WARN("failed to create rls context", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
case OB_DDL_DROP_RLS_CONTEXT: {
|
|
if (OB_ISNULL(old_schema)) {
|
|
ret = OB_ERR_DRIVING_CONTEXT_NOT_EXIST;
|
|
LOG_WARN("rls context not exist", K(ret), K(context_name), K(attribute), K(arg));
|
|
} else if (OB_FAIL(schema.assign(*old_schema))) {
|
|
LOG_WARN("failed to assign rls context schema", KPC(old_schema), K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_rls_context(schema, trans, ddl_stmt_str,
|
|
true, table_schema))) {
|
|
LOG_WARN("failed to drop rls context", K(ret));
|
|
}
|
|
break;
|
|
}
|
|
default: {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected stmt type", 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_SUCCESS == 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_proxy(const ObAlterUserProxyArg &arg)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
uint64_t tenant_id = arg.tenant_id_;
|
|
ObSchemaGetterGuard schema_guard;
|
|
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("check inner stat failed", K(ret));
|
|
} else if (OB_ISNULL(schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error", K(ret));
|
|
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input schema", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
|
} else if (OB_FAIL(check_parallel_ddl_conflict(schema_guard, arg))) {
|
|
LOG_WARN("check parallel ddl conflict failed", K(ret));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("start transaction failed", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
ObArray<ObUserInfo> users_to_update;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < arg.role_ids_.count(); i++) {
|
|
const ObUserInfo *role_user = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.role_ids_.at(i), role_user))) {
|
|
LOG_WARN("get user info failed", K(arg.role_ids_.at(i)), K(ret));
|
|
} else if (OB_ISNULL(role_user)) {
|
|
ret = OB_ROLE_NOT_EXIST;
|
|
LOG_WARN("role not existed", K(ret));
|
|
}
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < arg.client_user_ids_.count(); i++) {
|
|
const ObUserInfo *client_user_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.client_user_ids_.at(i), client_user_info))) {
|
|
LOG_WARN("get user failed", K(ret));
|
|
} else if (OB_ISNULL(client_user_info)) {
|
|
ret = OB_ERR_USER_NOT_EXIST;
|
|
LOG_WARN("user not existed", K(ret));
|
|
}
|
|
for (int64_t j = 0; OB_SUCC(ret) && j < arg.proxy_user_ids_.count(); j++) {
|
|
const ObUserInfo *proxy_user_info = NULL;
|
|
if (OB_FAIL(schema_guard.get_user_info(tenant_id, arg.proxy_user_ids_.at(j), proxy_user_info))) {
|
|
LOG_WARN("get user failed", K(ret));
|
|
} else if (OB_ISNULL(proxy_user_info)) {
|
|
ret = OB_ERR_USER_NOT_EXIST;
|
|
LOG_WARN("user not existed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_user_proxy(client_user_info, proxy_user_info,
|
|
arg.flags_, arg.is_grant_, arg.role_ids_, users_to_update, trans))) {
|
|
LOG_WARN("alter user proxy failed", K(client_user_info), K(proxy_user_info), K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (users_to_update.empty()) {
|
|
//do nothing
|
|
} else if (OB_FAIL(schema_service_->get_schema_service()->get_user_sql_service().update_user_proxy_info(
|
|
arg.tenant_id_,
|
|
users_to_update,
|
|
&arg.ddl_stmt_str_,
|
|
trans))) {
|
|
LOG_WARN("Failed to grant or revoke user", K(users_to_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_SUCCESS == 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::ddl_wlock()
|
|
{
|
|
const int64_t timeout_us = THIS_WORKER.is_timeout_ts_valid() ?
|
|
THIS_WORKER.get_timeout_remain() : GCONF.rpc_timeout;
|
|
|
|
return ddl_lock_.wrlock(ObLatchIds::DDL_EXECUTE_LOCK, ObTimeUtility::current_time() + timeout_us);
|
|
}
|
|
|
|
int ObDDLService::ddl_rlock()
|
|
{
|
|
const int64_t timeout_us = THIS_WORKER.is_timeout_ts_valid() ?
|
|
THIS_WORKER.get_timeout_remain() : GCONF.rpc_timeout;
|
|
|
|
return ddl_lock_.rdlock(ObLatchIds::DDL_EXECUTE_LOCK, ObTimeUtility::current_time() + timeout_us);
|
|
}
|
|
|
|
int ObDDLService::recompile_all_views_batch(const uint64_t tenant_id, const ObIArray<uint64_t > &view_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_TENANT_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else {
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < view_ids.count(); ++i) {
|
|
const ObTableSchema *table = nullptr;
|
|
if (OB_FAIL(schema_guard.get_table_schema(tenant_id, view_ids.at(i), table))) {
|
|
LOG_WARN("failed to get table schema", K(ret));
|
|
} else if (OB_ISNULL(table)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get table schema", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.reset_view_status(trans, tenant_id, table))) {
|
|
LOG_WARN("failed to reset view status", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::try_add_dep_info_for_all_synonyms_batch(const uint64_t tenant_id, const common::ObIArray<uint64_t> &synonym_ids)
|
|
{
|
|
// 2.sync add dependency infos for synonym in this tenant
|
|
int ret = OB_SUCCESS;
|
|
ObSchemaService *schema_service = schema_service_->get_schema_service();
|
|
ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_);
|
|
if (OB_FAIL(check_inner_stat())) {
|
|
LOG_WARN("variable is not init");
|
|
} else if (OB_INVALID_TENANT_ID == tenant_id) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid tenant_id", K(ret), K(tenant_id));
|
|
} else if (OB_ISNULL(schema_service)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service must not null", K(ret));
|
|
} else {
|
|
ObSchemaGetterGuard schema_guard;
|
|
int64_t refreshed_schema_version = 0;
|
|
if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) {
|
|
LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id));
|
|
} else {
|
|
ObDDLSQLTransaction trans(schema_service_);
|
|
if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id));
|
|
} else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) {
|
|
LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < synonym_ids.count(); ++i) {
|
|
const ObSimpleSynonymSchema *synonym_info = nullptr;
|
|
if (OB_FAIL(schema_guard.get_simple_synonym_info(tenant_id, synonym_ids.at(i), synonym_info))) {
|
|
LOG_WARN("failed to get synonym schema", K(ret));
|
|
} else if (OB_ISNULL(synonym_info)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("failed to get synonym schema", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.try_add_dep_info_for_synonym(synonym_info, trans))) {
|
|
LOG_WARN("failed to add dep for synonym", 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_SUCCESS == ret, K(temp_ret));
|
|
ret = (OB_SUCC(ret)) ? temp_ret : ret;
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(publish_schema(tenant_id))) {
|
|
LOG_WARN("publish_schema failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// alter tenant with primary_zone changed is not allowed when tenant rebalance is disabled.
|
|
int ObDDLService::check_alter_tenant_when_rebalance_is_disabled_(
|
|
const share::schema::ObTenantSchema &orig_tenant_schema,
|
|
const share::schema::ObTenantSchema &new_tenant_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const uint64_t tenant_id = orig_tenant_schema.get_tenant_id();
|
|
ObArray<ObZone> orig_first_primary_zone;
|
|
ObArray<ObZone> new_first_primary_zone;
|
|
bool is_allowed = true;
|
|
bool is_first_primary_zone_changed = false;
|
|
if (OB_UNLIKELY(orig_tenant_schema.get_tenant_id() != new_tenant_schema.get_tenant_id())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid input tenant schema", KR(ret), K(orig_tenant_schema), K(new_tenant_schema));
|
|
} else if (is_sys_tenant(tenant_id)) {
|
|
// primary_zone and locality changes in sys tenant do not cause rebalance,
|
|
// so alter sys tenant is not controlled by enable_rebalance.
|
|
is_allowed = true;
|
|
} else if (ObShareUtil::is_tenant_enable_rebalance(tenant_id)) {
|
|
is_allowed = true;
|
|
} else if (OB_FAIL(ObRootUtils::is_first_priority_primary_zone_changed(
|
|
orig_tenant_schema,
|
|
new_tenant_schema,
|
|
orig_first_primary_zone,
|
|
new_first_primary_zone,
|
|
is_first_primary_zone_changed))) {
|
|
LOG_WARN("fail to check is_first_priority_primary_zone_changed", KR(ret), K(orig_tenant_schema), K(new_tenant_schema));
|
|
} else if (is_first_primary_zone_changed) {
|
|
is_allowed = false;
|
|
}
|
|
if (OB_SUCC(ret) && !is_allowed) {
|
|
ObSqlString orig_str;
|
|
ObSqlString new_str;
|
|
ARRAY_FOREACH(orig_first_primary_zone, idx) {
|
|
if (OB_FAIL(orig_str.append_fmt(0 == idx ? "%s" : ",%s", orig_first_primary_zone.at(idx).ptr()))) {
|
|
LOG_WARN("append fmt failed", KR(ret), K(orig_first_primary_zone), K(idx));
|
|
}
|
|
}
|
|
ARRAY_FOREACH(new_first_primary_zone, idx) {
|
|
if (OB_FAIL(new_str.append_fmt(0 == idx ? "%s" : ",%s", new_first_primary_zone.at(idx).ptr()))) {
|
|
LOG_WARN("append fmt failed", KR(ret), K(new_first_primary_zone), K(idx));
|
|
}
|
|
}
|
|
ret = OB_OP_NOT_ALLOW;
|
|
LOG_WARN("enable_rebalance is disabled, alter tenant with primary zone changed not allowed", KR(ret),
|
|
K(tenant_id), K(orig_first_primary_zone), K(new_first_primary_zone));
|
|
char err_msg[DEFAULT_BUF_LENGTH];
|
|
(void)snprintf(err_msg, sizeof(err_msg),
|
|
"Tenant (%lu) Primary Zone with the first priority will be changed from '%s' to '%s', "
|
|
"but tenant 'enable_rebalance' is disabled, alter tenant", tenant_id, orig_str.ptr(), new_str.ptr());
|
|
LOG_USER_ERROR(OB_OP_NOT_ALLOW, err_msg);
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::pre_rename_mysql_columns_online(
|
|
const ObTableSchema &origin_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const bool is_oracle_mode,
|
|
ObTableSchema &new_table_schema,
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObSchemaChecker &schema_checker,
|
|
ObDDLOperator &ddl_operator,
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> &update_column_name_set,
|
|
common::ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObIArray<ObTableSchema> &idx_schema_array,
|
|
ObIArray<ObTableSchema> *global_idx_schema_array)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
const common::ObString *nls_formats = alter_table_arg.nls_formats_;
|
|
common::ObIAllocator &allocator = alter_table_arg.allocator_;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
const ObCollationType cur_extended_type_info_collation = ObCharset::get_system_collation();
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
ObSEArray<ObColumnSchemaV2, 16> new_col_schemas;
|
|
ObSEArray<ObString, 16> orig_col_names;
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
const ObString &dst_column_name = alter_column_schema->get_column_name_str();
|
|
ObColumnSchemaV2 new_column_schema;
|
|
/* rename column, no need to check gen col dup */
|
|
ObSEArray<ObString, 4> empty_expr_arr;
|
|
if (is_oracle_mode || alter_column_schema->alter_type_ != OB_DDL_CHANGE_COLUMN
|
|
|| !is_rename_column(*alter_column_schema)) {
|
|
continue;
|
|
} else if (OB_FAIL(prepare_change_modify_column_online(
|
|
*alter_column_schema, origin_table_schema, alter_table_schema, is_oracle_mode,
|
|
alter_table_arg, new_table_schema, schema_checker, ddl_operator, trans,
|
|
schema_guard, global_idx_schema_array, update_column_name_set,
|
|
new_column_schema))) {
|
|
LOG_WARN("prepare alter column schema failed", K(ret));
|
|
} else if (OB_FAIL(new_col_schemas.push_back(new_column_schema))) {
|
|
LOG_WARN("failed to push back element", K(ret));
|
|
} else if (OB_FAIL(orig_col_names.push_back(orig_column_name))) {
|
|
LOG_WARN("failed to push back element", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
// do nothing
|
|
} else if (new_col_schemas.empty()) { // do nothing
|
|
} else if (OB_FAIL(new_table_schema.alter_mysql_table_columns(
|
|
new_col_schemas, orig_col_names, ObTableSchema::CHECK_MODE_ONLINE))) {
|
|
LOG_WARN("failed to alter batch columns", K(ret));
|
|
} else if (OB_FAIL(check_new_columns_for_index(idx_schema_array, origin_table_schema,
|
|
new_col_schemas))) {
|
|
LOG_WARN("check new columns for indexes failed", K(ret));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
LOG_WARN("failed to check primary key over partition column", K(ret));
|
|
} else {
|
|
for (int i = 0; OB_SUCC(ret) && i < new_col_schemas.count(); i++) {
|
|
if (OB_FAIL(ddl_operator.update_single_column(trans, origin_table_schema, new_table_schema,
|
|
new_col_schemas.at(i)))) {
|
|
LOG_WARN("failed to alter column", K(ret), K(new_col_schemas.at(i)));
|
|
} else if (OB_FAIL(ddl_operator.update_single_column_group(trans, origin_table_schema, new_col_schemas.at(i)))) {
|
|
LOG_WARN("failed to alter column", K(ret), K(new_col_schemas.at(i)));
|
|
} else if (OB_FAIL(alter_table_update_index_and_view_column(
|
|
new_table_schema, new_col_schemas.at(i), ddl_operator, trans,
|
|
global_idx_schema_array))) {
|
|
LOG_WARN("failedt o update index column", K(ret));
|
|
} else {
|
|
ObColumnNameHashWrapper orig_col_name_key(orig_col_names.at(i));
|
|
if (OB_FAIL(update_column_name_set.exist_refactored(orig_col_name_key))) {
|
|
if (ret == OB_HASH_EXIST) {
|
|
LOG_WARN("duplicate index name", K(ret), K(orig_col_name_key.column_name_));
|
|
} else if (ret == OB_HASH_NOT_EXIST) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("exists refactored failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(orig_col_name_key))) {
|
|
LOG_WARN("failed to add index_name to hashset", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::prepare_change_modify_column_online(AlterColumnSchema &alter_column_schema,
|
|
const ObTableSchema &origin_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const bool is_oracle_mode,
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObTableSchema &new_table_schema,
|
|
sql::ObSchemaChecker &schema_checker,
|
|
ObDDLOperator &ddl_operator,
|
|
common::ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObIArray<ObTableSchema> *global_idx_schema_array,
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> &update_column_name_set,
|
|
ObColumnSchemaV2 &new_column_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
const common::ObString *nls_formats = alter_table_arg.nls_formats_;
|
|
common::ObIAllocator &allocator = alter_table_arg.allocator_;
|
|
const ObString &orig_column_name = alter_column_schema.get_origin_column_name();
|
|
const ObColumnSchemaV2 *orig_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
/* rename column, no need to check gen col dup */
|
|
ObSEArray<ObString, 4> empty_expr_arr;
|
|
if (OB_ISNULL(orig_column_schema)) {
|
|
ret = OB_ERR_BAD_FIELD_ERROR;
|
|
LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(),
|
|
origin_table_schema.get_table_name_str().length(),
|
|
origin_table_schema.get_table_name_str().ptr());
|
|
LOG_WARN("failed to find old column schema", K(ret), K(orig_column_name));
|
|
} else if (OB_FAIL(ObDDLResolver::check_default_value(
|
|
alter_column_schema.get_cur_default_value(), tz_info_wrap, nls_formats,
|
|
orig_column_schema->is_generated_column() ? &orig_column_schema->get_local_session_var() : NULL, allocator,
|
|
new_table_schema, alter_column_schema, empty_expr_arr, alter_table_schema.get_sql_mode(),
|
|
!alter_column_schema.is_generated_column(), &schema_checker))) {
|
|
LOG_WARN("failed to check default value", K(ret), K(alter_column_schema));
|
|
} else if (OB_FAIL(pre_check_orig_column_schema(alter_column_schema, origin_table_schema,
|
|
update_column_name_set))) {
|
|
LOG_WARN("failed to pre check orig column schema", K(ret));
|
|
} else if (orig_column_schema->has_generated_column_deps()) {
|
|
if (OB_FAIL(update_generated_column_schema(alter_column_schema, *orig_column_schema,
|
|
origin_table_schema, tz_info_wrap, NULL,
|
|
new_table_schema, true, false, &ddl_operator, &trans))) {
|
|
LOG_WARN("failed to rebuild generated column schema", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (alter_column_schema.is_primary_key_) {
|
|
if (new_table_schema.get_rowkey_column_num() > 0) {
|
|
if (new_table_schema.is_heap_table()) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to add primary key", K(ret));
|
|
} else {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret));
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret) && alter_column_schema.is_autoincrement_) {
|
|
if (alter_column_schema.is_autoincrement()) {
|
|
if (orig_column_schema->get_column_id() != new_table_schema.get_autoinc_column_id()) {
|
|
// not supported now; from non-auto-increment column to auto-increment column
|
|
ret = OB_NOT_SUPPORTED;
|
|
RS_LOG(WARN, "from non-auto-increment column to auto-increment column", "alter_column_id",
|
|
alter_column_schema.get_column_id(), "auto_inc_column_id",
|
|
new_table_schema.get_autoinc_column_id(), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& orig_column_schema->get_column_name_str() != alter_column_schema.get_column_name_str()) {
|
|
if (OB_FAIL(modify_part_func_expr(origin_table_schema,
|
|
*orig_column_schema,
|
|
alter_column_schema,
|
|
new_table_schema,
|
|
*tz_info_wrap.get_time_zone_info(),
|
|
allocator))) {
|
|
LOG_WARN("modify part func expr failed", K(ret));
|
|
} else if (OB_FAIL(modify_part_func_expr_for_global_index(
|
|
*orig_column_schema, alter_column_schema, new_table_schema,
|
|
*tz_info_wrap.get_time_zone_info(), allocator, &ddl_operator, &trans,
|
|
global_idx_schema_array))) {
|
|
LOG_WARN("failed to modify part_func_expr for global_index", K(ret));
|
|
} else if (OB_FAIL(modify_constraint_check_expr(
|
|
*orig_column_schema, alter_column_schema, new_table_schema, alter_table_arg,
|
|
is_oracle_mode, *tz_info_wrap.get_time_zone_info(), allocator, &ddl_operator,
|
|
&trans))) {
|
|
LOG_WARN("failed to modify check_expr constraint", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(check_can_alter_column_type(*orig_column_schema, alter_column_schema,
|
|
origin_table_schema, is_oracle_mode))) {
|
|
LOG_WARN("fail to check can alter column type", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_column_schema.assign(*orig_column_schema))) {
|
|
LOG_WARN("failed to asisgn column schema", KR(ret));
|
|
} else if (OB_FAIL(fill_new_column_attributes(alter_column_schema, new_column_schema))) {
|
|
LOG_WARN("failed to fill new column attributes", K(ret));
|
|
} else if (OB_FAIL(resolve_timestamp_column(&alter_column_schema, new_table_schema,
|
|
new_column_schema, tz_info_wrap, nls_formats,
|
|
allocator))) {
|
|
LOG_WARN("failed to resolve timestamp column", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// in generate_tables_array(), inc_aux_table_schema is copied from inc_table_schema which leads to partitions of inc_aux_table_schema
|
|
// recored the split_source_tablet_id of data table's partition rathan than that of aux table.
|
|
// thus, we need to correct split_source_tablet_id of inc_aux_table_schema's partition based on partition name.
|
|
int ObDDLService::correct_source_tablet_id_for_inc_aux_table_schema_(
|
|
const obrpc::ObAlterTableArg::AlterPartitionType op_type,
|
|
const ObPartitionLevel target_part_level,
|
|
const ObTableSchema &table_schema,
|
|
const ObTableSchema &aux_table_schema,
|
|
const AlterTableSchema &inc_table_schema,
|
|
ObTableSchema &inc_aux_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObPartitionLevel ori_part_level = table_schema.get_part_level();
|
|
int orig_table_part_num = table_schema.get_partition_num();
|
|
int orig_aux_table_part_num = aux_table_schema.get_partition_num();
|
|
int inc_aux_table_part_num = inc_aux_table_schema.get_partition_num();
|
|
ObPartition **orig_table_parts = table_schema.get_part_array();
|
|
ObPartition **orig_aux_table_parts = aux_table_schema.get_part_array();
|
|
ObPartition **inc_aux_table_parts = inc_aux_table_schema.get_part_array();
|
|
|
|
if (obrpc::ObAlterTableArg::SPLIT_PARTITION != op_type
|
|
&& obrpc::ObAlterTableArg::AUTO_SPLIT_PARTITION != op_type
|
|
&& obrpc::ObAlterTableArg::REORGANIZE_PARTITION != op_type) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("invalid type", KR(ret), K(op_type));
|
|
} else if (OB_UNLIKELY(orig_table_part_num != orig_aux_table_part_num)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid partition num", KR(ret), K(orig_table_part_num), K(orig_aux_table_part_num));
|
|
} else if (target_part_level == PARTITION_LEVEL_TWO) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to build part schema for aux table with subpartition", KR(ret));
|
|
} else if (target_part_level != PARTITION_LEVEL_ONE) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(ori_part_level), K(target_part_level));
|
|
} else { // target_part_level == PARTITION_LEVEL_ONE
|
|
if (ori_part_level == PARTITION_LEVEL_ZERO) {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_aux_table_part_num; i++) {
|
|
ObPartition *inc_aux_part = inc_aux_table_parts[i];
|
|
|
|
if (OB_ISNULL(inc_aux_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("partition is NULL", KR(ret), K(inc_aux_table_schema));
|
|
} else if (!aux_table_schema.get_tablet_id().is_valid()) { // never run here
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("src_aux_table_tablet_id is invalid", KR(ret), K(aux_table_schema.get_tablet_id()));
|
|
} else {
|
|
inc_aux_part->set_split_source_tablet_id(aux_table_schema.get_tablet_id());
|
|
}
|
|
}
|
|
} else if (ori_part_level == PARTITION_LEVEL_ONE) {
|
|
if (OB_UNLIKELY(!table_schema.is_valid_split_part_type())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("only support to split range or range column part", KR(ret), K(table_schema));
|
|
} else {
|
|
hash::ObHashMap<ObTabletID, int64_t> source_tablet_id_map;
|
|
|
|
if (OB_FAIL(source_tablet_id_map.create(orig_table_part_num, lib::ObLabel("SourceIdMap")))) {
|
|
LOG_WARN("failed to create source tablet id map", KR(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < orig_table_part_num; i++) {
|
|
ObPartition *orig_table_part = orig_table_parts[i];
|
|
if (OB_ISNULL(orig_table_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("partition is NULL", KR(ret));
|
|
} else if (OB_FAIL(source_tablet_id_map.set_refactored(orig_table_part->get_tablet_id(), i))) {
|
|
LOG_WARN("fail to set map", KR(ret), KPC(orig_table_part));
|
|
}
|
|
}
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObTabletID src_table_tablet_id;
|
|
ObTabletID src_aux_table_tablet_id;
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_aux_table_part_num; i++) {
|
|
ObPartition *inc_aux_part = inc_aux_table_parts[i];
|
|
int64_t ori_part_index = 0;
|
|
if (OB_ISNULL(inc_aux_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("partition is NULL", KR(ret));
|
|
} else if (inc_aux_part->get_split_source_tablet_id() != src_table_tablet_id) {
|
|
src_table_tablet_id = inc_aux_part->get_split_source_tablet_id();
|
|
if (OB_FAIL(source_tablet_id_map.get_refactored(src_table_tablet_id, ori_part_index))) {
|
|
LOG_WARN("fail to get part idx", KR(ret), K(src_table_tablet_id), K(table_schema));
|
|
} else if (OB_ISNULL(orig_aux_table_parts[ori_part_index])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("partition is NULL", KR(ret), K(inc_aux_table_schema));
|
|
} else if (orig_table_parts[ori_part_index]->get_high_bound_val() !=
|
|
orig_aux_table_parts[ori_part_index]->get_high_bound_val()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to find source partition in aux table",
|
|
KR(ret), KPC(inc_aux_part),
|
|
KPC(orig_table_parts[ori_part_index]),
|
|
KPC(orig_aux_table_parts[ori_part_index]),
|
|
K(table_schema),
|
|
K(aux_table_schema));
|
|
} else {
|
|
src_aux_table_tablet_id = orig_aux_table_parts[ori_part_index]->get_tablet_id();
|
|
}
|
|
} /* else {
|
|
the inc_aux_part is split from the same tablet with previous inc_aux_part,
|
|
no need to update src_aux_table_tablet_id
|
|
} */
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (!src_aux_table_tablet_id.is_valid()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("src_aux_table_tablet_id is invalid", KR(ret), K(src_table_tablet_id),
|
|
K(inc_aux_part->get_split_source_tablet_id()));
|
|
} else {
|
|
inc_aux_part->set_split_source_tablet_id(src_aux_table_tablet_id);
|
|
}
|
|
}
|
|
} // end for
|
|
}
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(target_part_level), K(ori_part_level));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_column_online(
|
|
const ObTableSchema &origin_table_schema, ObTableSchema &new_table_schema,
|
|
const ObString &orig_column_name, ObDDLOperator &ddl_operator, ObSchemaGetterGuard &schema_guard,
|
|
common::ObMySQLTransaction &trans,
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> &update_column_name_set)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObColumnSchemaV2 *orig_column_schema =
|
|
origin_table_schema.get_column_schema(orig_column_name);
|
|
if (OB_ISNULL(orig_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid null column schema", K(ret), K(orig_column_name));
|
|
} else {
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
if (OB_FAIL(check_can_drop_column(orig_column_name, orig_column_schema, origin_table_schema,
|
|
new_table_schema, new_table_schema.get_column_count(),
|
|
schema_guard))) {
|
|
LOG_WARN("check drop column failed", K(ret));
|
|
} else if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) {
|
|
ret = OB_ERR_BAD_FIELD_ERROR;
|
|
LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(),
|
|
origin_table_schema.get_table_name_str().length(),
|
|
origin_table_schema.get_table_name_str().ptr());
|
|
LOG_WARN("column has beed modified, can't drop", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.drop_sequence_in_drop_column(*orig_column_schema, trans,
|
|
schema_guard))) {
|
|
RS_LOG(WARN, "alter table drop identity column fail", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_table_drop_aux_column(
|
|
new_table_schema, *orig_column_schema, trans, USER_INDEX))) {
|
|
LOG_WARN("drop index column failed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.alter_table_drop_aux_column(
|
|
new_table_schema, *orig_column_schema, trans, AUX_VERTIAL_PARTITION_TABLE))) {
|
|
LOG_WARN("drop aux vp column failed", K(ret));
|
|
} else if (OB_FAIL(ddl_operator.update_prev_id_for_delete_column(
|
|
origin_table_schema, new_table_schema, *orig_column_schema, trans))) {
|
|
LOG_WARN("failed to update column previous id for delete column", K(ret));
|
|
} else if (OB_FAIL(
|
|
ddl_operator.delete_single_column(trans, new_table_schema, orig_column_name))) {
|
|
LOG_WARN("failed to delete column", K(ret), K(orig_column_name));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_column_offline(const ObTableSchema &origin_table_schema,
|
|
ObTableSchema &new_table_schema,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
const ObString &orig_column_name,
|
|
const int64_t new_tbl_cols_cnt)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObColumnSchemaV2 *orig_column_schema =
|
|
origin_table_schema.get_column_schema(orig_column_name);
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
if (OB_FAIL(check_can_drop_column(orig_column_name, orig_column_schema, origin_table_schema,
|
|
new_table_schema,
|
|
(new_table_schema.get_column_count() > new_tbl_cols_cnt + 1 ?
|
|
new_table_schema.get_column_count() :
|
|
new_tbl_cols_cnt + 1),
|
|
schema_guard))) {
|
|
LOG_WARN("column cannot be dropped", K(ret), K(new_tbl_cols_cnt), K(orig_column_schema));
|
|
} else if (OB_FAIL(drop_column_update_new_table(origin_table_schema, new_table_schema,
|
|
*orig_column_schema))) {
|
|
LOG_WARN("fail to drop column update new table", K(ret));
|
|
} else {
|
|
// do nothing
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::prepare_change_modify_column_offline(AlterColumnSchema &alter_column_schema,
|
|
const ObTableSchema &origin_table_schema,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const bool is_oracle_mode,
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
ObTableSchema &new_table_schema,
|
|
sql::ObSchemaChecker &schema_checker,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> &update_column_name_set,
|
|
ObColumnSchemaV2 &new_column_schema,
|
|
bool &is_contain_part_key)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const common::ObTimeZoneInfoWrap &tz_info_wrap = alter_table_arg.tz_info_wrap_;
|
|
const common::ObString *nls_formats = alter_table_arg.nls_formats_;
|
|
common::ObIAllocator &allocator = alter_table_arg.allocator_;
|
|
const ObString &orig_column_name = alter_column_schema.get_origin_column_name();
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
ObColumnSchemaV2 *orig_column_schema = new_table_schema.get_column_schema(orig_column_name);
|
|
bool need_update_default = OB_DDL_CHANGE_COLUMN == alter_column_schema.alter_type_;
|
|
bool need_update_local_var = OB_DDL_CHANGE_COLUMN == alter_column_schema.alter_type_
|
|
|| OB_DDL_MODIFY_COLUMN == alter_column_schema.alter_type_;
|
|
if (OB_ISNULL(orig_column_schema)) {
|
|
ret = OB_ERR_BAD_FIELD_ERROR;
|
|
LOG_USER_ERROR(OB_ERR_BAD_FIELD_ERROR, orig_column_name.length(), orig_column_name.ptr(),
|
|
origin_table_schema.get_table_name_str().length(),
|
|
origin_table_schema.get_table_name_str().ptr());
|
|
LOG_WARN("unknown column", K(ret), K(orig_column_name), K(new_table_schema));
|
|
} else if (!is_contain_part_key) {
|
|
if (OB_FAIL(origin_table_schema.is_tbl_partition_key(*orig_column_schema, is_contain_part_key,
|
|
false /* ignore_presetting_key */))) {
|
|
LOG_WARN("fail to check is partition key", KR(ret), K(origin_table_schema), KPC(orig_column_schema));
|
|
} else if (is_contain_part_key
|
|
&& OB_FAIL(new_table_schema.assign_partition_schema_without_auto_part_attr(alter_table_schema))) {
|
|
// the auto_part_attr of alter_table_schema is default value, no need to assign invalid attributes
|
|
LOG_WARN("failed to assign partition schema", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
// do nothing
|
|
} else if (alter_column_schema.is_generated_column()
|
|
&& OB_FAIL(ObDDLResolver::reformat_generated_column_expr(
|
|
alter_column_schema.get_cur_default_value(), tz_info_wrap, nls_formats, alter_table_arg.local_session_var_,
|
|
allocator, new_table_schema, alter_column_schema, alter_table_schema.get_sql_mode(),
|
|
&schema_checker))) {
|
|
LOG_WARN("faled to check default value", K(ret), K(alter_column_schema));
|
|
} else if (OB_FAIL(pre_check_orig_column_schema(alter_column_schema, origin_table_schema,
|
|
update_column_name_set))) {
|
|
LOG_WARN("failed to pre check orig column schema", K(ret));
|
|
} else if ((need_update_default || need_update_local_var)
|
|
&& orig_column_schema->has_generated_column_deps()) {
|
|
if (OB_FAIL(update_generated_column_schema(alter_column_schema, *orig_column_schema,
|
|
origin_table_schema, tz_info_wrap,
|
|
&alter_table_arg.local_session_var_,
|
|
new_table_schema,
|
|
need_update_default,
|
|
need_update_local_var))) {
|
|
LOG_WARN("failed to rebuild generated column schema", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
/* rename column, no need to check gen col dup */
|
|
ObSEArray<ObString, 4> empty_expr_arr;
|
|
if (OB_FAIL(ObDDLResolver::check_default_value(
|
|
alter_column_schema.get_cur_default_value(), tz_info_wrap, nls_formats, &alter_table_arg.local_session_var_,
|
|
allocator, new_table_schema, alter_column_schema, empty_expr_arr, alter_table_schema.get_sql_mode(),
|
|
!alter_column_schema.is_generated_column(), /* allow_sequence */
|
|
&schema_checker))) {
|
|
LOG_WARN("fail to check default value", K(alter_column_schema), K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)
|
|
&& orig_column_schema->get_column_name_str() != alter_column_schema.get_column_name_str()) {
|
|
if (OB_FAIL(modify_part_func_expr(origin_table_schema,
|
|
*orig_column_schema,
|
|
alter_column_schema,
|
|
new_table_schema,
|
|
*tz_info_wrap.get_time_zone_info(),
|
|
allocator))) {
|
|
LOG_WARN("modify part func expr failed", K(ret));
|
|
} else if (OB_FAIL(modify_part_func_expr_for_global_index(
|
|
*orig_column_schema,
|
|
alter_column_schema,
|
|
new_table_schema,
|
|
*tz_info_wrap.get_time_zone_info(),
|
|
allocator,
|
|
nullptr/* ddl_operator */,
|
|
nullptr/* trans */,
|
|
nullptr/* global_idx_schema_array */))) {
|
|
LOG_WARN("failed to modify part_func_expr for global_index", K(ret));
|
|
} else if (OB_FAIL(modify_constraint_check_expr(
|
|
*orig_column_schema,
|
|
alter_column_schema,
|
|
new_table_schema,
|
|
alter_table_arg,
|
|
is_oracle_mode,
|
|
*tz_info_wrap.get_time_zone_info(),
|
|
allocator,
|
|
nullptr/* ddl_operator */,
|
|
nullptr/* trans */))) {
|
|
LOG_WARN("failed to modify check_expr constraint", K(ret));
|
|
}
|
|
}
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(new_column_schema.assign(*orig_column_schema))) {
|
|
LOG_WARN("assign column schema failed", K(ret));
|
|
} else if (alter_column_schema.is_primary_key_) {
|
|
if (!new_table_schema.is_heap_table()) {
|
|
ret = OB_ERR_MULTIPLE_PRI_KEY;
|
|
LOG_WARN("multiple primary key defined", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
// do nothing
|
|
} else if (OB_FAIL(fill_new_column_attributes(alter_column_schema, new_column_schema))) {
|
|
LOG_WARN("fill new column attributes failed", K(ret));
|
|
} else if (OB_FAIL(new_column_schema.set_orig_default_value(
|
|
alter_column_schema.get_orig_default_value()))) {
|
|
LOG_WARN("set orig default value failed", K(ret));
|
|
} else if (OB_FAIL(resolve_timestamp_column(&alter_column_schema, new_table_schema,
|
|
new_column_schema, tz_info_wrap, nls_formats,
|
|
allocator))) {
|
|
LOG_WARN("resolve timestamp column failed", K(ret));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::generate_split_info_for_schemas_(const obrpc::ObAlterTableArg::AlterPartitionType type,
|
|
ObIArray<const ObTableSchema*>& ori_table_schemas,
|
|
ObIArray<AlterTableSchema*>& inc_table_schemas,
|
|
ObIArray<ObTableSchema*>& new_table_schemas,
|
|
ObIArray<ObTableSchema*>& upd_table_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartitionLevel target_part_level = ObPartitionLevel::PARTITION_LEVEL_MAX;
|
|
|
|
if (OB_UNLIKELY(ori_table_schemas.count() != inc_table_schemas.count() ||
|
|
ori_table_schemas.count() != new_table_schemas.count() ||
|
|
ori_table_schemas.count() != upd_table_schemas.count())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schemas' number is different", KR(ret), K(ori_table_schemas.count()), K(inc_table_schemas.count()),
|
|
K(new_table_schemas.count()), K(upd_table_schemas.count()));
|
|
} else if (OB_UNLIKELY(ori_table_schemas.empty())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("empty schemas", KR(ret));
|
|
} else if (OB_ISNULL(ori_table_schemas.at(0))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", KR(ret));
|
|
} else if (OB_UNLIKELY(ori_table_schemas.at(0)->is_global_index_table() && ori_table_schemas.count() != 1)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("global index should not have aux table", KR(ret), KPC(ori_table_schemas.at(0)), K(ori_table_schemas.count()));
|
|
} else if (FALSE_IT(target_part_level = obrpc::ObAlterTableArg::AUTO_SPLIT_PARTITION == type ?
|
|
ori_table_schemas.at(0)->get_target_part_level_for_auto_partitioned_table() :
|
|
ori_table_schemas.at(0)->get_part_level())) {
|
|
} else if (target_part_level == PARTITION_LEVEL_TWO) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to split subpartition", KR(ret), K(type), K(target_part_level),
|
|
K(ori_table_schemas.at(0)->get_part_level()));
|
|
} else if (target_part_level != PARTITION_LEVEL_ONE) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(type), K(target_part_level),
|
|
K(ori_table_schemas.at(0)->get_part_level()),
|
|
KPC(ori_table_schemas.at(0)));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < new_table_schemas.count(); ++i) {
|
|
if (OB_ISNULL(new_table_schemas.at(i)) || OB_ISNULL(inc_table_schemas.at(i))
|
|
|| OB_ISNULL(ori_table_schemas.at(i)) || OB_ISNULL(upd_table_schemas.at(i))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema ptr is NULL", KR(ret), K(new_table_schemas.at(i)), K(inc_table_schemas.at(i)),
|
|
K(ori_table_schemas.at(i)), K(upd_table_schemas.at(i)));
|
|
} else if (i > 0 && OB_FAIL(correct_source_tablet_id_for_inc_aux_table_schema_(type,
|
|
target_part_level,
|
|
*ori_table_schemas.at(0),
|
|
*ori_table_schemas.at(i),
|
|
*inc_table_schemas.at(0),
|
|
*inc_table_schemas.at(i)))) {
|
|
LOG_WARN("fail to correct source tablet id", KR(ret),
|
|
KPC(ori_table_schemas.at(0)),
|
|
KPC(ori_table_schemas.at(i)),
|
|
KPC(inc_table_schemas.at(0)),
|
|
KPC(inc_table_schemas.at(i)));
|
|
} else if (OB_FAIL(generate_split_info_for_schema_(target_part_level,
|
|
*ori_table_schemas.at(i),
|
|
*inc_table_schemas.at(i),
|
|
*new_table_schemas.at(i),
|
|
*upd_table_schemas.at(i)))) {
|
|
LOG_WARN("generate split info failed", KR(ret),
|
|
KPC(ori_table_schemas.at(i)),
|
|
KPC(inc_table_schemas.at(i)),
|
|
KPC(new_table_schemas.at(i)),
|
|
KPC(upd_table_schemas.at(i)));
|
|
}
|
|
}
|
|
|
|
if (OB_FAIL(ret)) {
|
|
} else if (new_table_schemas.count() > 1) {
|
|
// keep partition setting of aux table be same with data table
|
|
ObTableSchema* new_table_schema = new_table_schemas.at(0);
|
|
if (OB_ISNULL(new_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", KR(ret));
|
|
} else {
|
|
ObPartitionOption& new_table_option = new_table_schema->get_part_option();
|
|
|
|
for (int64_t i = 1; OB_SUCC(ret) && i < new_table_schemas.count(); ++i) {
|
|
ObTableSchema* new_aux_table_schema = new_table_schemas.at(i);
|
|
if (OB_ISNULL(new_aux_table_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected null", KR(ret));
|
|
} else {
|
|
ObPartitionOption& new_aux_table_option = new_aux_table_schema->get_part_option();
|
|
new_aux_table_option.set_part_func_type(new_table_option.get_part_func_type());
|
|
const ObString& part_func_expr = new_table_option.get_part_func_expr_str();
|
|
if (OB_FAIL(new_aux_table_option.set_part_expr(part_func_expr))) {
|
|
LOG_WARN("fail to set part expr", KR(ret), K(part_func_expr));
|
|
}
|
|
}
|
|
} // end for
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// 1. new_table_schema records the setting of table,
|
|
// 1.1 set part_num
|
|
// 1.2 set part_level
|
|
// 1.3 for auto-partitioned non-partitioned data_table/glbal_index, set part_func_expr and partition key
|
|
// 2. inc_table_schema records the setting of split partitions,
|
|
// 2.1 set part_idx of inc_table_schema, part_idx should keep partial order based on the value of partition key
|
|
// 2.2 generate part_id and tablet_id
|
|
// 2.3 for auto-partitioned non-partitioned data_table/glbal_index, genarate hidden part to mock a part with source tablet
|
|
// 3. upd_table_schema records the setting of changed origin partitions
|
|
// (only partitioned table has upd_table_schema)
|
|
// 3.1 set part_idx (need to be sorted with inc_table_schema)
|
|
// 3.2 set partition_type for source tablet (to be hidden part)
|
|
int ObDDLService::generate_split_info_for_schema_(const ObPartitionLevel target_part_level,
|
|
const ObTableSchema& ori_table_schema,
|
|
ObTableSchema& inc_table_schema,
|
|
ObTableSchema& new_table_schema,
|
|
ObTableSchema& upd_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const ObPartitionLevel part_level = ori_table_schema.get_part_level();
|
|
const int64_t ori_part_num = ori_table_schema.get_part_option().get_part_num();
|
|
const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num();
|
|
const int64_t all_part_num = ori_part_num + inc_part_num - 1;
|
|
new_table_schema.get_part_option().set_part_num(all_part_num);
|
|
new_table_schema.set_part_level(target_part_level);
|
|
|
|
if (target_part_level== PARTITION_LEVEL_ONE) {
|
|
if (part_level == PARTITION_LEVEL_ZERO) {
|
|
if (OB_FAIL(generate_partition_info_from_non_partitioned_table_(ori_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema))) {
|
|
LOG_WARN("fail to generate split partition info from non-partitioned table", KR(ret),
|
|
K(ori_table_schema),
|
|
K(inc_table_schema),
|
|
K(new_table_schema));
|
|
}
|
|
} else if (part_level == PARTITION_LEVEL_ONE) {
|
|
if (OB_FAIL(generate_partition_info_from_partitioned_table_(ori_table_schema, inc_table_schema,
|
|
upd_table_schema))) {
|
|
LOG_WARN("fail to generate split partition info from non-partitioned table", KR(ret),
|
|
K(ori_table_schema),
|
|
K(inc_table_schema),
|
|
K(upd_table_schema));
|
|
}
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(part_level), K(target_part_level));
|
|
}
|
|
} else {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("invalid part level", KR(ret), K(part_level), K(target_part_level), K(ori_table_schema));
|
|
}
|
|
|
|
// generate part id and tablet id for new splitting partitions
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(generate_object_id_for_partition_schema(inc_table_schema))) {
|
|
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(generate_tablet_id(inc_table_schema))) {
|
|
LOG_WARN("fail to fetch new table id", K(inc_table_schema), KR(ret));
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// for inc_table_schema:
|
|
// 1. add a mock hidden partition for origin tablet. it seems that we are splitting a "partitioned" table
|
|
// 2. set part_idx for all additional splitting partitions
|
|
// for new_table_schema:
|
|
// 1. set tablet_id as invalid (partitioned table don't record tablet id in table-level)
|
|
// 2. set partition key and part_func_expr based on presetting-partition-key
|
|
int ObDDLService::generate_partition_info_from_non_partitioned_table_(const ObTableSchema& ori_table_schema,
|
|
ObTableSchema& inc_table_schema,
|
|
ObTableSchema& new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num();
|
|
ObPartition **inc_part_array = nullptr;
|
|
|
|
if (OB_ISNULL(inc_part_array = inc_table_schema.get_part_array())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", KR(ret), K(inc_table_schema));
|
|
} else if (OB_FAIL(mock_hidden_partition_for_non_partitioned_table_(ori_table_schema,
|
|
inc_table_schema,
|
|
new_table_schema))) {
|
|
LOG_WARN("fail to mock hidden source partition for non-partitioned table", KR(ret),
|
|
K(ori_table_schema),
|
|
K(inc_table_schema));
|
|
} else if (ori_table_schema.is_user_table() || ori_table_schema.is_global_index_table()) {
|
|
ObArray<uint64_t> presetting_partition_keys;
|
|
if (OB_FAIL(ori_table_schema.get_presetting_partition_keys(presetting_partition_keys))) {
|
|
LOG_WARN("fail to get presetting partition key columns", KR(ret), K(ori_table_schema));
|
|
} else if (presetting_partition_keys.empty()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid schema for auto partitioning", KR(ret), K(ori_table_schema));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < presetting_partition_keys.count(); i++) {
|
|
uint64_t column_id = presetting_partition_keys.at(i);
|
|
if (OB_FAIL(new_table_schema.add_partition_key(column_id))){
|
|
LOG_WARN("fail to add partition keu", KR(ret), K(new_table_schema), K(column_id));
|
|
}
|
|
} // end for
|
|
|
|
const bool need_set_part_expr = ori_table_schema.get_part_option().get_part_func_expr_str().empty();
|
|
if (OB_FAIL(ret)) {
|
|
} else if (need_set_part_expr) {
|
|
ObArenaAllocator allocator(ObModIds::OB_SCHEMA);
|
|
int64_t buf_len = OB_MAX_TEXT_LENGTH;
|
|
int64_t pos = 0;
|
|
char* buf = static_cast<char *>(allocator.alloc(buf_len));
|
|
|
|
if (OB_ISNULL(buf)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to alloc", KR(ret));
|
|
}
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < presetting_partition_keys.count(); i++) {
|
|
ObColumnSchemaV2* column_schema = new_table_schema.get_column_schema(presetting_partition_keys.at(i));
|
|
if (OB_ISNULL(column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("column_schema is null", KR(ret));
|
|
} else if (FALSE_IT(pos += column_schema->get_column_name_str().to_string(buf + pos, buf_len - pos))) {
|
|
} else if (i != presetting_partition_keys.count() - 1) {
|
|
J_COMMA();
|
|
}
|
|
} // end for
|
|
|
|
if (OB_SUCC(ret)) {
|
|
ObString part_func_expr(pos, buf);
|
|
if (OB_FAIL(new_table_schema.get_part_option().set_part_expr(part_func_expr))) {
|
|
LOG_WARN("fail to set part expr", KR(ret));
|
|
}
|
|
}
|
|
} // end if (need_set_part_expr)
|
|
}
|
|
} else if (ori_table_schema.is_aux_lob_table() || ori_table_schema.is_index_local_storage()) {
|
|
// the partition info of local index and lob is same to data_table, no need to modify schema
|
|
} else {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid table type", KR(ret), K(ori_table_schema));
|
|
}
|
|
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) {
|
|
ObPartition* inc_part = inc_part_array[i];
|
|
if (OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("split partition is null", KR(ret));
|
|
} else {
|
|
inc_part->set_part_idx(i);
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::mock_hidden_partition_for_non_partitioned_table_(const ObTableSchema& ori_table_schema,
|
|
ObTableSchema& inc_table_schema,
|
|
ObTableSchema& new_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
const int64_t object_cnt = 1;
|
|
const uint64_t tenant_id = ori_table_schema.get_tenant_id();
|
|
share::schema::ObPartition mock_part;
|
|
ObObjectID object_id = OB_INVALID_ID;
|
|
|
|
if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("schema_service is empty", KR(ret), KP_(schema_service));
|
|
} else if (OB_FAIL(schema_service_->get_schema_service()
|
|
->fetch_new_partition_ids(tenant_id, object_cnt, object_id))) {
|
|
LOG_WARN("fail to get max object id", KR(ret), K(tenant_id), K(object_cnt));
|
|
} else if (OB_FAIL(mock_part.set_part_name(ObString::make_string("mock_part")))) {
|
|
LOG_WARN("failed to set part_name", KR(ret));
|
|
} else if (OB_FAIL(mock_part.set_high_bound_val(ObRowkey::MAX_ROWKEY))) {
|
|
LOG_WARN("failed to set high_bound_val", KR(ret));
|
|
} else {
|
|
mock_part.set_part_idx(0);
|
|
mock_part.set_tenant_id(ori_table_schema.get_tenant_id());
|
|
mock_part.set_table_id(ori_table_schema.get_table_id());
|
|
mock_part.set_tablet_id(ori_table_schema.get_tablet_id());
|
|
mock_part.set_partition_type(PartitionType::PARTITION_TYPE_SPLIT_SOURCE);
|
|
mock_part.set_part_id(object_id);
|
|
}
|
|
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(inc_table_schema.add_partition(mock_part))) {
|
|
LOG_WARN("fail to add partition", KR(ret), K(mock_part));
|
|
} else {
|
|
new_table_schema.set_tablet_id(ObTabletID::INVALID_TABLET_ID);
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
// inc_table_schema records the additional splitting partition,
|
|
// upd_table_schema records the modification of origin partition:
|
|
// 1. reorganize the part_idx of partition of inc_table_schema and upd_table_schema in partial order
|
|
// based on the value of partition key
|
|
// 2. set partition_type for source partition of upd_table_schema (to be hidden part)
|
|
int ObDDLService::generate_partition_info_from_partitioned_table_(const ObTableSchema& ori_table_schema,
|
|
ObTableSchema& inc_table_schema,
|
|
ObTableSchema& upd_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObPartition **ori_part_array = nullptr;
|
|
ObPartition **inc_part_array = nullptr;
|
|
const int64_t ori_part_num = ori_table_schema.get_part_option().get_part_num();
|
|
const int64_t inc_part_num = inc_table_schema.get_part_option().get_part_num();
|
|
|
|
// set part_idx and partition type
|
|
if (OB_ISNULL(inc_part_array = inc_table_schema.get_part_array())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", KR(ret), K(inc_table_schema));
|
|
} else if (OB_ISNULL(ori_part_array = ori_table_schema.get_part_array())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("part array is null", KR(ret), K(ori_table_schema));
|
|
} else if (OB_UNLIKELY(!ori_table_schema.is_valid_split_part_type())) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("only support to split range or range column part", KR(ret), K(ori_table_schema));
|
|
} else {
|
|
common::hash::ObHashSet<uint64_t> source_tablet_id_set;
|
|
if (OB_FAIL(source_tablet_id_set.create(32))) {
|
|
LOG_WARN("failed to create source part id set", KR(ret));
|
|
} else {
|
|
// check ori_part_array
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < ori_part_num; ++i) {
|
|
ObPartition* ori_part = ori_part_array[i];
|
|
if (OB_ISNULL(ori_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("origin partition is null", KR(ret));
|
|
} else if (OB_UNLIKELY(ori_part->get_split_source_tablet_id().is_valid())){
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("only split part could have split_source_tablet_id", KR(ret), KPC(ori_part));
|
|
}
|
|
} // end for
|
|
|
|
// build upd_part of upd_table_schema to update the partition_type of source split part
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) {
|
|
ObPartition* inc_part = inc_part_array[i];
|
|
if (OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("split partition is null", KR(ret));
|
|
} else {
|
|
ObTabletID source_tablet_id = inc_part->get_split_source_tablet_id();
|
|
if (OB_FAIL(source_tablet_id_set.set_refactored(source_tablet_id.id(), 0 /*flag*/))) {
|
|
if (OB_HASH_EXIST == ret) {
|
|
ret = OB_SUCCESS;
|
|
} else {
|
|
LOG_WARN("set_refactored from source_tablet_id_set failed", KR(ret), K(source_tablet_id));
|
|
}
|
|
} else {
|
|
ObPartition* source_split_part = nullptr;
|
|
for (int64_t j = 0; source_split_part == nullptr && OB_SUCC(ret) && j < ori_part_num; ++j) {
|
|
ObPartition* ori_part = ori_part_array[j];
|
|
if (OB_ISNULL(ori_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("source split part is null", KR(ret), K(j), K(ori_table_schema));
|
|
} else if (ori_part->get_tablet_id() == source_tablet_id) {
|
|
source_split_part = ori_part;
|
|
}
|
|
} // end for
|
|
|
|
ObPartition upd_part;
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_ISNULL(source_split_part)) {
|
|
ret = OB_UNKNOWN_PARTITION;
|
|
LOG_WARN("source split part is not existed", KR(ret), K(source_tablet_id), KPC(inc_part), K(ori_table_schema));
|
|
} else if (OB_FAIL(upd_part.assign(*source_split_part))) {
|
|
LOG_WARN("fail to assign part", KR(ret), KPC(source_split_part));
|
|
} else if (FALSE_IT(upd_part.set_partition_type(PartitionType::PARTITION_TYPE_SPLIT_SOURCE))) {
|
|
} else if (OB_FAIL(upd_table_schema.add_partition(upd_part))) {
|
|
LOG_WARN("add partition fail", KR(ret), K(upd_part));
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
}
|
|
|
|
// the part_idx is deprecated, so we need to set it to a invalid value
|
|
if (OB_FAIL(ret)) {
|
|
} else {
|
|
const int64_t deprecated_part_idx = 0;
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < inc_part_num; ++i) {
|
|
ObPartition *part = inc_part_array[i];
|
|
if (OB_ISNULL(part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("split partition is null", KR(ret));
|
|
} else {
|
|
part->set_part_idx(deprecated_part_idx);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::pre_rename_mysql_columns_offline(
|
|
const ObTableSchema &origin_table_schema, AlterTableSchema &alter_table_schema,
|
|
bool is_oracle_mode, obrpc::ObAlterTableArg &alter_table_arg, ObTableSchema &new_table_schema,
|
|
sql::ObSchemaChecker &schema_checker,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
common::hash::ObHashSet<ObColumnNameHashWrapper> &update_column_name_set,
|
|
bool &need_redistribute_column_id, bool &is_contain_part_key)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterColumnSchema *alter_column_schema;
|
|
ObTableSchema::const_column_iterator it_begin = alter_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator it_end = alter_table_schema.column_end();
|
|
ObSEArray<ObColumnSchemaV2, 16> new_column_schemas;
|
|
ObSEArray<ObString, 8> new_pk_column;
|
|
ObSEArray<AlterColumnSchema *, 16> alter_column_schemas;
|
|
ObSEArray<const ObColumnSchemaV2 *, 16> orig_column_schemas;
|
|
ObSEArray<ObString, 16> orig_column_names;
|
|
for (; OB_SUCC(ret) && it_begin != it_end; it_begin++) {
|
|
if (OB_ISNULL(alter_column_schema = static_cast<AlterColumnSchema *>(*it_begin))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("*it_begin is NULL", K(ret));
|
|
} else if (is_oracle_mode || alter_column_schema->alter_type_ != OB_DDL_CHANGE_COLUMN
|
|
|| !is_rename_column(*alter_column_schema)) {
|
|
// do nothing
|
|
} else {
|
|
const ObString &orig_column_name = alter_column_schema->get_origin_column_name();
|
|
ObColumnNameHashWrapper orig_column_key(orig_column_name);
|
|
ObColumnSchemaV2 new_column_schema;
|
|
const ObColumnSchemaV2 *orig_column_schema =
|
|
new_table_schema.get_column_schema(orig_column_name);
|
|
if (OB_FAIL(alter_column_schemas.push_back(alter_column_schema))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
} else if (OB_FAIL(orig_column_schemas.push_back(orig_column_schema))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
} else {
|
|
if (OB_FAIL(prepare_change_modify_column_offline(
|
|
*alter_column_schema, origin_table_schema, alter_table_schema, is_oracle_mode,
|
|
alter_table_arg, new_table_schema, schema_checker, schema_guard,
|
|
update_column_name_set, new_column_schema, is_contain_part_key))) {
|
|
LOG_WARN("prepare change column offline failed", K(ret));
|
|
} else if (alter_column_schema->is_primary_key_) {
|
|
if (OB_FAIL(new_pk_column.push_back(alter_column_schema->get_column_name_str()))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
}
|
|
}
|
|
if (OB_FAIL(ret)) {
|
|
} else if (OB_FAIL(new_column_schemas.push_back(new_column_schema))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
} else if (OB_FAIL(
|
|
orig_column_names.push_back(alter_column_schema->get_origin_column_name()))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
} else {
|
|
if (OB_HASH_EXIST == update_column_name_set.exist_refactored(orig_column_key)) {
|
|
ret = OB_HASH_EXIST;
|
|
RS_LOG(WARN, "duplicate index name", K(ret), K(orig_column_name));
|
|
} else if (OB_FAIL(update_column_name_set.set_refactored(orig_column_key))) {
|
|
RS_LOG(WARN, "failed to add index_name to hash set.", K(orig_column_name), K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
if (OB_FAIL(ret)) {
|
|
} else if (alter_column_schemas.empty()) { // do nothing
|
|
} else if (OB_FAIL(new_table_schema.alter_mysql_table_columns(
|
|
new_column_schemas, orig_column_names, ObTableSchema::CHECK_MODE_OFFLINE))) {
|
|
LOG_WARN("alter mysql table columns failed", K(ret));
|
|
} else if (!new_pk_column.empty() && OB_FAIL(add_primary_key(new_pk_column, new_table_schema))) {
|
|
LOG_WARN("failed to add pk to table", K(ret), K(new_pk_column), K(new_table_schema));
|
|
} else if (OB_FAIL(new_table_schema.check_primary_key_cover_partition_column())) {
|
|
RS_LOG(WARN, "fail to check primary key cover partition column", K(ret));
|
|
}
|
|
|
|
for (int i = 0; OB_SUCC(ret) && i < alter_column_schemas.count(); i++) {
|
|
bool is_change_column_order = false;
|
|
if (OB_FAIL(check_is_change_column_order(new_table_schema, *alter_column_schemas.at(i), is_change_column_order))) {
|
|
LOG_WARN("failed to check is change column order", K(ret));
|
|
} else if (is_change_column_order) {
|
|
if (alter_column_schemas.at(i)->get_column_id()
|
|
!= orig_column_schemas.at(i)->get_column_id()) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid alter column schema column id", K(ret), K(*alter_column_schemas.at(i)),
|
|
K(*orig_column_schemas.at(i)));
|
|
} else if (OB_FAIL(new_table_schema.reorder_column(
|
|
alter_column_schemas.at(i)->get_column_name_str(),
|
|
alter_column_schemas.at(i)->is_first_,
|
|
alter_column_schemas.at(i)->get_prev_column_name(),
|
|
alter_column_schemas.at(i)->get_next_column_name()))) {
|
|
LOG_WARN("failed to reorder column", K(ret));
|
|
} else {
|
|
need_redistribute_column_id = true;
|
|
}
|
|
}
|
|
}
|
|
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_new_columns_for_index(ObIArray<ObTableSchema> &idx_schemas,
|
|
const ObTableSchema &orig_table_schema,
|
|
ObIArray<ObColumnSchemaV2> &new_column_schemas)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObTableSchema *index_table_schema = NULL;
|
|
hash::ObHashSet<uint64_t> main_table_rowkeys;
|
|
if (OB_FAIL(main_table_rowkeys.create(32))) {
|
|
LOG_WARN("create bucket failed", K(ret));
|
|
} else {
|
|
ObTableSchema::const_column_iterator iter = orig_table_schema.column_begin();
|
|
ObTableSchema::const_column_iterator end = orig_table_schema.column_end();
|
|
ObColumnSchemaV2 *col = nullptr;
|
|
for(; OB_SUCC(ret) && iter != end; iter++) {
|
|
if (OB_ISNULL(col = static_cast<ObColumnSchemaV2 *>(*iter))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid null *iter", K(ret));
|
|
} else if (col->is_rowkey_column()) {
|
|
if (OB_FAIL(main_table_rowkeys.set_refactored(col->get_column_id()))) {
|
|
LOG_WARN("set hashset failed");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
for (int i = 0; OB_SUCC(ret) && i < idx_schemas.count(); i++) {
|
|
index_table_schema = &idx_schemas.at(i);
|
|
ObSEArray<ObColumnSchemaV2, 8> copy_index_column_schemas;
|
|
ObSEArray<ObString , 8> orig_column_names;
|
|
for (int j = 0; OB_SUCC(ret) && j < new_column_schemas.count(); j++) {
|
|
const ObColumnSchemaV2 *orig_idx_col_schema =
|
|
index_table_schema->get_column_schema(new_column_schemas.at(j).get_column_id());
|
|
bool is_main_table_rowkey = false;
|
|
int tmp_ret = main_table_rowkeys.exist_refactored(new_column_schemas.at(j).get_column_id());
|
|
if (tmp_ret == OB_HASH_EXIST) {
|
|
is_main_table_rowkey = true;
|
|
}
|
|
if (OB_ISNULL(orig_idx_col_schema)) {
|
|
RS_LOG(INFO, "index table do not contain this column", "column_name",
|
|
new_column_schemas.at(j).get_column_name_str(), "index_table",
|
|
index_table_schema->get_table_name_str());
|
|
continue;
|
|
} else if (!orig_idx_col_schema->is_rowkey_column()
|
|
&& !is_main_table_rowkey) {
|
|
RS_LOG(INFO, "ingore not rowkey column", "column_name",
|
|
new_column_schemas.at(j).get_column_name_str(), "index_table",
|
|
index_table_schema->get_table_name_str());
|
|
} else if (OB_FAIL(orig_column_names.push_back(orig_idx_col_schema->get_column_name_str()))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
} else {
|
|
ObColumnSchemaV2 copy_index_column_schema;
|
|
if (OB_FAIL(copy_index_column_schema.assign(new_column_schemas.at(j)))) {
|
|
LOG_WARN("fail to assign column", KR(ret), K(new_column_schemas.at(j)));
|
|
} else {
|
|
copy_index_column_schema.set_rowkey_position(orig_idx_col_schema->get_rowkey_position());
|
|
copy_index_column_schema.set_index_position(orig_idx_col_schema->get_index_position());
|
|
copy_index_column_schema.set_tbl_part_key_pos(orig_idx_col_schema->get_tbl_part_key_pos());
|
|
if (OB_FAIL(copy_index_column_schemas.push_back(copy_index_column_schema))) {
|
|
LOG_WARN("push back element failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
} // end for
|
|
if (OB_SUCC(ret)) {
|
|
if (OB_FAIL(index_table_schema->alter_mysql_table_columns(
|
|
copy_index_column_schemas, orig_column_names, ObTableSchema::CHECK_MODE_ONLINE))) {
|
|
LOG_WARN("alter index column schemas failed", K(ret));
|
|
} else if (!index_table_schema->is_valid()) {
|
|
ret = OB_SCHEMA_ERROR;
|
|
LOG_WARN("index table schema is invalid", K(ret));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
// if alter stmt contains column renaming request and dst column name already exists in table,
|
|
// renaming needs to execute first.
|
|
int ObDDLService::check_rename_first(const AlterTableSchema &alter_table_schema,
|
|
const ObTableSchema &table_schema,
|
|
const bool is_oracle_mode, bool &is_rename_first)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
AlterColumnSchema *alter_column_schema = nullptr;
|
|
const ObColumnSchemaV2 *column_schema = nullptr;
|
|
is_rename_first = false;
|
|
if (is_oracle_mode) {
|
|
// need to check rename cycle only in mysql mode.
|
|
} else {
|
|
hash::ObHashSet<ObColumnNameHashWrapper> orig_keys;
|
|
if (OB_FAIL(orig_keys.create(32))) {
|
|
LOG_WARN("init hashset failed", K(ret));
|
|
} else {
|
|
for (auto col_iter = table_schema.column_begin(); OB_SUCC(ret) && col_iter != table_schema.column_end(); col_iter++) {
|
|
if (OB_ISNULL(column_schema = static_cast<const ObColumnSchemaV2 *>(*col_iter))) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid null *col_iter", K(ret));
|
|
} else {
|
|
ObColumnNameHashWrapper orig_key(column_schema->get_column_name_str());
|
|
if (OB_FAIL(orig_keys.set_refactored(orig_key))) {
|
|
LOG_WARN("set hashset failed", K(ret));
|
|
}
|
|
}
|
|
} // end for
|
|
for (auto col_iter = alter_table_schema.column_begin();
|
|
!is_rename_first && OB_SUCC(ret) && col_iter != alter_table_schema.column_end();
|
|
col_iter++) {
|
|
alter_column_schema = static_cast<AlterColumnSchema *>(*col_iter);
|
|
if (OB_ISNULL(alter_column_schema)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("invalid null alter_column_schema", K(ret));
|
|
} else {
|
|
ObColumnNameHashWrapper orig_key(alter_column_schema->get_origin_column_name());
|
|
ObColumnNameHashWrapper new_key(alter_column_schema->get_column_name_str());
|
|
if (alter_column_schema->alter_type_ == OB_DDL_DROP_COLUMN) {
|
|
ret = orig_keys.erase_refactored(orig_key);
|
|
if (OB_FAIL(ret)) {
|
|
if (ret == OB_HASH_NOT_EXIST) {
|
|
ret = OB_ERR_CANT_DROP_FIELD_OR_KEY;
|
|
LOG_USER_ERROR(OB_ERR_CANT_DROP_FIELD_OR_KEY,
|
|
alter_column_schema->get_origin_column_name().length(),
|
|
alter_column_schema->get_origin_column_name().ptr());
|
|
LOG_WARN("invalid drop column", K(*alter_column_schema));
|
|
} else {
|
|
LOG_WARN("erase refactored failed", K(ret));
|
|
}
|
|
}
|
|
} else if (!(orig_key == new_key)) {
|
|
ret = orig_keys.exist_refactored(new_key);
|
|
if (ret == OB_HASH_EXIST || ret == OB_HASH_NOT_EXIST) {
|
|
is_rename_first = (ret == OB_HASH_EXIST);
|
|
ret = OB_SUCCESS;
|
|
} else if (OB_FAIL(ret)) {
|
|
LOG_WARN("exist refactored failed", K(ret));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fix_local_idx_part_name_(const ObSimpleTableSchemaV2 &ori_data_table_schema,
|
|
const ObSimpleTableSchemaV2 &ori_table_schema,
|
|
ObSimpleTableSchemaV2 &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool ori_oracle_mode = false;
|
|
bool is_matched = false;
|
|
const schema::ObPartitionOption &ori_part_option = ori_table_schema.get_part_option();
|
|
schema::ObPartitionFuncType ori_part_func_type = ori_part_option.get_part_func_type();
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
const ObPartitionLevel part_level = inc_table_schema.get_part_level();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
const int64_t inc_partition_num = inc_table_schema.get_partition_num();
|
|
const ObPartition *ori_data_part = nullptr;
|
|
const ObPartition *ori_part = nullptr;
|
|
ObPartition *inc_part = nullptr;
|
|
int64_t part_id = 0;
|
|
int64_t part_idx = 0;
|
|
ObString part_name;
|
|
if (!ori_table_schema.is_aux_table()) {
|
|
//no need to fix
|
|
} else if (OB_UNLIKELY(ori_data_table_schema.get_table_id() != ori_table_schema.get_data_table_id())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("data_table_id not same", KR(ret), K(ori_data_table_schema.get_table_id()), K(ori_table_schema.get_data_table_id()));
|
|
} else if (OB_UNLIKELY(ori_table_schema.get_table_id() != inc_table_schema.get_table_id())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("inc table id not euqal to ori table id", KR(ret), K(ori_table_schema.get_table_id()), K(inc_table_schema.get_table_id()));
|
|
} else if (OB_FAIL(ori_table_schema.check_if_oracle_compat_mode(ori_oracle_mode))) {
|
|
LOG_WARN("fail to get ori aux table oracle mode", KR(ret), K(ori_table_schema));
|
|
} else if (PARTITION_LEVEL_ZERO == part_level) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("inc table should be a partitioned table", KR(ret), K(part_level));
|
|
} else if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_table_part_array is null", KR(ret), KP(inc_part_array));
|
|
} else {
|
|
for (int64_t i = 0; i < inc_partition_num && OB_SUCC(ret); i++) {
|
|
if (FALSE_IT(inc_part = inc_part_array[i])) {
|
|
} else if (OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_part is null", KR(ret), KP(inc_part_array), K(i), K(inc_table_schema));
|
|
} else {
|
|
const ObString &inc_part_name = inc_part->get_part_name();
|
|
if (OB_FAIL(ori_data_table_schema.get_partition_by_name(inc_part_name, ori_data_part))) {
|
|
LOG_WARN("fail to get partition by name", KR(ret), K(inc_part_name), KPC(ori_data_part));
|
|
} else if (FALSE_IT(part_id = ori_data_part->get_part_id())) {
|
|
} else if (OB_FAIL(ori_data_table_schema.get_partition_index_by_id(part_id, check_partition_mode, part_idx))) {
|
|
LOG_WARN("fail to get part idx", KR(ret), K(part_id), K(part_idx));
|
|
} else if (OB_FAIL(ori_table_schema.get_partition_by_partition_index(part_idx, check_partition_mode, ori_part))) {
|
|
LOG_WARN("fail to get src part by idx", KR(ret), K(part_idx), KPC(ori_part));
|
|
} else if (OB_ISNULL(ori_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("src part is null", KR(ret));
|
|
} else if (FALSE_IT(part_name = ori_part->get_part_name())){
|
|
} else if (OB_FAIL(check_same_partition(ori_oracle_mode, *ori_data_part, *ori_part, ori_part_func_type, is_matched))) {
|
|
LOG_WARN("fail to check ori_table_part and ori_aux_part is the same", KR(ret), KPC(ori_data_part), KPC(ori_part), K(ori_part_func_type));
|
|
} else if (OB_UNLIKELY(!is_matched)) {
|
|
ret = OB_INDEX_INELIGIBLE;
|
|
LOG_WARN("part with the same offset not equal, maybe not the right index", KR(ret), KPC(ori_data_part), KPC(ori_part));
|
|
} else if (OB_FAIL(inc_part->set_part_name(part_name))) {
|
|
LOG_WARN("fail to set part name", KR(ret), KPC(inc_part), K(part_name));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fix_local_idx_subpart_name_(const ObSimpleTableSchemaV2 &ori_data_table_schema,
|
|
const ObSimpleTableSchemaV2 &ori_table_schema,
|
|
ObSimpleTableSchemaV2 &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool ori_oracle_mode = false;
|
|
bool is_matched = false;
|
|
const schema::ObPartitionOption &ori_part_option = ori_table_schema.get_part_option();
|
|
schema::ObPartitionFuncType ori_part_func_type = ori_part_option.get_part_func_type();
|
|
const schema::ObPartitionOption &ori_subpart_option = ori_table_schema.get_sub_part_option();
|
|
schema::ObPartitionFuncType ori_subpart_func_type = ori_subpart_option.get_sub_part_func_type();
|
|
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
|
|
const ObPartitionLevel part_level = inc_table_schema.get_part_level();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
ObSubPartition **inc_subpart_array = nullptr;
|
|
const int64_t inc_partition_num = inc_table_schema.get_partition_num();
|
|
const ObPartition *ori_data_part = nullptr;
|
|
const ObPartition *ori_part = nullptr;
|
|
ObPartition *inc_part = nullptr;
|
|
const ObSubPartition *ori_data_subpart = nullptr;
|
|
const ObSubPartition *ori_subpart = nullptr;
|
|
ObSubPartition *inc_subpart = nullptr;
|
|
int64_t part_id = 0;
|
|
int64_t part_idx = 0;
|
|
int64_t subpart_id = 0;
|
|
int64_t subpart_idx = 0;
|
|
ObString part_name;
|
|
if (!ori_table_schema.is_aux_table()) {
|
|
//no need to fix
|
|
} else if (OB_UNLIKELY(ori_data_table_schema.get_table_id() != ori_table_schema.get_data_table_id())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("data_table_id not same", KR(ret), K(ori_data_table_schema.get_table_id()), K(ori_table_schema.get_data_table_id()));
|
|
} else if (OB_UNLIKELY(ori_table_schema.get_table_id() != inc_table_schema.get_table_id())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("inc table id not euqal to ori table id", KR(ret), K(ori_table_schema.get_table_id()), K(inc_table_schema.get_table_id()));
|
|
} else if (OB_FAIL(ori_table_schema.check_if_oracle_compat_mode(ori_oracle_mode))) {
|
|
LOG_WARN("fail to get ori aux table oracle mode", KR(ret), K(ori_table_schema));
|
|
} else if (PARTITION_LEVEL_TWO != part_level) {
|
|
//no need to fix
|
|
} else if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_table_part_array is null", KR(ret), KP(inc_part_array));
|
|
} else {
|
|
for (int64_t i = 0; i < inc_partition_num && OB_SUCC(ret); i++){
|
|
inc_part = inc_part_array[i];
|
|
if (OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_part is null", KR(ret), KP(inc_part_array), K(i), K(inc_table_schema));
|
|
} else if (OB_ISNULL(inc_subpart_array = inc_part->get_subpart_array())) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_aux_table_subpart_array is null ptr", KR(ret));
|
|
} else {
|
|
const int64_t inc_subpartition_num = inc_part->get_subpartition_num();
|
|
for (int64_t j = 0; j < inc_subpartition_num && OB_SUCC(ret); j++) {
|
|
if (OB_ISNULL(inc_subpart = inc_subpart_array[j])) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_aux_table subpart is null ptr", KR(ret), K(j));
|
|
} else if (OB_FAIL(ori_data_table_schema.get_subpartition_by_name(inc_subpart->get_part_name(), ori_data_part, ori_data_subpart))) {
|
|
LOG_WARN("fail to get subpartition by name", KR(ret), K(inc_subpart->get_part_name()), KP(ori_data_part), KP(ori_data_subpart));
|
|
} else if (FALSE_IT(part_id = ori_data_part->get_part_id())) {
|
|
} else if (OB_FAIL(ori_data_table_schema.get_partition_index_by_id(part_id, check_partition_mode, part_idx))) {
|
|
LOG_WARN("fail to get part idx", KR(ret), K(part_id), K(part_idx));
|
|
} else if (OB_FAIL(ori_table_schema.get_partition_by_partition_index(part_idx, check_partition_mode, ori_part))) {
|
|
LOG_WARN("fail to get src part by idx", KR(ret), K(part_idx), KPC(ori_part));
|
|
} else if (OB_ISNULL(ori_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("src part is null", KR(ret));
|
|
} else if (OB_FAIL(check_same_partition(ori_oracle_mode, *ori_data_part, *ori_part, ori_part_func_type, is_matched))) {
|
|
LOG_WARN("fail to check ori_table_part and ori_aux_part is the same", KR(ret), KPC(ori_data_part), KPC(ori_part), K(ori_part_func_type));
|
|
} else if (OB_UNLIKELY(!is_matched)) {
|
|
ret = OB_INDEX_INELIGIBLE;
|
|
LOG_WARN("part with the same offset not equal, maybe not the right index", KR(ret), KPC(inc_part), KPC(ori_part));
|
|
} else if (FALSE_IT(subpart_id = ori_data_subpart->get_sub_part_id())) {
|
|
} else if (OB_FAIL(ori_data_part->get_normal_subpartition_index_by_id(subpart_id, subpart_idx))) {
|
|
LOG_WARN("src subpart array is null ptr", KR(ret), K(subpart_id), K(subpart_idx));
|
|
} else if (OB_FAIL(ori_part->get_normal_subpartition_by_subpartition_index(subpart_idx, ori_subpart))) {
|
|
LOG_WARN("fail to get src subpart by subpart index", KR(ret), K(subpart_idx));
|
|
} else if (OB_ISNULL(ori_subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("fail to get src subpart.", KR(ret));
|
|
} else if (FALSE_IT(part_name = ori_subpart->get_part_name())) {
|
|
} else if (OB_FAIL(check_same_subpartition(ori_oracle_mode, *ori_data_subpart, *ori_subpart, ori_subpart_func_type, is_matched))) {
|
|
LOG_WARN("fail to check ori_table_subpart and ori_aux_subpart is the same", KR(ret), KPC(ori_data_part), KPC(ori_part), K(ori_subpart_func_type));
|
|
} else if (OB_UNLIKELY(!is_matched)) {
|
|
ret = OB_INDEX_INELIGIBLE;
|
|
LOG_WARN("part with the same offset not equal, maybe not the right index", KR(ret), KPC(inc_subpart), KPC(ori_subpart));
|
|
} else if (OB_FAIL(inc_subpart->set_part_name(part_name))) {
|
|
LOG_WARN("fail to set subpart name", KR(ret), KPC(inc_part), K(part_name));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fix_local_idx_part_name_for_add_part_(const ObSimpleTableSchemaV2 &ori_table_schema,
|
|
ObSimpleTableSchemaV2 &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
const int64_t inc_partition_num = inc_table_schema.get_partition_num();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
ObPartition *inc_part = nullptr;
|
|
char buf[OB_MAX_PARTITION_NAME_LENGTH];
|
|
ObString part_name;
|
|
if (OB_UNLIKELY(ori_table_schema.get_table_id() != inc_table_schema.get_table_id())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("ori table id not equal to inc table schema id");
|
|
} else if (!ori_table_schema.is_aux_table()) {
|
|
//no need to fix
|
|
} else if (OB_FAIL(ori_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check oracle mode", KR(ret), K(ori_table_schema));
|
|
} else if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part array is null", KR(ret), K(inc_table_schema));
|
|
} else {
|
|
for (int64_t i = 0; i < inc_partition_num && OB_SUCC(ret); i++) {
|
|
inc_part = inc_part_array[i];
|
|
if (OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part is null", KR(ret));
|
|
} else if (OB_FAIL(ori_table_schema.check_partition_duplicate_with_name(inc_part->get_part_name()))) {
|
|
if (OB_DUPLICATE_OBJECT_NAME_EXIST == ret) {
|
|
MEMSET(buf, 0, OB_MAX_PARTITION_NAME_LENGTH);
|
|
if (OB_FAIL(ObPartitionSchema::gen_hash_part_name(
|
|
inc_part->get_part_id(), FIRST_PART, is_oracle_mode, buf, OB_MAX_PARTITION_NAME_LENGTH, NULL, NULL))) {
|
|
LOG_WARN("fail to get part name", KR(ret), K(i));
|
|
} else if (FALSE_IT(part_name.assign_ptr(buf, static_cast<int32_t>(strlen(buf))))) {
|
|
} else if (OB_FAIL(inc_part->set_part_name(part_name))) {
|
|
LOG_WARN("fail to set name", KR(ret), KPC(inc_part), K(part_name));
|
|
}
|
|
} else {
|
|
LOG_WARN("fail to check dupliate partition with name", KR(ret), K(ori_table_schema), K(inc_part->get_part_name()));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::fix_local_idx_part_name_for_add_subpart_(const ObSimpleTableSchemaV2 &ori_table_schema,
|
|
ObSimpleTableSchemaV2 &inc_table_schema)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool is_oracle_mode = false;
|
|
const int64_t inc_partition_num = inc_table_schema.get_partition_num();
|
|
ObPartition **inc_part_array = inc_table_schema.get_part_array();
|
|
ObPartition *inc_part = nullptr;
|
|
char buf[OB_MAX_PARTITION_NAME_LENGTH];
|
|
ObString part_name;
|
|
if (OB_UNLIKELY(ori_table_schema.get_table_id() != inc_table_schema.get_table_id())) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("ori table id not equal to inc table schema id");
|
|
} else if (!ori_table_schema.is_aux_table()) {
|
|
//no need to fix
|
|
} else if (OB_FAIL(ori_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
|
LOG_WARN("fail to check oracle mode", KR(ret), K(ori_table_schema));
|
|
} else if (OB_ISNULL(inc_part_array)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part array is null", KR(ret), K(inc_table_schema));
|
|
} else {
|
|
for (int64_t i = 0; i < inc_partition_num && OB_SUCC(ret); i++) {
|
|
inc_part = inc_part_array[i];
|
|
if (OB_ISNULL(inc_part)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc part is null", KR(ret));
|
|
} else {
|
|
const int64_t inc_subpartition_num = inc_part->get_subpartition_num();
|
|
ObSubPartition **inc_subpart_array = inc_part->get_subpart_array();
|
|
ObSubPartition *inc_subpart = nullptr;
|
|
for (int64_t j = 0; j < inc_subpartition_num && OB_SUCC(ret); j++) {
|
|
inc_subpart = inc_subpart_array[j];
|
|
if (OB_ISNULL(inc_subpart)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("inc_subpart is null", KR(ret));
|
|
} else if (OB_FAIL(ori_table_schema.check_partition_duplicate_with_name(inc_subpart->get_part_name()))) {
|
|
if (OB_DUPLICATE_OBJECT_NAME_EXIST == ret) {
|
|
MEMSET(buf, 0, OB_MAX_PARTITION_NAME_LENGTH);
|
|
if (OB_FAIL(ObPartitionSchema::gen_hash_part_name(
|
|
inc_subpart->get_sub_part_id(), TEMPLATE_SUB_PART, is_oracle_mode, buf, OB_MAX_PARTITION_NAME_LENGTH, NULL, NULL))) {
|
|
LOG_WARN("fail to get part name", KR(ret), K(i));
|
|
} else if (FALSE_IT(part_name.assign_ptr(buf, static_cast<int32_t>(strlen(buf))))) {
|
|
} else if (OB_FAIL(inc_subpart->set_part_name(part_name))) {
|
|
LOG_WARN("fail to set name", KR(ret), KPC(inc_subpart), K(part_name));
|
|
}
|
|
} else {
|
|
LOG_WARN("fail to check dupliate partition with name", KR(ret), K(ori_table_schema), K(inc_subpart->get_part_name()));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_same_partition(const bool is_oracle_mode, const ObPartition &l, const ObPartition &r,
|
|
const ObPartitionFuncType part_type, bool &is_matched) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_matched = false;
|
|
if (l.get_sub_part_num() == r.get_sub_part_num()
|
|
&& l.get_sub_interval_start() == r.get_sub_interval_start()
|
|
&& l.get_sub_part_interval() == r.get_sub_part_interval()) {
|
|
if (is_hash_like_part(part_type)) {
|
|
if(l.get_part_idx() == r.get_part_idx()) {
|
|
is_matched = true;
|
|
}
|
|
} else if (OB_FAIL(schema::ObPartitionUtils::check_partition_value(is_oracle_mode, l, r, part_type, is_matched))) {
|
|
LOG_WARN("fail to check partition value", KR(ret), K(l), K(r));
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_same_subpartition(const bool is_oracle_mode, const ObSubPartition &l, const ObSubPartition &r,
|
|
const ObPartitionFuncType part_type, bool &is_matched) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
is_matched = false;
|
|
if (is_hash_like_part(part_type)) {
|
|
if (l.get_sub_part_idx() == r.get_sub_part_idx()) {
|
|
is_matched = true;
|
|
}
|
|
} else if (OB_FAIL(schema::ObPartitionUtils::check_partition_value(is_oracle_mode, l, r, part_type, is_matched))) {
|
|
LOG_WARN("fail to check partition value", KR(ret), K(l), K(r));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
bool ObDDLService::need_modify_dep_obj_status(const obrpc::ObAlterTableArg &alter_table_arg) const
|
|
{
|
|
const AlterTableSchema &alter_table_schema = alter_table_arg.alter_table_schema_;
|
|
return (alter_table_arg.is_alter_columns_
|
|
|| (alter_table_arg.is_alter_options_
|
|
&& alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::TABLE_NAME)));
|
|
}
|
|
|
|
int ObDDLService::add_extra_tenant_init_config_(
|
|
const uint64_t tenant_id,
|
|
common::ObIArray<common::ObConfigPairs> &init_configs)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
bool find = false;
|
|
ObString config_name("_parallel_ddl_control");
|
|
ObSqlString config_value;
|
|
if (OB_FAIL(ObParallelDDLControlMode::generate_parallel_ddl_control_config_for_create_tenant(config_value))) {
|
|
LOG_WARN("fail to generate parallel ddl control config value", KR(ret));
|
|
}
|
|
for (int index = 0 ; !find && OB_SUCC(ret) && index < init_configs.count(); ++index) {
|
|
if (tenant_id == init_configs.at(index).get_tenant_id()) {
|
|
find = true;
|
|
common::ObConfigPairs ¶llel_table_config = init_configs.at(index);
|
|
if (OB_FAIL(parallel_table_config.add_config(config_name, config_value.string()))) {
|
|
LOG_WARN("fail to add config", KR(ret), K(config_name), K(config_value));
|
|
}
|
|
}
|
|
}
|
|
// ---- Add new tenant init config above this line -----
|
|
// At the same time, to verify modification, you need modify test case tenant_init_config(_oracle).test
|
|
if (OB_SUCC(ret) && !find) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("no matched tenant config", KR(ret), K(tenant_id), K(init_configs));
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
bool ObDDLService::is_dec_table_lob_inrow_threshold(
|
|
const obrpc::ObAlterTableArg &alter_table_arg,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const ObTableSchema *orig_table_schema) const
|
|
{
|
|
return OB_NOT_NULL(orig_table_schema)
|
|
&& alter_table_arg.is_alter_options_
|
|
&& alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::LOB_INROW_THRESHOLD)
|
|
&& alter_table_schema.get_lob_inrow_threshold() < orig_table_schema->get_lob_inrow_threshold();
|
|
}
|
|
|
|
int ObDDLService::check_alter_lob_inrow_threshold(
|
|
obrpc::ObAlterTableArg &alter_table_arg,
|
|
const AlterTableSchema &alter_table_schema,
|
|
const ObTableSchema *orig_table_schema,
|
|
bool &is_dec_lob_inrow_threshold,
|
|
share::ObDDLType &ddl_type) const
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
if (! is_dec_table_lob_inrow_threshold(alter_table_arg, alter_table_schema, orig_table_schema)) {
|
|
// inc or not change lob inrow threshold is online
|
|
} else if (ddl_type != ObDDLType::DDL_INVALID) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("There are several mutually exclusive DDL in single statement", K(ret), K(ddl_type));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement");
|
|
} else {
|
|
ObTableSchema::const_column_iterator tmp_begin = orig_table_schema->column_begin();
|
|
ObTableSchema::const_column_iterator tmp_end = orig_table_schema->column_end();
|
|
bool is_origin_table_has_lob_column = 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) && i != ObAlterTableArg::LOB_INROW_THRESHOLD) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("There are several mutually exclusive DDL in single statement", K(ret), K(ddl_type), K(alter_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "There are several mutually exclusive DDL in single statement");
|
|
}
|
|
}
|
|
|
|
for (; OB_SUCC(ret) && tmp_begin != tmp_end && ! is_origin_table_has_lob_column; tmp_begin++) {
|
|
ObColumnSchemaV2 *column = (*tmp_begin);
|
|
if (OB_ISNULL(column)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("col is NULL", K(ret));
|
|
} else if (is_lob_storage(column->get_data_type())) {
|
|
is_origin_table_has_lob_column = true;
|
|
}
|
|
}
|
|
|
|
// online if no lob storage column in origin table
|
|
// offline if has lob storage column and dec lob inrow threshold
|
|
if (OB_SUCC(ret) && is_origin_table_has_lob_column) {
|
|
ddl_type = ObDDLType::DDL_TABLE_REDEFINITION;
|
|
is_dec_lob_inrow_threshold = true;
|
|
}
|
|
LOG_TRACE("alter lob inrow threahold", K(ret), K(is_origin_table_has_lob_column), K(ddl_type),
|
|
"old_value", alter_table_schema.get_lob_inrow_threshold(), "new_value", orig_table_schema->get_lob_inrow_threshold());
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_fts_index_conflict(const uint64_t tenant_id, const uint64_t table_id) {
|
|
int ret = OB_SUCCESS;
|
|
ObArenaAllocator allocator(lib::ObLabel("DdlTasRecord"));
|
|
ObArray<ObDDLTaskRecord> task_records;
|
|
if (OB_FAIL(ObDDLTaskRecordOperator::get_ddl_task_record_by_table_id(tenant_id, table_id, get_sql_proxy(), allocator, task_records))) {
|
|
LOG_WARN("get task record failed", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < task_records.count(); ++i) {
|
|
const ObDDLTaskRecord &cur_record = task_records.at(i);
|
|
if (cur_record.ddl_type_ == ObDDLType::DDL_CREATE_FTS_INDEX
|
|
|| cur_record.ddl_type_ == ObDDLType::DDL_DROP_FTS_INDEX
|
|
|| cur_record.ddl_type_ == ObDDLType::DDL_CREATE_MULTIVALUE_INDEX
|
|
|| cur_record.ddl_type_ == ObDDLType::DDL_DROP_MULVALUE_INDEX) {
|
|
LOG_INFO("cur_record.ddl_type is: ", K(cur_record.ddl_type_));
|
|
ret = OB_EAGAIN;
|
|
LOG_WARN("fts index is building, will retry later", K(ret), K(table_id), K(cur_record));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::check_vec_index_conflict(const uint64_t tenant_id, const uint64_t table_id) {
|
|
int ret = OB_SUCCESS;
|
|
ObArenaAllocator allocator(lib::ObLabel("DdlTasRecord"));
|
|
ObArray<ObDDLTaskRecord> task_records;
|
|
if (OB_FAIL(ObDDLTaskRecordOperator::get_ddl_task_record_by_table_id(tenant_id, table_id, get_sql_proxy(), allocator, task_records))) {
|
|
LOG_WARN("get task record failed", K(ret));
|
|
} else {
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < task_records.count(); ++i) {
|
|
const ObDDLTaskRecord &cur_record = task_records.at(i);
|
|
if (cur_record.ddl_type_ == ObDDLType::DDL_CREATE_VEC_INDEX ||
|
|
cur_record.ddl_type_ == ObDDLType::DDL_DROP_VEC_INDEX ||
|
|
cur_record.ddl_type_ == ObDDLType::DDL_REBUILD_INDEX) {
|
|
ret = OB_EAGAIN;
|
|
LOG_WARN("vector index is building, will retry later", K(ret), K(table_id), K(cur_record));
|
|
}
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
int ObDDLService::drop_index_to_scheduler_(ObMySQLTransaction &trans,
|
|
ObSchemaGetterGuard &schema_guard,
|
|
ObArenaAllocator &allocator,
|
|
const ObTableSchema &orig_table_schema,
|
|
const common::ObIArray<common::ObTabletID> *inc_data_tablet_ids,
|
|
const common::ObIArray<common::ObTabletID> *del_data_tablet_ids,
|
|
obrpc::ObDropIndexArg *drop_index_arg,
|
|
ObDDLOperator &ddl_operator,
|
|
obrpc::ObAlterTableRes &res,
|
|
ObIArray<ObDDLTaskRecord> &ddl_tasks)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
ObDDLRes ddl_res;
|
|
ObDDLTaskRecord task_record;
|
|
const ObTableSchema *index_table_schema = nullptr;
|
|
ObIndexBuilder index_builder(*this);
|
|
ObIArray<obrpc::ObDDLRes> &ddl_res_array = res.ddl_res_array_;
|
|
if (OB_ISNULL(drop_index_arg)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("drop index arg is nullptr", KR(ret));
|
|
} else if (drop_index_arg->index_name_.empty()) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("index name is empty", KR(ret), K(drop_index_arg->index_name_));
|
|
} else {
|
|
const ObString &index_name = drop_index_arg->index_name_;
|
|
if (OB_FAIL(get_index_schema_by_name(
|
|
orig_table_schema.get_table_id(),
|
|
orig_table_schema.get_database_id(),
|
|
*drop_index_arg,
|
|
schema_guard,
|
|
index_table_schema))) {
|
|
LOG_WARN("get index schema by name failed", KR(ret), K(orig_table_schema.get_table_id()),
|
|
K(orig_table_schema.get_database_id()), KPC(drop_index_arg));
|
|
} else if (OB_ISNULL(index_table_schema)) {
|
|
ret = OB_TABLE_NOT_EXIST;
|
|
LOG_WARN("index not exist", KR(ret), K(orig_table_schema.get_table_id()), K(drop_index_arg->index_name_));
|
|
} else {
|
|
const bool is_fts_or_multivalue_or_vec_index = (index_table_schema->is_fts_or_multivalue_index() || index_table_schema->is_vec_index());
|
|
const bool is_inner_and_domain_index = drop_index_arg->is_inner_ && is_fts_or_multivalue_or_vec_index;
|
|
const bool need_check_fts_index_conflict = !drop_index_arg->is_inner_ && index_table_schema->is_fts_or_multivalue_index();
|
|
const bool need_check_vec_index_conflict = !drop_index_arg->is_inner_ && index_table_schema->is_vec_index();
|
|
bool has_index_task = false;
|
|
typedef common::ObSEArray<share::schema::ObTableSchema, 4> TableSchemaArray;
|
|
SMART_VAR(TableSchemaArray, new_index_schemas) {
|
|
if (need_check_fts_index_conflict && OB_FAIL(check_fts_index_conflict(orig_table_schema.get_tenant_id(), orig_table_schema.get_table_id()))) {
|
|
if (OB_EAGAIN != ret) {
|
|
LOG_WARN("failed to check fts index ", K(ret));
|
|
}
|
|
} else if (need_check_vec_index_conflict && OB_FAIL(check_vec_index_conflict(orig_table_schema.get_tenant_id(), orig_table_schema.get_table_id()))) {
|
|
if (OB_EAGAIN != ret) {
|
|
LOG_WARN("failed to check vec index ", K(ret));
|
|
}
|
|
} else if (!drop_index_arg->is_inner_ && !index_table_schema->can_read_index() && OB_FAIL(ObDDLTaskRecordOperator::check_has_index_or_mlog_task(
|
|
trans, *index_table_schema, orig_table_schema.get_tenant_id(), orig_table_schema.get_table_id(), has_index_task))) {
|
|
LOG_WARN("failed to check ddl conflict", KR(ret));
|
|
} else if (has_index_task) {
|
|
ret = OB_NOT_SUPPORTED;
|
|
LOG_WARN("not support to drop a building or dropping index", K(ret), K(drop_index_arg->is_inner_), KPC(index_table_schema));
|
|
LOG_USER_ERROR(OB_NOT_SUPPORTED, "dropping a building or dropping index is");
|
|
} else if (OB_FAIL(rename_dropping_index_name(orig_table_schema.get_table_id(),
|
|
orig_table_schema.get_database_id(),
|
|
is_inner_and_domain_index,
|
|
*drop_index_arg,
|
|
schema_guard,
|
|
ddl_operator,
|
|
trans,
|
|
new_index_schemas))) {
|
|
LOG_WARN("submit drop index arg failed", KR(ret));
|
|
} else if (OB_UNLIKELY(!is_fts_or_multivalue_or_vec_index && new_index_schemas.count() != 1)
|
|
|| OB_UNLIKELY(!drop_index_arg->is_inner_ && index_table_schema->is_vec_delta_buffer_type() && new_index_schemas.count() != 5)
|
|
|| OB_UNLIKELY(index_table_schema->is_fts_index_aux() && new_index_schemas.count() != 4)
|
|
|| OB_UNLIKELY(index_table_schema->is_multivalue_index_aux() && new_index_schemas.count() != 3)) {
|
|
ret = OB_ERR_UNEXPECTED;
|
|
LOG_WARN("unexpected error, invalid new index schema count", KR(ret),
|
|
"count", new_index_schemas.count(),
|
|
"is fts index", index_table_schema->is_fts_index_aux(),
|
|
"is multivalue index", index_table_schema->is_multivalue_index_aux(),
|
|
"is vector index", index_table_schema->is_vec_delta_buffer_type(),
|
|
K(new_index_schemas));
|
|
} else {
|
|
const ObTableSchema &new_index_schema = new_index_schemas.at(new_index_schemas.count() - 1);
|
|
drop_index_arg->index_table_id_ = new_index_schema.get_table_id();
|
|
if (OB_FAIL(submit_drop_index_task_and_fill_ddl_result_(allocator, trans, new_index_schemas,
|
|
orig_table_schema, drop_index_arg,
|
|
inc_data_tablet_ids, del_data_tablet_ids, has_index_task,
|
|
ddl_tasks, ddl_res_array))) {
|
|
LOG_WARN("fail to submit drop index task", KR(ret), KPC(drop_index_arg));
|
|
}
|
|
}
|
|
} // end smart var
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
template <class TTableSchema>
|
|
int ObDDLService:: get_tablets_with_table_id_(const ObArray<TTableSchema *> &table_schemas,
|
|
const int table_id,
|
|
ObArray<ObTabletID> &tablet_ids)
|
|
{
|
|
int ret = OB_SUCCESS;
|
|
tablet_ids.reset();
|
|
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); i++) {
|
|
ObTableSchema *table_schema = table_schemas[i];
|
|
if (OB_ISNULL(table_schema)) {
|
|
ret = OB_INVALID_ARGUMENT;
|
|
LOG_WARN("table schema is null ptr", KR(ret));
|
|
} else if (table_schema->get_table_id() == table_id) {
|
|
// some table types don't have tablet like external table
|
|
if (table_schema->has_tablet()
|
|
&& OB_FAIL(table_schema->get_tablet_ids(tablet_ids))) {
|
|
LOG_WARN("failed to get table tablet ids", KR(ret), KPC(table_schema));
|
|
}
|
|
break;
|
|
}
|
|
}
|
|
return ret;
|
|
}
|
|
|
|
} // end namespace rootserver
|
|
} // end namespace oceanbase
|