diff --git a/deps/oblib/src/lib/utility/ob_tracepoint.h b/deps/oblib/src/lib/utility/ob_tracepoint.h index 1aa25623bb..18f5a1df87 100644 --- a/deps/oblib/src/lib/utility/ob_tracepoint.h +++ b/deps/oblib/src/lib/utility/ob_tracepoint.h @@ -704,6 +704,8 @@ class EventTable EN_BACKUP_READ_MACRO_BLOCK_FAILED = 1111, EN_FETCH_TABLE_INFO_RPC = 1112, EN_RESTORE_TABLET_TASK_FAILED = 1113, + EN_INSERT_USER_RECOVER_JOB_FAILED = 1114, + EN_INSERT_AUX_TENANT_RESTORE_JOB_FAILED = 1115, // END OF STORAGE HA - 1101 - 2000 // sql parameterization 1170-1180 diff --git a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h index 46f308c30f..b23c4f69ab 100644 --- a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h +++ b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h @@ -922,9 +922,9 @@ PCODE_DEF(OB_CHECK_BACKUP_DEST_CONNECTIVITY, 0x1423) PCODE_DEF(OB_BACKUP_META, 0x1424) //PCODE_DEF(OB_BACKUP_CHECK_TABLET_CREATE_TS, 0x1425) not used anymore PCODE_DEF(OB_DELETE_POLICY, 0x1426) -// 0x1427 for OB_RECOVER_TABLE +PCODE_DEF(OB_RECOVER_TABLE, 0x1427) // 0x1428 for OB_BACKUP_ADVANCE_CHECKPOINT -// 0x1429 for OB_RECOVER_RESTORE_TABLE_DDL +PCODE_DEF(OB_RECOVER_RESTORE_TABLE_DDL, 0x1429) PCODE_DEF(OB_NOTIFY_ARCHIVE, 0x142A) // backup and restore end 0x14ff diff --git a/src/objit/include/objit/common/ob_item_type.h b/src/objit/include/objit/common/ob_item_type.h index 9fdf6be737..d3566eded9 100755 --- a/src/objit/include/objit/common/ob_item_type.h +++ b/src/objit/include/objit/common/ob_item_type.h @@ -2029,17 +2029,23 @@ typedef enum ObItemType T_OBCONFIG_URL, T_ARCHIVE_LOG, T_BACKUP_DATABASE, + T_RECOVER_TABLE, + T_REMAP_TABLE, + T_REMAP_TABLEGROUP, + T_REMAP_TABLESPACE, T_BACKUP_MANAGE, T_BACKUP_CLEAN, T_DELETE_POLICY, T_BACKUP_KEY, T_RESTORE_TENANT_2, T_CANCEL_RESTORE, + T_CANCEL_RECOVER_TABLE, T_GEN_ROWS, T_LOAD_BATCH_SIZE, T_DIRECT, // direct load data T_REMOTE_OSS, T_PHYSICAL_RESTORE_TENANT, + T_RECOVER_TABLE_REMAP, T_PHYSICAL_RESTORE_UNTIL, T_REVOKE_SYSAUTH, diff --git a/src/observer/CMakeLists.txt b/src/observer/CMakeLists.txt index 87a2a0857c..2e41316f34 100644 --- a/src/observer/CMakeLists.txt +++ b/src/observer/CMakeLists.txt @@ -360,6 +360,7 @@ ob_set_subtarget(ob_server virtual_table virtual_table/ob_table_columns.cpp virtual_table/ob_table_index.cpp virtual_table/ob_tenant_all_tables.cpp + virtual_table/ob_tenant_show_restore_preview.cpp virtual_table/ob_tenant_show_tables.cpp virtual_table/ob_tenant_virtual_charset.cpp virtual_table/ob_tenant_virtual_privilege.cpp diff --git a/src/observer/ob_service.cpp b/src/observer/ob_service.cpp index da48b8a0de..ed31aab592 100644 --- a/src/observer/ob_service.cpp +++ b/src/observer/ob_service.cpp @@ -2358,42 +2358,41 @@ int ObService::build_ddl_single_replica_request(const ObDDLBuildSingleReplicaReq } else { if (DDL_DROP_COLUMN == arg.ddl_type_ || DDL_ADD_COLUMN_OFFLINE == arg.ddl_type_ - || DDL_COLUMN_REDEFINITION == arg.ddl_type_) { - MTL_SWITCH(arg.tenant_id_) { - int saved_ret = OB_SUCCESS; - ObTenantDagScheduler *dag_scheduler = nullptr; - ObComplementDataDag *dag = nullptr; - if (OB_ISNULL(dag_scheduler = MTL(ObTenantDagScheduler *))) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("dag scheduler is null", K(ret)); - } else if (OB_FAIL(dag_scheduler->alloc_dag(dag))) { - LOG_WARN("fail to alloc dag", K(ret)); - } else if (OB_ISNULL(dag)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected error, dag is null", K(ret), KP(dag)); - } else if (OB_FAIL(dag->init(arg))) { - LOG_WARN("fail to init complement data dag", K(ret), K(arg)); - } else if (OB_FAIL(dag->create_first_task())) { - LOG_WARN("create first task failed", K(ret)); - } else if (OB_FAIL(dag_scheduler->add_dag(dag))) { - saved_ret = ret; - LOG_WARN("add dag failed", K(ret), K(arg)); - if (OB_EAGAIN == saved_ret) { - dag_scheduler->get_complement_data_dag_progress(dag, res.row_scanned_, res.row_inserted_); - } - } else { - dag = nullptr; - } - if (OB_NOT_NULL(dag)) { - (void) dag->handle_init_failed_ret_code(ret); - dag_scheduler->free_dag(*dag); - dag = nullptr; - } - if (OB_FAIL(ret)) { - // RS does not retry send RPC to tablet leader when the dag exists. - ret = OB_EAGAIN == saved_ret ? OB_SUCCESS : ret; - ret = OB_SIZE_OVERFLOW == saved_ret ? OB_EAGAIN : ret; + || DDL_COLUMN_REDEFINITION == arg.ddl_type_ + || DDL_TABLE_RESTORE == arg.ddl_type_) { + int saved_ret = OB_SUCCESS; + ObTenantDagScheduler *dag_scheduler = nullptr; + ObComplementDataDag *dag = nullptr; + if (OB_ISNULL(dag_scheduler = MTL(ObTenantDagScheduler *))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dag scheduler is null", K(ret)); + } else if (OB_FAIL(dag_scheduler->alloc_dag(dag))) { + LOG_WARN("fail to alloc dag", K(ret)); + } else if (OB_ISNULL(dag)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error, dag is null", K(ret), KP(dag)); + } else if (OB_FAIL(dag->init(arg))) { + LOG_WARN("fail to init complement data dag", K(ret), K(arg)); + } else if (OB_FAIL(dag->create_first_task())) { + LOG_WARN("create first task failed", K(ret)); + } else if (OB_FAIL(dag_scheduler->add_dag(dag))) { + saved_ret = ret; + LOG_WARN("add dag failed", K(ret), K(arg)); + if (OB_EAGAIN == saved_ret) { + dag_scheduler->get_complement_data_dag_progress(dag, res.row_scanned_, res.row_inserted_); } + } else { + dag = nullptr; + } + if (OB_NOT_NULL(dag)) { + (void) dag->handle_init_failed_ret_code(ret); + dag_scheduler->free_dag(*dag); + dag = nullptr; + } + if (OB_FAIL(ret)) { + // RS does not retry send RPC to tablet leader when the dag exists. + ret = OB_EAGAIN == saved_ret ? OB_SUCCESS : ret; + ret = OB_SIZE_OVERFLOW == saved_ret ? OB_EAGAIN : ret; } LOG_INFO("obs get rpc to build drop column dag", K(ret)); } else { diff --git a/src/observer/ob_srv_xlator_rootserver.cpp b/src/observer/ob_srv_xlator_rootserver.cpp index 28c6359481..ce427bd077 100644 --- a/src/observer/ob_srv_xlator_rootserver.cpp +++ b/src/observer/ob_srv_xlator_rootserver.cpp @@ -96,6 +96,7 @@ void oceanbase::observer::init_srv_xlator_for_rootserver(ObSrvRpcXlator *xlator) RPC_PROCESSOR(rootserver::ObRpcDropTablegroupP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObRpcAlterTablegroupP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObRpcCreateTableP, *gctx_.root_service_); + RPC_PROCESSOR(rootserver::ObRpcRecoverRestoreTableDDLP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObRpcParallelCreateTableP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObRpcStartRedefTableP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObRpcCopyTableDependentsP, *gctx_.root_service_); @@ -248,6 +249,7 @@ void oceanbase::observer::init_srv_xlator_for_rootserver(ObSrvRpcXlator *xlator) RPC_PROCESSOR(rootserver::ObBackupManageP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObBackupCleanP, *gctx_.root_service_); RPC_PROCESSOR(rootserver::ObDeletePolicyP, *gctx_.root_service_); + RPC_PROCESSOR(rootserver::ObRecoverTableP, *gctx_.root_service_); // auto part ddl diff --git a/src/observer/omt/ob_multi_tenant.cpp b/src/observer/omt/ob_multi_tenant.cpp index e90f600d0c..ea3032e9a5 100644 --- a/src/observer/omt/ob_multi_tenant.cpp +++ b/src/observer/omt/ob_multi_tenant.cpp @@ -100,7 +100,7 @@ #include "rootserver/ob_primary_ls_service.h"//ObLSService #include "rootserver/ob_recovery_ls_service.h"//ObRecoveryLSService #include "rootserver/ob_common_ls_service.h"//ObCommonLSService -#include "rootserver/restore/ob_restore_scheduler.h" //ObRestoreService +#include "rootserver/restore/ob_restore_service.h" //ObRestoreService #include "rootserver/ob_tenant_transfer_service.h" // ObTenantTransferService #include "rootserver/ob_balance_task_execute_service.h" //ObBalanceTaskExecuteService #include "rootserver/backup/ob_backup_service.h" //ObBackupDataService and ObBackupCleanService diff --git a/src/observer/table_load/ob_table_load_redef_table.cpp b/src/observer/table_load/ob_table_load_redef_table.cpp index 1048163a54..e2dc8fbddb 100644 --- a/src/observer/table_load/ob_table_load_redef_table.cpp +++ b/src/observer/table_load/ob_table_load_redef_table.cpp @@ -42,8 +42,8 @@ int ObTableLoadRedefTable::start(const ObTableLoadRedefTableStartArg &arg, create_table_arg.reset(); create_table_arg.exec_tenant_id_ = arg.tenant_id_; create_table_arg.tenant_id_ = arg.tenant_id_; - create_table_arg.table_id_ = arg.table_id_; create_table_arg.dest_tenant_id_ = arg.tenant_id_; + create_table_arg.table_id_ = arg.table_id_; create_table_arg.parallelism_ = arg.parallelism_; create_table_arg.ddl_type_ = arg.is_load_data_ ? share::DDL_DIRECT_LOAD : share::DDL_DIRECT_LOAD_INSERT; create_table_arg.session_id_ = session_info.get_sessid_for_table(); @@ -97,16 +97,19 @@ int ObTableLoadRedefTable::finish(const ObTableLoadRedefTableFinishArg &arg, finish_redef_table_arg.tenant_id_ = arg.tenant_id_; ObDDLBuildSingleReplicaResponseArg build_single_replica_response_arg; - build_single_replica_response_arg.task_id_ = arg.task_id_; - build_single_replica_response_arg.tenant_id_ = arg.tenant_id_; - build_single_replica_response_arg.ls_id_ = share::ObLSID(1); - build_single_replica_response_arg.tablet_id_ = ObTableID(-1); - build_single_replica_response_arg.source_table_id_ = arg.table_id_; - build_single_replica_response_arg.dest_schema_id_ = arg.dest_table_id_; - build_single_replica_response_arg.ret_code_ = ret; - build_single_replica_response_arg.snapshot_version_ = 1; - build_single_replica_response_arg.schema_version_ = arg.schema_version_; - build_single_replica_response_arg.execution_id_ = 1; + build_single_replica_response_arg.task_id_ = arg.task_id_; + build_single_replica_response_arg.tenant_id_ = arg.tenant_id_; + build_single_replica_response_arg.dest_tenant_id_ = arg.tenant_id_; + build_single_replica_response_arg.source_table_id_ = arg.table_id_; + build_single_replica_response_arg.dest_schema_id_ = arg.dest_table_id_; + build_single_replica_response_arg.schema_version_ = arg.schema_version_; + build_single_replica_response_arg.dest_schema_version_ = arg.schema_version_; + build_single_replica_response_arg.ls_id_ = share::ObLSID(1); + build_single_replica_response_arg.dest_ls_id_ = share::ObLSID(1); + build_single_replica_response_arg.tablet_id_ = ObTableID(-1); + build_single_replica_response_arg.snapshot_version_ = 1; + build_single_replica_response_arg.execution_id_ = 1; + build_single_replica_response_arg.ret_code_ = ret; if (OB_FAIL(ObDDLServerClient::finish_redef_table( finish_redef_table_arg, build_single_replica_response_arg, session_info))) { LOG_WARN("failed to finish redef table", KR(ret), K(finish_redef_table_arg)); @@ -131,7 +134,7 @@ int ObTableLoadRedefTable::abort(const ObTableLoadRedefTableAbortArg &arg, ObAbortRedefTableArg abort_redef_table_arg; abort_redef_table_arg.task_id_ = arg.task_id_; abort_redef_table_arg.tenant_id_ = arg.tenant_id_; - if (OB_FAIL(ObDDLServerClient::abort_redef_table(abort_redef_table_arg, session_info))) { + if (OB_FAIL(ObDDLServerClient::abort_redef_table(abort_redef_table_arg, &session_info))) { LOG_WARN("failed to abort redef table", KR(ret), K(abort_redef_table_arg)); } else { LOG_INFO("succeed to abort hidden table", K(arg)); diff --git a/src/observer/virtual_table/ob_tenant_show_restore_preview.cpp b/src/observer/virtual_table/ob_tenant_show_restore_preview.cpp new file mode 100644 index 0000000000..4037f586e0 --- /dev/null +++ b/src/observer/virtual_table/ob_tenant_show_restore_preview.cpp @@ -0,0 +1,297 @@ +/** + * Copyright (c) 2023 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. + */ + +#include "observer/virtual_table/ob_tenant_show_restore_preview.h" +#include "lib/string/ob_fixed_length_string.h" +#include "sql/session/ob_sql_session_info.h" +#include "rootserver/restore/ob_restore_util.h" +#include "share/restore/ob_restore_uri_parser.h" + +#include + +using namespace oceanbase::common; +using namespace oceanbase::share; + +namespace oceanbase +{ +namespace observer +{ + +ObTenantShowRestorePreview::ObTenantShowRestorePreview() + : ObVirtualTableIterator(), + is_inited_(false), + idx_(-1), + total_cnt_(0), + uri_(), + restore_scn_(), + only_contain_backup_set_(false), + backup_set_list_(), + backup_piece_list_(), + log_path_list_(), + allocator_() +{ +} + +ObTenantShowRestorePreview::~ObTenantShowRestorePreview() +{ +} + +void ObTenantShowRestorePreview::reset() +{ + ObVirtualTableIterator::reset(); + backup_set_list_.reset(); + backup_piece_list_.reset(); + log_path_list_.reset(); + uri_.reset(); + idx_ = -1; + total_cnt_ = 0; +} + +int ObTenantShowRestorePreview::init() +{ + int ret = OB_SUCCESS; + ObObj backup_dest_value; + ObObj passwd; + ObArenaAllocator allocator; + ObArray tenant_path_array; + ObString backup_passwd(0, ""); + if (OB_ISNULL(session_)) { + ret = OB_BAD_NULL_ERROR; + SHARE_LOG(WARN, "session should not be null", KR(ret)); + } else if (!session_->user_variable_exists(OB_RESTORE_PREVIEW_BACKUP_DEST_SESSION_STR) + || !session_->user_variable_exists(OB_RESTORE_PREVIEW_SCN_SESSION_STR)) { + ret = OB_NOT_SUPPORTED; + SHARE_LOG(WARN, "no restore preview backup dest specified before", KR(ret)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "show restore preview do not specify backup dest"); + } else if (OB_FAIL(session_->get_user_variable_value(OB_RESTORE_PREVIEW_BACKUP_DEST_SESSION_STR, backup_dest_value))) { + SHARE_LOG(WARN, "failed to get user variable value", KR(ret)); + } else if (OB_FAIL(backup_dest_value.get_varchar(uri_))) { + SHARE_LOG(WARN, "failed to varchar", KR(ret), K(backup_dest_value)); + } else if (OB_FAIL(ObPhysicalRestoreUriParser::parse(uri_, allocator, tenant_path_array))) { + SHARE_LOG(WARN, "fail to parse uri", K(ret), K(uri_)); + } else if (OB_FAIL(rootserver::ObRestoreUtil::check_restore_using_complement_log_(tenant_path_array, only_contain_backup_set_))) { + SHARE_LOG(WARN, "check restore using complement log failed", K(ret), K(tenant_path_array)); + } else if (!session_->user_variable_exists(OB_BACKUP_DECRYPTION_PASSWD_ARRAY_SESSION_STR)) { + } else if (OB_FAIL(session_->get_user_variable_value(OB_BACKUP_DECRYPTION_PASSWD_ARRAY_SESSION_STR, passwd))) { + SHARE_LOG(WARN, "failed to get user passwd", K(ret)); + } else if (OB_FAIL(passwd.get_varchar(backup_passwd))) { + SHARE_LOG(WARN, "failed to parser passwd", K(ret)); + } + + if (OB_FAIL(ret)) { + } else if (OB_FAIL(parse_restore_scn_from_session_(backup_passwd, tenant_path_array))) { + SHARE_LOG(WARN, "failed to parse restore timestamp from session", KR(ret)); + } else if (OB_FAIL(rootserver::ObRestoreUtil::get_restore_source(only_contain_backup_set_, + tenant_path_array, backup_passwd, restore_scn_, backup_set_list_, backup_piece_list_, log_path_list_))) { + SHARE_LOG(WARN, "failed to get restore source", K(ret), K(tenant_path_array), K(restore_scn_), K(backup_passwd)); + } else { + idx_ = 0; + total_cnt_ = backup_set_list_.count() + backup_piece_list_.count(); + is_inited_ = true; + } + return ret; +} + +int ObTenantShowRestorePreview::parse_restore_scn_from_session_( + const ObString &backup_passwd, ObIArray &tenant_path_array) +{ + int ret = OB_SUCCESS; + uint64_t restore_scn = 0; + ObObj restore_scn_obj; + ObObj restore_timestamp_obj; + ObString restore_scn_str; + ObString restore_timestamp_str; + ObFixedLengthString fixed_string; + if (OB_ISNULL(session_)) { + ret = OB_BAD_NULL_ERROR; + SHARE_LOG(WARN, "session should not be null", KR(ret)); + } else if (OB_FAIL(session_->get_user_variable_value( + OB_RESTORE_PREVIEW_SCN_SESSION_STR, restore_scn_obj))) { + SHARE_LOG(WARN, "failed to get user variable value", KR(ret)); + } else if (OB_FAIL(session_->get_user_variable_value( + OB_RESTORE_PREVIEW_TIMESTAMP_SESSION_STR, restore_timestamp_obj))) { + SHARE_LOG(WARN, "failed to get user variable value", KR(ret)); + } else if (OB_FAIL(restore_scn_obj.get_varchar(restore_scn_str))) { + SHARE_LOG(WARN, "failed to varchar", KR(ret), K(restore_scn_obj)); + } else if (OB_FAIL(restore_timestamp_obj.get_varchar(restore_timestamp_str))) { + SHARE_LOG(WARN, "failed to varchar", KR(ret), K(restore_scn_obj)); + } else if (OB_FAIL(fixed_string.assign(restore_scn_str))) { + SHARE_LOG(WARN, "failed to assign tenant id str", KR(ret), K(restore_scn_str)); + } else if (1 != sscanf(fixed_string.ptr(), "%lu", &restore_scn)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "failed to get uint64_t from value", KR(ret), K(restore_scn)); + } else if (restore_scn != 0) { + if (OB_FAIL(restore_scn_.convert_for_inner_table_field(restore_scn))) { + SHARE_LOG(WARN, "failed to convert for inner table field", K(ret), K(restore_scn)); + } + } else if (OB_FAIL(rootserver::ObRestoreUtil::fill_restore_scn_( + restore_scn_, restore_timestamp_str, false, tenant_path_array, backup_passwd, only_contain_backup_set_, restore_scn_))) { + SHARE_LOG(WARN, "failed to parse restore scn", K(ret)); + } + return ret; +} + +int ObTenantShowRestorePreview::inner_get_next_row(common::ObNewRow *&row) +{ + int ret = OB_SUCCESS; + if (!is_sys_tenant(effective_tenant_id_)) { + ret = OB_OP_NOT_ALLOW; + SHARE_LOG(WARN, "show restore preview is sys only", K(ret), K(effective_tenant_id_)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "SHOW RESTORE PREVIEW in user tenant"); + } else if (OB_FAIL(inner_get_next_row_())) { + if (OB_ITER_END != ret) { + SERVER_LOG(WARN, "failed to get next row", KR(ret), K(cur_row_)); + } + } else { + row = &cur_row_; + } + return ret; +} + +int ObTenantShowRestorePreview::inner_get_next_row_() +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + SHARE_LOG(WARN, "not init", KR(ret)); + } else if (idx_ == total_cnt_) { + ret = OB_ITER_END; + SHARE_LOG(INFO, "iterator end", KR(ret), K(idx_), K(total_cnt_)); + } else { + const int64_t col_count = output_column_ids_.count(); + for (int64_t i = 0; OB_SUCC(ret) && i < col_count; ++i) { + const uint64_t col_id = output_column_ids_.at(i); + switch (col_id) { + case BACKUP_TYPE: { + BackupType type; + if (OB_FAIL(get_backup_type_(type))) { + SHARE_LOG(WARN, "failed to get backup id", KR(ret)); + } else { + if (type == BACKUP_TYPE_PIECE) { + cur_row_.cells_[i].set_varchar("BACKUP_PIECE"); + } else if (type == BACKUP_TYPE_SET) { + cur_row_.cells_[i].set_varchar("BACKUP_SET"); + } + cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation( + ObCharset::get_default_charset())); + } + break; + } + case BACKUP_ID: { + int64_t id = 0; + if (OB_FAIL(get_backup_id_(id))) { + SHARE_LOG(WARN, "failed to get backup id", KR(ret)); + } else { + cur_row_.cells_[i].set_int(id); + } + break; + } + case PREVIEW_PATH: { + common::ObString str; + if (OB_FAIL(get_backup_path_(str))) { + SHARE_LOG(WARN, "failed to get backup path", KR(ret)); + } else { + cur_row_.cells_[i].set_varchar(str); + cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation( + ObCharset::get_default_charset())); + } + break; + } + case BACKUP_DESC: { + common::ObString str; + if (OB_FAIL(get_backup_desc_(str))) { + SHARE_LOG(WARN, "failed to get backup desc", KR(ret)); + } else { + cur_row_.cells_[i].set_varchar(str); + cur_row_.cells_[i].set_collation_type(ObCharset::get_default_collation( + ObCharset::get_default_charset())); + } + break; + } + } + } + if (OB_SUCC(ret)) { + ++idx_; + } + } + return ret; +} + +int ObTenantShowRestorePreview::get_backup_id_(int64_t &backup_id) +{ + int ret = OB_SUCCESS; + backup_id = -1; + if (idx_ < 0) { + ret = OB_ERR_UNEXPECTED; + SHARE_LOG(WARN, "idx should be less than 0", KR(ret), K(idx_)); + } else if (idx_ <= backup_set_list_.count() - 1) { + backup_id = backup_set_list_.at(idx_).backup_set_desc_.backup_set_id_; + } else if (idx_ >= backup_set_list_.count() && idx_ <= total_cnt_ - 1) { + backup_id = backup_piece_list_.at(idx_ - backup_set_list_.count()).piece_id_; + } + return ret; +} + +int ObTenantShowRestorePreview::get_backup_type_(BackupType &type) +{ + int ret = OB_SUCCESS; + type = BACKUP_TYPE_MAX; + if (idx_ < 0) { + ret = OB_ERR_UNEXPECTED; + SHARE_LOG(WARN, "idx should be less than 0", KR(ret), K(idx_)); + } else if (idx_ <= backup_set_list_.count() - 1) { + type = BACKUP_TYPE_SET; + } else if (idx_ >= backup_set_list_.count() && idx_ <= total_cnt_ - 1) { + type = BACKUP_TYPE_PIECE; + } else { + type = BACKUP_TYPE_MAX; + } + return ret; +} + +int ObTenantShowRestorePreview::get_backup_path_(common::ObString &str) +{ + int ret = OB_SUCCESS; + str.reset(); + if (idx_ < 0) { + ret = OB_ERR_UNEXPECTED; + SHARE_LOG(WARN, "idx should be less than 0", KR(ret), K(idx_)); + } else if (idx_ <= backup_set_list_.count() - 1) { + str = backup_set_list_.at(idx_).backup_set_path_.str(); + } else if (idx_ >= backup_set_list_.count() && idx_ <= total_cnt_ - 1) { + str = backup_piece_list_.at(idx_ - backup_set_list_.count()).piece_path_.str(); + } + return ret; +} + +int ObTenantShowRestorePreview::get_backup_desc_(common::ObString &str) +{ + int ret = OB_SUCCESS; + str.reset(); + if (idx_ < 0) { + ret = OB_ERR_UNEXPECTED; + SHARE_LOG(WARN, "idx should be less than 0", KR(ret), K(idx_)); + } else if (idx_ <= backup_set_list_.count() - 1) { + if (OB_FAIL(backup_set_list_.at(idx_).get_restore_backup_set_brief_info_str(allocator_, str))) { + SHARE_LOG(WARN, "failed to get restore backup set brief info str", K(ret)); + } + } else if (idx_ >= backup_set_list_.count() && idx_ <= total_cnt_ - 1) { + if (OB_FAIL(backup_piece_list_.at(idx_ - backup_set_list_.count()).get_restore_log_piece_brief_info_str(allocator_, str))) { + SHARE_LOG(WARN, "failed to get restore backup set brief info str", K(ret)); + } + } + return ret; +} + + +} // end namespace observer +} // end namespace oceanbase \ No newline at end of file diff --git a/src/observer/virtual_table/ob_tenant_show_restore_preview.h b/src/observer/virtual_table/ob_tenant_show_restore_preview.h new file mode 100644 index 0000000000..14c41b947d --- /dev/null +++ b/src/observer/virtual_table/ob_tenant_show_restore_preview.h @@ -0,0 +1,76 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_OBSERVER_VIRTUAL_TABLE_OB_TENANT_SHOW_RESTORE_PREVIEW_H_ +#define OCEANBASE_OBSERVER_VIRTUAL_TABLE_OB_TENANT_SHOW_RESTORE_PREVIEW_H_ + +#include "share/ob_virtual_table_iterator.h" +#include "share/backup/ob_backup_struct.h" +#include "lib/container/ob_array.h" +#include "share/scn.h" + +namespace oceanbase +{ +namespace observer +{ + +class ObTenantShowRestorePreview : public common::ObVirtualTableIterator +{ + enum BackupType + { + BACKUP_TYPE_SET = 0, + BACKUP_TYPE_PIECE = 1, + BACKUP_TYPE_MAX = 2, + }; + + enum RestorePreviewColumn + { + BACKUP_TYPE = common::OB_APP_MIN_COLUMN_ID, + BACKUP_ID = common::OB_APP_MIN_COLUMN_ID + 1, + PREVIEW_PATH = common::OB_APP_MIN_COLUMN_ID + 2, + BACKUP_DESC = common::OB_APP_MIN_COLUMN_ID + 3, + }; + + static const int64_t MAX_INT64_STR_LENGTH = 100; +public: + ObTenantShowRestorePreview(); + virtual ~ObTenantShowRestorePreview(); + int init(); + virtual int inner_get_next_row(common::ObNewRow *&row); + virtual void reset(); +private: + int parse_restore_scn_from_session_(const ObString &backup_passwd, ObIArray &tenant_path_array); + int inner_get_next_row_(); + int get_backup_type_(BackupType &type); + int get_backup_id_(int64_t &backup_id); + int get_backup_path_(common::ObString &str); + int get_backup_desc_(common::ObString &str); +private: + bool is_inited_; + int64_t idx_; + int64_t total_cnt_; + ObString uri_; + share::SCN restore_scn_; + bool only_contain_backup_set_; + ObArray backup_set_list_; + ObArray backup_piece_list_; + ObArray log_path_list_; + common::ObArenaAllocator allocator_; +private: + DISALLOW_COPY_AND_ASSIGN(ObTenantShowRestorePreview); + +}; + +} // end namespace observer +} // end namespace oceanbase + +#endif \ No newline at end of file diff --git a/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp b/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp index 3c23714d36..4671772d23 100644 --- a/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp +++ b/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp @@ -1143,6 +1143,18 @@ int ObVTIterCreator::create_vt_iter(ObVTableScanParam ¶ms, } break; } + case OB_TENANT_VIRTUAL_SHOW_RESTORE_PREVIEW_TID: + { + ObTenantShowRestorePreview *restore_preview = NULL; + if (OB_SUCC(NEW_VIRTUAL_TABLE(ObTenantShowRestorePreview, restore_preview))) { + if (OB_FAIL(restore_preview->init())) { + SERVER_LOG(WARN, "failed to init restore preview", K(ret)); + } else { + vt_iter = static_cast(restore_preview); + } + } + break; + } case OB_TENANT_VIRTUAL_OBJECT_DEFINITION_TID: { ObGetObjectDefinition *get_object_def = NULL; diff --git a/src/rootserver/CMakeLists.txt b/src/rootserver/CMakeLists.txt index b1d4b3b2c9..b8beb598e9 100644 --- a/src/rootserver/CMakeLists.txt +++ b/src/rootserver/CMakeLists.txt @@ -120,6 +120,7 @@ ob_set_subtarget(ob_rootserver ddl_task ddl_task/ob_index_build_task.cpp ddl_task/ob_modify_autoinc_task.cpp ddl_task/ob_table_redefinition_task.cpp + ddl_task/ob_recover_restore_table_task.cpp ) ob_set_subtarget(ob_rootserver parallel_ddl @@ -148,6 +149,11 @@ ob_set_subtarget(ob_rootserver freeze ) ob_set_subtarget(ob_rootserver restore + restore/ob_import_table_job_scheduler.cpp + restore/ob_recover_table_initiator.cpp + restore/ob_import_table_task_generator.cpp + restore/ob_recover_table_job_scheduler.cpp + restore/ob_restore_service.cpp restore/ob_restore_scheduler.cpp restore/ob_restore_util.cpp ) diff --git a/src/rootserver/ddl_task/ob_column_redefinition_task.cpp b/src/rootserver/ddl_task/ob_column_redefinition_task.cpp index cc5d5d5393..7b06e276e8 100644 --- a/src/rootserver/ddl_task/ob_column_redefinition_task.cpp +++ b/src/rootserver/ddl_task/ob_column_redefinition_task.cpp @@ -29,8 +29,8 @@ using namespace oceanbase::share::schema; using namespace oceanbase::rootserver; ObColumnRedefinitionTask::ObColumnRedefinitionTask() - : ObDDLRedefinitionTask(ObDDLType::DDL_COLUMN_REDEFINITION), sstable_complete_request_time_(0), has_rebuild_index_(false), has_rebuild_constraint_(false), has_rebuild_foreign_key_(false), - is_sstable_complete_task_submitted_(false), allocator_(lib::ObLabel("RedefTask")) + : ObDDLRedefinitionTask(ObDDLType::DDL_COLUMN_REDEFINITION), has_rebuild_index_(false), has_rebuild_constraint_(false), has_rebuild_foreign_key_(false), + allocator_(lib::ObLabel("RedefTask")) { } @@ -76,6 +76,11 @@ int ObColumnRedefinitionTask::init(const uint64_t tenant_id, const int64_t task_ if (OB_FAIL(init_ddl_task_monitor_info(target_object_id_))) { LOG_WARN("init ddl task monitor info failed", K(ret)); } else { + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; + alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); + alter_table_arg_.alter_table_schema_.set_schema_version(schema_version_); + alter_table_arg_.exec_tenant_id_ = dst_tenant_id_; data_format_version_ = tenant_data_format_version; is_inited_ = true; ddl_tracing_.open(); @@ -90,6 +95,7 @@ int ObColumnRedefinitionTask::init(const ObDDLTaskRecord &task_record) const uint64_t data_table_id = task_record.object_id_; const uint64_t dest_table_id = task_record.target_object_id_; const int64_t schema_version = task_record.schema_version_; + task_type_ = task_record.ddl_type_; int64_t pos = 0; const ObTableSchema *data_schema = nullptr; ObSchemaGetterGuard schema_guard; @@ -115,7 +121,6 @@ int ObColumnRedefinitionTask::init(const ObDDLTaskRecord &task_record) LOG_WARN("fail to get table schema", K(ret), K(data_schema)); } else { task_id_ = task_record.task_id_; - task_type_ = task_record.ddl_type_; object_id_ = data_table_id; target_object_id_ = dest_table_id; schema_version_ = schema_version; @@ -125,6 +130,11 @@ int ObColumnRedefinitionTask::init(const ObDDLTaskRecord &task_record) tenant_id_ = task_record.tenant_id_; ret_code_ = task_record.ret_code_; start_time_ = ObTimeUtility::current_time(); + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; + alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); + alter_table_arg_.alter_table_schema_.set_schema_version(schema_version_); + alter_table_arg_.exec_tenant_id_ = dst_tenant_id_; if (OB_FAIL(init_ddl_task_monitor_info(target_object_id_))) { LOG_WARN("init ddl task monitor info failed", K(ret)); } else { @@ -159,7 +169,7 @@ int ObColumnRedefinitionTask::wait_data_complement(const ObDDLTaskStatus next_ta DEBUG_SYNC(COLUMN_REDEFINITION_REPLICA_BUILD); if (is_build_replica_end) { ret = complete_sstable_job_ret_code_; - if (OB_SUCC(ret) && OB_FAIL(check_data_dest_tables_columns_checksum(1))) { + if (OB_SUCC(ret) && OB_FAIL(check_data_dest_tables_columns_checksum(get_execution_id()))) { LOG_WARN("fail to check the columns checkum between data table and hidden one", K(ret)); } if (OB_FAIL(switch_status(next_task_status, true, ret))) { @@ -170,61 +180,6 @@ int ObColumnRedefinitionTask::wait_data_complement(const ObDDLTaskStatus next_ta return ret; } -// send the request of complementing data to each primary server through rpc -int ObColumnRedefinitionTask::send_build_single_replica_request() -{ - int ret = OB_SUCCESS; - if (OB_UNLIKELY(!is_inited_)) { - ret = OB_NOT_INIT; - LOG_WARN("ObColumnRedefinitionTask has not been inited", K(ret)); - } else { - ObDDLSingleReplicaExecutorParam param; - param.tenant_id_ = tenant_id_; - param.type_ = task_type_; - param.source_table_id_ = object_id_; - param.dest_table_id_ = target_object_id_; - param.schema_version_ = schema_version_; - param.snapshot_version_ = snapshot_version_; - param.task_id_ = task_id_; - param.parallelism_ = alter_table_arg_.parallelism_; - param.execution_id_ = execution_id_; - param.data_format_version_ = data_format_version_; - param.consumer_group_id_ = alter_table_arg_.consumer_group_id_; - if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, object_id_, param.source_tablet_ids_))) { - LOG_WARN("fail to get tablets", K(ret), K(tenant_id_), K(object_id_)); - } else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, target_object_id_, param.dest_tablet_ids_))) { - LOG_WARN("fail to get tablets", K(ret), K(tenant_id_), K(target_object_id_)); - } else if (OB_FAIL(replica_builder_.build(param))) { - LOG_WARN("fail to send build single replica", K(ret)); - } else { - LOG_INFO("start to build single replica", K(target_object_id_)); - is_sstable_complete_task_submitted_ = true; - sstable_complete_request_time_ = ObTimeUtility::current_time(); - } - } - return ret; -} - -// check whether all leaders have completed the complement task -int ObColumnRedefinitionTask::check_build_single_replica(bool &is_end) -{ - int ret = OB_SUCCESS; - is_end = false; - TCRLockGuard guard(lock_); - if (OB_UNLIKELY(!is_inited_)) { - ret = OB_NOT_INIT; - LOG_WARN("ObColumnRedefinitionTask has not been inited", K(ret)); - } else if (OB_FAIL(replica_builder_.check_build_end(is_end, complete_sstable_job_ret_code_))) { - LOG_WARN("fail to check build end", K(ret)); - } else if (!is_end) { - if (sstable_complete_request_time_ + OB_MAX_DDL_SINGLE_REPLICA_BUILD_TIMEOUT < ObTimeUtility::current_time()) { // timeout, retry - is_sstable_complete_task_submitted_ = false; - sstable_complete_request_time_ = 0; - } - } - return ret; -} - // update sstable complement status for all leaders int ObColumnRedefinitionTask::update_complete_sstable_job_status(const common::ObTabletID &tablet_id, const int64_t snapshot_version, @@ -252,7 +207,6 @@ int ObColumnRedefinitionTask::update_complete_sstable_job_status(const common::O return ret; } - // Now, rebuild index table in schema and tablet. // Next, we only rebuild index in schema and remap new index schema to old tablet by sending RPC(REMAP_INDEXES_AND_TAKE_EFFECT_TASK) to RS. int ObColumnRedefinitionTask::copy_table_indexes() @@ -281,7 +235,6 @@ int ObColumnRedefinitionTask::copy_table_indexes() alter_table_arg_.ddl_task_type_ = share::REBUILD_INDEX_TASK; alter_table_arg_.table_id_ = object_id_; alter_table_arg_.hidden_table_id_ = target_object_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); if (OB_FAIL(root_service->get_ddl_service().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(schema_guard.get_table_schema(tenant_id_, target_object_id_, table_schema))) { @@ -427,7 +380,6 @@ int ObColumnRedefinitionTask::copy_table_constraints() alter_table_arg_.ddl_task_type_ = share::REBUILD_CONSTRAINT_TASK; alter_table_arg_.table_id_ = object_id_; alter_table_arg_.hidden_table_id_ = target_object_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(tenant_id_, target_object_id_, rpc_timeout))) { LOG_WARN("get ddl rpc timeout failed", K(ret)); } else if (OB_FAIL(root_service->get_ddl_service().get_common_rpc()->to(obrpc::ObRpcProxy::myaddr_).timeout(rpc_timeout). @@ -475,7 +427,6 @@ int ObColumnRedefinitionTask::copy_table_foreign_keys() alter_table_arg_.ddl_task_type_ = share::REBUILD_FOREIGN_KEY_TASK; alter_table_arg_.table_id_ = object_id_; alter_table_arg_.hidden_table_id_ = target_object_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(tenant_id_, target_object_id_, rpc_timeout))) { LOG_WARN("get ddl rpc timeout failed", K(ret)); } else if (OB_FAIL(root_service->get_ddl_service().get_common_rpc()->to(obrpc::ObRpcProxy::myaddr_).timeout(rpc_timeout). @@ -519,7 +470,7 @@ int ObColumnRedefinitionTask::deserlize_params_from_message(const uint64_t tenan LOG_WARN("ObDDLTask deserlize failed", K(ret)); } else if (OB_FAIL(tmp_arg.deserialize(buf, data_len, pos))) { LOG_WARN("serialize table failed", K(ret)); - } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(tenant_id, tmp_arg))) { + } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(task_type_, tenant_id, tmp_arg))) { LOG_WARN("replace user tenant id failed", K(ret), K(tenant_id), K(tmp_arg)); } else if (OB_FAIL(deep_copy_table_arg(allocator_, tmp_arg, alter_table_arg_))) { LOG_WARN("deep copy table arg failed", K(ret)); @@ -538,7 +489,6 @@ int ObColumnRedefinitionTask::copy_table_dependent_objects(const ObDDLTaskStatus ObRootService *root_service = GCTX.root_service_; int64_t finished_task_cnt = 0; bool state_finish = false; - ObSchemaGetterGuard schema_guard; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("ObColumnRedefinitionTask has not been inited", K(ret)); @@ -619,7 +569,6 @@ int ObColumnRedefinitionTask::take_effect(const ObDDLTaskStatus next_task_status // offline ddl is allowed on table with trigger(enable/disable). alter_table_arg_.need_rebuild_trigger_ = true; alter_table_arg_.task_id_ = task_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); ObRootService *root_service = GCTX.root_service_; ObSchemaGetterGuard schema_guard; const ObTableSchema *table_schema = nullptr; diff --git a/src/rootserver/ddl_task/ob_column_redefinition_task.h b/src/rootserver/ddl_task/ob_column_redefinition_task.h index fd0c553a1e..15cc3e520b 100644 --- a/src/rootserver/ddl_task/ob_column_redefinition_task.h +++ b/src/rootserver/ddl_task/ob_column_redefinition_task.h @@ -56,8 +56,6 @@ public: K(has_rebuild_index_), K(has_rebuild_constraint_), K(has_rebuild_foreign_key_), K(is_sstable_complete_task_submitted_)); private: int wait_data_complement(const share::ObDDLTaskStatus next_task_status); - int send_build_single_replica_request(); - int check_build_single_replica(bool &is_end); int copy_table_dependent_objects(const share::ObDDLTaskStatus next_task_status); int take_effect(const share::ObDDLTaskStatus next_task_status); int copy_table_indexes(); @@ -68,13 +66,10 @@ private: virtual int64_t get_serialize_param_size() const override; private: static const int64_t OB_COLUMN_REDEFINITION_TASK_VERSION = 1L; - int64_t sstable_complete_request_time_; bool has_rebuild_index_; bool has_rebuild_constraint_; bool has_rebuild_foreign_key_; - bool is_sstable_complete_task_submitted_; common::ObArenaAllocator allocator_; - ObDDLSingleReplicaExecutor replica_builder_; }; } // end namespace rootserver diff --git a/src/rootserver/ddl_task/ob_constraint_task.cpp b/src/rootserver/ddl_task/ob_constraint_task.cpp index 61a5a1079d..b828390d6a 100755 --- a/src/rootserver/ddl_task/ob_constraint_task.cpp +++ b/src/rootserver/ddl_task/ob_constraint_task.cpp @@ -522,6 +522,8 @@ int ObConstraintTask::init( consumer_group_id_ = consumer_group_id; task_version_ = OB_CONSTRAINT_TASK_VERSION; is_table_hidden_ = table_schema->is_user_hidden_table(); + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; is_inited_ = true; ddl_tracing_.open(); } @@ -534,6 +536,7 @@ int ObConstraintTask::init(const ObDDLTaskRecord &task_record) const uint64_t table_id = task_record.object_id_; const uint64_t target_object_id = task_record.target_object_id_; const int64_t schema_version = task_record.schema_version_; + task_type_ = task_record.ddl_type_; ObSchemaGetterGuard schema_guard; ObRootService *root_service = GCTX.root_service_; const ObTableSchema *table_schema = nullptr; @@ -561,7 +564,6 @@ int ObConstraintTask::init(const ObDDLTaskRecord &task_record) target_object_id_ = target_object_id; tenant_id_ = task_record.tenant_id_; task_status_ = static_cast(task_record.task_status_); - task_type_ = task_record.ddl_type_; snapshot_version_ = task_record.snapshot_version_; schema_version_ = task_record.schema_version_; root_service_ = root_service; @@ -569,6 +571,8 @@ int ObConstraintTask::init(const ObDDLTaskRecord &task_record) parent_task_id_ = task_record.parent_task_id_; is_table_hidden_ = table_schema->is_user_hidden_table(); ret_code_ = task_record.ret_code_; + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; is_inited_ = true; // set up span during recover task @@ -1913,7 +1917,7 @@ int ObConstraintTask::deserlize_params_from_message(const uint64_t tenant_id, co LOG_WARN("ObDDLTask deserlize failed", K(ret)); } else if (OB_FAIL(tmp_arg.deserialize(buf, data_len, pos))) { LOG_WARN("serialize table failed", K(ret)); - } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(tenant_id, tmp_arg))) { + } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(task_type_, tenant_id, tmp_arg))) { LOG_WARN("replace user tenant id failed", K(ret), K(tenant_id), K(tmp_arg)); } else if (OB_FAIL(deep_copy_table_arg(allocator_, tmp_arg, alter_table_arg_))) { LOG_WARN("deep copy table arg failed", K(ret)); diff --git a/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp b/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp index cf0a17c1bc..02471810ba 100644 --- a/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp +++ b/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp @@ -115,17 +115,17 @@ int ObDDLRedefinitionSSTableBuildTask::process() LOG_WARN("error unexpected, table schema must not be nullptr", K(ret), K(tenant_id_), K(data_table_id_)); } else { if (OB_FAIL(ObDDLUtil::generate_build_replica_sql(tenant_id_, - data_table_id_, - dest_table_id_, - data_table_schema->get_schema_version(), - snapshot_version_, - execution_id_, - task_id_, - parallelism_, - use_heap_table_ddl_plan_, - true/*use_schema_version_hint_for_src_table*/, - &col_name_map_, - sql_string))) { + data_table_id_, + dest_table_id_, + data_table_schema->get_schema_version(), + snapshot_version_, + execution_id_, + task_id_, + parallelism_, + use_heap_table_ddl_plan_, + true/*use_schema_version_hint_for_src_table*/, + &col_name_map_, + sql_string))) { LOG_WARN("fail to generate build replica sql", K(ret)); } else { ObTimeoutCtx timeout_ctx; @@ -249,7 +249,7 @@ int ObDDLRedefinitionTask::check_table_empty(const ObDDLTaskStatus next_task_sta LOG_WARN("failed to check need check table empty", K(ret)); } else if (need_check_table_empty) { if (!is_check_replica_end && 0 == check_table_empty_job_time_) { - ObCheckConstraintValidationTask task(tenant_id_, object_id_, -1/*constraint id*/, target_object_id_, + ObCheckConstraintValidationTask task(dst_tenant_id_, object_id_, -1/*constraint id*/, target_object_id_, schema_version_, trace_id_, task_id_, true/*check_table_empty*/, obrpc::ObAlterTableArg::AlterConstraintType::ADD_CONSTRAINT); if (OB_FAIL(root_service->submit_ddl_single_replica_build_task(task))) { @@ -589,23 +589,115 @@ int ObDDLRedefinitionTask::get_validate_checksum_columns_id(const ObTableSchema return ret; } -int ObDDLRedefinitionTask::check_data_dest_tables_columns_checksum(const int64_t execution_id) +int ObDDLRedefinitionTask::wait_data_complement(const ObDDLTaskStatus next_task_status) { int ret = OB_SUCCESS; - ObSchemaGetterGuard schema_guard; - const ObTableSchema *data_table_schema = nullptr; - const ObTableSchema *dest_table_schema = nullptr; - hash::ObHashMap validate_checksum_columns_id; - ObMultiVersionSchemaService &schema_service = ObMultiVersionSchemaService::get_instance(); + bool is_build_replica_end = false; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObColumnRedefinitionTask is not inited", K(ret)); + } else if (ObDDLTaskStatus::REDEFINITION != task_status_) { + ret = OB_TASK_EXPIRED; + LOG_WARN("task status not match", K(ret), K(task_status_)); + } else if (OB_UNLIKELY(snapshot_version_ <= 0)) { + is_build_replica_end = true; // switch to fail. + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected snapshot", K(ret), KPC(this)); + } else if (!is_sstable_complete_task_submitted_ && OB_FAIL(send_build_single_replica_request())) { + LOG_WARN("fail to send build single replica request", K(ret)); + } else if (is_sstable_complete_task_submitted_ && OB_FAIL(check_build_single_replica(is_build_replica_end))) { + LOG_WARN("fail to check build single replica", K(ret), K(is_build_replica_end)); + } + DEBUG_SYNC(COLUMN_REDEFINITION_REPLICA_BUILD); + if (is_build_replica_end) { + ret = OB_SUCC(ret) ? complete_sstable_job_ret_code_ : ret; + if (OB_SUCC(ret) && OB_FAIL(check_data_dest_tables_columns_checksum(get_execution_id()))) { + LOG_WARN("fail to check the columns checkum between data table and hidden one", K(ret)); + } + if (OB_FAIL(switch_status(next_task_status, true, ret))) { + LOG_WARN("fail to swith task status", K(ret)); + } + LOG_INFO("wait data complement finished", K(ret), K(*this)); + } + return ret; +} + +int ObDDLRedefinitionTask::send_build_single_replica_request() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObColumnRedefinitionTask has not been inited", K(ret)); + } else { + ObDDLSingleReplicaExecutorParam param; + param.tenant_id_ = tenant_id_; + param.dest_tenant_id_ = dst_tenant_id_; + param.type_ = task_type_; + param.source_table_id_ = object_id_; + param.dest_table_id_ = target_object_id_; + param.schema_version_ = schema_version_; + param.dest_schema_version_ = dst_schema_version_; + param.snapshot_version_ = snapshot_version_; + param.task_id_ = task_id_; + param.parallelism_ = std::max(alter_table_arg_.parallelism_, 1L); + param.execution_id_ = execution_id_; + param.data_format_version_ = data_format_version_; + param.consumer_group_id_ = alter_table_arg_.consumer_group_id_; + if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, object_id_, param.source_tablet_ids_))) { + LOG_WARN("fail to get tablets", K(ret), K(tenant_id_), K(object_id_)); + } else if (OB_FAIL(ObDDLUtil::get_tablets(dst_tenant_id_, target_object_id_, param.dest_tablet_ids_))) { + LOG_WARN("fail to get tablets", K(ret), K(tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(replica_builder_.build(param))) { + LOG_WARN("fail to send build single replica", K(ret)); + } else { + LOG_INFO("start to build single replica", K(target_object_id_)); + is_sstable_complete_task_submitted_ = true; + sstable_complete_request_time_ = ObTimeUtility::current_time(); + } + } + return ret; +} + +// check whether all leaders have completed the complement task +int ObDDLRedefinitionTask::check_build_single_replica(bool &is_end) +{ + int ret = OB_SUCCESS; + is_end = false; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("ObDDLRedefinitionTask has not been inited", 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_table_schema(tenant_id_, object_id_, data_table_schema))) { + } else if (OB_FAIL(replica_builder_.check_build_end(is_end, complete_sstable_job_ret_code_))) { + LOG_WARN("fail to check build end", K(ret)); + } else if (!is_end) { + if (sstable_complete_request_time_ + OB_MAX_DDL_SINGLE_REPLICA_BUILD_TIMEOUT < ObTimeUtility::current_time()) { // timeout, retry + is_sstable_complete_task_submitted_ = false; + sstable_complete_request_time_ = 0; + } + } + return ret; +} + +int ObDDLRedefinitionTask::check_data_dest_tables_columns_checksum(const int64_t execution_id) +{ + int ret = OB_SUCCESS; + 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; + const ObTableSchema *data_table_schema = nullptr; + const ObTableSchema *dest_table_schema = nullptr; + hash::ObHashMap validate_checksum_columns_id; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObDDLRedefinitionTask has not been inited", K(ret)); + } else if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(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("get tenant schema guard failed", K(ret), K(tenant_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(tenant_id_, object_id_, data_table_schema))) { LOG_WARN("get data table schema failed", K(ret), K(tenant_id_), K(object_id_)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, dest_table_schema))) { - LOG_WARN("get dest table schema failed", K(ret), K(tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(dst_tenant_id_, target_object_id_, dest_table_schema))) { + LOG_WARN("get data table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(data_table_schema) || OB_ISNULL(dest_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_INFO("table is not exist", K(ret), K(object_id_), K(target_object_id_), KP(data_table_schema), KP(dest_table_schema)); @@ -627,10 +719,15 @@ int ObDDLRedefinitionTask::check_data_dest_tables_columns_checksum(const int64_t } else if (OB_UNLIKELY(OB_INVALID_ID == target_object_id_ || !dest_table_column_checksums.created())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), "dest_table_id", target_object_id_, K(dest_table_column_checksums.created())); - } else if (OB_FAIL(ObDDLChecksumOperator::get_table_column_checksum(tenant_id_, execution_id, object_id_, task_id_, false/*replica build*/, data_table_column_checksums, GCTX.root_service_->get_sql_proxy()))) { - LOG_WARN("fail to get table column checksum", K(ret), K(execution_id), "table_id", object_id_, K_(task_id), K(data_table_column_checksums.created()), KP(GCTX.root_service_)); - } else if (OB_FAIL(ObDDLChecksumOperator::get_table_column_checksum(tenant_id_, execution_id, target_object_id_, task_id_, false /*replica build*/, dest_table_column_checksums, GCTX.root_service_->get_sql_proxy()))) { - LOG_WARN("fail to get table column checksum", K(ret), K(execution_id), "table_id", target_object_id_, K_(task_id), K(dest_table_column_checksums.created()), KP(GCTX.root_service_)); + } else if (OB_FAIL(ObDDLChecksumOperator::get_table_column_checksum(dst_tenant_id_, execution_id, object_id_, task_id_, false/*replica build*/, data_table_column_checksums, GCTX.root_service_->get_sql_proxy()))) { + LOG_WARN("fail to get table column checksum", K(ret), K(dst_tenant_id_), K(execution_id), "table_id", object_id_, K_(task_id), K(data_table_column_checksums.created()), KP(GCTX.root_service_)); + } else if (OB_FAIL(ObDDLChecksumOperator::get_table_column_checksum(dst_tenant_id_, execution_id, target_object_id_, task_id_, false /*replica build*/, dest_table_column_checksums, GCTX.root_service_->get_sql_proxy()))) { + /** + * For DDL_RESTORE_TABLE, dst tenant id is differen to source tenant id. + * Meanwhile, the original tenant is a backup one, can not support write operation, + * and its' checksum is recorded into to the dest tenant. + */ + LOG_WARN("fail to get table column checksum", K(ret), K(dst_tenant_id_), K(execution_id), "table_id", target_object_id_, K_(task_id), K(dest_table_column_checksums.created()), KP(GCTX.root_service_)); } else { uint64_t dest_column_id = 0; for (hash::ObHashMap::const_iterator iter = data_table_column_checksums.begin(); @@ -690,10 +787,10 @@ int ObDDLRedefinitionTask::add_constraint_ddl_task(const int64_t constraint_id) } else if (OB_UNLIKELY(OB_INVALID_ID == constraint_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(constraint_id)); - } else if (OB_FAIL(root_service->get_ddl_service().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(schema_guard.get_table_schema(tenant_id_, target_object_id_, table_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_)); + } else if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(dst_tenant_id_, schema_guard))) { + LOG_WARN("get schema guard failed", K(ret), K(dst_tenant_id_)); + } else if (OB_FAIL(schema_guard.get_table_schema(dst_tenant_id_, target_object_id_, table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_SYS; LOG_WARN("table schema must not be nullptr", K(ret)); @@ -702,14 +799,14 @@ int ObDDLRedefinitionTask::add_constraint_ddl_task(const int64_t constraint_id) } else if (OB_ISNULL(constraint = table_schema->get_constraint(constraint_id))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get constraint failed", K(ret), K(constraint_id)); - } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id_, table_schema->get_database_id(), database_schema))) { - LOG_WARN("get database schema failed", K(ret), K_(tenant_id)); + } else if (OB_FAIL(schema_guard.get_database_schema(dst_tenant_id_, table_schema->get_database_id(), database_schema))) { + LOG_WARN("get database schema failed", K(ret), K(dst_tenant_id_)); } else if (OB_FAIL(alter_table_arg.tz_info_wrap_.deep_copy(alter_table_arg_.tz_info_wrap_))) { LOG_WARN("deep copy timezone info failed", K(ret)); } else if (OB_FAIL(alter_table_arg.set_nls_formats(alter_table_arg_.nls_formats_))) { LOG_WARN("set nls formats failed", K(ret)); } else { - alter_table_arg.exec_tenant_id_ = tenant_id_; + alter_table_arg.exec_tenant_id_ = dst_tenant_id_; alter_table_arg.alter_constraint_type_ = obrpc::ObAlterTableArg::ADD_CONSTRAINT; alter_table_schema.clear_constraint(); alter_table_schema.set_origin_database_name(database_schema->get_database_name_str()); @@ -721,7 +818,7 @@ int ObDDLRedefinitionTask::add_constraint_ddl_task(const int64_t constraint_id) if (need_check) { //TODO: shanting not null ObDDLTaskRecord task_record; - ObCreateDDLTaskParam param(tenant_id_, + ObCreateDDLTaskParam param(dst_tenant_id_, ObDDLType::DDL_CHECK_CONSTRAINT, table_schema, nullptr, @@ -766,7 +863,10 @@ int ObDDLRedefinitionTask::add_constraint_ddl_task(const int64_t constraint_id) int ObDDLRedefinitionTask::add_fk_ddl_task(const int64_t fk_id) { int ret = OB_SUCCESS; - ObSchemaGetterGuard schema_guard; + 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; const ObTableSchema *orig_table_schema = nullptr; const ObTableSchema *hidden_table_schema = nullptr; SMART_VAR(obrpc::ObAlterTableArg, alter_table_arg) { @@ -783,21 +883,23 @@ int ObDDLRedefinitionTask::add_fk_ddl_task(const int64_t fk_id) } else if (OB_UNLIKELY(OB_INVALID_ID == fk_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(fk_id)); - } else if (OB_FAIL(root_service->get_ddl_service().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(schema_guard.get_table_schema(tenant_id_, object_id_, orig_table_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_)); + } else if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(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("get schema guard failed", K(ret), K(tenant_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(tenant_id_, object_id_, orig_table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(object_id_)); } else if (OB_ISNULL(orig_table_schema)) { ret = OB_ERR_SYS; LOG_WARN("error sys, table schema must not be nullptr", K(ret)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, hidden_table_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(dst_tenant_id_, target_object_id_, hidden_table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(hidden_table_schema)) { ret = OB_ERR_SYS; - LOG_WARN("error sys, table schema must not be nullptr", K(ret)); + LOG_WARN("error sys, table schema must not be nullptr", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_FAIL(alter_table_schema.assign(*hidden_table_schema))) { LOG_WARN("assign table schema failed", K(ret)); - } else if (OB_FAIL(schema_guard.get_database_schema(tenant_id_, orig_table_schema->get_database_id(), database_schema))) { + } else if (OB_FAIL(src_tenant_schema_guard->get_database_schema(tenant_id_, orig_table_schema->get_database_id(), database_schema))) { LOG_WARN("get database schema failed", K(ret), K_(tenant_id)); } else if (OB_FAIL(alter_table_arg.tz_info_wrap_.deep_copy(alter_table_arg_.tz_info_wrap_))) { LOG_WARN("deep copy timezone info failed", K(ret)); @@ -812,7 +914,7 @@ int ObDDLRedefinitionTask::add_fk_ddl_task(const int64_t fk_id) alter_table_schema.set_origin_table_name(orig_table_schema->get_table_name_str()); alter_table_arg.table_id_ = object_id_; alter_table_arg.hidden_table_id_ = target_object_id_; - alter_table_arg.exec_tenant_id_ = tenant_id_; + alter_table_arg.exec_tenant_id_ = dst_tenant_id_; for (int64_t i = 0; OB_SUCC(ret) && i < fk_info_array.count(); ++i) { const ObForeignKeyInfo &tmp_fk_info = fk_info_array.at(i); if (tmp_fk_info.foreign_key_id_ == fk_id) { @@ -838,7 +940,7 @@ int ObDDLRedefinitionTask::add_fk_ddl_task(const int64_t fk_id) fk_arg.is_modify_fk_state_ = fk_info.is_modify_fk_state_; fk_arg.need_validate_data_ = fk_info.validate_flag_; ObDDLTaskRecord task_record; - ObCreateDDLTaskParam param(tenant_id_, + ObCreateDDLTaskParam param(dst_tenant_id_, ObDDLType::DDL_FOREIGN_KEY_CONSTRAINT, hidden_table_schema, nullptr, @@ -917,8 +1019,10 @@ int ObDDLRedefinitionTask::on_child_task_finish( int ObDDLRedefinitionTask::sync_auto_increment_position() { int ret = OB_SUCCESS; - ObMultiVersionSchemaService &schema_service = ObMultiVersionSchemaService::get_instance(); - ObSchemaGetterGuard schema_guard; + 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; const ObTableSchema *data_table_schema = nullptr; const ObTableSchema *dest_table_schema = nullptr; if (OB_UNLIKELY(!is_inited_)) { @@ -926,18 +1030,20 @@ int ObDDLRedefinitionTask::sync_auto_increment_position() LOG_WARN("ObDDLRedefinitionTask has not been inited", K(ret)); } else if (has_synced_autoincrement_) { // do nothing - } 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_table_schema(tenant_id_, object_id_, data_table_schema))) { - LOG_WARN("get data table schema failed", KR(ret), K(tenant_id_), K(object_id_)); + } else if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(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("get tenant schema guard failed", K(ret), K(tenant_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(tenant_id_, object_id_, data_table_schema))) { + LOG_WARN("get data table schema failed", K(ret), K(tenant_id_), K(object_id_)); } else if (OB_ISNULL(data_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("data_table_schema is NULL", KR(ret), K_(tenant_id), K_(object_id)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, dest_table_schema))) { - LOG_WARN("get dest table schema failed", KR(ret), K(tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(dst_tenant_id_, target_object_id_, dest_table_schema))) { + LOG_WARN("get data table schema failed", KR(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(dest_table_schema)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("dest_table_schema is NULL", KR(ret), K_(tenant_id), K_(target_object_id)); + LOG_WARN("data_table_schema is NULL", K(ret), K_(dst_tenant_id), K_(target_object_id)); } else { ObArray column_ids; if (OB_FAIL(data_table_schema->get_column_ids(column_ids))) { @@ -961,7 +1067,7 @@ int ObDDLRedefinitionTask::sync_auto_increment_position() ObAutoincrementService &auto_inc_service = ObAutoincrementService::get_instance(); uint64_t sequence_value = 0; AutoincParam param; - param.tenant_id_ = tenant_id_; + param.tenant_id_ = dst_tenant_id_; param.autoinc_table_id_ = target_object_id_; param.autoinc_first_part_num_ = dest_table_schema->get_first_part_num(); param.autoinc_table_part_num_ = dest_table_schema->get_all_part_num(); @@ -980,9 +1086,23 @@ int ObDDLRedefinitionTask::sync_auto_increment_position() LOG_WARN("get sequence value failed", KR(ret), K(tenant_id_), K(object_id_), K(cur_column_id)); } else if (FALSE_IT(param.global_value_to_sync_ = sequence_value - 1)) { // as sequence_value is an avaliable value. sync value will not be avaliable to user - } else if (OB_FAIL(auto_inc_service.sync_insert_value_global(param))) { - LOG_WARN("set auto increment position failed", K(ret), K(tenant_id_), K(target_object_id_), K(cur_column_id), K(param)); } else { + while (OB_SUCC(ret)) { + if (OB_FAIL(auto_inc_service.sync_insert_value_global(param))) { + if (DDL_TABLE_RESTORE == task_type_ && OB_TENANT_NOT_IN_SERVER == ret) { + if (TC_REACH_TIME_INTERVAL(10L * 1000L * 1000L)) { + LOG_INFO("set auto increment position failed, retry", K(ret), K(dst_tenant_id_), K(target_object_id_), K(cur_column_id), K(param)); + } + ret = OB_SUCCESS; + } else { + LOG_WARN("set auto increment position failed", K(ret), K(dst_tenant_id_), K(target_object_id_), K(cur_column_id), K(param)); + } + } else { + break; + } + } + } + if (OB_SUCC(ret)) { has_synced_autoincrement_ = true; LOG_INFO("sync auto increment position succ", K(ret), K(sequence_value), K(object_id_), K(target_object_id_), K(dst_column_schema->get_column_id())); @@ -1008,21 +1128,23 @@ int ObDDLRedefinitionTask::modify_autoinc(const ObDDLTaskStatus next_task_status } else if (OB_FAIL(check_update_autoinc_end(is_update_autoinc_end))) { LOG_WARN("update autoinc failed", K(ret)); } else { - ObDDLService &ddl_service = root_service->get_ddl_service(); - ObMultiVersionSchemaService &schema_service = ddl_service.get_schema_service(); - ObMySQLProxy &sql_proxy = ddl_service.get_sql_proxy(); - const ObTableSchema *orig_table_schema = nullptr; - ObSchemaGetterGuard schema_guard; + 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; AlterTableSchema &alter_table_schema = alter_table_arg_.alter_table_schema_; + const ObTableSchema *orig_table_schema = nullptr; const ObTableSchema *new_table_schema = nullptr; uint64_t alter_autoinc_column_id = 0; ObColumnNameMap col_name_map; - 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_, object_id_, orig_table_schema))) { + if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(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("get tenant schema guard failed", K(ret), K(tenant_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(tenant_id_, object_id_, orig_table_schema))) { LOG_WARN("get data table schema failed", K(ret), K(tenant_id_), K(object_id_)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, new_table_schema))) { - LOG_WARN("get data table schema failed", K(ret), K(tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(dst_tenant_id_, target_object_id_, new_table_schema))) { + LOG_WARN("get data table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(orig_table_schema) || OB_ISNULL(new_table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schemas should not be null", K(ret), K(orig_table_schema), K(new_table_schema)); @@ -1085,12 +1207,9 @@ int ObDDLRedefinitionTask::modify_autoinc(const ObDDLTaskStatus next_task_status const int64_t save_timeout_ts = THIS_WORKER.get_timeout_ts(); THIS_WORKER.set_timeout_ts(ObTimeUtility::current_time() + max(GCONF.rpc_timeout, 1000 * 1000 * 20L)); ObAutoincrementService &auto_inc_service = ObAutoincrementService::get_instance(); - ObDDLService &ddl_service = root_service->get_ddl_service(); - ObMultiVersionSchemaService &schema_service = ddl_service.get_schema_service(); - ObMySQLProxy &sql_proxy = ddl_service.get_sql_proxy(); const uint64_t autoinc_val = alter_table_schema.get_auto_increment(); AutoincParam param; - param.tenant_id_ = tenant_id_; + param.tenant_id_ = dst_tenant_id_; param.autoinc_table_id_ = target_object_id_; param.autoinc_first_part_num_ = new_table_schema->get_first_part_num(); param.autoinc_table_part_num_ = new_table_schema->get_all_part_num(); @@ -1272,17 +1391,20 @@ int ObDDLRedefinitionTask::check_health() } else if (OB_FAIL(refresh_schema_version())) { LOG_WARN("refresh schema version failed", K(ret)); } else { - ObMultiVersionSchemaService &schema_service = root_service->get_schema_service(); - ObSchemaGetterGuard schema_guard; - const ObTableSchema *index_schema = nullptr; + 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; bool is_source_table_exist = false; bool is_dest_table_exist = false; - if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id_, schema_guard))) { - LOG_WARN("get tanant schema guard failed", K(ret), K(tenant_id_)); - } else if (OB_FAIL(schema_guard.check_table_exist(tenant_id_, object_id_, is_source_table_exist))) { + if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(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("get tenant schema guard failed", K(ret), K(tenant_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->check_table_exist(tenant_id_, object_id_, is_source_table_exist))) { LOG_WARN("check data table exist failed", K(ret), K_(tenant_id), K(object_id_)); - } else if (OB_FAIL(schema_guard.check_table_exist(tenant_id_, target_object_id_, is_dest_table_exist))) { - LOG_WARN("check index table exist failed", K(ret), K_(tenant_id), K(is_dest_table_exist)); + } else if (OB_FAIL(dst_tenant_schema_guard->check_table_exist(dst_tenant_id_, target_object_id_, is_dest_table_exist))) { + LOG_WARN("check index table exist failed", K(ret), K_(dst_tenant_id), K(is_dest_table_exist)); } else if (!is_source_table_exist || !is_dest_table_exist) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("data table or dest table not exist", K(ret), K(is_source_table_exist), K(is_dest_table_exist)); @@ -1365,22 +1487,25 @@ int ObDDLRedefinitionTask::sync_stats_info() ret = OB_ERR_SYS; LOG_WARN("error sys, root service must not be nullptr", K(ret)); } else if (check_need_sync_stats()) { - ObMultiVersionSchemaService &schema_service = root_service->get_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; ObMySQLTransaction trans; - ObSchemaGetterGuard schema_guard; const ObTableSchema *data_table_schema = nullptr; const ObTableSchema *new_table_schema = nullptr; ObTimeoutCtx timeout_ctx; int64_t timeout = 0; const int64_t start_time = ObTimeUtility::current_time(); bool need_sync_history = check_need_sync_stats_history(); - - if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id_, schema_guard))) { - LOG_WARN("get tanant schema guard failed", K(ret), K(tenant_id_)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, object_id_, data_table_schema))) { - LOG_WARN("fail to get data table schema", K(ret), K(object_id_)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, new_table_schema))) { - LOG_WARN("fail to get data table schema", K(ret), K(target_object_id_)); + if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(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("get tenant schema guard failed", K(ret), K(tenant_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(tenant_id_, object_id_, data_table_schema))) { + LOG_WARN("fail to get data table schema", K(ret), K(tenant_id_), K(object_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(dst_tenant_id_, target_object_id_, new_table_schema))) { + LOG_WARN("fail to get data table schema", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(data_table_schema) || OB_ISNULL(new_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("table schema is null", K(ret)); @@ -1390,7 +1515,7 @@ int ObDDLRedefinitionTask::sync_stats_info() LOG_WARN("set timeout ctx failed", K(ret)); } else if (OB_FAIL(timeout_ctx.set_timeout(timeout))) { LOG_WARN("set timeout failed", K(ret)); - } else if (OB_FAIL(trans.start(&root_service->get_sql_proxy(), tenant_id_))) { + } else if (OB_FAIL(trans.start(&root_service->get_sql_proxy(), dst_tenant_id_))) { LOG_WARN("fail to start transaction", K(ret)); } else if (OB_FAIL(sync_table_level_stats_info(trans, *data_table_schema, need_sync_history))) { LOG_WARN("fail to sync table level stats", K(ret)); @@ -1403,7 +1528,7 @@ int ObDDLRedefinitionTask::sync_stats_info() } else if (OB_FAIL(sync_column_level_stats_info(trans, *data_table_schema, *new_table_schema, - schema_guard, + *src_tenant_schema_guard, need_sync_history))) { LOG_WARN("fail to sync column level stats", K(ret)); } @@ -1437,7 +1562,7 @@ int ObDDLRedefinitionTask::sync_table_level_stats_info(common::ObMySQLTransactio // for partitioned table, table-level stat is -1, for non-partitioned table, table-level stat is table id int64_t partition_id = -1; int64_t target_partition_id = -1; - const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id_); + const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(dst_tenant_id_); if (!data_table_schema.is_partitioned_table()) { partition_id = object_id_; target_partition_id = target_object_id_; @@ -1445,9 +1570,9 @@ int ObDDLRedefinitionTask::sync_table_level_stats_info(common::ObMySQLTransactio if (OB_FAIL(sql_string.assign_fmt("UPDATE %s SET table_id = %ld, partition_id = %ld" " WHERE tenant_id = %ld and table_id = %ld and partition_id = %ld", OB_ALL_TABLE_STAT_TNAME, target_object_id_, target_partition_id, - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), object_id_, partition_id))) { + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), object_id_, partition_id))) { LOG_WARN("fail to assign sql string", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_table_stat", K(ret), K(sql_string)); } else if (OB_UNLIKELY(affected_rows < 0)) { ret = OB_ERR_UNEXPECTED; @@ -1456,9 +1581,9 @@ int ObDDLRedefinitionTask::sync_table_level_stats_info(common::ObMySQLTransactio } else if (OB_FAIL(history_sql_string.assign_fmt("UPDATE %s SET table_id = %ld, partition_id = %ld" " WHERE tenant_id = %ld and table_id = %ld and partition_id = %ld", OB_ALL_TABLE_STAT_HISTORY_TNAME, target_object_id_, target_partition_id, - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), object_id_, partition_id))) { + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), object_id_, partition_id))) { LOG_WARN("fail to assign history sql string", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, history_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, history_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_table_stat_history", K(ret), K(sql_string)); } return ret; @@ -1472,7 +1597,7 @@ int ObDDLRedefinitionTask::sync_partition_level_stats_info(common::ObMySQLTransa int ret = OB_SUCCESS; ObArray src_partition_ids; ObArray dest_partition_ids; - const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id_); + const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(dst_tenant_id_); if (!data_table_schema.is_partitioned_table()) { // if not partition table, no need to sync partition level stats } else if (OB_FAIL(pl::ObDbmsStats::get_part_ids_from_schema(&data_table_schema, src_partition_ids))) { @@ -1494,7 +1619,7 @@ int ObDDLRedefinitionTask::sync_partition_level_stats_info(common::ObMySQLTransa batch_end, sql_string))) { LOG_WARN("fail to generate sql", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_table_stat", K(ret), K(sql_string)); } else if (OB_UNLIKELY(affected_rows < 0)) { ret = OB_ERR_UNEXPECTED; @@ -1507,7 +1632,7 @@ int ObDDLRedefinitionTask::sync_partition_level_stats_info(common::ObMySQLTransa batch_end, history_sql_string))) { LOG_WARN("fail to generate sql", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, history_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, history_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_table_stat_history", K(ret), K(history_sql_string)); } else if (OB_UNLIKELY(affected_rows < 0)) { ret = OB_ERR_UNEXPECTED; @@ -1599,7 +1724,7 @@ int ObDDLRedefinitionTask::sync_one_column_table_level_stats_info(common::ObMySQ // for partitioned table, table-level stat is -1, for non-partitioned table, table-level stat is table id int64_t partition_id = -1; int64_t target_partition_id = -1; - const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id_); + const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(dst_tenant_id_); if (!data_table_schema.is_partitioned_table()) { partition_id = object_id_; target_partition_id = target_object_id_; @@ -1607,35 +1732,35 @@ int ObDDLRedefinitionTask::sync_one_column_table_level_stats_info(common::ObMySQ if (OB_FAIL(column_sql_string.assign_fmt("UPDATE %s SET table_id = %ld, partition_id = %ld, column_id = %ld" " WHERE tenant_id = %ld and table_id = %ld and partition_id = %ld and column_id = %ld", OB_ALL_COLUMN_STAT_TNAME, target_object_id_, target_partition_id, new_col_id, - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), object_id_, partition_id, + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), object_id_, partition_id, old_col_id))) { LOG_WARN("fail to assign sql string", K(ret)); } else if (OB_FAIL(histogram_sql_string.assign_fmt("UPDATE %s SET table_id = %ld, partition_id = %ld, column_id = %ld" " WHERE tenant_id = %ld and table_id = %ld and partition_id = %ld and column_id = %ld", OB_ALL_HISTOGRAM_STAT_TNAME, target_object_id_, target_partition_id, new_col_id, - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), object_id_, partition_id, + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), object_id_, partition_id, old_col_id))) { LOG_WARN("fail to assign sql string", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, column_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, column_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_column_stat", K(ret), K(column_sql_string)); - } else if (OB_FAIL(trans.write(tenant_id_, histogram_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, histogram_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_histogram_stat_history", K(ret), K(histogram_sql_string)); } else if (!need_sync_history) { // do not need to sync history } else if (OB_FAIL(column_history_sql_string.assign_fmt( "UPDATE %s SET table_id = %ld, partition_id = %ld, column_id = %ld" " WHERE tenant_id = %ld and table_id = %ld and partition_id = %ld and column_id = %ld", OB_ALL_COLUMN_STAT_HISTORY_TNAME, target_object_id_, target_partition_id, new_col_id, - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), object_id_, partition_id, old_col_id))) { + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), object_id_, partition_id, old_col_id))) { LOG_WARN("fail to assign history sql string", K(ret)); } else if (OB_FAIL(histogram_history_sql_string.assign_fmt( "UPDATE %s SET table_id = %ld, partition_id = %ld, column_id = %ld" " WHERE tenant_id = %ld and table_id = %ld and partition_id = %ld and column_id = %ld", OB_ALL_HISTOGRAM_STAT_HISTORY_TNAME, target_object_id_, target_partition_id, new_col_id, - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), object_id_, partition_id, old_col_id))) { + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), object_id_, partition_id, old_col_id))) { LOG_WARN("fail to assign history sql string", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, column_history_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, column_history_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_column_stat_history", K(ret), K(column_history_sql_string)); - } else if (OB_FAIL(trans.write(tenant_id_, histogram_history_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, histogram_history_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_histogram_stat_history", K(ret), K(histogram_history_sql_string)); } return ret; @@ -1651,7 +1776,7 @@ int ObDDLRedefinitionTask::sync_one_column_partition_level_stats_info(common::Ob int ret = OB_SUCCESS; ObArray src_partition_ids; ObArray dest_partition_ids; - const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id_); + const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(dst_tenant_id_); if (!data_table_schema.is_partitioned_table()) { // if not partition table, no need to sync partition level stats } else if (OB_FAIL(pl::ObDbmsStats::get_part_ids_from_schema(&data_table_schema, src_partition_ids))) { @@ -1686,9 +1811,9 @@ int ObDDLRedefinitionTask::sync_one_column_partition_level_stats_info(common::Ob batch_end, histogram_sql_string))) { LOG_WARN("fail to generate sql", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, column_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, column_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_column_stat", K(ret), K(column_sql_string)); - } else if (OB_FAIL(trans.write(tenant_id_, histogram_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, histogram_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_histogram_stat_history", K(ret), K(histogram_sql_string)); } else if (!need_sync_history) { } else if (OB_FAIL(generate_sync_column_partition_level_stats_sql(OB_ALL_COLUMN_STAT_HISTORY_TNAME, @@ -1709,9 +1834,9 @@ int ObDDLRedefinitionTask::sync_one_column_partition_level_stats_info(common::Ob batch_end, histogram_history_sql_string))) { LOG_WARN("fail to generate sql", K(ret)); - } else if (OB_FAIL(trans.write(tenant_id_, column_history_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, column_history_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_column_stat_history", K(ret), K(column_history_sql_string)); - } else if (OB_FAIL(trans.write(tenant_id_, histogram_history_sql_string.ptr(), affected_rows))) { + } else if (OB_FAIL(trans.write(dst_tenant_id_, histogram_history_sql_string.ptr(), affected_rows))) { LOG_WARN("fail to update __all_histogram_stat_history", K(ret), K(histogram_history_sql_string)); } batch_start += BATCH_SIZE; @@ -1729,7 +1854,7 @@ int ObDDLRedefinitionTask::generate_sync_partition_level_stats_sql(const char *t { int ret = OB_SUCCESS; sql_string.reset(); - const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id_); + const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(dst_tenant_id_); if (OB_UNLIKELY(src_partition_ids.count() != dest_partition_ids.count() || batch_end < batch_start || batch_end >= dest_partition_ids.count())) { ret = OB_INVALID_ARGUMENT; @@ -1747,9 +1872,9 @@ int ObDDLRedefinitionTask::generate_sync_partition_level_stats_sql(const char *t } if (OB_FAIL(ret)) { } else if (OB_FAIL(sql_string.append_fmt(" else partition_id end) where tenant_id=%ld and table_id=%ld", - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), batch_start == 0 ? object_id_ : target_object_id_))) { - LOG_WARN("fail to append sql string", K(ret), K(object_id_), K(tenant_id_), K(exec_tenant_id)); + LOG_WARN("fail to append sql string", K(ret), K(object_id_), K(dst_tenant_id_), K(exec_tenant_id)); } } return ret; @@ -1766,7 +1891,7 @@ int ObDDLRedefinitionTask::generate_sync_column_partition_level_stats_sql(const { int ret = OB_SUCCESS; sql_string.reset(); - const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id_); + const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(dst_tenant_id_); if (OB_UNLIKELY(src_partition_ids.count() != dest_partition_ids.count() || batch_end < batch_start || batch_end >= dest_partition_ids.count())) { ret = OB_INVALID_ARGUMENT; @@ -1784,9 +1909,9 @@ int ObDDLRedefinitionTask::generate_sync_column_partition_level_stats_sql(const } if (OB_FAIL(ret)) { } else if (OB_FAIL(sql_string.append_fmt(" else partition_id end) where tenant_id=%ld and table_id=%ld and column_id=%ld", - ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id_), + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, dst_tenant_id_), batch_start == 0 ? object_id_ : target_object_id_, batch_start == 0 ? old_col_id : new_col_id))) { - LOG_WARN("fail to append sql string", K(ret), K(object_id_), K(tenant_id_), K(exec_tenant_id), K(old_col_id)); + LOG_WARN("fail to append sql string", K(ret), K(object_id_), K(dst_tenant_id_), K(exec_tenant_id), K(old_col_id)); } } return ret; @@ -1796,7 +1921,7 @@ int ObDDLRedefinitionTask::sync_tablet_autoinc_seq() { int ret = OB_SUCCESS; if (!sync_tablet_autoinc_seq_ctx_.is_inited() - && OB_FAIL(sync_tablet_autoinc_seq_ctx_.init(tenant_id_, object_id_, target_object_id_))) { + && OB_FAIL(sync_tablet_autoinc_seq_ctx_.init(tenant_id_/*src_tenant_id*/, dst_tenant_id_, object_id_, target_object_id_))) { LOG_WARN("failed to init sync tablet autoinc seq ctx", K(ret)); } else if (OB_FAIL(sync_tablet_autoinc_seq_ctx_.sync())) { LOG_WARN("failed to sync tablet autoinc seq", K(ret)); @@ -1906,24 +2031,30 @@ int ObDDLRedefinitionTask::get_child_task_ids(char *buf, int64_t len) } ObSyncTabletAutoincSeqCtx::ObSyncTabletAutoincSeqCtx() - : is_inited_(false), is_synced_(false), tenant_id_(OB_INVALID_ID), orig_src_tablet_ids_(), + : is_inited_(false), is_synced_(false), src_tenant_id_(OB_INVALID_ID), dst_tenant_id_(OB_INVALID_ID), orig_src_tablet_ids_(), src_tablet_ids_(), dest_tablet_ids_(), autoinc_params_() {} -int ObSyncTabletAutoincSeqCtx::init(uint64_t tenant_id, int64_t src_table_id, int64_t dest_table_id) +int ObSyncTabletAutoincSeqCtx::init( + const uint64_t src_tenant_id, + const uint64_t dst_tenant_id, + int64_t src_table_id, + int64_t dest_table_id) { int ret = OB_SUCCESS; - if (OB_UNLIKELY(tenant_id == OB_INVALID_ID || src_table_id == OB_INVALID_ID || dest_table_id == OB_INVALID_ID)) { + if (OB_UNLIKELY(OB_INVALID_ID == src_tenant_id || OB_INVALID_ID == dst_tenant_id + || src_table_id == OB_INVALID_ID || dest_table_id == OB_INVALID_ID)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(tenant_id), K(src_table_id), K(dest_table_id)); - } else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id, src_table_id, orig_src_tablet_ids_))) { + LOG_WARN("invalid argument", K(ret), K(src_tenant_id), K(dst_tenant_id), K(src_table_id), K(dest_table_id)); + } else if (OB_FAIL(ObDDLUtil::get_tablets(src_tenant_id, src_table_id, orig_src_tablet_ids_))) { LOG_WARN("failed to get data table snapshot", K(ret)); } else if (OB_FAIL(src_tablet_ids_.assign(orig_src_tablet_ids_))) { LOG_WARN("failed to assign src_tablet_ids", K(ret)); - } else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id, dest_table_id, dest_tablet_ids_))) { + } else if (OB_FAIL(ObDDLUtil::get_tablets(dst_tenant_id, dest_table_id, dest_tablet_ids_))) { LOG_WARN("failed to get dest table snapshot", K(ret)); } else { - tenant_id_ = tenant_id; + src_tenant_id_ = src_tenant_id; + dst_tenant_id_ = dst_tenant_id; is_synced_ = false; is_inited_ = true; } @@ -2020,6 +2151,7 @@ int ObSyncTabletAutoincSeqCtx::call_and_process_all_tablet_autoinc_seqs(P &proxy const int64_t tablet_count = src_tablet_ids_.count(); share::ObLocationService *location_service = nullptr; ObHashMap> ls_to_tablet_map; + const uint64_t target_tenant_id = is_get ? src_tenant_id_ : dst_tenant_id_; if (OB_ISNULL(location_service = GCTX.location_service_)) { ret = OB_ERR_SYS; LOG_WARN("location_cache is null", K(ret)); @@ -2043,7 +2175,7 @@ int ObSyncTabletAutoincSeqCtx::call_and_process_all_tablet_autoinc_seqs(P &proxy } } if (OB_SUCC(ret) && OB_FAIL(build_ls_to_tablet_map(location_service, - tenant_id_, + target_tenant_id, tmp_autoinc_params, rpc_timeout, force_renew, @@ -2053,7 +2185,7 @@ int ObSyncTabletAutoincSeqCtx::call_and_process_all_tablet_autoinc_seqs(P &proxy } } else { if (OB_FAIL(build_ls_to_tablet_map(location_service, - tenant_id_, + target_tenant_id, autoinc_params_, rpc_timeout, force_renew, @@ -2071,14 +2203,14 @@ int ObSyncTabletAutoincSeqCtx::call_and_process_all_tablet_autoinc_seqs(P &proxy const ObLSID &ls_id = map_iter->first; ObAddr leader_addr; if (OB_FAIL(location_service->get_leader(GCONF.cluster_id, - tenant_id_, + target_tenant_id, ls_id, force_renew, leader_addr))) { LOG_WARN("failed to get leader", K(ret)); - } else if (OB_FAIL(arg.init(tenant_id_, ls_id, map_iter->second))) { + } else if (OB_FAIL(arg.init(target_tenant_id, ls_id, map_iter->second))) { LOG_WARN("failed to init arg", K(ret)); - } else if (OB_FAIL(proxy.call(leader_addr, rpc_timeout, tenant_id_, arg))) { + } else if (OB_FAIL(proxy.call(leader_addr, rpc_timeout, target_tenant_id, arg))) { LOG_WARN("send rpc failed", K(ret), K(arg), K(leader_addr)); } } diff --git a/src/rootserver/ddl_task/ob_ddl_redefinition_task.h b/src/rootserver/ddl_task/ob_ddl_redefinition_task.h index 65eff717e4..6fe39c53d3 100644 --- a/src/rootserver/ddl_task/ob_ddl_redefinition_task.h +++ b/src/rootserver/ddl_task/ob_ddl_redefinition_task.h @@ -73,10 +73,14 @@ class ObSyncTabletAutoincSeqCtx final public: ObSyncTabletAutoincSeqCtx(); ~ObSyncTabletAutoincSeqCtx() {} - int init(uint64_t tenant_id, int64_t src_table_id, int64_t dest_table_id); + int init( + const uint64_t src_tenant_id, + const uint64_t dst_tenant_id, + int64_t src_table_id, + int64_t dest_table_id); int sync(); bool is_inited() const { return is_inited_; } - TO_STRING_KV(K_(is_inited), K_(is_synced), K_(tenant_id), K_(orig_src_tablet_ids), K_(src_tablet_ids), + TO_STRING_KV(K_(is_inited), K_(is_synced), K_(src_tenant_id), K_(dst_tenant_id), K_(orig_src_tablet_ids), K_(src_tablet_ids), K_(dest_tablet_ids), K_(autoinc_params)); private: int build_ls_to_tablet_map( @@ -98,7 +102,8 @@ private: static const int64_t MAP_BUCKET_NUM = 1024; bool is_inited_; bool is_synced_; - uint64_t tenant_id_; + uint64_t src_tenant_id_; + uint64_t dst_tenant_id_; ObSEArray orig_src_tablet_ids_; ObSEArray src_tablet_ids_; ObSEArray dest_tablet_ids_; @@ -113,7 +118,9 @@ public: build_replica_request_time_(0), complete_sstable_job_ret_code_(INT64_MAX), alter_table_arg_(), dependent_task_result_map_(), snapshot_held_(false), has_synced_autoincrement_(false), has_synced_stats_info_(false), update_autoinc_job_ret_code_(INT64_MAX), update_autoinc_job_time_(0), - check_table_empty_job_ret_code_(INT64_MAX), check_table_empty_job_time_(0) {} + check_table_empty_job_ret_code_(INT64_MAX), check_table_empty_job_time_(0), + is_sstable_complete_task_submitted_(false), sstable_complete_request_time_(0), replica_builder_() + {} virtual ~ObDDLRedefinitionTask() {} virtual int process() = 0; virtual int update_complete_sstable_job_status( @@ -138,7 +145,10 @@ public: protected: int prepare(const share::ObDDLTaskStatus next_task_status); int check_table_empty(const share::ObDDLTaskStatus next_task_status); - int obtain_snapshot(const share::ObDDLTaskStatus next_task_status); + virtual int obtain_snapshot(const share::ObDDLTaskStatus next_task_status); + virtual int wait_data_complement(const share::ObDDLTaskStatus next_task_status); + int send_build_single_replica_request(); + int check_build_single_replica(bool &is_end); bool check_can_validate_column_checksum( const bool is_oracle_mode, const share::schema::ObColumnSchemaV2 &src_column_schema, @@ -148,8 +158,8 @@ protected: const share::schema::ObTableSchema &dest_table_schema, common::hash::ObHashMap &validate_checksum_column_ids); int check_data_dest_tables_columns_checksum(const int64_t execution_id); - int fail(); - int success(); + virtual int fail(); + virtual int success(); int hold_snapshot(const int64_t snapshot_version); int release_snapshot(const int64_t snapshot_version); int add_constraint_ddl_task(const int64_t constraint_id); @@ -240,6 +250,9 @@ protected: int64_t update_autoinc_job_time_; int64_t check_table_empty_job_ret_code_; int64_t check_table_empty_job_time_; + bool is_sstable_complete_task_submitted_; + int64_t sstable_complete_request_time_; + ObDDLSingleReplicaExecutor replica_builder_; }; } // end namespace rootserver diff --git a/src/rootserver/ddl_task/ob_ddl_retry_task.cpp b/src/rootserver/ddl_task/ob_ddl_retry_task.cpp index 70e95370fb..1421288c7d 100644 --- a/src/rootserver/ddl_task/ob_ddl_retry_task.cpp +++ b/src/rootserver/ddl_task/ob_ddl_retry_task.cpp @@ -194,6 +194,8 @@ int ObDDLRetryTask::init(const uint64_t tenant_id, task_version_ = OB_DDL_RETRY_TASK_VERSION; task_status_ = static_cast(task_status); is_schema_change_done_ = false; + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; is_inited_ = true; ddl_tracing_.open(); } @@ -221,6 +223,8 @@ int ObDDLRetryTask::init(const ObDDLTaskRecord &task_record) ret_code_ = task_record.ret_code_; task_status_ = static_cast(task_record.task_status_); is_schema_change_done_ = false; // do not worry about it, check_schema_change_done will correct it. + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; if (nullptr != task_record.message_) { int64_t pos = 0; if (OB_FAIL(deserlize_params_from_message(task_record.tenant_id_, task_record.message_.ptr(), task_record.message_.length(), pos))) { @@ -666,7 +670,7 @@ int ObDDLRetryTask::deserlize_params_from_message(const uint64_t tenant_id, cons obrpc::ObAlterTableArg tmp_arg; if (OB_FAIL(tmp_arg.deserialize(buf, buf_size, pos))) { LOG_WARN("serialize table failed", K(ret)); - } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(tenant_id, tmp_arg))) { + } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(task_type_, tenant_id, tmp_arg))) { LOG_WARN("replace user tenant id failed", K(ret), K(tenant_id), K(tmp_arg)); } else if (OB_FAIL(deep_copy_ddl_arg(allocator_, task_type_, &tmp_arg))) { LOG_WARN("deep copy table arg failed", K(ret)); diff --git a/src/rootserver/ddl_task/ob_ddl_scheduler.cpp b/src/rootserver/ddl_task/ob_ddl_scheduler.cpp index 2db4266889..c4b638f8a8 100755 --- a/src/rootserver/ddl_task/ob_ddl_scheduler.cpp +++ b/src/rootserver/ddl_task/ob_ddl_scheduler.cpp @@ -26,6 +26,7 @@ #include "rootserver/ddl_task/ob_index_build_task.h" #include "rootserver/ddl_task/ob_modify_autoinc_task.h" #include "rootserver/ddl_task/ob_table_redefinition_task.h" +#include "rootserver/ddl_task/ob_recover_restore_table_task.h" #include "share/ob_ddl_common.h" #include "share/ob_rpc_struct.h" #include "share/longops_mgr/ob_longops_mgr.h" @@ -907,6 +908,20 @@ int ObDDLScheduler::create_ddl_task(const ObCreateDDLTaskParam ¶m, LOG_WARN("fail to create table redefinition task", K(ret)); } break; + case DDL_TABLE_RESTORE: + if (OB_FAIL(create_recover_restore_table_task(proxy, + param.type_, + param.src_table_schema_, + param.dest_table_schema_, + param.parallelism_, + param.consumer_group_id_, + param.task_id_, + static_cast(param.ddl_arg_), + *param.allocator_, + task_record))) { + LOG_WARN("fail to create recover restore table task", K(ret)); + } + break; case DDL_DROP_PRIMARY_KEY: alter_table_arg = static_cast(param.ddl_arg_); if (OB_FAIL(create_drop_primary_key_task(proxy, @@ -1460,11 +1475,13 @@ int ObDDLScheduler::create_table_redefinition_task( ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(task_id), KP(alter_table_arg), KP(src_schema), KP(dest_schema)); } else if (OB_FAIL(redefinition_task.init(src_schema->get_tenant_id(), + dest_schema->get_tenant_id(), task_id, type, src_schema->get_table_id(), dest_schema->get_table_id(), dest_schema->get_schema_version(), + dest_schema->get_schema_version(), parallelism, consumer_group_id, *alter_table_arg))) { @@ -1627,6 +1644,48 @@ int ObDDLScheduler::create_ddl_retry_task( return ret; } +int ObDDLScheduler::create_recover_restore_table_task( + common::ObISQLClient &proxy, + const share::ObDDLType &type, + const share::schema::ObTableSchema *src_schema, + const share::schema::ObTableSchema *dest_schema, + const int64_t parallelism, + const int64_t consumer_group_id, + const int64_t task_id, + const obrpc::ObAlterTableArg *alter_table_arg, + ObIAllocator &allocator, + ObDDLTaskRecord &task_record) +{ + int ret = OB_SUCCESS; + SMART_VAR(ObRecoverRestoreTableTask, redefinition_task) { + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObDDLScheduler has not been inited", K(ret)); + } else if (OB_UNLIKELY(0 == task_id) || OB_ISNULL(alter_table_arg) || OB_ISNULL(src_schema) || OB_ISNULL(dest_schema)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments", K(ret), K(task_id), KP(alter_table_arg), KP(src_schema), KP(dest_schema)); + } else if (OB_FAIL(redefinition_task.init(src_schema->get_tenant_id(), + dest_schema->get_tenant_id(), + task_id, + type, + src_schema->get_table_id(), + dest_schema->get_table_id(), + src_schema->get_schema_version(), + dest_schema->get_schema_version(), + parallelism, + consumer_group_id, + *alter_table_arg))) { + LOG_WARN("fail to init redefinition task", K(ret)); + } else if (OB_FAIL(redefinition_task.set_trace_id(*ObCurTraceId::get_trace_id()))) { + LOG_WARN("set trace id failed", K(ret)); + } else if (OB_FAIL(insert_task_record(proxy, redefinition_task, allocator, task_record))) { + LOG_WARN("fail to insert task record", K(ret)); + } + LOG_INFO("ddl_scheduler create table redefinition task finished", K(ret), K(redefinition_task), K(common::lbt())); + } + return ret; +} + int ObDDLScheduler::insert_task_record( common::ObISQLClient &proxy, ObDDLTask &ddl_task, @@ -1806,6 +1865,9 @@ int ObDDLScheduler::schedule_ddl_task(const ObDDLTaskRecord &record) case DDL_TRUNCATE_SUB_PARTITION: ret = schedule_ddl_retry_task(record); break; + case DDL_TABLE_RESTORE: + ret = schedule_recover_restore_table_task(record); + break; default: { ret = OB_NOT_SUPPORTED; LOG_WARN("not supported task type", K(ret), K(record)); @@ -2034,6 +2096,35 @@ int ObDDLScheduler::schedule_drop_index_task(const ObDDLTaskRecord &task_record) return ret; } +int ObDDLScheduler::schedule_recover_restore_table_task(const ObDDLTaskRecord &task_record) +{ + int ret = OB_SUCCESS; + ObRecoverRestoreTableTask *redefinition_task = nullptr; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObDDLScheduler has not been inited", K(ret)); + } else if (OB_FAIL(alloc_ddl_task(redefinition_task))) { + LOG_WARN("alloc ddl task failed", K(ret)); + } else if (OB_FAIL(redefinition_task->init(task_record))) { + LOG_WARN("init table redefinition task failed", K(ret)); + } else if (OB_FAIL(redefinition_task->set_trace_id(task_record.trace_id_))) { + LOG_WARN("set trace id failed", K(ret)); + } else if (OB_FAIL(inner_schedule_ddl_task(redefinition_task, task_record))) { + if (OB_ENTRY_EXIST != ret) { + LOG_WARN("inner schedule task failed", K(ret), K(*redefinition_task)); + } + } else if (ObDDLTask::check_is_load_data(task_record.ddl_type_) + && OB_FAIL(manager_reg_heart_beat_task_.update_task_active_time(ObDDLTaskID(task_record.tenant_id_, task_record.task_id_)))) { + LOG_WARN("register_task_time recover fail", K(ret)); + } + if (OB_FAIL(ret) && nullptr != redefinition_task) { + redefinition_task->~ObRecoverRestoreTableTask(); + allocator_.free(redefinition_task); + redefinition_task = nullptr; + } + return ret; +} + int ObDDLScheduler::add_task_to_longops_mgr(ObDDLTask *ddl_task) { int ret = OB_SUCCESS; @@ -2211,6 +2302,11 @@ int ObDDLScheduler::on_sstable_complement_job_reply( LOG_WARN("update complete sstable job status", K(ret)); } break; + case ObDDLType::DDL_TABLE_RESTORE: + if (OB_FAIL(static_cast(&task)->update_complete_sstable_job_status(tablet_id, snapshot_version, execution_id, ret_code, addition_info))) { + LOG_WARN("update complete sstable job status", K(ret)); + } + break; case ObDDLType::DDL_CHECK_CONSTRAINT: case ObDDLType::DDL_FOREIGN_KEY_CONSTRAINT: case ObDDLType::DDL_ADD_NOT_NULL_COLUMN: diff --git a/src/rootserver/ddl_task/ob_ddl_scheduler.h b/src/rootserver/ddl_task/ob_ddl_scheduler.h index 02989dc7a2..386a7472ff 100755 --- a/src/rootserver/ddl_task/ob_ddl_scheduler.h +++ b/src/rootserver/ddl_task/ob_ddl_scheduler.h @@ -406,6 +406,19 @@ private: const obrpc::ObDDLArg *arg, ObIAllocator &allocator, ObDDLTaskRecord &task_record); + + int create_recover_restore_table_task( + common::ObISQLClient &proxy, + const share::ObDDLType &type, + const share::schema::ObTableSchema *src_schema, + const share::schema::ObTableSchema *dest_schema, + const int64_t parallelism, + const int64_t consumer_group_id, + const int64_t task_id, + const obrpc::ObAlterTableArg *alter_table_arg, + ObIAllocator &allocator, + ObDDLTaskRecord &task_record); + int schedule_build_index_task( const ObDDLTaskRecord &task_record); int schedule_drop_primary_key_task(const ObDDLTaskRecord &task_record); @@ -415,6 +428,7 @@ private: int schedule_modify_autoinc_task(const ObDDLTaskRecord &task_record); int schedule_drop_index_task(const ObDDLTaskRecord &task_record); int schedule_ddl_retry_task(const ObDDLTaskRecord &task_record); + int schedule_recover_restore_table_task(const ObDDLTaskRecord &task_record); int add_sys_task(ObDDLTask *task); int remove_sys_task(ObDDLTask *task); int add_task_to_longops_mgr(ObDDLTask *ddl_task); diff --git a/src/rootserver/ddl_task/ob_ddl_single_replica_executor.cpp b/src/rootserver/ddl_task/ob_ddl_single_replica_executor.cpp index 4555b7873d..d2454b6a3f 100644 --- a/src/rootserver/ddl_task/ob_ddl_single_replica_executor.cpp +++ b/src/rootserver/ddl_task/ob_ddl_single_replica_executor.cpp @@ -38,10 +38,12 @@ int ObDDLSingleReplicaExecutor::build(const ObDDLSingleReplicaExecutorParam &par LOG_WARN("fail to assign tablet ids", K(ret)); } else { tenant_id_ = param.tenant_id_; + dest_tenant_id_ = param.dest_tenant_id_; type_ = param.type_; source_table_id_ = param.source_table_id_; dest_table_id_ = param.dest_table_id_; schema_version_ = param.schema_version_; + dest_schema_version_ = param.dest_schema_version_; snapshot_version_ = param.snapshot_version_; task_id_ = param.task_id_; execution_id_ = param.execution_id_; @@ -98,63 +100,75 @@ int ObDDLSingleReplicaExecutor::schedule_task() const bool force_renew = true; bool is_cache_hit = false; const int64_t expire_renew_time = force_renew ? INT64_MAX : 0; - ObLSID ls_id; + ObLSID orig_ls_id; + ObLSID dest_ls_id; common::ObArray ret_array; ObArray request_source_tablet_ids; ObArray request_dest_tablet_ids; ObArray request_tablet_task_ids; int tmp_ret = OB_SUCCESS; { - { - ObSpinLockGuard guard(lock_); - // send rpc request may cost too much time, thus set some status before rpc request under lock. - for (int64_t i = 0; OB_SUCC(ret) && i < build_infos.count(); i++) { - ObPartitionBuildInfo &build_info = build_infos.at(i); - if (ObPartitionBuildStat::BUILD_INIT == build_info.stat_ || build_info.need_schedule()) { - if (OB_FAIL(request_source_tablet_ids.push_back(source_tablet_ids_.at(i)))) { - LOG_WARN("push backed failed", K(ret)); - } else if (OB_FAIL(request_dest_tablet_ids.push_back(dest_tablet_ids_.at(i)))) { - LOG_WARN("push back failed", K(ret)); - } else if (OB_FAIL(request_tablet_task_ids.push_back(tablet_task_ids_.at(i)))) { - LOG_WARN("push back failed", K(ret)); - } else { - build_info.stat_ = ObPartitionBuildStat::BUILD_INIT; - } + ObSpinLockGuard guard(lock_); + // send rpc request may cost too much time, thus set some status before rpc request under lock. + for (int64_t i = 0; OB_SUCC(ret) && i < build_infos.count(); i++) { + ObPartitionBuildInfo &build_info = build_infos.at(i); + if (ObPartitionBuildStat::BUILD_INIT == build_info.stat_|| build_info.need_schedule()) { + if (OB_FAIL(request_source_tablet_ids.push_back(source_tablet_ids_.at(i)))) { + LOG_WARN("push backed failed", K(ret)); + } else if (OB_FAIL(request_dest_tablet_ids.push_back(dest_tablet_ids_.at(i)))) { + LOG_WARN("push back failed", K(ret)); + } else if (OB_FAIL(request_tablet_task_ids.push_back(tablet_task_ids_.at(i)))) { + LOG_WARN("push back failed", K(ret)); + } else { + build_info.stat_ = ObPartitionBuildStat::BUILD_INIT; } } } - - for (int64_t i = 0; OB_SUCC(ret) && i < request_source_tablet_ids.count(); ++i) { - ObAddr leader_addr; - obrpc::ObDDLBuildSingleReplicaRequestArg arg; - arg.ls_id_ = share::ObLSID::INVALID_LS_ID; - arg.tenant_id_ = tenant_id_; - arg.source_tablet_id_ = request_source_tablet_ids.at(i); - arg.dest_tablet_id_ = request_dest_tablet_ids.at(i); - arg.source_table_id_ = source_table_id_; - arg.dest_schema_id_ = dest_table_id_; - arg.schema_version_ = schema_version_; - arg.snapshot_version_ = snapshot_version_; - arg.ddl_type_ = type_; - arg.task_id_ = task_id_; - arg.parallelism_ = parallelism_; - arg.execution_id_ = execution_id_; - arg.data_format_version_ = data_format_version_; - arg.tablet_task_id_ = request_tablet_task_ids.at(i); - arg.consumer_group_id_ = consumer_group_id_; - if (OB_FAIL(location_service->get(tenant_id_, arg.source_tablet_id_, - expire_renew_time, is_cache_hit, ls_id))) { - LOG_WARN("get ls failed", K(ret), K(arg.source_tablet_id_)); - } else if (OB_FAIL(location_service->get_leader(GCONF.cluster_id, tenant_id_, ls_id, force_renew, leader_addr))) { - LOG_WARN("get leader failed", K(ret), K(tenant_id_), K(ls_id)); - } else if (FALSE_IT(arg.ls_id_ = ls_id)) { - } else if (OB_FAIL(proxy.call(leader_addr, rpc_timeout, tenant_id_, arg))) { - LOG_WARN("fail to send rpc", K(ret), K(rpc_timeout)); - } else if (OB_FAIL(idxs.push_back(i))) { - LOG_WARN("fail to push back idx", K(ret)); - } else { - LOG_INFO("send build single replica request", K(arg)); - } + } + for (int64_t i = 0; OB_SUCC(ret) && i < request_source_tablet_ids.count(); ++i) { + // get leader of partition + ObAddr orig_leader_addr; + ObAddr dest_leader_addr; + obrpc::ObDDLBuildSingleReplicaRequestArg arg; + arg.ls_id_ = share::ObLSID::INVALID_LS_ID; + arg.dest_ls_id_ = share::ObLSID::INVALID_LS_ID; + arg.tenant_id_ = tenant_id_; + arg.dest_tenant_id_ = dest_tenant_id_; + arg.source_tablet_id_ = request_source_tablet_ids.at(i); + arg.dest_tablet_id_ = request_dest_tablet_ids.at(i); + arg.source_table_id_ = source_table_id_; + arg.dest_schema_id_ = dest_table_id_; + arg.schema_version_ = schema_version_; + arg.dest_schema_version_ = dest_schema_version_; + arg.snapshot_version_ = snapshot_version_; + arg.ddl_type_ = type_; + arg.task_id_ = task_id_; + arg.parallelism_ = parallelism_; + arg.execution_id_ = execution_id_; + arg.data_format_version_ = data_format_version_; + arg.tablet_task_id_ = request_tablet_task_ids.at(i); + arg.consumer_group_id_ = consumer_group_id_; + if (OB_FAIL(location_service->get(tenant_id_, arg.source_tablet_id_, + expire_renew_time, is_cache_hit, orig_ls_id))) { + LOG_WARN("get ls failed", K(ret), K(arg.source_tablet_id_)); + } else if (OB_FAIL(location_service->get_leader(GCONF.cluster_id, tenant_id_, orig_ls_id, force_renew, orig_leader_addr))) { + LOG_WARN("get leader failed", K(ret), K(tenant_id_), K(orig_ls_id)); + } else if (FALSE_IT(arg.ls_id_ = orig_ls_id)) { + } else if (OB_FAIL(location_service->get(dest_tenant_id_, arg.dest_tablet_id_, + expire_renew_time, is_cache_hit, dest_ls_id))) { + LOG_WARN("get ls failed", K(ret), K(arg.dest_tablet_id_)); + } else if (OB_FAIL(location_service->get_leader(GCONF.cluster_id, dest_tenant_id_, dest_ls_id, force_renew, dest_leader_addr))) { + LOG_WARN("get leader failed", K(ret), K(dest_tenant_id_), K(dest_ls_id)); + } else if (ObDDLType::DDL_TABLE_RESTORE != arg.ddl_type_ && orig_leader_addr != dest_leader_addr) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("orig leader addr not equal dest leader addr", K(ret), K(orig_leader_addr), K(dest_leader_addr), K(arg)); + } else if (FALSE_IT(arg.dest_ls_id_ = dest_ls_id)) { + } else if (OB_FAIL(proxy.call(dest_leader_addr, rpc_timeout, dest_tenant_id_, arg))) { + LOG_WARN("fail to send rpc", K(ret), K(rpc_timeout)); + } else if (OB_FAIL(idxs.push_back(i))) { + LOG_WARN("fail to push back idx", K(ret)); + } else { + LOG_INFO("send build single replica request", K(arg), K(dest_leader_addr)); } } if (OB_SUCCESS != (tmp_ret = proxy.wait_all(ret_array))) { @@ -221,9 +235,9 @@ int ObDDLSingleReplicaExecutor::check_build_end(bool &is_end, int64_t &ret_code) } if (OB_SUCC(ret) && build_infos.count() == succ_cnt) { if (OB_FAIL(ObCheckTabletDataComplementOp::check_finish_report_checksum( - tenant_id_, dest_table_id_, execution_id_, task_id_))) { + dest_tenant_id_, dest_table_id_, execution_id_, task_id_))) { LOG_WARN("fail to check sstable checksum_report_finish", - K(ret), K(tenant_id_), K(dest_table_id_), K(execution_id_), K(task_id_)); + K(ret), K(dest_tenant_id_), K(dest_table_id_), K(execution_id_), K(task_id_)); } is_end = true; ret_code = ret; @@ -245,13 +259,13 @@ int ObDDLSingleReplicaExecutor::set_partition_task_status(const common::ObTablet const int64_t row_scanned, const int64_t row_inserted) { + LOG_INFO("ObDDLSingleReplicaExecutor::set_partition_task_status", K(ret_code)); int ret = OB_SUCCESS; common::ObIArray &build_infos = partition_build_stat_; if (OB_UNLIKELY(!tablet_id.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(tablet_id)); } else { - LOG_INFO("receive tablet task status", K(tablet_id), K(ret_code)); ObSpinLockGuard guard(lock_); for (int64_t i = 0; OB_SUCC(ret) && i < source_tablet_ids_.count(); ++i) { if (tablet_id == source_tablet_ids_.at(i)) { diff --git a/src/rootserver/ddl_task/ob_ddl_single_replica_executor.h b/src/rootserver/ddl_task/ob_ddl_single_replica_executor.h index 7045efd3cc..2c1c472238 100644 --- a/src/rootserver/ddl_task/ob_ddl_single_replica_executor.h +++ b/src/rootserver/ddl_task/ob_ddl_single_replica_executor.h @@ -27,12 +27,14 @@ struct ObDDLSingleReplicaExecutorParam final public: ObDDLSingleReplicaExecutorParam() : tenant_id_(common::OB_INVALID_TENANT_ID), + dest_tenant_id_(common::OB_INVALID_TENANT_ID), type_(share::DDL_INVALID), source_tablet_ids_(), dest_tablet_ids_(), source_table_id_(common::OB_INVALID_ID), dest_table_id_(common::OB_INVALID_ID), schema_version_(0), + dest_schema_version_(0), snapshot_version_(0), task_id_(0), parallelism_(0), @@ -42,24 +44,26 @@ public: {} ~ObDDLSingleReplicaExecutorParam() = default; bool is_valid() const { - return common::OB_INVALID_TENANT_ID != tenant_id_ && share::DDL_INVALID != type_ - && source_tablet_ids_.count() > 0 && dest_tablet_ids_.count() > 0 + return common::OB_INVALID_TENANT_ID != tenant_id_ && common::OB_INVALID_TENANT_ID != dest_tenant_id_ + && share::DDL_INVALID != type_ && source_tablet_ids_.count() > 0 && dest_tablet_ids_.count() > 0 && common::OB_INVALID_ID != source_table_id_ && common::OB_INVALID_ID != dest_table_id_ - && schema_version_ > 0 && snapshot_version_ > 0 && task_id_ > 0 && execution_id_ >= 0 - && data_format_version_ > 0 && consumer_group_id_ >= 0; + && schema_version_ > 0 && dest_schema_version_ > 0 && snapshot_version_ > 0 && task_id_ > 0 + && execution_id_ >= 0 && data_format_version_ > 0 && consumer_group_id_ >= 0; } - TO_STRING_KV(K_(tenant_id), K_(type), K_(source_tablet_ids), K_(dest_tablet_ids), - K_(source_table_id), K_(dest_table_id), K_(schema_version), + TO_STRING_KV(K_(tenant_id), K_(dest_tenant_id), K_(type), K_(source_tablet_ids), K_(dest_tablet_ids), + K_(source_table_id), K_(dest_table_id), K_(schema_version), K_(dest_schema_version), K_(snapshot_version), K_(task_id), K_(parallelism), K_(execution_id), K_(data_format_version), K_(consumer_group_id)); public: uint64_t tenant_id_; + uint64_t dest_tenant_id_; share::ObDDLType type_; common::ObArray source_tablet_ids_; common::ObArray dest_tablet_ids_; int64_t source_table_id_; int64_t dest_table_id_; int64_t schema_version_; + int64_t dest_schema_version_; int64_t snapshot_version_; int64_t task_id_; int64_t parallelism_; @@ -113,6 +117,7 @@ private: }; private: uint64_t tenant_id_; + uint64_t dest_tenant_id_; share::ObDDLType type_; common::ObArray source_tablet_ids_; common::ObArray dest_tablet_ids_; @@ -120,6 +125,7 @@ private: int64_t source_table_id_; int64_t dest_table_id_; int64_t schema_version_; + int64_t dest_schema_version_; int64_t snapshot_version_; int64_t task_id_; int64_t parallelism_; diff --git a/src/rootserver/ddl_task/ob_ddl_task.cpp b/src/rootserver/ddl_task/ob_ddl_task.cpp index a796c2fc17..be5bd3b9e1 100644 --- a/src/rootserver/ddl_task/ob_ddl_task.cpp +++ b/src/rootserver/ddl_task/ob_ddl_task.cpp @@ -743,6 +743,9 @@ int ObDDLTask::get_ddl_type_str(const int64_t ddl_type, const char *&ddl_type_st case DDL_DIRECT_LOAD_INSERT: ddl_type_str = "direct load insert"; break; + case DDL_TABLE_RESTORE: + ddl_type_str = "recover table restore ddl"; + break; case DDL_MODIFY_AUTO_INCREMENT: ddl_type_str = "modify auto increment"; break; @@ -884,10 +887,10 @@ int ObDDLTask::convert_to_record( const int64_t serialize_param_size = get_serialize_param_size(); int64_t pos = 0; task_record.gmt_create_ = get_gmt_create(); - task_record.tenant_id_ = get_tenant_id(); + task_record.tenant_id_ = get_tenant_id(); //dest tenant id. task_record.object_id_ = get_object_id(); task_record.target_object_id_ = get_target_object_id(); - task_record.schema_version_ = get_schema_version(); + task_record.schema_version_ = get_schema_version(); // dest schema version. task_record.ddl_type_ = get_task_type(); task_record.trace_id_ = get_trace_id(); task_record.task_status_ = get_task_status(); @@ -976,22 +979,22 @@ int ObDDLTask::switch_status(const ObDDLTaskStatus new_status, const bool enable if (OB_ISNULL(root_service = GCTX.root_service_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, root service must not be nullptr", K(ret)); - } else if (OB_FAIL(root_service->get_schema_service().check_if_tenant_has_been_dropped(tenant_id_, is_tenant_dropped))) { - LOG_WARN("check if tenant has been dropped failed", K(ret), K(tenant_id_)); + } else if (OB_FAIL(root_service->get_schema_service().check_if_tenant_has_been_dropped(dst_tenant_id_, is_tenant_dropped))) { + LOG_WARN("check if tenant has been dropped failed", K(ret), K(dst_tenant_id_)); } else if (is_tenant_dropped) { need_retry_ = false; - LOG_INFO("tenant has been dropped, exit anyway", K(ret), K(task_id_), K(parent_task_id_), K(tenant_id_)); - } else if (OB_FAIL(ObAllTenantInfoProxy::is_standby_tenant(&root_service->get_sql_proxy(), tenant_id_, is_standby_tenant))) { - LOG_WARN("check is standby tenant failed", K(ret), K(tenant_id_)); + LOG_INFO("tenant has been dropped, exit anyway", K(ret), K(task_id_), K(parent_task_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(ObAllTenantInfoProxy::is_standby_tenant(&root_service->get_sql_proxy(), dst_tenant_id_, is_standby_tenant))) { + LOG_WARN("check is standby tenant failed", K(ret), K(dst_tenant_id_)); } else if (is_standby_tenant) { need_retry_ = false; - LOG_INFO("tenant is standby, exit anyway", K(ret), K(task_id_), K(parent_task_id_), K(tenant_id_)); - } else if (OB_FAIL(trans.start(&root_service->get_sql_proxy(), tenant_id_))) { + LOG_INFO("tenant is standby, exit anyway", K(ret), K(task_id_), K(parent_task_id_), K(dst_tenant_id_)); + } else if (OB_FAIL(trans.start(&root_service->get_sql_proxy(), dst_tenant_id_))) { LOG_WARN("start transaction failed", K(ret)); } else { int64_t table_task_status = 0; int64_t execution_id = -1; - if (OB_FAIL(ObDDLTaskRecordOperator::select_for_update(trans, tenant_id_, task_id_, table_task_status, execution_id))) { + if (OB_FAIL(ObDDLTaskRecordOperator::select_for_update(trans, dst_tenant_id_, task_id_, table_task_status, execution_id))) { if (OB_ENTRY_NOT_EXIST == ret) { need_retry_ = false; } @@ -1012,11 +1015,11 @@ int ObDDLTask::switch_status(const ObDDLTaskStatus new_status, const bool enable ret = update_task_record_status_and_msg(trans, real_new_status); } } else if (OB_FAIL(ObDDLTaskRecordOperator::update_task_status( - trans, tenant_id_, task_id_, static_cast(real_new_status)))) { + trans, dst_tenant_id_, task_id_, static_cast(real_new_status)))) { LOG_WARN("update task status failed", K(ret), K(task_id_), K(real_new_status)); } if (OB_SUCC(ret)) { - if (OB_FAIL(ObDDLTaskRecordOperator::update_ret_code(trans, tenant_id_, task_id_, ret_code_))) { + if (OB_FAIL(ObDDLTaskRecordOperator::update_ret_code(trans, dst_tenant_id_, task_id_, ret_code_))) { LOG_WARN("failed to update ret code", K(ret)); } } @@ -1030,10 +1033,10 @@ int ObDDLTask::switch_status(const ObDDLTaskStatus new_status, const bool enable if (OB_SUCC(ret) && old_status != real_new_status) { const char *status_str = ddl_task_status_to_str(real_new_status); if (status_str) { - ROOTSERVICE_EVENT_ADD("ddl_scheduler", "switch_state", K_(tenant_id), K_(task_id), K_(object_id), K_(target_object_id), + ROOTSERVICE_EVENT_ADD("ddl_scheduler", "switch_state", "tenant_id", dst_tenant_id_, K_(task_id), K_(object_id), K_(target_object_id), "new_state", status_str, K_(snapshot_version), ret_code_); } else { - ROOTSERVICE_EVENT_ADD("ddl_scheduler", "switch_state", K_(tenant_id), K_(task_id), K_(object_id), K_(target_object_id), + ROOTSERVICE_EVENT_ADD("ddl_scheduler", "switch_state", "tenant_id", dst_tenant_id_, K_(task_id), K_(object_id), K_(target_object_id), "new_state", real_new_status, K_(snapshot_version), ret_code_); } task_status_ = real_new_status; @@ -1041,7 +1044,7 @@ int ObDDLTask::switch_status(const ObDDLTaskStatus new_status, const bool enable if (OB_CANCELED == real_ret_code) { (void)ObDDLTaskRecordOperator::kill_task_inner_sql(root_service->get_sql_proxy(), - trace_id_, tenant_id_, task_id_, snapshot_version_, sql_exec_addr_); // ignore return code + trace_id_, dst_tenant_id_, task_id_, snapshot_version_, sql_exec_addr_); // ignore return code LOG_WARN("ddl_task switch_status kill_task_inner_sql"); } } @@ -1089,9 +1092,9 @@ int ObDDLTask::remove_task_record() ret = OB_ERR_SYS; LOG_WARN("error sys, root service must not be nullptr", K(ret)); } else if (OB_FAIL(ObDDLTaskRecordOperator::delete_record(root_service->get_sql_proxy(), - tenant_id_, + dst_tenant_id_, task_id_))) { - LOG_WARN("delete record failed", K(ret), K(task_id_)); + LOG_WARN("delete record failed", K(ret), K(dst_tenant_id_), K(task_id_)); } return ret; } @@ -1113,9 +1116,9 @@ int ObDDLTask::report_error_code(const ObString &forward_user_message, const int error_message.affected_rows_ = affected_rows; const bool is_ddl_retry_task = is_drop_schema_block_concurrent_trans(task_type_); if (OB_SUCCESS != ret_code_) { - if (OB_FAIL(ObDDLErrorMessageTableOperator::load_ddl_user_error(tenant_id_, task_id_, object_id_, + if (OB_FAIL(ObDDLErrorMessageTableOperator::load_ddl_user_error(dst_tenant_id_, task_id_, object_id_, *GCTX.sql_proxy_, error_message))) { - LOG_WARN("load ddl user error failed", K(ret), K(tenant_id_), K(task_id_), K(object_id_)); + LOG_WARN("load ddl user error failed", K(ret), K(dst_tenant_id_), K(task_id_), K(object_id_)); if (OB_ITER_END == ret) { // no single replica error message found, use ret_code_ ret = OB_SUCCESS; if (is_oracle_mode && DDL_CREATE_INDEX != task_type_ && OB_ERR_DUPLICATED_UNIQUE_KEY == ret_code_) { @@ -1167,7 +1170,7 @@ int ObDDLTask::report_error_code(const ObString &forward_user_message, const int } if (OB_SUCC(ret)) { - if (OB_FAIL(ObDDLErrorMessageTableOperator::report_ddl_error_message(error_message, tenant_id_, task_id_, + if (OB_FAIL(ObDDLErrorMessageTableOperator::report_ddl_error_message(error_message, dst_tenant_id_, task_id_, target_object_id_, schema_version_, -1/*object id*/, GCTX.self_addr(), GCTX.root_service_->get_sql_proxy()))) { LOG_WARN("report ddl error message failed", K(ret)); } @@ -2012,11 +2015,6 @@ int ObDDLWaitTransEndCtx::get_snapshot(int64_t &snapshot_version) { int ret = OB_SUCCESS; snapshot_version = 0; - ObRootService *root_service = nullptr; - ObFreezeInfoProxy freeze_info_proxy(tenant_id_); - ObSimpleFrozenStatus frozen_status; - const int64_t timeout_us = ObDDLUtil::get_default_ddl_rpc_timeout(); - SCN curr_ts; bool is_external_consistent = false; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; @@ -2024,42 +2022,20 @@ int ObDDLWaitTransEndCtx::get_snapshot(int64_t &snapshot_version) } else if (!is_trans_end_) { ret = OB_STATE_NOT_MATCH; LOG_WARN("not all transactions are end", K(ret)); - } else if (OB_ISNULL(root_service = GCTX.root_service_)) { - ret = OB_ERR_SYS; - LOG_WARN("root service is null", K(ret), KP(root_service)); } else { - { - MAKE_TENANT_SWITCH_SCOPE_GUARD(tenant_guard); - // ignore return, MTL is only used in get_ts_sync, which will handle switch failure. - // for performance, everywhere calls get_ts_sync should ensure using correct tenant ctx - tenant_guard.switch_to(tenant_id_); - if (OB_FAIL(OB_TS_MGR.get_ts_sync(tenant_id_, - timeout_us, - curr_ts, - is_external_consistent))) { - LOG_WARN("fail to get gts sync", K(ret), K(tenant_id_), K(timeout_us), K(curr_ts), K(is_external_consistent)); + int64_t max_snapshot = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < snapshot_array_.count(); ++i) { + int64_t cur_snapshot = snapshot_array_.at(i); + if (0 >= cur_snapshot) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("current snapshot is invalid", K(ret), K(cur_snapshot)); + } else { + max_snapshot = max(max_snapshot, cur_snapshot); } } if (OB_SUCC(ret)) { - int64_t max_snapshot = 0; - for (int64_t i = 0; OB_SUCC(ret) && i < snapshot_array_.count(); ++i) { - int64_t cur_snapshot = snapshot_array_.at(i); - if (0 >= cur_snapshot) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("current snapshot is invalid", K(ret), K(cur_snapshot)); - } else { - max_snapshot = max(max_snapshot, cur_snapshot); - } - } - if (OB_SUCC(ret)) { - snapshot_version = max(max_snapshot, curr_ts.get_val_for_tx() - INDEX_SNAPSHOT_VERSION_DIFF); - if (OB_FAIL(freeze_info_proxy.get_freeze_info( - root_service->get_sql_proxy(), SCN::min_scn(), frozen_status))) { - LOG_WARN("get freeze info failed", K(ret)); - } else { - const int64_t frozen_scn_val = frozen_status.frozen_scn_.get_val_for_tx(); - snapshot_version = max(snapshot_version, frozen_scn_val); - } + if (OB_FAIL(calc_snapshot_with_gts(tenant_id_, max_snapshot, snapshot_version))) { + LOG_WARN("calc snapshot with gts failed", K(ret), K(tenant_id_), K(max_snapshot), K(snapshot_version)); } } } @@ -2072,6 +2048,52 @@ bool ObDDLWaitTransEndCtx::is_wait_trans_type_valid(const WaitTransType wait_tra && wait_trans_type < WaitTransType::MAX_WAIT_TYPE; } +int ObDDLWaitTransEndCtx::calc_snapshot_with_gts( + const uint64_t tenant_id, + const int64_t trans_end_snapshot, + int64_t &snapshot) +{ + int ret = OB_SUCCESS; + snapshot = 0; + SCN curr_ts; + bool is_external_consistent = false; + ObRootService *root_service = nullptr; + const int64_t timeout_us = ObDDLUtil::get_default_ddl_rpc_timeout(); + ObFreezeInfoProxy freeze_info_proxy(tenant_id); + ObSimpleFrozenStatus frozen_status; + if (OB_UNLIKELY(tenant_id == common::OB_INVALID_ID)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(tenant_id)); + } else if (OB_ISNULL(root_service = GCTX.root_service_)) { + ret = OB_ERR_SYS; + LOG_WARN("root service is null", K(ret), KP(root_service)); + } else { + { + MAKE_TENANT_SWITCH_SCOPE_GUARD(tenant_guard); + // ignore return, MTL is only used in get_ts_sync, which will handle switch failure. + // for performance, everywhere calls get_ts_sync should ensure using correct tenant ctx + tenant_guard.switch_to(tenant_id); + if (OB_FAIL(OB_TS_MGR.get_ts_sync(tenant_id, + timeout_us, + curr_ts, + is_external_consistent))) { + LOG_WARN("fail to get gts sync", K(ret), K(tenant_id), K(timeout_us), K(curr_ts), K(is_external_consistent)); + } + } + if (OB_SUCC(ret)) { + snapshot = max(trans_end_snapshot, curr_ts.get_val_for_tx() - INDEX_SNAPSHOT_VERSION_DIFF); + if (OB_FAIL(freeze_info_proxy.get_freeze_info( + root_service->get_sql_proxy(), SCN::min_scn(), frozen_status))) { + LOG_WARN("get freeze info failed", K(ret)); + } else { + const int64_t frozen_scn_val = frozen_status.frozen_scn_.get_val_for_tx(); + snapshot = max(snapshot, frozen_scn_val); + } + } + } + return ret; +} + int ObDDLWaitTransEndCtx::get_snapshot_check_list( ObIArray &need_check_tablets, ObIArray &tablet_pos_indexes) @@ -2465,6 +2487,7 @@ bool ObDDLTaskRecord::is_valid() const && tenant_id_ > 0 && task_version_ > 0 && OB_INVALID_ID != object_id_ + && OB_INVALID_ID != target_object_id_ && schema_version_ > 0 && execution_id_ >= -1; return is_valid; diff --git a/src/rootserver/ddl_task/ob_ddl_task.h b/src/rootserver/ddl_task/ob_ddl_task.h index 06afd3f21a..9d3fa8f91e 100755 --- a/src/rootserver/ddl_task/ob_ddl_task.h +++ b/src/rootserver/ddl_task/ob_ddl_task.h @@ -39,7 +39,7 @@ public: uint64_t hash() const; int hash(uint64_t &hash_val) const { hash_val = hash(); return OB_SUCCESS; } bool operator==(const ObDDLTaskKey &other) const; - bool is_valid() const { return OB_INVALID_TENANT_ID != tenant_id_ && OB_INVALID_ID != object_id_ && schema_version_ > 0; } + bool is_valid() const { return OB_INVALID_TENANT_ID != tenant_id_ && OB_INVALID_ID != object_id_ && schema_version_ > 0;} int assign(const ObDDLTaskKey &other); TO_STRING_KV(K_(tenant_id), K_(object_id), K_(schema_version)); public: @@ -328,6 +328,17 @@ public: TO_STRING_KV(K(is_inited_), K_(tenant_id), K(table_id_), K(is_trans_end_), K(wait_type_), K(wait_version_), K_(pending_tx_id), K(tablet_ids_.count()), K(snapshot_array_.count())); +public: + /** + * To calculate the final snapshot version used for writing macro block. + * @param [in] tenant_id + * @param [in] trans_end_snapshot: usually the snapshot version obtained after wait trans end. + * @param [out] snapshot: used for data scan, row trans version for ddl. + */ + static int calc_snapshot_with_gts( + const uint64_t tenant_id, + const int64_t trans_end_snapshot, + int64_t &snapshot); private: static bool is_wait_trans_type_valid(const WaitTransType wait_trans_type); int get_snapshot_check_list( @@ -451,7 +462,7 @@ class ObDDLTask : public common::ObDLinkBase public: explicit ObDDLTask(const share::ObDDLType task_type) : lock_(), ddl_tracing_(this), is_inited_(false), need_retry_(true), is_running_(false), is_abort_(false), - task_type_(task_type), trace_id_(), tenant_id_(0), object_id_(0), schema_version_(0), + task_type_(task_type), trace_id_(), tenant_id_(0), dst_tenant_id_(0), object_id_(0), schema_version_(0), dst_schema_version_(0), target_object_id_(0), task_status_(share::ObDDLTaskStatus::PREPARE), snapshot_version_(0), ret_code_(OB_SUCCESS), task_id_(0), parent_task_id_(0), parent_task_key_(), task_version_(0), parallelism_(0), allocator_(lib::ObLabel("DdlTask")), compat_mode_(lib::Worker::CompatMode::INVALID), err_code_occurence_cnt_(0), @@ -472,17 +483,17 @@ public: bool get_is_abort() { return is_abort_; } void set_consumer_group_id(const int64_t group_id) { consumer_group_id_ = group_id; } bool try_set_running() { return !ATOMIC_CAS(&is_running_, false, true); } - uint64_t get_tenant_id() const { return tenant_id_; } + uint64_t get_tenant_id() const { return dst_tenant_id_; } uint64_t get_object_id() const { return object_id_; } - int64_t get_schema_version() const { return schema_version_; } + int64_t get_schema_version() const { return dst_schema_version_; } uint64_t get_target_object_id() const { return target_object_id_; } int64_t get_task_status() const { return task_status_; } int64_t get_snapshot_version() const { return snapshot_version_; } int get_ddl_type_str(const int64_t ddl_type, const char *&ddl_type_str); int64_t get_ret_code() const { return ret_code_; } int64_t get_task_id() const { return task_id_; } - ObDDLTaskID get_ddl_task_id() const { return ObDDLTaskID(tenant_id_, task_id_); } - ObDDLTaskKey get_task_key() const { return ObDDLTaskKey(tenant_id_, target_object_id_, schema_version_); } + ObDDLTaskID get_ddl_task_id() const { return ObDDLTaskID(dst_tenant_id_, task_id_); } + ObDDLTaskKey get_task_key() const { return ObDDLTaskKey(dst_tenant_id_, target_object_id_, dst_schema_version_); } int64_t get_parent_task_id() const { return parent_task_id_; } int64_t get_task_version() const { return task_version_; } int64_t get_parallelism() const { return parallelism_; } @@ -543,7 +554,8 @@ public: K_(ret_code), K_(task_id), K_(parent_task_id), K_(parent_task_key), K_(task_version), K_(parallelism), K_(ddl_stmt_str), K_(compat_mode), K_(sys_task_id), K_(err_code_occurence_cnt), K_(stat_info), - K_(next_schedule_ts), K_(delay_schedule_time), K(execution_id_), K(sql_exec_addr_), K_(data_format_version), K(consumer_group_id_)); + K_(next_schedule_ts), K_(delay_schedule_time), K(execution_id_), K(sql_exec_addr_), K_(data_format_version), K(consumer_group_id_), + K_(dst_tenant_id), K_(dst_schema_version)); protected: int gather_redefinition_stats(const uint64_t tenant_id, const int64_t task_id, @@ -585,8 +597,10 @@ protected: share::ObDDLType task_type_; TraceId trace_id_; uint64_t tenant_id_; + uint64_t dst_tenant_id_; uint64_t object_id_; uint64_t schema_version_; + uint64_t dst_schema_version_; uint64_t target_object_id_; share::ObDDLTaskStatus task_status_; int64_t snapshot_version_; diff --git a/src/rootserver/ddl_task/ob_drop_index_task.cpp b/src/rootserver/ddl_task/ob_drop_index_task.cpp index 158d5d606e..b858b5886f 100644 --- a/src/rootserver/ddl_task/ob_drop_index_task.cpp +++ b/src/rootserver/ddl_task/ob_drop_index_task.cpp @@ -65,6 +65,8 @@ int ObDropIndexTask::init( parent_task_id_ = parent_task_id; consumer_group_id_ = consumer_group_id; task_version_ = OB_DROP_INDEX_TASK_VERSION; + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; is_inited_ = true; ddl_tracing_.open(); } @@ -91,6 +93,8 @@ int ObDropIndexTask::init( parent_task_id_ = task_record.parent_task_id_; task_version_ = task_record.task_version_; ret_code_ = task_record.ret_code_; + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; if (nullptr != task_record.message_.ptr()) { int64_t pos = 0; if (OB_FAIL(deserlize_params_from_message(task_record.tenant_id_, task_record.message_.ptr(), task_record.message_.length(), pos))) { diff --git a/src/rootserver/ddl_task/ob_drop_primary_key_task.cpp b/src/rootserver/ddl_task/ob_drop_primary_key_task.cpp index 6af717a429..ab2846a06b 100644 --- a/src/rootserver/ddl_task/ob_drop_primary_key_task.cpp +++ b/src/rootserver/ddl_task/ob_drop_primary_key_task.cpp @@ -43,8 +43,8 @@ int ObDropPrimaryKeyTask::init(const uint64_t tenant_id, const int64_t task_id, const int64_t consumer_group_id, const obrpc::ObAlterTableArg &alter_table_arg, const int64_t task_status, const int64_t snapshot_version) { int ret = OB_SUCCESS; - if (OB_FAIL(ObTableRedefinitionTask::init(tenant_id, task_id, ddl_type, data_table_id, - dest_table_id, schema_version, parallelism, consumer_group_id, + if (OB_FAIL(ObTableRedefinitionTask::init(tenant_id, tenant_id, task_id, ddl_type, data_table_id, + dest_table_id, schema_version, schema_version, parallelism, consumer_group_id, alter_table_arg, task_status, snapshot_version))) { LOG_WARN("fail to init ObDropPrimaryKeyTask", K(ret)); } else { diff --git a/src/rootserver/ddl_task/ob_index_build_task.cpp b/src/rootserver/ddl_task/ob_index_build_task.cpp index 072b86a37b..98e319b7a5 100755 --- a/src/rootserver/ddl_task/ob_index_build_task.cpp +++ b/src/rootserver/ddl_task/ob_index_build_task.cpp @@ -59,7 +59,8 @@ int ObIndexSSTableBuildTask::process() const ObSysVariableSchema *sys_variable_schema = NULL; bool oracle_mode = false; ObTabletID unused_tablet_id; - const ObTableSchema *table_schema = nullptr; + const ObTableSchema *data_schema = nullptr; + const ObTableSchema *index_schema = nullptr; bool need_padding = false; bool need_exec_new_inner_sql = true; @@ -77,25 +78,30 @@ int ObIndexSSTableBuildTask::process() LOG_WARN("sys variable schema is NULL", K(ret)); } else if (OB_FAIL(sys_variable_schema->get_oracle_mode(oracle_mode))) { LOG_WARN("get oracle mode failed", K(ret)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, data_table_id_, table_schema))) { + } 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 (nullptr == table_schema) { + } else if (nullptr == data_schema) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("error unexpected, table schema must not be nullptr", K(ret)); + LOG_WARN("error unexpected, table schema must not be nullptr", K(ret), K(tenant_id_), K(data_table_id_)); + } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, dest_table_id_, index_schema))) { + LOG_WARN("get index schema failed", K(ret), K(tenant_id_), K(dest_table_id_)); + } else if (nullptr == index_schema) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("error unexpected, index schema must not be nullptr", K(ret), K(tenant_id_), K(dest_table_id_)); } else { if (OB_FAIL(ObDDLUtil::generate_build_replica_sql(tenant_id_, data_table_id_, dest_table_id_, - table_schema->get_schema_version(), + data_schema->get_schema_version(), snapshot_version_, execution_id_, task_id_, parallelism_, false/*use_heap_table_ddl*/, - !table_schema->is_user_hidden_table()/*use_schema_version_hint_for_src_table*/, + !data_schema->is_user_hidden_table()/*use_schema_version_hint_for_src_table*/, nullptr, sql_string))) { LOG_WARN("fail to generate build replica sql", K(ret)); - } else if (OB_FAIL(table_schema->is_need_padding_for_generated_column(need_padding))) { + } else if (OB_FAIL(data_schema->is_need_padding_for_generated_column(need_padding))) { LOG_WARN("fail to check need padding", K(ret)); } else { common::ObCommonSqlProxy *user_sql_proxy = nullptr; @@ -105,8 +111,8 @@ int ObIndexSSTableBuildTask::process() session_param.sql_mode_ = (int64_t *)&sql_mode; session_param.tz_info_wrap_ = nullptr; session_param.ddl_info_.set_is_ddl(true); - session_param.ddl_info_.set_source_table_hidden(table_schema->is_user_hidden_table()); - session_param.ddl_info_.set_dest_table_hidden(table_schema->is_user_hidden_table()); + session_param.ddl_info_.set_source_table_hidden(data_schema->is_user_hidden_table()); + session_param.ddl_info_.set_dest_table_hidden(index_schema->is_user_hidden_table()); session_param.nls_formats_[ObNLSFormatEnum::NLS_DATE] = nls_date_format_; session_param.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP] = nls_timestamp_format_; session_param.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP_TZ] = nls_timestamp_tz_format_; @@ -380,6 +386,8 @@ int ObIndexBuildTask::init( } else if (OB_FAIL(init_ddl_task_monitor_info(index_schema->get_table_id()))) { LOG_WARN("init ddl task monitor info failed", K(ret)); } else { + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; is_inited_ = true; } ddl_tracing_.open(); @@ -444,6 +452,8 @@ int ObIndexBuildTask::init(const ObDDLTaskRecord &task_record) ret_code_ = task_record.ret_code_; start_time_ = ObTimeUtility::current_time(); + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; if (OB_FAIL(init_ddl_task_monitor_info(index_schema->get_table_id()))) { LOG_WARN("init ddl task monitor info failed", K(ret)); } else { @@ -1211,7 +1221,7 @@ int ObIndexBuildTask::update_index_status_in_schema(const ObTableSchema &index_s arg.index_table_id_ = index_schema.get_table_id(); arg.status_ = new_status; arg.exec_tenant_id_ = tenant_id_; - arg.in_offline_ddl_white_list_ = index_schema.get_table_state_flag() != TABLE_STATE_NORMAL; + arg.in_offline_ddl_white_list_ = true; arg.task_id_ = task_id_; int64_t ddl_rpc_timeout = 0; int64_t tmp_timeout = 0; diff --git a/src/rootserver/ddl_task/ob_modify_autoinc_task.cpp b/src/rootserver/ddl_task/ob_modify_autoinc_task.cpp index da3930f1eb..c88e5ee4bd 100644 --- a/src/rootserver/ddl_task/ob_modify_autoinc_task.cpp +++ b/src/rootserver/ddl_task/ob_modify_autoinc_task.cpp @@ -183,6 +183,8 @@ int ObModifyAutoincTask::init(const uint64_t tenant_id, tenant_id_ = tenant_id; task_version_ = OB_MODIFY_AUTOINC_TASK_VERSION; task_id_ = task_id; + dst_tenant_id_ = tenant_id; + dst_schema_version_ = schema_version; is_inited_ = true; ddl_tracing_.open(); } @@ -195,6 +197,7 @@ int ObModifyAutoincTask::init(const ObDDLTaskRecord &task_record) const uint64_t data_table_id = task_record.object_id_; const uint64_t target_schema_id = task_record.target_object_id_; const int64_t schema_version = task_record.schema_version_; + task_type_ = ObDDLType::DDL_MODIFY_AUTO_INCREMENT; int64_t pos = 0; if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; @@ -207,7 +210,6 @@ int ObModifyAutoincTask::init(const ObDDLTaskRecord &task_record) } else if (OB_FAIL(set_ddl_stmt_str(task_record.ddl_stmt_str_))) { LOG_WARN("set ddl stmt str failed", K(ret)); } else { - task_type_ = ObDDLType::DDL_MODIFY_AUTO_INCREMENT; object_id_ = data_table_id; target_object_id_ = target_schema_id; schema_version_ = schema_version; @@ -216,6 +218,8 @@ int ObModifyAutoincTask::init(const ObDDLTaskRecord &task_record) tenant_id_ = task_record.tenant_id_; task_id_ = task_record.task_id_; ret_code_ = task_record.ret_code_; + dst_tenant_id_ = tenant_id_; + dst_schema_version_ = schema_version_; is_inited_ = true; // set up span during recover task @@ -626,7 +630,7 @@ int ObModifyAutoincTask::deserlize_params_from_message(const uint64_t tenant_id, LOG_WARN("ObDDLTask deserlize failed", K(ret)); } else if (OB_FAIL(tmp_arg.deserialize(buf, data_len, pos))) { LOG_WARN("serialize table failed", K(ret)); - } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(tenant_id, tmp_arg))) { + } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(task_type_, tenant_id, tmp_arg))) { LOG_WARN("replace user tenant id failed", K(ret), K(tenant_id), K(tmp_arg)); } else if (OB_FAIL(deep_copy_table_arg(allocator_, tmp_arg, alter_table_arg_))) { LOG_WARN("deep copy table arg failed", K(ret)); diff --git a/src/rootserver/ddl_task/ob_recover_restore_table_task.cpp b/src/rootserver/ddl_task/ob_recover_restore_table_task.cpp new file mode 100755 index 0000000000..a337f1296f --- /dev/null +++ b/src/rootserver/ddl_task/ob_recover_restore_table_task.cpp @@ -0,0 +1,293 @@ +/** + * 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 "ob_recover_restore_table_task.h" +#include "lib/rc/context.h" +#include "share/schema/ob_multi_version_schema_service.h" +#include "rootserver/ddl_task/ob_ddl_scheduler.h" +#include "rootserver/ob_root_service.h" +#include "rootserver/ddl_task/ob_ddl_redefinition_task.h" +#include "observer/ob_server_event_history_table_operator.h" +#include "lib/mysqlclient/ob_mysql_proxy.h" +#include "lib/mysqlclient/ob_mysql_result.h" + +using namespace oceanbase::lib; +using namespace oceanbase::common; +using namespace oceanbase::share; +using namespace oceanbase::share::schema; +using namespace oceanbase::rootserver; +using namespace oceanbase::obrpc; + +ObRecoverRestoreTableTask::ObRecoverRestoreTableTask() + : ObTableRedefinitionTask() +{ +} + +ObRecoverRestoreTableTask::~ObRecoverRestoreTableTask() +{ +} + +int ObRecoverRestoreTableTask::init(const uint64_t src_tenant_id, const uint64_t dst_tenant_id, const int64_t task_id, + const share::ObDDLType &ddl_type, const int64_t data_table_id, const int64_t dest_table_id, const int64_t src_schema_version, + const int64_t dst_schema_version, const int64_t parallelism, const int64_t consumer_group_id, + const ObAlterTableArg &alter_table_arg, const int64_t task_status, const int64_t snapshot_version) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", K(ret)); + } else if (OB_UNLIKELY(ObDDLType::DDL_TABLE_RESTORE != ddl_type)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(ddl_type), K(src_tenant_id), K(data_table_id)); + } else if (OB_FAIL(ObTableRedefinitionTask::init(src_tenant_id, dst_tenant_id, task_id, ddl_type, data_table_id, + dest_table_id, src_schema_version, dst_schema_version, parallelism, consumer_group_id, alter_table_arg, task_status, 0/*snapshot*/))) { + LOG_WARN("fail to init ObDropPrimaryKeyTask", K(ret)); + } else { + execution_id_ = 1L; + task_version_ = OB_RECOVER_RESTORE_TABLE_TASK_VERSION; + set_is_copy_foreign_keys(false); + set_is_ignore_errors(true); + } + LOG_INFO("init recover restore table ddl task finished", K(ret), KPC(this)); + return ret; +} + +int ObRecoverRestoreTableTask::init(const ObDDLTaskRecord &task_record) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_WARN("ObRecoverRestoreTableTask has already been inited", K(ret)); + } else if (!task_record.is_valid() || ObDDLType::DDL_TABLE_RESTORE != task_record.ddl_type_) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments", K(ret), K(task_record)); + } else if (OB_FAIL(ObTableRedefinitionTask::init(task_record))) { + LOG_WARN("deserialize to init task failed", K(ret), K(task_record)); + } else { + set_is_copy_foreign_keys(false); + set_is_ignore_errors(true); + } + LOG_INFO("init recover table restore ddl task finished", K(ret), KPC(this)); + return ret; +} + +int ObRecoverRestoreTableTask::obtain_snapshot(const ObDDLTaskStatus next_task_status) +{ + int ret = OB_SUCCESS; + ObRootService *root_service = GCTX.root_service_; + ObDDLTaskStatus new_status = ObDDLTaskStatus::OBTAIN_SNAPSHOT; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverRestoreTableTask has not been inited", K(ret)); + } else if (OB_ISNULL(root_service)) { + ret = OB_ERR_SYS; + LOG_WARN("error sys, root service must not be nullptr", K(ret)); + } else if (snapshot_version_ > 0) { + // do nothing, already hold snapshot. + } else if (OB_FAIL(ObDDLWaitTransEndCtx::calc_snapshot_with_gts(dst_tenant_id_, 0/*trans_end_snapshot*/, snapshot_version_))) { + // fetch snapshot. + LOG_WARN("calc snapshot with gts failed", K(ret), K(dst_tenant_id_)); + } else if (snapshot_version_ <= 0) { + // the snapshot version obtained here must be valid. + ret = OB_ERR_UNEXPECTED; + LOG_WARN("snapshot version is invalid", K(ret), K(snapshot_version_)); + } else if (OB_FAIL(ObDDLTaskRecordOperator::update_snapshot_version(root_service->get_sql_proxy(), + dst_tenant_id_, + task_id_, + snapshot_version_))) { + LOG_WARN("update snapshot version failed", K(ret), K(dst_tenant_id_), K(task_id_), K(snapshot_version_)); + } + + if (OB_FAIL(ret)) { + snapshot_version_ = 0; // reset snapshot if failed. + } else { + new_status = next_task_status; + } + if (new_status == next_task_status || OB_FAIL(ret)) { + if (OB_FAIL(switch_status(new_status, true, ret))) { + LOG_WARN("fail to switch task status", K(ret)); + } + } + return ret; +} + +// update sstable complement status for all leaders +int ObRecoverRestoreTableTask::update_complete_sstable_job_status(const common::ObTabletID &tablet_id, + const int64_t snapshot_version, + const int64_t execution_id, + const int ret_code, + const ObDDLTaskInfo &addition_info) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObColumnRedefinitionTask has not been inited", K(ret)); + } else if (ObDDLTaskStatus::REDEFINITION != task_status_) { + // by pass, may be network delay + } else if (snapshot_version != snapshot_version_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("snapshot version not match", K(ret), K(snapshot_version), K(snapshot_version_)); + } else if (execution_id < execution_id_) { + LOG_INFO("receive a mismatch execution result, ignore", K(ret_code), K(execution_id), K(execution_id_)); + } else if (OB_FAIL(replica_builder_.set_partition_task_status(tablet_id, + ret_code, + addition_info.row_scanned_, + addition_info.row_inserted_))) { + LOG_WARN("fail to set partition task status", K(ret)); + } + return ret; +} + +int ObRecoverRestoreTableTask::success() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else if (OB_FAIL(cleanup())) { + LOG_WARN("clean up failed", K(ret)); + } + return ret; +} + +int ObRecoverRestoreTableTask::fail() +{ + int ret = OB_SUCCESS; + ObArenaAllocator tmp_arena; + int64_t rpc_timeout = 0; + int64_t all_orig_index_tablet_count = 0; + const ObDatabaseSchema *db_schema = nullptr; + const ObTableSchema *table_schema = nullptr; + bool is_oracle_mode = false; + ObRootService *root_service = GCTX.root_service_; + obrpc::ObTableItem table_item; + obrpc::ObDropTableArg drop_table_arg; + obrpc::ObDDLRes drop_table_res; + { + ObSchemaGetterGuard src_tenant_schema_guard; + ObSchemaGetterGuard dst_tenant_schema_guard; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else 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(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(tenant_id_, src_tenant_schema_guard))) { + LOG_WARN("get schema guard failed", K(ret), K(tenant_id_)); + } else if (OB_FAIL(get_orig_all_index_tablet_count(src_tenant_schema_guard, all_orig_index_tablet_count))) { + LOG_WARN("get orig all tablet count failed", K(ret)); + } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(dst_tenant_id_, dst_tenant_schema_guard))) { + LOG_WARN("get schema guard failed", K(ret), K(dst_tenant_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard.get_table_schema(dst_tenant_id_, target_object_id_, table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); + } else if (OB_ISNULL(table_schema)) { + // already dropped. + LOG_INFO("already dropped", K(ret), K(dst_tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard.get_database_schema(dst_tenant_id_, table_schema->get_database_id(), db_schema))) { + LOG_WARN("get db schema failed", K(ret), K(dst_tenant_id_), KPC(table_schema)); + } else if (OB_ISNULL(db_schema)) { + ret = OB_ERR_BAD_DATABASE; + LOG_WARN("database id is invalid", K(dst_tenant_id_), "db_id", table_schema->get_database_id(), K(ret)); + } else if (OB_FAIL(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(ObDDLUtil::get_ddl_rpc_timeout(max(all_orig_index_tablet_count, table_schema->get_all_part_num()), rpc_timeout))) { + LOG_WARN("get ddl rpc timeout failed", K(ret)); + } else if (OB_FAIL(ob_write_string(tmp_arena, db_schema->get_database_name_str(), table_item.database_name_))) { + LOG_WARN("deep cpy database name failed", K(ret), "db_name", db_schema->get_database_name_str()); + } else if (OB_FAIL(ob_write_string(tmp_arena, table_schema->get_table_name_str(), table_item.table_name_))) { + LOG_WARN("deep cpy table name failed", K(ret), "table_name", table_schema->get_table_name_str()); + } else { + // for drop table item. + table_item.mode_ = table_schema->get_name_case_mode(); + table_item.is_hidden_ = table_schema->is_hidden_schema(); + // for drop table arg. + drop_table_arg.tenant_id_ = dst_tenant_id_; + drop_table_arg.exec_tenant_id_ = dst_tenant_id_; + drop_table_arg.session_id_ = table_schema->get_session_id(); + drop_table_arg.table_type_ = table_schema->get_table_type(); + drop_table_arg.foreign_key_checks_ = false; + drop_table_arg.force_drop_ = true; + drop_table_arg.compat_mode_ = is_oracle_mode ? lib::Worker::CompatMode::ORACLE : lib::Worker::CompatMode::MYSQL; + } + } + if (OB_SUCC(ret)) { + obrpc::ObCommonRpcProxy common_rpc_proxy = root_service->get_common_rpc_proxy().to(GCTX.self_addr()).timeout(rpc_timeout); + if (OB_FAIL(drop_table_arg.tables_.push_back(table_item))) { + LOG_WARN("push back failed", K(ret), K(drop_table_arg)); + } else if (OB_FAIL(common_rpc_proxy.drop_table(drop_table_arg, drop_table_res))) { + LOG_WARN("drop table failed", K(ret), K(rpc_timeout), K(drop_table_arg)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(cleanup())) { + LOG_WARN("clean up failed", K(ret)); + } + } + return ret; +} + +int ObRecoverRestoreTableTask::process() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverRestoreTableTask has not been inited", K(ret)); + } else if (OB_FAIL(check_health())) { + LOG_WARN("check task health failed", K(ret)); + } else { + ddl_tracing_.restore_span_hierarchy(); + switch(task_status_) { + case ObDDLTaskStatus::PREPARE: + if (OB_FAIL(prepare(ObDDLTaskStatus::OBTAIN_SNAPSHOT))) { + LOG_WARN("fail to prepare table redefinition task", K(ret)); + } + break; + case ObDDLTaskStatus::OBTAIN_SNAPSHOT: + if (OB_FAIL(obtain_snapshot(ObDDLTaskStatus::REDEFINITION))) { + LOG_WARN("fail to lock table", K(ret)); + } + break; + case ObDDLTaskStatus::REDEFINITION: + if (OB_FAIL(wait_data_complement(ObDDLTaskStatus::COPY_TABLE_DEPENDENT_OBJECTS))) { + LOG_WARN("fail to do table redefinition", K(ret)); + } + break; + case ObDDLTaskStatus::COPY_TABLE_DEPENDENT_OBJECTS: + if (OB_FAIL(copy_table_dependent_objects(ObDDLTaskStatus::TAKE_EFFECT))) { + LOG_WARN("fail to copy table dependent objects", K(ret)); + } + break; + case ObDDLTaskStatus::TAKE_EFFECT: + if (OB_FAIL(take_effect(ObDDLTaskStatus::SUCCESS))) { + LOG_WARN("fail to take effect", K(ret)); + } + break; + case ObDDLTaskStatus::FAIL: + if (OB_FAIL(fail())) { + LOG_WARN("fail to do clean up", K(ret)); + } + break; + case ObDDLTaskStatus::SUCCESS: + if (OB_FAIL(success())) { + LOG_WARN("fail to success", K(ret)); + } + break; + default: + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected table redefinition task state", K(task_status_)); + break; + } + ddl_tracing_.release_span_hierarchy(); + } + return ret; +} \ No newline at end of file diff --git a/src/rootserver/ddl_task/ob_recover_restore_table_task.h b/src/rootserver/ddl_task/ob_recover_restore_table_task.h new file mode 100644 index 0000000000..ac0aca1f2c --- /dev/null +++ b/src/rootserver/ddl_task/ob_recover_restore_table_task.h @@ -0,0 +1,67 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_ROOTSERVER_OB_RECOVER_RESTORE_TABLE_TASK_H +#define OCEANBASE_ROOTSERVER_OB_RECOVER_RESTORE_TABLE_TASK_H + +#include "rootserver/ddl_task/ob_ddl_task.h" +#include "rootserver/ddl_task/ob_table_redefinition_task.h" + +namespace oceanbase +{ +namespace sql +{ + class ObLoadDataStat; +} +namespace rootserver +{ +class ObRootService; + +class ObRecoverRestoreTableTask final : public ObTableRedefinitionTask +{ +public: + ObRecoverRestoreTableTask(); + virtual ~ObRecoverRestoreTableTask(); + int init( + const uint64_t src_tenant_id, + const uint64_t dst_tenant_id, + const int64_t task_id, + const share::ObDDLType &ddl_type, + const int64_t data_table_id, + const int64_t dest_table_id, + const int64_t src_schema_version, + const int64_t dest_schema_version, + const int64_t parallelism, + const int64_t consumer_group_id, + const obrpc::ObAlterTableArg &alter_table_arg, + const int64_t task_status = share::ObDDLTaskStatus::PREPARE, + const int64_t snapshot_version = 0); + int init(const ObDDLTaskRecord &task_record); + virtual int process() override; + virtual int update_complete_sstable_job_status( + const common::ObTabletID &tablet_id, + const int64_t snapshot_version, + const int64_t execution_id, + const int ret_code, + const ObDDLTaskInfo &addition_info) override; +protected: + virtual int obtain_snapshot(const share::ObDDLTaskStatus next_task_status) override; + virtual int fail() override; + virtual int success() override; +private: + static const int64_t OB_RECOVER_RESTORE_TABLE_TASK_VERSION = 1L; +}; + +} // end namespace rootserver +} // end namespace oceanbase + +#endif // OCEANBASE_ROOTSERVER_OB_RECOVER_RESTORE_TABLE_TASK_H diff --git a/src/rootserver/ddl_task/ob_table_redefinition_task.cpp b/src/rootserver/ddl_task/ob_table_redefinition_task.cpp index 2d18b52d79..cd7922705b 100755 --- a/src/rootserver/ddl_task/ob_table_redefinition_task.cpp +++ b/src/rootserver/ddl_task/ob_table_redefinition_task.cpp @@ -34,8 +34,11 @@ using namespace oceanbase::rootserver; using namespace oceanbase::obrpc; ObTableRedefinitionTask::ObTableRedefinitionTask() - : ObDDLRedefinitionTask(ObDDLType::DDL_TABLE_REDEFINITION), has_rebuild_index_(false), has_rebuild_constraint_(false), has_rebuild_foreign_key_(false), allocator_(lib::ObLabel("RedefTask")), - is_copy_indexes_(true), is_copy_triggers_(true), is_copy_constraints_(true), is_copy_foreign_keys_(true), is_ignore_errors_(false), is_do_finish_(false) + : ObDDLRedefinitionTask(ObDDLType::DDL_TABLE_REDEFINITION), + has_rebuild_index_(false), has_rebuild_constraint_(false), has_rebuild_foreign_key_(false), + allocator_(lib::ObLabel("RedefTask")), + is_copy_indexes_(true), is_copy_triggers_(true), is_copy_constraints_(true), is_copy_foreign_keys_(true), + is_ignore_errors_(false), is_do_finish_(false) { } @@ -43,41 +46,53 @@ ObTableRedefinitionTask::~ObTableRedefinitionTask() { } -int ObTableRedefinitionTask::init(const uint64_t tenant_id, const int64_t task_id, const share::ObDDLType &ddl_type, - const int64_t data_table_id, const int64_t dest_table_id, const int64_t schema_version, const int64_t parallelism, - const int64_t consumer_group_id, const ObAlterTableArg &alter_table_arg, const int64_t task_status, const int64_t snapshot_version) +int ObTableRedefinitionTask::init(const uint64_t src_tenant_id, const uint64_t dst_tenant_id, const int64_t task_id, + const share::ObDDLType &ddl_type, const int64_t data_table_id, const int64_t dest_table_id, const int64_t src_schema_version, + const int64_t dst_schema_version, const int64_t parallelism, const int64_t consumer_group_id, + const ObAlterTableArg &alter_table_arg, const int64_t task_status, const int64_t snapshot_version) { int ret = OB_SUCCESS; uint64_t tenant_data_format_version = 0; if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; LOG_WARN("ObTableRedefinitionTask has already been inited", K(ret)); - } else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || OB_INVALID_ID == data_table_id || OB_INVALID_ID == dest_table_id || schema_version <= 0 || task_status < ObDDLTaskStatus::PREPARE - || task_status > ObDDLTaskStatus::SUCCESS || snapshot_version < 0 || (snapshot_version > 0 && task_status < ObDDLTaskStatus::WAIT_TRANS_END))) { + } else if (OB_UNLIKELY(OB_INVALID_ID == src_tenant_id || OB_INVALID_ID == dst_tenant_id + || task_id <= 0 || OB_INVALID_ID == data_table_id || OB_INVALID_ID == dest_table_id + || src_schema_version <= 0 || dst_schema_version <= 0 + || task_status < ObDDLTaskStatus::PREPARE || task_status > ObDDLTaskStatus::SUCCESS || snapshot_version < 0 + || (snapshot_version > 0 && task_status < ObDDLTaskStatus::WAIT_TRANS_END))) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid arguments", K(ret), K(tenant_id), K(task_id), K(data_table_id), K(dest_table_id), K(schema_version), K(task_status), K(snapshot_version)); + LOG_WARN("invalid arguments", K(ret), K(src_tenant_id), K(dst_tenant_id), K(task_id), + K(data_table_id), K(dest_table_id), K(src_schema_version), K(dst_schema_version), + K(task_status), K(snapshot_version)); } else if (OB_FAIL(deep_copy_table_arg(allocator_, alter_table_arg, alter_table_arg_))) { LOG_WARN("deep copy alter table arg failed", K(ret)); } else if (OB_FAIL(set_ddl_stmt_str(alter_table_arg_.ddl_stmt_str_))) { LOG_WARN("set ddl stmt str failed", K(ret)); - } else if (OB_FAIL(ObShareUtil::fetch_current_data_version(*GCTX.sql_proxy_, tenant_id, tenant_data_format_version))) { - LOG_WARN("get min data version failed", K(ret), K(tenant_id)); + } else if (OB_FAIL(ObShareUtil::fetch_current_data_version(*GCTX.sql_proxy_, src_tenant_id, tenant_data_format_version))) { + LOG_WARN("get min data version failed", K(ret), K(src_tenant_id)); } else { set_gmt_create(ObTimeUtility::current_time()); consumer_group_id_ = consumer_group_id; task_type_ = ddl_type; object_id_ = data_table_id; target_object_id_ = dest_table_id; - schema_version_ = schema_version; + schema_version_ = src_schema_version; task_status_ = static_cast(task_status); snapshot_version_ = snapshot_version; - tenant_id_ = tenant_id; + tenant_id_ = src_tenant_id; task_version_ = OB_TABLE_REDEFINITION_TASK_VERSION; task_id_ = task_id; parallelism_ = parallelism; data_format_version_ = tenant_data_format_version; - alter_table_arg_.exec_tenant_id_ = tenant_id_; start_time_ = ObTimeUtility::current_time(); + // For common offline ddl, dest_tenant_id is also the tenant_id_, i.e., tenant id of the data table. + // But for DDL_RESTORE_TABLE, dst_tenant_id_ is different to the tenant_id_. + dst_tenant_id_ = dst_tenant_id; + dst_schema_version_ = dst_schema_version; + alter_table_arg_.alter_table_schema_.set_tenant_id(src_tenant_id); + alter_table_arg_.alter_table_schema_.set_schema_version(src_schema_version); + alter_table_arg_.exec_tenant_id_ = dst_tenant_id_; if (OB_FAIL(init_ddl_task_monitor_info(target_object_id_))) { LOG_WARN("init ddl task monitor info failed", K(ret)); } else { @@ -85,15 +100,20 @@ int ObTableRedefinitionTask::init(const uint64_t tenant_id, const int64_t task_i ddl_tracing_.open(); } } + LOG_INFO("init table redefinition task finished", K(ret), KPC(this)); return ret; } int ObTableRedefinitionTask::init(const ObDDLTaskRecord &task_record) { int ret = OB_SUCCESS; + uint64_t src_tenant_id = common::OB_INVALID_ID; + uint64_t dst_tenant_id = common::OB_INVALID_ID; + int64_t src_schema_version = 0; + int64_t dst_schema_version = 0; const uint64_t data_table_id = task_record.object_id_; const uint64_t dest_table_id = task_record.target_object_id_; - const int64_t schema_version = task_record.schema_version_; + task_type_ = task_record.ddl_type_; // put here to decide whether to replace user tenant id. int64_t pos = 0; if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; @@ -105,20 +125,37 @@ int ObTableRedefinitionTask::init(const ObDDLTaskRecord &task_record) LOG_WARN("deserialize params from message failed", K(ret), K(task_record.message_), K(common::lbt())); } else if (OB_FAIL(set_ddl_stmt_str(task_record.ddl_stmt_str_))) { LOG_WARN("set ddl stmt str failed", K(ret)); + } else if (FALSE_IT(src_tenant_id = alter_table_arg_.alter_table_schema_.get_tenant_id())) { + } else if (FALSE_IT(dst_tenant_id = task_record.tenant_id_)) { + } else if (FALSE_IT(src_schema_version = alter_table_arg_.alter_table_schema_.get_schema_version())) { + } else if (FALSE_IT(dst_schema_version = task_record.schema_version_)) { + } else if (OB_UNLIKELY(common::OB_INVALID_ID == src_tenant_id + || common::OB_INVALID_ID == dst_tenant_id + || src_schema_version <= 0 + || dst_schema_version <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected err", K(ret), K(task_record), K(src_tenant_id), K(dst_tenant_id), K(src_schema_version), K(dst_schema_version)); + } else if (OB_UNLIKELY(ObDDLType::DDL_TABLE_RESTORE != task_record.ddl_type_ + && (src_tenant_id != dst_tenant_id || src_schema_version != dst_schema_version))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected err", K(ret), K(task_record), K(src_tenant_id), K(dst_tenant_id), K(src_schema_version), K(dst_schema_version)); + } else if (OB_UNLIKELY(ObDDLType::DDL_TABLE_RESTORE == task_record.ddl_type_ + && (src_tenant_id == dst_tenant_id))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected err", K(ret), K(task_record), K(src_tenant_id), K(dst_tenant_id), K(src_schema_version), K(dst_schema_version)); } else { task_id_ = task_record.task_id_; - task_type_ = task_record.ddl_type_; object_id_ = data_table_id; target_object_id_ = dest_table_id; - schema_version_ = schema_version; + schema_version_ = src_schema_version; task_status_ = static_cast(task_record.task_status_); snapshot_version_ = task_record.snapshot_version_; execution_id_ = task_record.execution_id_; - tenant_id_ = task_record.tenant_id_; + tenant_id_ = src_tenant_id; ret_code_ = task_record.ret_code_; - alter_table_arg_.exec_tenant_id_ = tenant_id_; start_time_ = ObTimeUtility::current_time(); - + dst_tenant_id_ = dst_tenant_id; + dst_schema_version_ = dst_schema_version; if (OB_FAIL(init_ddl_task_monitor_info(target_object_id_))) { LOG_WARN("init ddl task monitor info failed", K(ret)); } else { @@ -128,6 +165,7 @@ int ObTableRedefinitionTask::init(const ObDDLTaskRecord &task_record) ddl_tracing_.open_for_recovery(); } } + LOG_INFO("init table redefinition task finished", K(ret), KPC(this)); return ret; } @@ -301,9 +339,9 @@ int ObTableRedefinitionTask::check_use_heap_table_ddl_plan(bool &use_heap_table_ 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))) { + .get_tenant_schema_guard_with_version_in_inner_table(dst_tenant_id_, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, target_table_schema))) { + } else if (OB_FAIL(schema_guard.get_table_schema(dst_tenant_id_, target_object_id_, target_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(target_object_id_)); } else if (OB_ISNULL(target_table_schema)) { ret = OB_ERR_UNEXPECTED; @@ -359,7 +397,7 @@ int ObTableRedefinitionTask::table_redefinition(const ObDDLTaskStatus next_task_ // overwrite ret if (is_build_replica_end) { - ret = complete_sstable_job_ret_code_; + ret = OB_SUCC(ret) ? complete_sstable_job_ret_code_ : ret; if (OB_SUCC(ret)) { if (OB_FAIL(replica_end_check(ret))) { LOG_WARN("fail to check", K(ret)); @@ -416,11 +454,10 @@ int ObTableRedefinitionTask::copy_table_indexes() alter_table_arg_.ddl_task_type_ = share::REBUILD_INDEX_TASK; alter_table_arg_.table_id_ = object_id_; alter_table_arg_.hidden_table_id_ = target_object_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); - if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(tenant_id_, schema_guard))) { + if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(dst_tenant_id_, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, table_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(schema_guard.get_table_schema(dst_tenant_id_, target_object_id_, table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, table schema must not be nullptr", K(ret), K(target_object_id_)); @@ -438,7 +475,10 @@ int ObTableRedefinitionTask::copy_table_indexes() // if there is no indexes in new tables, we need to rebuild indexes in new table int64_t ddl_rpc_timeout = 0; int64_t all_tablet_count = 0; - if (OB_FAIL(get_orig_all_index_tablet_count(schema_guard, all_tablet_count))) { + ObSchemaGetterGuard orig_schema_guard; + if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(tenant_id_, orig_schema_guard))) { + LOG_WARN("get schema guard failed", K(ret), K(tenant_id_)); + } else if (OB_FAIL(get_orig_all_index_tablet_count(orig_schema_guard, all_tablet_count))) { LOG_WARN("get all tablet count failed", K(ret)); } else if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(all_tablet_count, ddl_rpc_timeout))) { LOG_WARN("get ddl rpc timeout failed", K(ret)); @@ -451,7 +491,7 @@ int ObTableRedefinitionTask::copy_table_indexes() DEBUG_SYNC(TABLE_REDEFINITION_COPY_TABLE_INDEXES); if (OB_SUCC(ret) && index_ids.count() > 0) { ObSchemaGetterGuard new_schema_guard; - if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(tenant_id_, new_schema_guard))) { + if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(dst_tenant_id_, new_schema_guard))) { LOG_WARN("failed to refresh schema guard", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < index_ids.count(); ++i) { @@ -464,8 +504,8 @@ int ObTableRedefinitionTask::copy_table_indexes() create_index_arg.nls_date_format_ = alter_table_arg_.nls_formats_[0]; create_index_arg.nls_timestamp_format_ = alter_table_arg_.nls_formats_[1]; create_index_arg.nls_timestamp_tz_format_ = alter_table_arg_.nls_formats_[2]; - if (OB_FAIL(new_schema_guard.get_table_schema(tenant_id_, index_ids.at(i), index_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(index_ids.at(i))); + if (OB_FAIL(new_schema_guard.get_table_schema(dst_tenant_id_, index_ids.at(i), index_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_), K(index_ids.at(i))); } else if (OB_ISNULL(index_schema)) { ret = OB_ERR_SYS; LOG_WARN("error sys, index schema must not be nullptr", K(ret), K(index_ids.at(i))); @@ -477,7 +517,7 @@ int ObTableRedefinitionTask::copy_table_indexes() ret = OB_EAGAIN; } else { create_index_arg.index_type_ = index_schema->get_index_type(); - ObCreateDDLTaskParam param(tenant_id_, + ObCreateDDLTaskParam param(dst_tenant_id_, ObDDLType::DDL_CREATE_INDEX, table_schema, index_schema, @@ -544,10 +584,10 @@ int ObTableRedefinitionTask::copy_table_constraints() ObSArray constraint_ids; ObSArray new_constraint_ids; bool need_rebuild_constraint = true; - if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(tenant_id_, schema_guard))) { + if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(dst_tenant_id_, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, table_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(schema_guard.get_table_schema(dst_tenant_id_, target_object_id_, table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, table schema must not be nullptr", K(ret), K(target_object_id_)); @@ -559,9 +599,8 @@ int ObTableRedefinitionTask::copy_table_constraints() alter_table_arg_.ddl_task_type_ = share::REBUILD_CONSTRAINT_TASK; alter_table_arg_.table_id_ = object_id_; alter_table_arg_.hidden_table_id_ = target_object_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); int64_t ddl_rpc_timeout = 0; - if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(tenant_id_, target_object_id_, ddl_rpc_timeout))) { + if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(dst_tenant_id_, target_object_id_, ddl_rpc_timeout))) { LOG_WARN("get ddl rpc timeout fail", K(ret)); } else if (OB_FAIL(root_service->get_ddl_service().get_common_rpc()->to(obrpc::ObRpcProxy::myaddr_).timeout(ddl_rpc_timeout). execute_ddl_task(alter_table_arg_, constraint_ids))) { @@ -609,10 +648,10 @@ int ObTableRedefinitionTask::copy_table_foreign_keys() if (has_rebuild_foreign_key_) { // do nothing } else { - if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(tenant_id_, schema_guard))) { + if (OB_FAIL(root_service->get_ddl_service().get_tenant_schema_guard_with_version_in_inner_table(dst_tenant_id_, schema_guard))) { LOG_WARN("get schema guard failed", K(ret)); - } else if (OB_FAIL(schema_guard.get_simple_table_schema(tenant_id_, target_object_id_, table_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(schema_guard.get_simple_table_schema(dst_tenant_id_, target_object_id_, table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_), K(target_object_id_)); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected, table schema must not be nullptr", K(ret), K(target_object_id_)); @@ -631,9 +670,8 @@ int ObTableRedefinitionTask::copy_table_foreign_keys() alter_table_arg_.ddl_task_type_ = share::REBUILD_FOREIGN_KEY_TASK; alter_table_arg_.table_id_ = object_id_; alter_table_arg_.hidden_table_id_ = target_object_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); int64_t ddl_rpc_timeout = 0; - if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(tenant_id_, target_object_id_, ddl_rpc_timeout))) { + if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(dst_tenant_id_, target_object_id_, ddl_rpc_timeout))) { LOG_WARN("get ddl rpc timeout fail", K(ret)); } else if (OB_FAIL(root_service->get_ddl_service().get_common_rpc()->to(obrpc::ObRpcProxy::myaddr_).timeout(ddl_rpc_timeout). execute_ddl_task(alter_table_arg_, fk_ids))) { @@ -698,11 +736,11 @@ int ObTableRedefinitionTask::copy_table_dependent_objects(const ObDDLTaskStatus // maybe ddl already finish when switching rs HEAP_VAR(ObDDLErrorMessageTableOperator::ObBuildDDLErrorMessage, error_message) { int64_t unused_user_msg_len = 0; - if (OB_FAIL(ObDDLErrorMessageTableOperator::get_ddl_error_message(tenant_id_, child_task_id, target_object_id, + if (OB_FAIL(ObDDLErrorMessageTableOperator::get_ddl_error_message(dst_tenant_id_, child_task_id, target_object_id, unused_addr, false /* is_ddl_retry_task */, *GCTX.sql_proxy_, error_message, unused_user_msg_len))) { if (OB_ENTRY_NOT_EXIST == ret) { ret = OB_SUCCESS; - LOG_INFO("ddl task not finish", K(task_key), K(child_task_id), K(target_object_id)); + LOG_INFO("ddl task not finish", K(dst_tenant_id_), K(task_key), K(child_task_id), K(target_object_id)); } else { LOG_WARN("fail to get ddl error message", K(ret), K(task_key), K(child_task_id), K(target_object_id)); } @@ -752,28 +790,30 @@ int ObTableRedefinitionTask::take_effect(const ObDDLTaskStatus next_task_status) #endif ObSArray objs; int64_t ddl_rpc_timeout = 0; - alter_table_arg_.ddl_task_type_ = share::MAKE_DDL_TAKE_EFFECT_TASK; + alter_table_arg_.ddl_task_type_ = ObDDLType::DDL_TABLE_RESTORE != task_type_ ? + share::MAKE_DDL_TAKE_EFFECT_TASK : share::MAKE_RECOVER_RESTORE_TABLE_TASK_TAKE_EFFECT; alter_table_arg_.table_id_ = object_id_; alter_table_arg_.hidden_table_id_ = target_object_id_; // offline ddl is allowed on table with trigger(enable/disable). alter_table_arg_.need_rebuild_trigger_ = true; alter_table_arg_.task_id_ = task_id_; - alter_table_arg_.alter_table_schema_.set_tenant_id(tenant_id_); ObRootService *root_service = GCTX.root_service_; ObSchemaGetterGuard schema_guard; const ObTableSchema *table_schema = nullptr; bool use_heap_table_ddl_plan = false; ObDDLTaskStatus new_status = next_task_status; + // FIXME YIREN, recover restore table ddl need to sync stats. + const bool need_sync_stats = DDL_TABLE_RESTORE != task_type_; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("ObTableRedefinitionTask has not been inited", K(ret)); } else 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(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(tenant_id_, schema_guard))) { + } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(dst_tenant_id_, schema_guard))) { LOG_WARN("get tenant schema guard failed", K(ret)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, table_schema))) { - LOG_WARN("get table schema failed", K(ret), K(tenant_id_)); + } else if (OB_FAIL(schema_guard.get_table_schema(dst_tenant_id_, target_object_id_, table_schema))) { + LOG_WARN("get table schema failed", K(ret), K(dst_tenant_id_)); } else if (OB_ISNULL(table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("table schema not exist", K(ret), K(target_object_id_)); @@ -795,9 +835,9 @@ int ObTableRedefinitionTask::take_effect(const ObDDLTaskStatus next_task_status) } else { LOG_WARN("sync auto increment position failed", K(ret), K(object_id_), K(target_object_id_)); } - } else if (OB_FAIL(sync_stats_info())) { + } else if (OB_FAIL(need_sync_stats && sync_stats_info())) { LOG_WARN("fail to sync stats info", K(ret), K(object_id_), K(target_object_id_)); - } else if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(tenant_id_, target_object_id_, ddl_rpc_timeout))) { + } else if (OB_FAIL(ObDDLUtil::get_ddl_rpc_timeout(dst_tenant_id_, target_object_id_, ddl_rpc_timeout))) { LOG_WARN("get ddl rpc timeout fail", K(ret)); } else if (OB_FAIL(root_service->get_ddl_service().get_common_rpc()->to(obrpc::ObRpcProxy::myaddr_).timeout(ddl_rpc_timeout). execute_ddl_task(alter_table_arg_, objs))) { @@ -826,6 +866,7 @@ int ObTableRedefinitionTask::repending(const share::ObDDLTaskStatus next_task_st switch (task_type_) { case DDL_DIRECT_LOAD: case DDL_DIRECT_LOAD_INSERT: + case DDL_TABLE_RESTORE: if (get_is_do_finish()) { if (OB_FAIL(switch_status(next_task_status, true, ret))) { LOG_WARN("fail to switch status", K(ret)); @@ -983,7 +1024,7 @@ int ObTableRedefinitionTask::serialize_params_to_message(char *buf, const int64_ } else if (OB_FAIL(serialization::encode_i8(buf, buf_len, pos, ignore_errors))) { LOG_WARN("fail to serialize is_ignore_errors", K(ret)); } else if (OB_FAIL(serialization::encode_i8(buf, buf_len, pos, do_finish))) { - LOG_WARN("fail to serialize is_do_finish"); + LOG_WARN("fail to serialize is_do_finish", K(ret)); } FLOG_INFO("serialize message for table redefinition", K(ret), K(copy_indexes), K(copy_triggers), K(copy_constraints), K(copy_foreign_keys), K(ignore_errors), K(do_finish), K(*this)); @@ -1007,7 +1048,7 @@ int ObTableRedefinitionTask::deserlize_params_from_message(const uint64_t tenant LOG_WARN("ObDDLTask deserlize failed", K(ret)); } else if (OB_FAIL(tmp_arg.deserialize(buf, data_len, pos))) { LOG_WARN("serialize table failed", K(ret)); - } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(tenant_id, tmp_arg))) { + } else if (OB_FAIL(ObDDLUtil::replace_user_tenant_id(task_type_, tenant_id, tmp_arg))) { LOG_WARN("replace user tenant id failed", K(ret), K(tenant_id), K(tmp_arg)); } else if (OB_FAIL(deep_copy_table_arg(allocator_, tmp_arg, alter_table_arg_))) { LOG_WARN("deep copy table arg failed", K(ret)); @@ -1061,7 +1102,7 @@ int ObTableRedefinitionTask::collect_longops_stat(ObLongopsValue &value) int ret = OB_SUCCESS; int64_t pos = 0; const ObDDLTaskStatus status = static_cast(task_status_); - databuff_printf(stat_info_.message_, MAX_LONG_OPS_MESSAGE_LENGTH, pos, "TENANT_ID: %ld, TASK_ID: %ld, ", tenant_id_, task_id_); + databuff_printf(stat_info_.message_, MAX_LONG_OPS_MESSAGE_LENGTH, pos, "TENANT_ID: %ld, TASK_ID: %ld, ", dst_tenant_id_, task_id_); switch (status) { case ObDDLTaskStatus::PREPARE: { if (OB_FAIL(databuff_printf(stat_info_.message_, @@ -1114,7 +1155,7 @@ int ObTableRedefinitionTask::collect_longops_stat(ObLongopsValue &value) int64_t row_scanned = 0; int64_t row_sorted = 0; int64_t row_inserted = 0; - if (OB_FAIL(gather_redefinition_stats(tenant_id_, task_id_, *GCTX.sql_proxy_, row_scanned, row_sorted, row_inserted))) { + if (OB_FAIL(gather_redefinition_stats(dst_tenant_id_, task_id_, *GCTX.sql_proxy_, row_scanned, row_sorted, row_inserted))) { LOG_WARN("failed to gather redefinition stats", K(ret)); } else if (OB_FAIL(databuff_printf(stat_info_.message_, MAX_LONG_OPS_MESSAGE_LENGTH, diff --git a/src/rootserver/ddl_task/ob_table_redefinition_task.h b/src/rootserver/ddl_task/ob_table_redefinition_task.h index b8ccd62992..7bfbf1d409 100644 --- a/src/rootserver/ddl_task/ob_table_redefinition_task.h +++ b/src/rootserver/ddl_task/ob_table_redefinition_task.h @@ -33,11 +33,13 @@ public: virtual ~ObTableRedefinitionTask(); int init( const uint64_t tenant_id, + const uint64_t dest_tenant_id, const int64_t task_id, const share::ObDDLType &ddl_type, const int64_t data_table_id, const int64_t dest_table_id, const int64_t schema_version, + const int64_t dest_schema_version, const int64_t parallelism, const int64_t consumer_group_id, const obrpc::ObAlterTableArg &alter_table_arg, @@ -74,6 +76,10 @@ protected: int table_redefinition(const share::ObDDLTaskStatus next_task_status); int copy_table_dependent_objects(const share::ObDDLTaskStatus next_task_status); int take_effect(const share::ObDDLTaskStatus next_task_status); + int set_partition_task_status(const common::ObTabletID &tablet_id, + const int ret_code, + const int64_t row_scanned, + const int64_t row_inserted); int repending(const share::ObDDLTaskStatus next_task_status); private: inline bool get_is_copy_indexes() const {return is_copy_indexes_;} diff --git a/src/rootserver/ob_create_standby_from_net_actor.cpp b/src/rootserver/ob_create_standby_from_net_actor.cpp index d95762bf20..f41caf2220 100644 --- a/src/rootserver/ob_create_standby_from_net_actor.cpp +++ b/src/rootserver/ob_create_standby_from_net_actor.cpp @@ -230,7 +230,7 @@ int ObCreateStandbyFromNetActor::refresh_schema_() } else if (OB_FAIL(schema_status_proxy->get_refresh_schema_status(tenant_id_, refresh_schema_status))) { LOG_WARN("fail to get refresh schema status", KR(ret), K_(tenant_id)); } else if (refresh_schema_status.snapshot_timestamp_ == 0) { - if (OB_FAIL(ObRestoreService::reset_schema_status(tenant_id_, sql_proxy_))) { + if (OB_FAIL(ObRestoreScheduler::reset_schema_status(tenant_id_, sql_proxy_))) { WSTAT("failed to reset schema status", KR(ret), K_(tenant_id)); } } diff --git a/src/rootserver/ob_ddl_service.cpp b/src/rootserver/ob_ddl_service.cpp index b32a7b94d2..4a1d884c51 100755 --- a/src/rootserver/ob_ddl_service.cpp +++ b/src/rootserver/ob_ddl_service.cpp @@ -310,6 +310,49 @@ int ObDDLService::get_tenant_schema_guard_with_version_in_inner_table(const uint 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; + bool is_standby = false; + 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 (src_tenant_id != dst_tenant_id + && OB_FAIL(ObAllTenantInfoProxy::is_standby_tenant(sql_proxy_, src_tenant_id, is_standby))) { + LOG_WARN("check tenant standby failed", K(ret), K(src_tenant_id), K(dst_tenant_id)); + } else if (src_tenant_id != dst_tenant_id && !is_standby) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg, src tenant should be standby", 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) @@ -3093,6 +3136,7 @@ int ObDDLService::create_hidden_table_with_pk_changed( &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, + schema_guard, ddl_operator, trans, allocator, @@ -3904,6 +3948,7 @@ int ObDDLService::alter_table_partition_by( &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, + schema_guard, ddl_operator, trans, alter_table_arg.allocator_)); @@ -4057,6 +4102,7 @@ int ObDDLService::convert_to_character( &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, + schema_guard, ddl_operator, trans, alter_table_arg.allocator_)); @@ -4851,12 +4897,14 @@ int ObDDLService::remap_index_tablets_and_take_effect( ObSArray 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 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_); @@ -4864,12 +4912,15 @@ int ObDDLService::remap_index_tablets_and_take_effect( 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))) { @@ -9684,6 +9735,8 @@ const char* ObDDLService::ddl_type_str(const ObDDLType 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"; } return str; } @@ -11500,6 +11553,7 @@ int ObDDLService::do_offline_ddl_in_trans(obrpc::ObAlterTableArg &alter_table_ar &alter_table_arg.sequence_ddl_arg_, bind_tablets, schema_guard, + schema_guard, ddl_operator, trans, alter_table_arg.allocator_))) { @@ -11653,7 +11707,7 @@ int ObDDLService::create_hidden_table( int ret = OB_SUCCESS; const uint64_t tenant_id = create_hidden_table_arg.tenant_id_; const int64_t table_id = create_hidden_table_arg.table_id_; - const uint64_t dest_tenant_id = create_hidden_table_arg.dest_tenant_id_; + const uint64_t dest_tenant_id = tenant_id; ObRootService *root_service = GCTX.root_service_; bool bind_tablets = true; ObSchemaGetterGuard schema_guard; @@ -11710,6 +11764,7 @@ int ObDDLService::create_hidden_table( nullptr, bind_tablets, schema_guard, + schema_guard, ddl_operator, trans, allocator))) { @@ -11787,6 +11842,122 @@ int ObDDLService::create_hidden_table( 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. + 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. + 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_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 { + ObDDLOperator ddl_operator(*schema_service_, *sql_proxy_); + 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(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))) { + LOG_WARN("create user hidden table failed", K(ret), K(arg)); + } else { + ObPrepareAlterTableArgParam param; + if (OB_FAIL(param.init(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_table_name(arg.target_schema_.get_table_name_str()); + 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(), + arg.parallelism_, + arg.consumer_group_id_, + &allocator, + &alter_table_arg, + 0, + arg.ddl_task_id_); + 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, @@ -13028,7 +13199,7 @@ int ObDDLService::rename_table(const obrpc::ObRenameTableArg &rename_table_arg) lib::Worker::CompatMode compat_mode = lib::Worker::CompatMode::MYSQL; if (OB_ISNULL(schema_service)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("schema_guard or scheam service is null", + 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; @@ -14202,7 +14373,7 @@ int ObDDLService::prepare_hidden_table_schema(const ObTableSchema &orig_table_sc LOG_WARN("schema_service must not null", K(ret)); } else if (OB_FAIL(orig_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) { LOG_WARN("fail to check is oracle mode", K(ret)); - } else if (OB_FAIL(schema_service->fetch_new_table_id(orig_table_schema.get_tenant_id(), + } 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 { @@ -14225,7 +14396,6 @@ int ObDDLService::prepare_hidden_table_schema(const ObTableSchema &orig_table_sc } else if (OB_FAIL(generate_tablet_id(hidden_table_schema))) { LOG_WARN("fail to generate tablet id for hidden table", K(ret), K(hidden_table_schema)); } else { - hidden_table_schema.set_database_id(orig_table_schema.get_database_id()); // 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); @@ -14239,7 +14409,7 @@ int ObDDLService::prepare_hidden_table_schema(const ObTableSchema &orig_table_sc ObString pk_name; ObConstraint cst; cst.set_constraint_type(CONSTRAINT_TYPE_PRIMARY_KEY); - if (OB_FAIL(schema_service->fetch_new_constraint_id(orig_table_schema.get_tenant_id(), + 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))) { @@ -14281,6 +14451,12 @@ int ObDDLService::rebuild_hidden_table_priv(const ObTableSchema &orig_table_sche 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(), @@ -14353,7 +14529,8 @@ int ObDDLService::create_user_hidden_table(const ObTableSchema &orig_table_schem ObTableSchema &hidden_table_schema, const obrpc::ObSequenceDDLArg *sequence_ddl_arg, const bool bind_tablets, - ObSchemaGetterGuard &schema_guard, + share::schema::ObSchemaGetterGuard &src_tenant_schema_guard, + share::schema::ObSchemaGetterGuard &dst_tenant_schema_guard, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans, ObIAllocator &allocator, @@ -14381,7 +14558,7 @@ int ObDDLService::create_user_hidden_table(const ObTableSchema &orig_table_schem LOG_WARN("failed to prepare hidden table schema", K(ret)); } else if (OB_FAIL(ddl_operator.create_sequence_in_create_table(hidden_table_schema, trans, - schema_guard, + 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; @@ -14392,18 +14569,18 @@ int ObDDLService::create_user_hidden_table(const ObTableSchema &orig_table_schem LOG_WARN("failed to build_aux_lob_table_schema_if_need", K(ret), K(hidden_table_schema)); } else if (OB_FAIL(rebuild_hidden_table_priv(orig_table_schema, hidden_table_schema, - schema_guard, + 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, - schema_guard, + 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, schema_guard))) { + } 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))) { @@ -14427,7 +14604,7 @@ int ObDDLService::create_user_hidden_table(const ObTableSchema &orig_table_schem trans); ObNewTableTabletAllocator new_table_tablet_allocator( tenant_id, - schema_guard, + dst_tenant_schema_guard, sql_proxy_); common::ObArray ls_id_array; @@ -14496,7 +14673,9 @@ int ObDDLService::create_user_hidden_table(const ObTableSchema &orig_table_schem } } - if (OB_SUCC(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))) { @@ -14697,6 +14876,7 @@ 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 &new_table_schemas, @@ -14833,11 +15013,13 @@ int ObDDLService::add_new_index_schema(obrpc::ObAlterTableArg &alter_table_arg, bool is_exist = false; index_schema.set_table_id(new_idx_tid); index_schema.set_data_table_id(new_table_schema.get_table_id()); + index_schema.set_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); if (OB_FAIL(ret)) { - } else if (OB_FAIL(schema_guard.check_table_exist(index_schema.get_tenant_id(), + } else if (OB_FAIL(dest_schema_guard.check_table_exist(index_schema.get_tenant_id(), index_schema.get_database_id(), index_schema.get_table_name_str(), true/*is_index*/, @@ -14960,9 +15142,11 @@ int ObDDLService::check_index_table_need_rebuild(const share::schema::ObTableSch return ret; } -int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schema, +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 &drop_cols_id_arr, const ObColumnNameMap &col_name_map, const common::ObTimeZoneInfo &tz_info, @@ -14971,7 +15155,8 @@ int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schem ObSArray &index_ids) { int ret = OB_SUCCESS; - const uint64_t tenant_id = orig_table_schema.get_tenant_id(); + 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())) { @@ -14992,12 +15177,28 @@ int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schem 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( - 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)); + 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)); @@ -15012,6 +15213,7 @@ int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schem // No need to convert hidden table column id since indexes are rebuilt based on column names. if (OB_FAIL(new_index_schema.assign(*index_table_schema))) { LOG_WARN("fail to assign schema", K(ret)); + } else if (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(), @@ -15020,6 +15222,12 @@ int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schem 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); @@ -15057,7 +15265,7 @@ int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schem } else { uint64_t new_idx_tid = OB_INVALID_ID; if (OB_FAIL(schema_service->fetch_new_table_id( - tenant_id, new_idx_tid))) { + 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)); @@ -15069,20 +15277,20 @@ int ObDDLService::reconstruct_index_schema(const ObTableSchema &orig_table_schem new_index_schema.set_table_id(new_idx_tid); new_index_schema.set_data_table_id(hidden_table_schema.get_table_id()); new_index_schema.set_index_status(INDEX_STATUS_UNAVAILABLE); - new_index_schema.set_table_name(new_index_table_name); - // set the hidden attributes of the table - new_index_schema.set_table_state_flag(ObTableStateFlag::TABLE_STATE_HIDDEN_OFFLINE_DDL); + 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); bool is_exist = false; if (OB_FAIL(ret)) { - } else if (OB_FAIL(schema_guard.check_table_exist(new_index_schema.get_tenant_id(), + } 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*/, - ObSchemaGetterGuard::USER_HIDDEN_TABLE_TYPE/*check_type*/, + check_table_type, is_exist))) { LOG_WARN("failed to check table exist", K(ret)); } else if (is_exist) { - LOG_INFO("index already rebuilt, skip", K(new_index_schema.get_table_id()), K(new_index_schema.get_table_name_str())); + 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))) { @@ -15157,9 +15365,11 @@ int ObDDLService::rebuild_hidden_table_index(obrpc::ObAlterTableArg &alter_table ObSArray &index_ids) { int ret = OB_SUCCESS; + ObSArray 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 tenant_id = alter_table_schema.get_tenant_id(); + 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)); @@ -15168,22 +15378,27 @@ int ObDDLService::rebuild_hidden_table_index(obrpc::ObAlterTableArg &alter_table LOG_WARN("invalid tz_info_wrap", K(tz_info_wrap), K(ret)); } else { ObDDLSQLTransaction trans(schema_service_); - ObSArray new_table_schemas; - - ObSchemaGetterGuard schema_guard; - const ObTableSchema *orig_table_schema = NULL; - const ObTableSchema *hidden_table_schema = NULL; - schema_guard.set_session_id(alter_table_arg.session_id_); + 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 drop_cols_id_arr; - if (OB_FAIL(get_tenant_schema_guard_with_version_in_inner_table(tenant_id, schema_guard))) { - LOG_WARN("fail to get schema guard with version in inner table", K(ret), K(tenant_id)); - } else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, refreshed_schema_version))) { - LOG_WARN("failed to get tenant schema version", KR(ret), K(tenant_id)); - } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { - LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); + 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, - schema_guard, + *src_tenant_schema_guard, + *dst_tenant_schema_guard, alter_table_schema, orig_table_schema, hidden_table_schema))) { @@ -15202,9 +15417,11 @@ int ObDDLService::rebuild_hidden_table_index(obrpc::ObAlterTableArg &alter_table LOG_WARN("failed to init column name map", K(ret), K(alter_table_schema), KPC(orig_table_schema)); } else if (OB_FAIL(get_all_dropped_column_ids(alter_table_arg, *orig_table_schema, drop_cols_id_arr))) { LOG_WARN("fail to get drop cols id set", K(ret)); - } else if (OB_FAIL(reconstruct_index_schema(*orig_table_schema, + } else if (OB_FAIL(reconstruct_index_schema(alter_table_arg, + *orig_table_schema, *hidden_table_schema, - schema_guard, + *src_tenant_schema_guard, + *dst_tenant_schema_guard, drop_cols_id_arr, col_name_map, *tz_info_wrap.get_time_zone_info(), @@ -15215,24 +15432,25 @@ int ObDDLService::rebuild_hidden_table_index(obrpc::ObAlterTableArg &alter_table } else if (OB_FAIL(add_new_index_schema(alter_table_arg, *orig_table_schema, *hidden_table_schema, - schema_guard, + *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 (OB_FAIL(rebuild_hidden_table_index_in_trans(tenant_id, - schema_guard, - ddl_operator, - trans, - new_table_schemas))) { + } else if (OB_FAIL(rebuild_hidden_table_index_in_trans(dst_tenant_id, + *dst_tenant_schema_guard, + ddl_operator, + trans, + new_table_schemas))) { LOG_WARN("failed to rebuild hidden table index in trans", K(ret)); } } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { - } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { + } else if (OB_SUCCESS != (tmp_ret = publish_schema(dst_tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { @@ -15442,39 +15660,47 @@ int ObDDLService::rebuild_hidden_table_constraints(ObAlterTableArg &alter_table_ { 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(); + 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_); - const ObTableSchema *orig_table_schema = NULL; - const ObTableSchema *hidden_table_schema = NULL; - ObSchemaGetterGuard schema_guard; - schema_guard.set_session_id(alter_table_arg.session_id_); + 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(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)); + 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, - schema_guard, + *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_in_trans(alter_table_arg, - *orig_table_schema, - *hidden_table_schema, - trans, - cst_ids))) { + *orig_table_schema, + *hidden_table_schema, + trans, + cst_ids))) { LOG_WARN("failed to rebuild hidden table constraints in trans", K(ret)); } } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { - } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { + } else if (OB_SUCCESS != (tmp_ret = publish_schema(dst_tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { @@ -15956,35 +16182,43 @@ int ObDDLService::rebuild_hidden_table_foreign_key(ObAlterTableArg &alter_table_ { 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(); + 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_); - const ObTableSchema *orig_table_schema = NULL; - const ObTableSchema *hidden_table_schema = NULL; - ObSchemaGetterGuard schema_guard; - schema_guard.set_session_id(alter_table_arg.session_id_); + 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(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)); + 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, - schema_guard, + *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_foreign_key_in_trans(alter_table_arg, - *orig_table_schema, - *hidden_table_schema, - false/*rebuild_child_table_fk*/, - schema_guard, - trans, - cst_ids))) { + *orig_table_schema, + *hidden_table_schema, + false/*rebuild_child_table_fk*/, + *dst_tenant_schema_guard, + trans, + cst_ids))) { LOG_WARN("failed to rebuild hidden table foreign key in trans", K(ret)); } if (trans.is_started()) { @@ -15997,7 +16231,7 @@ int ObDDLService::rebuild_hidden_table_foreign_key(ObAlterTableArg &alter_table_ } int tmp_ret = OB_SUCCESS; if (OB_FAIL(ret)) { - } else if (OB_SUCCESS != (tmp_ret = publish_schema(tenant_id))) { + } else if (OB_SUCCESS != (tmp_ret = publish_schema(dst_tenant_id))) { LOG_WARN("publish_schema failed", K(tmp_ret)); } if (OB_SUCC(ret)) { @@ -16009,6 +16243,7 @@ int ObDDLService::rebuild_hidden_table_foreign_key(ObAlterTableArg &alter_table_ 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) @@ -16017,19 +16252,22 @@ int ObDDLService::get_orig_and_hidden_table_schema( 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(schema_guard.get_table_schema(tenant_id, hidden_table_id, hidden_table_schema))) { + } 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(orig_table_id)); + 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(hidden_table_id)); + 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(orig_table_schema->get_association_table_id())); + 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; } @@ -16144,12 +16382,10 @@ int ObDDLService::unbind_hidden_tablets( } int ObDDLService::write_ddl_barrier( - const ObTableSchema &orig_table_schema, const ObTableSchema &hidden_table_schema, const uint64_t session_id, ObDDLSQLTransaction &trans) { - UNUSED(orig_table_schema); int ret = OB_SUCCESS; const uint64_t tenant_id = hidden_table_schema.get_tenant_id(); ObArray hidden_tablets; @@ -16280,16 +16516,17 @@ int ObDDLService::swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter } 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))) { + 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))) { + 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))) { @@ -16307,8 +16544,8 @@ int ObDDLService::swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter *orig_table_schema, *hidden_table_schema, true/*rebuild_child_table_fk*/, schema_guard, trans, fk_cst_ids))) { LOG_WARN("failed to rebuild hidden table fk", K(ret)); } else if (OB_FAIL(check_hidden_table_constraint_exist(hidden_table_schema, - orig_table_schema, - schema_guard))) { + 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_) { @@ -16329,7 +16566,7 @@ int ObDDLService::swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter 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)); + "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)); @@ -16396,7 +16633,7 @@ int ObDDLService::swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter } } if (OB_SUCC(ret)) { - if (OB_FAIL(write_ddl_barrier(*orig_table_schema, *hidden_table_schema, alter_table_arg.session_id_, trans))) { + if (OB_FAIL(write_ddl_barrier(*hidden_table_schema, alter_table_arg.session_id_, trans))) { LOG_WARN("failed to write ddl barrier", K(ret)); } } @@ -16433,6 +16670,230 @@ int ObDDLService::swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter 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(); + const char *cst_type_name = is_unique_index ? "_OBUNIQUE_" : "_OBIDX_"; + const int64_t cst_type_name_len = static_cast(strlen(cst_type_name)); + if (0 != ObCharset::instr(ObCollationType::CS_TYPE_UTF8MB4_BIN, + new_index_schema.get_table_name_str().ptr(), new_index_schema.get_table_name_str().length(), cst_type_name, cst_type_name_len)) { + // 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)); + } + } + } + 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 ObString &old_cst_name = (*iter)->get_constraint_name(); + const ObConstraintType &cst_type = (*iter)->get_constraint_type(); + const char *cst_type_name = CONSTRAINT_TYPE_PRIMARY_KEY == cst_type ? "_OBPK_" : + CONSTRAINT_TYPE_CHECK == cst_type ? "_OBCHECK_" : + CONSTRAINT_TYPE_UNIQUE_KEY == cst_type ? "_OBUNIQUE_" : + CONSTRAINT_TYPE_NOT_NULL == cst_type ? "_OBNOTNULL_" : nullptr; + const int64_t cst_type_name_len = static_cast(strlen(cst_type_name)); + if (nullptr == cst_type_name) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected err", K(ret), K(cst_type), K(old_cst_name), K(**iter)); + } else if (OB_FAIL(new_constraint.assign(**iter))) { + LOG_WARN("assign failed", K(ret)); + } else if (0 != ObCharset::instr(ObCollationType::CS_TYPE_UTF8MB4_BIN, + old_cst_name.ptr(), old_cst_name.length(), cst_type_name, cst_type_name_len)) { + 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)); + } + } + 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) { + LOG_INFO("duplicated constraint, can ignore", 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::make_recover_restore_tables_visible(obrpc::ObAlterTableArg &alter_table_arg) +{ + int ret = OB_SUCCESS; + common::ObArenaAllocator allocator(lib::ObLabel("RebuildCST")); + ObDDLSQLTransaction trans(schema_service_); + 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_; + const int64_t hidden_table_id = alter_table_arg.hidden_table_id_; + if (OB_FAIL(check_inner_stat())) { + LOG_WARN("variable is not init", K(ret)); + } else { + bool is_oracle_mode = false; + 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(dst_tenant_schema_guard->get_table_schema(dst_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(hidden_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("failed to get orig table schema", K(ret), K(dst_tenant_id), K(hidden_table_id)); + } 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); + ObArray 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(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, alter_table_arg.session_id_, trans))) { + LOG_WARN("failed to write ddl barrier", K(ret)); + } + } + if (trans.is_started()) { + int temp_ret = OB_SUCCESS; + if (OB_SUCCESS != (temp_ret = trans.end(OB_SUCC(ret)))) { + LOG_WARN("trans end failed", "is_commit", OB_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; @@ -16460,6 +16921,7 @@ int ObDDLService::modify_hidden_table_fk_state(obrpc::ObAlterTableArg &alter_tab } else if (OB_FAIL(trans.start(sql_proxy_, tenant_id, refreshed_schema_version))) { LOG_WARN("failed to start trans, ", KR(ret), K(tenant_id), K(refreshed_schema_version)); } else if (OB_FAIL(get_orig_and_hidden_table_schema(alter_table_arg, + schema_guard, schema_guard, alter_table_schema, orig_table_schema, @@ -16540,7 +17002,7 @@ int ObDDLService::modify_hidden_table_fk_state(obrpc::ObAlterTableArg &alter_tab 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))) { + 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, @@ -16705,6 +17167,7 @@ int ObDDLService::cleanup_garbage(ObAlterTableArg &alter_table_arg) } 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, diff --git a/src/rootserver/ob_ddl_service.h b/src/rootserver/ob_ddl_service.h index 3b09d1513c..b1c911a6df 100644 --- a/src/rootserver/ob_ddl_service.h +++ b/src/rootserver/ob_ddl_service.h @@ -220,6 +220,14 @@ public: const share::schema::ObTableSchema **table_schema); int create_hidden_table(const obrpc::ObCreateHiddenTableArg &create_hidden_table_arg, obrpc::ObCreateHiddenTableRes &res); + /** + * For recover restore table ddl task, it is a cross-tenant task, including, + * 1. Create a hidden table under different tenant but associated with the source table; + * 2. Do not change any attribute of the source table; + * 3. Create a recover ddl task to complement the data of the hidden table; + */ + int recover_restore_table_ddl_task( + const obrpc::ObRecoverRestoreTableDDLArg &arg); int check_index_on_foreign_key(const ObTableSchema *index_table_schema, const common::ObIArray &foreign_key_infos, bool &have_index); @@ -476,7 +484,6 @@ public: const uint64_t session_id, ObDDLSQLTransaction &trans); int write_ddl_barrier( - const share::schema::ObTableSchema &orig_table_schema, const share::schema::ObTableSchema &hidden_table_schema, const uint64_t session_id, ObDDLSQLTransaction &trans); @@ -497,6 +504,35 @@ public: * step5: rename hidden table name to orig table name and modify the state to non-hidden */ int swap_orig_and_hidden_table_state(obrpc::ObAlterTableArg &alter_table_arg); + + /** + * The function is designed for the recover restore table ddl, which is to check whether the object + * in table schema is duplicated with others in the sample table space. + * If the object is named by the default function of the system, then a new object name will be + * generated to replace the old. + */ + int check_and_replace_default_index_name_on_demand( + const bool is_oracle_mode, + common::ObIAllocator &allocator, + const ObTableSchema &hidden_data_schema, + const ObString &target_data_table_name, + ObTableSchema &new_index_schema); + int check_and_replace_dup_constraint_name_on_demand( + const bool is_oracle_mode, + ObSchemaGetterGuard &tenant_schema_guard, + ObTableSchema &hidden_data_schema, + common::ObIAllocator &allocator, + ObDDLOperator &ddl_operator, + ObDDLSQLTransaction &trans); + + + /** + * This function is called by the DDL RESTORE TABLE TASK. + * This task will create a hidden table, but will not be associated with the original table, + * and any attribute of the origin table will not change to avoid cross-tenant transactions. + * And the following function will make the hidden table and its' rebuilt indexes visible after data filling. + */ + int make_recover_restore_tables_visible(obrpc::ObAlterTableArg &alter_table_arg); /** * This function is called by the storage layer in the second stage of offline ddl * For foreign keys that refer to the columns of the original table, a corresponding hidden @@ -929,6 +965,34 @@ int check_table_udt_id_is_exist(share::schema::ObSchemaGetterGuard &schema_guard obrpc::ObCommonRpcProxy *get_common_rpc() { return common_rpc_; } int get_tenant_schema_guard_with_version_in_inner_table(const uint64_t tenant_id, share::schema::ObSchemaGetterGuard &schema_guard); + /** + * NOTICE: The interface is designed for Offline DDL operation only. + * The caller can not obtain the schema via the hold_buf_src_tenant_schema_guard whose + * validity is limited by whether src_tenant_id and dst_tenant_id are the same. + * + * 1. This interface will provide the same tenant schema guard when src_tenant_id = dst_tenant_id, + * to avoid using two different versions of the guard caused by the parallel ddl under the tenant. + * 2. This interface will provide corresponding tenant schema guard when src_tenant_id != dst_tenant_id. + * + * @param [in] src_tenant_id + * @param [in] dst_tenant_id + * @param [in] hold_buf_src_tenant_schema_guard: hold buf, invalid when src_tenant_id = dst_tenant_id. + * @param [in] hold_buf_dst_tenant_schema_guard: hold buf. + * @param [out] src_tenant_schema_guard: + * pointer to the hold_buf_dst_tenant_schema_guard if src_tenant_id = dst_tenant_id, + * pointer to the hold_buf_src_tenant_schema_guard if src_tenant_id != dst_tenant_id, + * is always not nullptr if the interface return OB_SUCC. +* @param [out] dst_tenant_schema_guard: + * pointer to the hold_buf_dst_tenant_schema_guard, + * is always not nullptr if the interface return OB_SUCC. + */ + int 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 drop_index_to_recyclebin(const share::schema::ObTableSchema &table_schema); int check_tenant_in_alter_locality(const uint64_t tenant_id, bool &in_alter_locality); // trigger @@ -1316,15 +1380,17 @@ private: bool &is_match); // 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 - int create_user_hidden_table(const share::schema::ObTableSchema &orig_table_schema, - share::schema::ObTableSchema &hidden_table_schema, - const obrpc::ObSequenceDDLArg *sequence_ddl_arg, - const bool bind_tablets, - share::schema::ObSchemaGetterGuard &schema_guard, - ObDDLOperator &ddl_operator, - common::ObMySQLTransaction &trans, - common::ObIAllocator &allocator, - const ObString &index_name = ObString("")); + int create_user_hidden_table( + const share::schema::ObTableSchema &orig_table_schema, + share::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, + common::ObMySQLTransaction &trans, + common::ObIAllocator &allocator, + const ObString &index_name = ObString("")); int rebuild_triggers_on_hidden_table( const share::schema::ObTableSchema &orig_table_schema, const share::schema::ObTableSchema &hidden_table_schema, @@ -1454,6 +1520,7 @@ private: int get_orig_and_hidden_table_schema( const obrpc::ObAlterTableArg &alter_table_arg, share::schema::ObSchemaGetterGuard &schema_guard, + share::schema::ObSchemaGetterGuard &dest_schema_guard, const share::schema::AlterTableSchema &alter_table_schema, const share::schema::ObTableSchema *&orig_table_schema, const share::schema::ObTableSchema *&hidden_table_schema); @@ -1513,6 +1580,7 @@ private: 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 &new_table_schemas, @@ -1523,9 +1591,11 @@ private: const bool is_oracle_mode, bool &need_rebuild); int reconstruct_index_schema( + obrpc::ObAlterTableArg &alter_table_arg, 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, const common::ObIArray &drop_cols_id_arr, const share::ObColumnNameMap &col_name_map, const common::ObTimeZoneInfo &tz_info, @@ -1905,7 +1975,6 @@ public: int ddl_rlock(); int ddl_wlock(); int ddl_unlock() { return ddl_lock_.unlock(); } - private: int generate_tenant_schema( const obrpc::ObCreateTenantArg &arg, diff --git a/src/rootserver/ob_root_service.cpp b/src/rootserver/ob_root_service.cpp index 596bf5a7b3..504f85be0f 100755 --- a/src/rootserver/ob_root_service.cpp +++ b/src/rootserver/ob_root_service.cpp @@ -103,9 +103,11 @@ #include "share/restore/ob_physical_restore_table_operator.h"//ObPhysicalRestoreTableOperator #include "share/ob_cluster_event_history_table_operator.h"//CLUSTER_EVENT_INSTANCE #include "share/scn.h" +#include "share/restore/ob_recover_table_util.h" #include "rootserver/backup/ob_backup_proxy.h" //ObBackupServiceProxy #include "logservice/palf_handle_guard.h" #include "logservice/ob_log_service.h" +#include "rootserver/restore/ob_recover_table_initiator.h" #include "rootserver/ob_heartbeat_service.h" #include "parallel_ddl/ob_create_table_helper.h" // ObCreateTableHelper @@ -3883,6 +3885,12 @@ int ObRootService::execute_ddl_task(const obrpc::ObAlterTableArg &arg, } break; } + case share::MAKE_RECOVER_RESTORE_TABLE_TASK_TAKE_EFFECT: { + if (OB_FAIL(ddl_service_.make_recover_restore_tables_visible(const_cast(arg)))) { + LOG_WARN("make recovert restore task visible failed", K(ret), K(arg)); + } + break; + } default: ret = OB_ERR_UNEXPECTED; LOG_WARN("unknown ddl task type", K(ret), K(arg.ddl_task_type_)); @@ -3968,7 +3976,8 @@ int ObRootService::precheck_interval_part(const obrpc::ObAlterTableArg &arg) return ret; } -int ObRootService::create_hidden_table(const obrpc::ObCreateHiddenTableArg &arg, obrpc::ObCreateHiddenTableRes &res) +int ObRootService::create_hidden_table(const obrpc::ObCreateHiddenTableArg &arg, + obrpc::ObCreateHiddenTableRes &res) { LOG_DEBUG("receive create hidden table arg", K(arg)); int ret = OB_SUCCESS; @@ -3986,7 +3995,7 @@ int ObRootService::create_hidden_table(const obrpc::ObCreateHiddenTableArg &arg, ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(ret), K(arg)); } else if (OB_FAIL(ddl_service_.create_hidden_table(arg, res))) { - LOG_WARN("do create hidden table in trans failed", K(ret)); + LOG_WARN("do create hidden table in trans failed", K(ret), K(arg)); } return ret; } @@ -4120,6 +4129,28 @@ int ObRootService::start_redef_table(const obrpc::ObStartRedefTableArg &arg, obr return ret; } +int ObRootService::recover_restore_table_ddl(const obrpc::ObRecoverRestoreTableDDLArg &arg) +{ + int ret = OB_SUCCESS; + uint64_t compat_version = 0; + if (OB_FAIL(GET_MIN_DATA_VERSION(arg.src_tenant_id_, compat_version))) { + LOG_WARN("fail to get data version", K(ret), K(arg)); + } else if (compat_version < DATA_VERSION_4_2_1_0) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("version 4.0 does not support this operation", K(ret)); + } else if (OB_UNLIKELY(!inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else if (OB_UNLIKELY(!arg.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(arg)); + } else if (OB_FAIL(ddl_service_.recover_restore_table_ddl_task(arg))) { + LOG_WARN("recover restore table ddl task failed", K(ret), K(arg)); + } + LOG_INFO("recover restore table ddl finish", K(ret), K(arg)); + return ret; +} + int ObRootService::alter_table(const obrpc::ObAlterTableArg &arg, obrpc::ObAlterTableRes &res) { LOG_DEBUG("receive alter table arg", K(arg)); @@ -4172,7 +4203,7 @@ int ObRootService::alter_table(const obrpc::ObAlterTableArg &arg, obrpc::ObAlter } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, nonconst_arg.alter_table_schema_.get_database_name(), nonconst_arg.alter_table_schema_.get_origin_table_name(), - false /* is_index */, + false /* is_index*/, orig_table_schema))) { LOG_WARN("fail to get and check table schema", K(ret)); } else if (OB_ISNULL(orig_table_schema)) { @@ -4189,7 +4220,7 @@ int ObRootService::alter_table(const obrpc::ObAlterTableArg &arg, obrpc::ObAlter arg.consumer_group_id_, &allocator, &arg, - 0 /* parent task id*/); + 0 /*parent task id*/); if (OB_FAIL(ddl_scheduler_.create_ddl_task(param, sql_proxy_, task_record))) { LOG_WARN("submit ddl task failed", K(ret), K(arg)); } else if (OB_FAIL(ddl_scheduler_.schedule_ddl_task(task_record))) { @@ -8549,7 +8580,7 @@ int ObRootService::check_restore_tenant_valid(const share::ObPhysicalRestoreJob ObArray zones; if (OB_FAIL(pool_list_str.assign(job_info.get_pool_list()))) { LOG_WARN("failed to assign pool list", KR(ret), K(job_info)); - } else if (OB_FAIL(ObRestoreService::assign_pool_list(pool_list_str.ptr(), pool_list))) { + } else if (OB_FAIL(ObRestoreScheduler::assign_pool_list(pool_list_str.ptr(), pool_list))) { LOG_WARN("failed to assgin pool list", KR(ret), K(pool_list_str)); } else if (OB_FAIL(ObUnitManager::convert_pool_name_list(pool_list, pools))) { LOG_WARN("fail to convert pools", KR(ret), K(pool_list)); @@ -10095,7 +10126,7 @@ int ObRootService::build_ddl_single_replica_response(const obrpc::ObDDLBuildSing ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(arg)); } else if (OB_FAIL(ddl_scheduler_.on_sstable_complement_job_reply( - arg.tablet_id_/*source tablet id*/, ObDDLTaskKey(arg.tenant_id_, arg.dest_schema_id_, arg.schema_version_), arg.snapshot_version_, arg.execution_id_, arg.ret_code_, info))) { + arg.tablet_id_/*source tablet id*/, ObDDLTaskKey(arg.dest_tenant_id_, arg.dest_schema_id_, arg.dest_schema_version_), arg.snapshot_version_, arg.execution_id_, arg.ret_code_, info))) { LOG_WARN("handle column checksum calc response failed", K(ret), K(arg)); } return ret; @@ -10436,6 +10467,48 @@ int ObRootService::set_cpu_quota_concurrency_config_() return ret; } +int ObRootService::handle_recover_table(const obrpc::ObRecoverTableArg &arg) +{ + int ret = OB_SUCCESS; + LOG_INFO("succeed received recover table arg", K(arg)); + uint64_t data_version = 0; + if (!inited_) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else if (!arg.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(arg)); + } else if (GCTX.is_standby_cluster()) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("recover table in standby tenant is not allowed", K(ret), K(arg)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "recover table in standby tenant"); + } else if (GCONF.in_upgrade_mode()) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("recover table in upgrade mode is not allowed", K(ret), K(arg)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "Cluster is in upgrade mode, recover table is"); + } else if (OB_FAIL(ObRecoverTableUtil::check_compatible(arg.tenant_id_))) { + LOG_WARN("check recover table compatible failed", K(ret), K(arg)); + } else { + ObRecoverTableInitiator initiator; + bool is_exist = false; + if (OB_FAIL(initiator.init(schema_service_, &sql_proxy_))) { + LOG_WARN("failed to init ObRecoverTableInitiator", K(ret)); + } else if (ObRecoverTableArg::Action::INITIATE == arg.action_ + && OB_FAIL(initiator.is_recover_job_exist(arg.tenant_id_, is_exist))) { + LOG_WARN("failed to check recover job exist", K(ret), K(arg)); + } else if (is_exist) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("recover job is exist", K(ret), K(arg)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "recover table when recover table job exists is"); + } else if (OB_FAIL(initiator.initiate_recover_table(arg))) { + LOG_WARN("failed to initiate table recover", K(ret), K(arg)); + } else { + LOG_INFO("[RECOVER_TABLE] initiate recover table succeed", K(arg)); + } + } + return ret; +} + int ObRootService::recompile_all_views_batch(const obrpc::ObRecompileAllViewsBatchArg &arg) { int ret = OB_SUCCESS; diff --git a/src/rootserver/ob_root_service.h b/src/rootserver/ob_root_service.h index 8114b6db55..cbaa33051b 100644 --- a/src/rootserver/ob_root_service.h +++ b/src/rootserver/ob_root_service.h @@ -487,6 +487,14 @@ public: int abort_redef_table(const obrpc::ObAbortRedefTableArg &arg); int update_ddl_task_active_time(const obrpc::ObUpdateDDLTaskActiveTimeArg &arg); int create_hidden_table(const obrpc::ObCreateHiddenTableArg &arg, obrpc::ObCreateHiddenTableRes &res); + /** + * For recover restore table ddl, data insert into the target table is selected from another tenant. + * The function is used to create a hidden target table without any change on the source table, + * and then register a recover task into ddl task queue to finish the all procedures. + * The format about the command is, + * alter system recover table test.t1 to tenant backup_oracle_tenant from '$ARCHIVE_FILES_PATH' with 'pool_list=small_pool_0&primary_zone=z1' remap table test.t1:recover_test.t3; + */ + int recover_restore_table_ddl(const obrpc::ObRecoverRestoreTableDDLArg &arg); int execute_ddl_task(const obrpc::ObAlterTableArg &arg, common::ObSArray &obj_ids); int cancel_ddl_task(const obrpc::ObCancelDDLTaskArg &arg); int alter_tablegroup(const obrpc::ObAlterTablegroupArg &arg); @@ -789,6 +797,7 @@ public: int handle_validate_database(const obrpc::ObBackupManageArg &arg); int handle_validate_backupset(const obrpc::ObBackupManageArg &arg); int handle_cancel_validate(const obrpc::ObBackupManageArg &arg); + int handle_recover_table(const obrpc::ObRecoverTableArg &arg); int disaster_recovery_task_reply(const obrpc::ObDRTaskReplyResult &arg); int standby_upgrade_virtual_schema(const obrpc::ObDDLNopOpreatorArg &arg); int check_backup_scheduler_working(obrpc::Bool &is_working); diff --git a/src/rootserver/ob_rs_rpc_processor.h b/src/rootserver/ob_rs_rpc_processor.h index dd55ffa116..d9f0e6346a 100644 --- a/src/rootserver/ob_rs_rpc_processor.h +++ b/src/rootserver/ob_rs_rpc_processor.h @@ -340,6 +340,7 @@ DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_CREATE_TABLEGROUP, ObRpcCreateTablegroupP, DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_DROP_TABLEGROUP, ObRpcDropTablegroupP, drop_tablegroup(arg_)); DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_ALTER_TABLEGROUP, ObRpcAlterTablegroupP, alter_tablegroup(arg_)); DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_CREATE_TABLE, ObRpcCreateTableP, create_table(arg_, result_)); +DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_RECOVER_RESTORE_TABLE_DDL, ObRpcRecoverRestoreTableDDLP, recover_restore_table_ddl(arg_)); DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_PARALLEL_CREATE_TABLE, ObRpcParallelCreateTableP, parallel_create_table(arg_, result_)); DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_ALTER_TABLE, ObRpcAlterTableP, alter_table(arg_, result_)); DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_DROP_TABLE, ObRpcDropTableP, drop_table(arg_, result_)); @@ -524,6 +525,7 @@ DEFINE_RS_RPC_PROCESSOR(obrpc::OB_BACKUP_MANAGE, ObBackupManageP, handle_backup_ DEFINE_RS_RPC_PROCESSOR(obrpc::OB_BACKUP_CLEAN, ObBackupCleanP, handle_backup_delete(arg_)); DEFINE_RS_RPC_PROCESSOR(obrpc::OB_DELETE_POLICY, ObDeletePolicyP, handle_delete_policy(arg_)); DEFINE_RS_RPC_PROCESSOR(obrpc::OB_PHYSICAL_RESTORE_RES, ObRpcPhysicalRestoreResultP, send_physical_restore_result(arg_)); +DEFINE_RS_RPC_PROCESSOR(obrpc::OB_RECOVER_TABLE, ObRecoverTableP, handle_recover_table(arg_)); DEFINE_RS_RPC_PROCESSOR(obrpc::OB_RS_FLUSH_OPT_STAT_MONITORING_INFO, ObRpcFlushOptStatMonitoringInfoP, flush_opt_stat_monitoring_info(arg_)); diff --git a/src/rootserver/restore/ob_import_table_job_scheduler.cpp b/src/rootserver/restore/ob_import_table_job_scheduler.cpp new file mode 100644 index 0000000000..f59c168d82 --- /dev/null +++ b/src/rootserver/restore/ob_import_table_job_scheduler.cpp @@ -0,0 +1,791 @@ +/** + * 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 "ob_import_table_job_scheduler.h" +#include "ob_recover_table_initiator.h" +#include "lib/mysqlclient/ob_mysql_transaction.h" +#include "storage/ddl/ob_ddl_server_client.h" +#include "share/backup/ob_backup_struct.h" +#include "rootserver/ddl_task/ob_ddl_task.h" +#include "sql/engine/cmd/ob_ddl_executor_util.h" +#include "share/ob_ddl_error_message_table_operator.h" +#include "rootserver/ob_rs_event_history_table_operator.h" +#include "share/restore/ob_import_util.h" + +using namespace oceanbase; +using namespace rootserver; +using namespace common; +using namespace share; + +ObImportTableJobScheduler::ObImportTableJobScheduler() + : is_inited_(false), tenant_id_(OB_INVALID_TENANT_ID), + schema_service_(nullptr), sql_proxy_(nullptr), + job_helper_(), task_helper_() +{} + +int ObImportTableJobScheduler::init( + share::schema::ObMultiVersionSchemaService &schema_service, common::ObMySQLProxy &sql_proxy) +{ + int ret = OB_SUCCESS; + const uint64_t tenant_id = gen_user_tenant_id(MTL_ID()); + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObImportTableJobScheduler init twice", K(ret)); + } else if (OB_FAIL(job_helper_.init(tenant_id))) { + LOG_WARN("failed to init table op", K(ret), K(tenant_id)); + } else if (OB_FAIL(task_helper_.init(tenant_id))) { + LOG_WARN("failed to init table op", K(ret), K(tenant_id)); + } else { + schema_service_ = &schema_service; + sql_proxy_ = &sql_proxy; + tenant_id_ = tenant_id; + is_inited_ = true; + } + return ret; +} + +void ObImportTableJobScheduler::do_work() +{ + int ret = OB_SUCCESS; + uint64_t data_version = 0; + ObArray jobs; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init ObImportTableJobScheduler", K(ret)); + } else if (is_sys_tenant(tenant_id_)) { + // no import table job in sys tenant + } else if (OB_FAIL(check_compatible_())) { + LOG_WARN("check compatible failed", K(ret)); + } else if (OB_FAIL(job_helper_.get_all_import_table_jobs(*sql_proxy_, jobs))) { + LOG_WARN("failed to get recover all recover table job", K(ret)); + } else { + ARRAY_FOREACH(jobs, i) { + ObImportTableJob &job = jobs.at(i); + if (!job.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("recover table job is not valid", K(ret), K(job)); + } else if (is_user_tenant(job.get_tenant_id())) { + process_(job); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid tenant", K(ret), K(job)); + } + } + } +} + +int ObImportTableJobScheduler::check_compatible_() const +{ + int ret = OB_SUCCESS; + uint64_t data_version = 0; + if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id_, data_version))) { + LOG_WARN("fail to get data version", K(ret), K_(tenant_id)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("min data version is smaller than v4.2.1", K(ret), K_(tenant_id), K(data_version)); + } else if (OB_FAIL(GET_MIN_DATA_VERSION(gen_meta_tenant_id(tenant_id_), data_version))) { + LOG_WARN("fail to get data version", K(ret), "tenant_id", gen_meta_tenant_id(tenant_id_)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("min data version is smaller than v4.2.1", K(ret), K_(tenant_id), K(data_version)); + } + return ret; +} + +int ObImportTableJobScheduler::process_(share::ObImportTableJob &job) +{ + int ret = OB_SUCCESS; + switch(job.get_status()) { + case ObImportTableJobStatus::INIT: { + if (OB_FAIL(gen_import_table_task_(job))) { + LOG_WARN("failed to gen import table task", K(ret), K(job)); + } + break; + } + case ObImportTableJobStatus::IMPORT_TABLE: { + if (OB_FAIL(deal_with_import_table_task_(job))) { + LOG_WARN("failed to deal with import table task", K(ret), K(job)); + } + break; + } + case ObImportTableJobStatus::RECONSTRUCT_REF_CONSTRAINT: { + if (OB_FAIL(reconstruct_ref_constraint_(job))) { + LOG_WARN("failed to deal with reconstrcut ref constraint", K(ret)); + } + break; + } + case ObImportTableJobStatus::CANCELING: { + if (OB_FAIL(canceling_(job))) { + LOG_WARN("failed to cancel", K(ret), K(job)); + } + break; + } + case ObImportTableJobStatus::IMPORT_FINISH: { + if (OB_FAIL(finish_(job))) { + LOG_WARN("failed to cancel", K(ret), K(job)); + } + break; + } + default: { + ret = OB_ERR_SYS; + LOG_WARN("invalid import job status", K(ret)); + break; + } + } + return ret; +} + +int ObImportTableJobScheduler::reconstruct_ref_constraint_(share::ObImportTableJob &job) +{ + int ret = OB_SUCCESS; + ObArray import_tasks; + ObImportTableJobStatus next_status = ObImportTableJobStatus::get_next_status(job.get_status()); + LOG_INFO("[IMPORT_TABLE]start reconstruct ref constraint", K(job)); + if (OB_FAIL(get_import_table_tasks_(job, import_tasks))) { + LOG_WARN("failed to get import table task", K(ret)); + } else if (OB_FALSE_IT(job.set_end_ts(ObTimeUtility::current_time()))) { + } else if (OB_FAIL(advance_status_(*sql_proxy_, job, next_status))) { + LOG_WARN("failed to advance status", K(ret), K(job), K(next_status)); + } else { + LOG_INFO("[IMPORT_TABLE]finish reconstruct ref constraint", K(job)); + ROOTSERVICE_EVENT_ADD("import_table", "reconstruct_ref_constraint", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id()); + } + return ret; +} + +int ObImportTableJobScheduler::finish_(const share::ObImportTableJob &job) +{ + int ret = OB_SUCCESS; + const uint64_t tenant_id = job.get_tenant_id(); + const int64_t job_id = job.get_job_id(); + if (OB_FAIL(task_helper_.move_import_task_to_history(*sql_proxy_, tenant_id, job_id))) { + LOG_WARN("failed to move import task to history", K(ret), K(tenant_id), K(job_id)); + } else if (OB_FAIL(job_helper_.move_import_job_to_history(*sql_proxy_, tenant_id, job_id))) { + LOG_WARN("failed to move import job to history", K(ret), K(tenant_id), K(job_id)); + } else { + LOG_INFO("[IMPORT_TABLE]import table job finish", K(job)); + ROOTSERVICE_EVENT_ADD("import_table", "import table finish", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id()); + } + return ret; +} + +int ObImportTableJobScheduler::gen_import_table_task_(share::ObImportTableJob &job) +{ + int ret = OB_SUCCESS; + ObImportTableTaskGenerator generator; + ObArray import_tasks; + ObMySQLTransaction trans; + uint64_t meta_tenant_id = gen_meta_tenant_id(job.get_tenant_id()); + DEBUG_SYNC(BEFORE_GENERATE_IMPORT_TABLE_TASK); + if (OB_FAIL(generator.init(*schema_service_, *sql_proxy_))) { + LOG_WARN("failed to init import task generator", K(ret)); + } else if (OB_FAIL(generator.gen_import_task(job, import_tasks))) { + LOG_WARN("failed to gen import table task", K(ret), K(job)); + if (!ObImportTableUtil::can_retrieable_err(ret)) { + int tmp_ret = OB_SUCCESS; + ObImportTableJobStatus next_status(ObImportTableJobStatus::IMPORT_FINISH); + job.set_end_ts(ObTimeUtility::current_time()); + if (OB_TMP_FAIL(advance_status_(*sql_proxy_, job, next_status))) { + LOG_WARN("failed to advance status", K(ret)); + } + } + } else if (OB_FAIL(trans.start(sql_proxy_, meta_tenant_id))) { + LOG_WARN("failed to start trans", K(ret), K(meta_tenant_id)); + } else { + ARRAY_FOREACH(import_tasks, i) { + const ObImportTableTask &task = import_tasks.at(i); + if (OB_FAIL(persist_import_table_task_(trans, task))) { + LOG_WARN("failed to persist import table task", K(ret), K(task)); + } else { + job.set_total_bytes(job.get_total_bytes() + task.get_total_bytes()); + job.set_total_table_count(job.get_total_table_count() + 1); + } + } + + ObImportTableJobStatus next_status = ObImportTableJobStatus::get_next_status(job.get_status()); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(job_helper_.report_import_job_statistics(*sql_proxy_, job))) { + LOG_WARN("failed to report import job statistics", K(ret)); + } else if (!next_status.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("error import table job status is unexpected", K(ret), K(next_status)); + } else if (OB_FAIL(advance_status_(trans, job, next_status))) { + LOG_WARN("failed to advance to next status", K(ret)); + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(trans.end(true))) { + LOG_WARN("failed to commit", K(ret)); + } else { + LOG_INFO("[IMPORT_TABLE] succeed generate import table task", K(import_tasks), K(next_status)); + ROOTSERVICE_EVENT_ADD("import_table", "generate import table task", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id(), + "task_count", import_tasks.count()); + } + } else { + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = trans.end(false))) { + LOG_WARN("failed to roll back", K(ret), K(tmp_ret)); + } + } + } + return ret; +} + +int ObImportTableJobScheduler::deal_with_import_table_task_(share::ObImportTableJob &job) +{ + int ret = OB_SUCCESS; + ObImportTableTask task; + bool all_finish = false; + if (OB_FAIL(task_helper_.get_one_not_finish_task_by_initiator(*sql_proxy_, job, all_finish, task))) { + LOG_WARN("failed to get import table task", K(ret), K(job)); + } else if (!all_finish) { + if (OB_FAIL(process_import_table_task_(task))) { + LOG_WARN("failed to process import table task", K(ret), K(task)); + } + } else if (OB_FAIL(do_after_import_all_table_(job))) { + LOG_WARN("failed to do after import all table", K(ret), K(job)); + } + return ret; +} + +int ObImportTableJobScheduler::process_import_table_task_(share::ObImportTableTask &task) +{ + int ret = OB_SUCCESS; + ObImportTableTaskScheduler task_mgr; + if (OB_FAIL(task_mgr.init(*schema_service_, *sql_proxy_, task))) { + LOG_WARN("failed to init task mgr", K(ret)); + } else { + task_mgr.process(); + } + return ret; +} + +int ObImportTableJobScheduler::do_after_import_all_table_(share::ObImportTableJob &job) +{ + int ret = OB_SUCCESS; + common::ObArray import_tasks; + if (OB_FAIL(get_import_table_tasks_(job, import_tasks))) { + LOG_WARN("failed to get import table tasks", K(ret), K(job)); + } + int64_t succeed_task_cnt = 0; + int64_t failed_task_cnt = 0; + ObImportResult::Comment comment; + int64_t pos = 0; + ARRAY_FOREACH(import_tasks, i) { + const ObImportTableTask &task = import_tasks.at(i); + if (task.get_result().is_succeed()) { + succeed_task_cnt++; + } else { + failed_task_cnt++; + if (OB_FAIL(databuff_printf(comment.ptr(), comment.capacity(), pos, + "%s%s%.*s", failed_task_cnt == 1 ? "import failed table list: " : "", + failed_task_cnt == 1 ? "" : ",", + task.get_src_table().length(), task.get_src_table().ptr()))) { + if (OB_SIZE_OVERFLOW == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to databuff_printf", K(ret)); + } + } + } + } + + ObImportTableJobStatus next_status = ObImportTableJobStatus::get_next_status(job.get_status()); + job.get_result().set_result(true, comment); + if (OB_FAIL(ret)) { + } else if (!next_status.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid import job status", K(ret), K(next_status)); + } else if (OB_FAIL(advance_status_(*sql_proxy_, job, next_status))) { + LOG_WARN("failed to advance to next status", K(ret)); + } else { + LOG_INFO("[IMPORT_TABLE]importing table finished", K(import_tasks), K(next_status)); + ROOTSERVICE_EVENT_ADD("import_table", "import table task finish", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id(), + "succeed_import_table_count", succeed_task_cnt, + "failed_import_table_count", failed_task_cnt); + } + return ret; +} + +int ObImportTableJobScheduler::canceling_(share::ObImportTableJob &job) +{ + int ret = OB_SUCCESS; + LOG_INFO("[IMPORT_TABLE]cancel import table job", K(job)); + ObArray import_tasks; + if (OB_FAIL(get_import_table_tasks_(job, import_tasks))) { + if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to get import table task", K(ret)); + } + } else { + ObImportTableTaskStatus next_status(ObImportTableTaskStatus::FINISH); + ARRAY_FOREACH(import_tasks, i) { + ObImportTableTask &task = import_tasks.at(i); + obrpc::ObAbortRedefTableArg arg; + arg.task_id_ = task.get_task_id(); + arg.tenant_id_ = task.get_tenant_id(); + bool is_exist = false; + if (task.get_status().is_finish()) { + } else if (OB_FAIL(check_import_ddl_task_exist_(task, is_exist))) { + LOG_WARN("failed to check import ddl task", K(ret)); + } else if (!is_exist) { + } else { + LOG_INFO("[IMPORT_TABLE]cancel import table task", K(arg)); + if (OB_FAIL(ObDDLServerClient::abort_redef_table(arg))) { + LOG_WARN("failed to abort redef table", K(ret), K(arg)); + } else if (OB_FAIL(task_helper_.advance_status(*sql_proxy_, task, next_status))) { + LOG_WARN("failed to cancel import task", K(ret), K(task)); + } else { + LOG_INFO("[IMPORT_TABLE]succeed cancel import table task", K(arg)); + } + } + } + } + + if (OB_SUCC(ret)) { + share::ObTaskId trace_id(*ObCurTraceId::get_trace_id()); + ObImportResult result; + ObImportTableJobStatus next_status(ObImportTableJobStatus::IMPORT_FINISH); + job.set_end_ts(ObTimeUtility::current_time()); + if (OB_FAIL(result.set_result(OB_CANCELED, trace_id, GCONF.self_addr_))) { + LOG_WARN("failed to set result", K(ret)); + } else if (OB_FALSE_IT(job.set_result(result))) { + } else if (OB_FAIL(advance_status_(*sql_proxy_, job, next_status))) { + LOG_WARN("failed to advance status", K(ret)); + } else { + LOG_INFO("[IMPORT_TABLE]succeed to cancel import table job", K(job)); + ROOTSERVICE_EVENT_ADD("import_table", "cancel import table task", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id()); + } + } + return ret; +} + +int ObImportTableJobScheduler::check_import_ddl_task_exist_(const share::ObImportTableTask &task, bool &is_exist) +{ + int ret = OB_SUCCESS; + is_exist = false; + const uint64_t tenant_id = task.get_tenant_id(); + const int64_t task_id = task.get_task_id(); + int64_t unused_user_msg_len = 0; + ObDDLErrorMessageTableOperator::ObBuildDDLErrorMessage error_message; + if (OB_FAIL(ObDDLTaskRecordOperator::check_task_id_exist(*sql_proxy_, tenant_id, task_id, is_exist))) { + LOG_WARN("failed to check task id exist", K(ret), K(tenant_id), K(task_id)); + } else if (is_exist) { + } else if (OB_FAIL(ObDDLErrorMessageTableOperator::get_ddl_error_message(tenant_id, + task_id, + -1 /* target_object_id */, + ObAddr()/*unused addr*/, + false /* is_ddl_retry_task */, + *sql_proxy_, + error_message, + unused_user_msg_len))) { + if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to load ddl user error", K(ret), K(tenant_id), K(task_id)); + } + } else { + is_exist = true; + } + return ret; +} + +int ObImportTableJobScheduler::persist_import_table_task_( + common::ObMySQLTransaction &trans, const share::ObImportTableTask &task) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(task_helper_.insert_import_table_task(trans, task))) { + LOG_WARN("failed to get import table job", K(ret), K(task)); + } else { + LOG_INFO("succeed to persist import table task", K(task)); + } + return ret; +} + +int ObImportTableJobScheduler::get_import_table_tasks_( + const share::ObImportTableJob &job, common::ObIArray &import_tasks) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(task_helper_.get_all_import_table_tasks_by_initiator(*sql_proxy_, job, import_tasks))) { + LOG_WARN("failed to get import table task", K(ret), K(job)); + } + return ret; +} + +int ObImportTableJobScheduler::advance_status_( + common::ObISQLClient &sql_proxy, const share::ObImportTableJob &job, const share::ObImportTableJobStatus &next_status) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(job_helper_.advance_status(sql_proxy, job, next_status))) { + LOG_WARN("failed to advance status", K(ret), K(job), K(next_status)); + } + return ret; +} + +int ObImportTableTaskScheduler::init(share::schema::ObMultiVersionSchemaService &schema_service, + common::ObMySQLProxy &sql_proxy, share::ObImportTableTask &task) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObImportTableTaskScheduler init twice", K(ret)); + } else if (OB_FAIL(helper_.init(task.get_tenant_id()))) { + LOG_WARN("failed to init recover table persist helper", K(ret)); + } else { + schema_service_ = &schema_service; + sql_proxy_ = &sql_proxy; + import_task_ = &task; + is_inited_ = true; + } + return ret; +} + +void ObImportTableTaskScheduler::reset() +{ + is_inited_ = false; + schema_service_ = nullptr; + sql_proxy_ = nullptr; + import_task_ = nullptr; +} + +int ObImportTableTaskScheduler::process() +{ + int ret = OB_SUCCESS; + LOG_INFO("ready process import table task", KPC_(import_task)); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObIImportTableTaskMgr not inited", K(ret)); + } else { + const ObImportTableTaskStatus &status = import_task_->get_status(); + switch(status) { + case ObImportTableTaskStatus::INIT: { + if (OB_FAIL(init_())) { + LOG_WARN("failed to do init work", K(ret), KPC_(import_task)); + } + break; + } + case ObImportTableTaskStatus::DOING: { + if (OB_FAIL(doing_())) { + LOG_WARN("failed to do doing work", K(ret), KPC_(import_task)); + } + break; + } + case ObImportTableTaskStatus::FINISH: { + break; // do nothing + } + default: { + ret = OB_ERR_SYS; + LOG_WARN("invalid recover task status", K(ret)); + } + } + } + return ret; +} + +int ObImportTableTaskScheduler::init_() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(gen_import_ddl_task_())) { + LOG_WARN("failed to generate import ddl task", K(ret), KPC_(import_task)); + } + + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(try_advance_status_(ret))) { + LOG_WARN("failed to advance status", K(tmp_ret), K(ret)); + ret = OB_SUCC(ret) ? tmp_ret : ret; + } + return ret; +} + +int ObImportTableTaskScheduler::doing_() +{ + int ret = OB_SUCCESS; + bool is_finish = false; + if (OB_FAIL(wait_import_ddl_task_finish_(is_finish))) { + LOG_WARN("failed to do doing work", K(ret), KPC_(import_task)); + } else if (!is_finish) { + } else if (OB_FAIL(try_advance_status_(ret))) { + LOG_WARN("failed to advance status", K(ret)); + } + return ret; +} + +int ObImportTableTaskScheduler::try_advance_status_(const int err_code) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(err_code) && ObImportTableUtil::can_retrieable_err(err_code)) { // do nothing + } else { + share::ObImportTableTaskStatus next_status = import_task_->get_status().get_next_status(err_code); + if (OB_FAIL(helper_.advance_status(*sql_proxy_, *import_task_, next_status))) { + LOG_WARN("failed to advance status", K(ret), KPC_(import_task), K(next_status)); + } + } + return ret; +} + +int ObImportTableTaskScheduler::gen_import_ddl_task_() +{ + int ret = OB_SUCCESS; + obrpc::ObRecoverRestoreTableDDLArg arg; + bool is_exist = false; + LOG_INFO("[IMPORT_TABLE]start to create import table", KPC_(import_task)); + if (OB_FAIL(check_import_ddl_task_exist_(is_exist))) { + LOG_WARN("failed to check import ddl task", K(ret)); + } else if (is_exist) { + LOG_INFO("[IMPORT_TABLE]import ddl task exist, skip it", KPC_(import_task), K(arg)); + } else if (OB_FAIL(construct_import_table_arg_(arg))) { + LOG_WARN("failed to construct import table arg", K(ret)); + } else if (OB_FAIL(ObDDLServerClient::execute_recover_restore_table(arg))) { + LOG_WARN("fail to start import table", K(ret), K(arg)); + } else { + LOG_INFO("[IMPORT_TABLE]succeed execute_recover_restore_table", KPC_(import_task), K(arg)); + } + return ret; +} + +int ObImportTableTaskScheduler::check_import_ddl_task_exist_(bool &is_exist) +{ + int ret = OB_SUCCESS; + is_exist = false; + const uint64_t tenant_id = import_task_->get_tenant_id(); + const int64_t task_id = import_task_->get_task_id(); + int64_t unused_user_msg_len = 0; + ObDDLErrorMessageTableOperator::ObBuildDDLErrorMessage error_message; + if (OB_FAIL(ObDDLTaskRecordOperator::check_task_id_exist(*sql_proxy_, tenant_id, task_id, is_exist))) { + LOG_WARN("failed to check task id exist", K(ret), K(tenant_id), K(task_id)); + } else if (is_exist) { + } else if (OB_FAIL(ObDDLErrorMessageTableOperator::get_ddl_error_message(tenant_id, + task_id, + -1 /* target_object_id */, + ObAddr()/*unused addr*/, + false /* is_ddl_retry_task */, + *sql_proxy_, + error_message, + unused_user_msg_len))) { + if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to load ddl user error", K(ret), K(tenant_id), K(task_id)); + } + } else { + is_exist = true; + } + return ret; +} + +int ObImportTableTaskScheduler::construct_import_table_arg_(obrpc::ObRecoverRestoreTableDDLArg &arg) +{ + int ret = OB_SUCCESS; + ObSchemaGetterGuard src_tenant_guard; + const ObTableSchema *src_table_schema = nullptr; + ObFixedLengthString time_zone; + MTL_SWITCH(OB_SYS_TENANT_ID) { + if (OB_FAIL(schema_service_->get_tenant_schema_guard(import_task_->get_src_tenant_id(), src_tenant_guard))) { + LOG_WARN("failed to get tenant schema guard", K(ret), KPC_(import_task)); + } else if (OB_FAIL(src_tenant_guard.get_table_schema(import_task_->get_src_tenant_id(), + import_task_->get_src_database(), + import_task_->get_src_table(), + false, + src_table_schema))) { + LOG_WARN("failed to get table schema", K(ret), KPC_(import_task)); + } + } + if (FAILEDx(construct_import_table_schema_(*src_table_schema, arg.target_schema_))) { + LOG_WARN("failed to construct import table schema", K(ret)); + } else { + arg.src_tenant_id_ = src_table_schema->get_tenant_id(); + arg.src_table_id_ = src_table_schema->get_table_id(); + arg.ddl_task_id_ = import_task_->get_task_id(); + arg.exec_tenant_id_ = import_task_->get_tenant_id(); + const ObSysVarSchema *data_format_schema = nullptr; + const ObSysVarSchema *nls_timestamp_format = nullptr; + const ObSysVarSchema *nls_timestamp_tz_format = nullptr; + if (OB_FAIL(share::ObBackupUtils::get_tenant_sys_time_zone_wrap(import_task_->get_tenant_id(), + time_zone, + arg.tz_info_wrap_))) { + LOG_WARN("failed to get tenant sys timezoen wrap", K(ret), KPC_(import_task)); + } else if (OB_FAIL(src_tenant_guard.get_tenant_system_variable(import_task_->get_src_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(src_tenant_guard.get_tenant_system_variable(import_task_->get_src_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(src_tenant_guard.get_tenant_system_variable(import_task_->get_src_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.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(); + } + } + return ret; +} + +int ObImportTableTaskScheduler::construct_import_table_schema_( + const share::schema::ObTableSchema &src_table_schema, share::schema::ObTableSchema &target_table_schema) +{ + int ret = OB_SUCCESS; + ObSchemaGetterGuard target_tenant_guard; + if (OB_FAIL(schema_service_->get_tenant_schema_guard(import_task_->get_tenant_id(), target_tenant_guard))) { + LOG_WARN("failed to get tenant schema guard", K(ret), KPC_(import_task)); + } else if (OB_FAIL(target_table_schema.assign(src_table_schema))) { + LOG_WARN("failed to assign target table schema", K(ret)); + } else { + target_table_schema.set_tenant_id(import_task_->get_tenant_id()); + target_table_schema.set_table_name(import_task_->get_target_table()); + target_table_schema.set_table_id(OB_INVALID_ID); + + target_table_schema.set_data_table_id(0); + target_table_schema.clear_constraint(); + target_table_schema.clear_foreign_key_infos(); + + uint64_t database_id = OB_INVALID_ID; + if (OB_FAIL(target_tenant_guard.get_database_id(import_task_->get_tenant_id(), + import_task_->get_target_database(), + database_id))) { + LOG_WARN("failed to get database id", K(ret), KPC_(import_task)); + } else if (OB_INVALID_ID == database_id) { + ret = OB_ERR_BAD_DATABASE; + LOG_WARN("invalid target database name", K(ret), K(database_id), KPC_(import_task)); + } else { + target_table_schema.set_database_id(database_id); + } + + uint64_t table_group_id = OB_INVALID_ID; + if (import_task_->get_target_tablegroup().empty()) { + } else if (FAILEDx(target_tenant_guard.get_tablegroup_id(import_task_->get_tenant_id(), + import_task_->get_target_tablegroup(), + table_group_id))) { + LOG_WARN("failed to get table group id", K(ret), KPC_(import_task)); + } else if (OB_INVALID_ID == table_group_id) { + ret = OB_TABLEGROUP_NOT_EXIST; + LOG_WARN("invalid target tablegroup id", K(ret), K(table_group_id), KPC_(import_task)); + } else { + target_table_schema.set_tablegroup_id(table_group_id); + } + + const schema::ObTablespaceSchema *schema = nullptr; + if (import_task_->get_target_tablespace().empty()) { + } else if (FAILEDx(target_tenant_guard.get_tablespace_schema_with_name(import_task_->get_tenant_id(), + import_task_->get_target_tablespace(), + schema))) { + LOG_WARN("failed to get tablespace schema", K(ret), KPC_(import_task)); + } else if (OB_ISNULL(schema)) { + ret = OB_TABLESPACE_NOT_EXIST; + LOG_WARN("tablespace must not be null", K(ret), KPC_(import_task)); + } else { + target_table_schema.set_tablespace_id(schema->get_tablespace_id()); + } + } + return ret; +} + +int ObImportTableTaskScheduler::wait_import_ddl_task_finish_(bool &is_finish) +{ + int ret = OB_SUCCESS; + int64_t unused_user_msg_len = 0; + ObDDLErrorMessageTableOperator::ObBuildDDLErrorMessage error_message; + uint64_t tenant_id = import_task_->get_tenant_id(); + int64_t task_id = import_task_->get_task_id(); + is_finish = false; + if (OB_FAIL(ObDDLErrorMessageTableOperator::get_ddl_error_message(tenant_id, + task_id, + -1 /* target_object_id */, + ObAddr()/*unused addr*/, + false /* is_ddl_retry_task */, + *sql_proxy_, + error_message, + unused_user_msg_len))) { + if (OB_ENTRY_NOT_EXIST == ret) { + ret = OB_SUCCESS; + if(REACH_TIME_INTERVAL(120 * 1000 * 1000)) { + LOG_WARN("[IMPORT_TABLE]import ddl task does not finish, retry again", K(tenant_id), K(task_id)); + } + } else { + LOG_WARN("failed to load ddl user error", K(ret), K(tenant_id), K(task_id)); + } + } else if (OB_SUCCESS != error_message.ret_code_) { + ObImportResult result; + if (OB_FAIL(result.set_result(false, error_message.user_message_))) { + LOG_WARN("failed to set result", K(ret), K(error_message)); + } else { + import_task_->set_result(result); + is_finish = true; + LOG_INFO("[IMPORT_TABLE]import table failed", KPC_(import_task), K(error_message)); + } + } else if (OB_FAIL(statistics_import_results_())) { + LOG_WARN("failed to statistics import result", K(ret)); + } else if (OB_FAIL(helper_.report_import_task_statistics(*sql_proxy_, *import_task_))) { + LOG_WARN("failed to report import task statistics", K(ret), KPC_(import_task)); + } else { + is_finish = true; + LOG_INFO("[IMPORT_TABLE]import table succeed", KPC_(import_task), K(error_message)); + } + return ret; +} + +int ObImportTableTaskScheduler::statistics_import_results_() +{ + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + ObSchemaGetterGuard guard; + const ObTableSchema * table_schema = nullptr; + const int64_t tenant_id = import_task_->get_tenant_id(); + const ObString &db_name = import_task_->get_target_database(); + const ObString &table_name = import_task_->get_target_table(); + if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, guard))) { + LOG_WARN("failed get tenant schema guard", K(ret), K(tenant_id)); + } else if (OB_FAIL(guard.get_table_schema(tenant_id, db_name, table_name, false/*no index*/, table_schema))) { + LOG_WARN("failed to get table schema", K(ret), K(tenant_id), K(db_name), K(table_name)); + } else if (OB_ISNULL(table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table is not exist", K(tenant_id), K(db_name), K(table_name)); + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), + "table %.*s has been deleted by user", table_name.length(), table_name.ptr()))) { + LOG_WARN("failed to databuff printf", K(ret), K(tmp_ret)); + } else { + import_task_->get_result().set_result(true, comment); + } + } else { + import_task_->set_completion_ts(ObTimeUtility::current_time()); + import_task_->set_imported_index_count(table_schema->get_simple_index_infos().count()); + import_task_->set_failed_index_count(import_task_->get_total_index_count() - import_task_->get_imported_index_count()); + import_task_->set_imported_constraint_count(table_schema->get_constraint_count()); + import_task_->set_failed_constraint_count(import_task_->get_total_constraint_count() - import_task_->get_imported_constraint_count()); + import_task_->set_imported_ref_constraint_count(table_schema->get_foreign_key_infos().count()); + import_task_->set_failed_ref_constraint_count(import_task_->get_total_ref_constraint_count() - import_task_->get_imported_ref_constraint_count()); + import_task_->set_imported_trigger_count(table_schema->get_trigger_list().count()); + import_task_->set_failed_trigger_count(import_task_->get_total_trigger_count() - import_task_->get_imported_trigger_count()); + } + return ret; +} diff --git a/src/rootserver/restore/ob_import_table_job_scheduler.h b/src/rootserver/restore/ob_import_table_job_scheduler.h new file mode 100644 index 0000000000..b7a20bb09f --- /dev/null +++ b/src/rootserver/restore/ob_import_table_job_scheduler.h @@ -0,0 +1,109 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_ROOTSERVER_IMPORT_TABLE_JOB_SCHEDULER_H +#define OCEANBASE_ROOTSERVER_IMPORT_TABLE_JOB_SCHEDULER_H + +#include "share/restore/ob_import_table_struct.h" +#include "share/restore/ob_import_table_persist_helper.h" +namespace oceanbase +{ +namespace obrpc +{ +struct ObRecoverRestoreTableDDLArg; +} +namespace share +{ +namespace schema +{ +class ObMultiVersionSchemaService; +} +} + +namespace common +{ +class ObMySQLProxy; +class ObString; +class ObMySQLTransaction; +class ObISQLClient; +} + +namespace rootserver +{ +class ObImportTableJobScheduler final +{ +public: + ObImportTableJobScheduler(); + ~ObImportTableJobScheduler() {} + int init(share::schema::ObMultiVersionSchemaService &schema_service, common::ObMySQLProxy &sql_proxy); + void do_work(); + +private: + int check_compatible_() const; + int process_(share::ObImportTableJob &job); + int gen_import_table_task_(share::ObImportTableJob &job); + int deal_with_import_table_task_(share::ObImportTableJob &job); + int process_import_table_task_(share::ObImportTableTask &task); + int do_after_import_all_table_(share::ObImportTableJob &job); + int canceling_(share::ObImportTableJob &job); + int finish_(const share::ObImportTableJob &job); + int persist_import_table_task_(common::ObMySQLTransaction &trans, const share::ObImportTableTask &task); + int get_import_table_tasks_(const share::ObImportTableJob &job, common::ObIArray &import_tasks); + int reconstruct_ref_constraint_(share::ObImportTableJob &job); + int check_import_ddl_task_exist_(const share::ObImportTableTask &task, bool &is_exist); + + int advance_status_(common::ObISQLClient &sql_proxy, const share::ObImportTableJob &job, const share::ObImportTableJobStatus &next_status); +private: + bool is_inited_; + uint64_t tenant_id_; + share::schema::ObMultiVersionSchemaService *schema_service_; + common::ObMySQLProxy *sql_proxy_; + share::ObImportTableJobPersistHelper job_helper_; + share::ObImportTableTaskPersistHelper task_helper_; + DISALLOW_COPY_AND_ASSIGN(ObImportTableJobScheduler); +}; + +class ObImportTableTaskScheduler +{ +public: + ObImportTableTaskScheduler() + : is_inited_(false), schema_service_(nullptr), sql_proxy_(nullptr), import_task_(nullptr), helper_() {} + virtual ~ObImportTableTaskScheduler() { reset(); } + int init(share::schema::ObMultiVersionSchemaService &schema_service, + common::ObMySQLProxy &sql_proxy, share::ObImportTableTask &task); + void reset(); + int process(); + TO_STRING_KV(KPC_(import_task)); +private: + int init_(); + int doing_(); + int try_advance_status_(const int err_code); + int construct_import_table_arg_(obrpc::ObRecoverRestoreTableDDLArg &arg); + int construct_import_table_schema_( + const share::schema::ObTableSchema &src_table_schema, share::schema::ObTableSchema &tartget_table_schema); + int check_import_ddl_task_exist_(bool &is_exist); + int statistics_import_results_(); + int gen_import_ddl_task_(); + int wait_import_ddl_task_finish_(bool &is_finish); +protected: + bool is_inited_; + share::schema::ObMultiVersionSchemaService *schema_service_; + common::ObMySQLProxy *sql_proxy_; + share::ObImportTableTask *import_task_; + share::ObImportTableTaskPersistHelper helper_; + DISALLOW_COPY_AND_ASSIGN(ObImportTableTaskScheduler); +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/rootserver/restore/ob_import_table_task_generator.cpp b/src/rootserver/restore/ob_import_table_task_generator.cpp new file mode 100644 index 0000000000..9c6c8481dd --- /dev/null +++ b/src/rootserver/restore/ob_import_table_task_generator.cpp @@ -0,0 +1,502 @@ +/** + * 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 "ob_recover_table_initiator.h" +#include "lib/hash/ob_hashset.h" +#include "lib/charset/ob_charset.h" +#include "share/ob_rpc_struct.h" +#include "share/backup/ob_backup_data_table_operator.h" +#include "rootserver/ob_rs_event_history_table_operator.h" +#include "ob_restore_util.h" +#include "share/restore/ob_recover_table_persist_helper.h" +#include "sql/parser/parse_node.h" +#include "rootserver/ddl_task/ob_ddl_task.h" +#include "share/restore/ob_import_util.h" + +using namespace oceanbase; +using namespace share::schema; +using namespace common; +using namespace obrpc; +using namespace rootserver; +using namespace share; + + + +int ObImportTableTaskGenerator::init( + share::schema::ObMultiVersionSchemaService &schema_service, common::ObMySQLProxy &sql_proxy) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObImportTableTaskGenerator init twice", K(ret)); + } else { + schema_service_ = &schema_service; + sql_proxy_ = &sql_proxy; + is_inited_ = true; + } + return ret; +} + +int ObImportTableTaskGenerator::gen_import_task( + share::ObImportTableJob &import_job, + common::ObIArray &import_tasks) +{ + int ret = OB_SUCCESS; + import_tasks.reset(); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableTaskGenerator not init", K(ret)); + } else if (!import_job.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid import job", K(ret), K(import_job)); + } else if (OB_FAIL(gen_db_import_tasks_(import_job, import_tasks))) { + LOG_WARN("failed to gen db import tasks", K(ret), K(import_job)); + } else if (OB_FAIL(gen_table_import_tasks_(import_job, import_tasks))) { + LOG_WARN("failed to gen table import tasks", K(ret), K(import_job)); + } else { + LOG_INFO("finish gen import task", K(ret), K(import_job), K(import_tasks)); + } + return ret; +} + +int ObImportTableTaskGenerator::gen_db_import_tasks_( + share::ObImportTableJob &import_job, + common::ObIArray &import_tasks) +{ + int ret = OB_SUCCESS; + MTL_SWITCH(OB_SYS_TENANT_ID) { + if (import_job.get_import_arg().get_import_table_arg().is_import_all()) { + ObSchemaGetterGuard guard; + uint64_t src_tenant_id = import_job.get_src_tenant_id(); + common::ObArray database_schemas; + if (OB_FAIL(ObImportTableUtil::get_tenant_schema_guard(*schema_service_, src_tenant_id, guard))) { + LOG_WARN("failed to get tenant schema guard", K(ret), K(src_tenant_id)); + } else if (OB_FAIL(guard.get_database_schemas_in_tenant(src_tenant_id, database_schemas))) { + LOG_WARN("failed to get database id", K(ret), K(src_tenant_id)); + } + ARRAY_FOREACH(database_schemas, i) { + const ObSimpleDatabaseSchema* database_schema = database_schemas.at(i); + const share::ObImportDatabaseItem db_item(database_schema->get_name_case_mode(), + database_schema->get_database_name_str().ptr(), + database_schema->get_database_name_str().length()); + if (OB_FAIL(gen_one_db_import_tasks_(import_job, db_item, import_tasks))) { + LOG_WARN("failed to generate one db import tasks", K(ret), K(db_item)); + } + } + } else { + const common::ObSArray &database_array = import_job.get_import_arg().get_import_database_array().get_items(); + ARRAY_FOREACH(database_array, i) { + const share::ObImportDatabaseItem &db_item = database_array.at(i); + if (OB_FAIL(gen_one_db_import_tasks_(import_job, db_item, import_tasks))) { + LOG_WARN("failed to generate one db import tasks", K(ret), K(db_item)); + } + } + } + } + return ret; +} + +int ObImportTableTaskGenerator::gen_one_db_import_tasks_( + share::ObImportTableJob &import_job, + const share::ObImportDatabaseItem &db_item, + common::ObIArray &import_tasks) +{ + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + ObSchemaGetterGuard guard; + uint64_t database_id = 0; + uint64_t src_tenant_id = import_job.get_src_tenant_id(); + common::ObArray table_schemas; + + if (OB_FAIL(ObImportTableUtil::get_tenant_schema_guard(*schema_service_, src_tenant_id, guard))) { + LOG_WARN("failed to get tenant schema guard", K(ret), K(src_tenant_id)); + } else if (OB_FAIL(guard.get_database_id(src_tenant_id, db_item.name_, database_id))) { + LOG_WARN("failed to get database id", K(ret), K(src_tenant_id), K(db_item)); + } else if (OB_INVALID_ID == database_id) { + ret = OB_ERR_BAD_DATABASE; + LOG_WARN("database not exist", K(ret)); + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), "import database not exist, %.*s", + db_item.name_.length(), db_item.name_.ptr()))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + import_job.get_result().set_result(false, comment); + } + } else if (OB_FAIL(guard.get_table_schemas_in_database(src_tenant_id, database_id, table_schemas))) { + LOG_WARN("failed to get table schemas", K(ret), K(src_tenant_id), K(database_id)); + } + ARRAY_FOREACH(table_schemas, i) { + share::ObImportTableTask import_task; + const ObTableSchema *table_schema = table_schemas.at(i); + ObImportTableItem table_item; + table_item.mode_ = db_item.mode_; + table_item.database_name_ = db_item.name_; + table_item.table_name_ = table_schema->get_table_name_str(); + if (!table_schema->is_user_table()) { + } else if (OB_FAIL(fill_import_task_from_import_db_(import_job, guard, db_item, table_item, *table_schema, import_task))) { + LOG_WARN("failed to fill import task", K(ret), K(import_job), K(db_item), K(table_item)); + } else if (OB_FAIL(import_tasks.push_back(import_task))) { + LOG_WARN("failed to push back import task", K(ret), K(import_task)); + } else { + LOG_INFO("[RECOVER_TABLE]succeed gen one import task", K(import_task)); + } + } + return ret; +} + +int ObImportTableTaskGenerator::gen_table_import_tasks_( + share::ObImportTableJob &import_job, + common::ObIArray &import_tasks) +{ + int ret = OB_SUCCESS; + ObSchemaGetterGuard guard; + const ObTableSchema * table_schema = nullptr; + const share::ObImportTableArg &import_arg = import_job.get_import_arg().get_import_table_arg(); + const common::ObSArray &table_array = import_arg.get_import_table_array().get_items(); + ARRAY_FOREACH(table_array, i) { + const share::ObImportTableItem &table_item = table_array.at(i); + share::ObImportTableTask import_task; + if (OB_FAIL(gen_table_import_task_(import_job, table_item, import_task))) { + LOG_WARN("failed to gen import task", K(ret), K(table_item)); + } else if (OB_FAIL(import_tasks.push_back(import_task))) { + LOG_WARN("failed to push back import task", K(ret), K(import_task)); + } else { + LOG_INFO("[RECOVER_TABLE]succeed gen one import task", K(import_task)); + } + } + + return ret; +} + +int ObImportTableTaskGenerator::gen_table_import_task_( + share::ObImportTableJob &import_job, + const share::ObImportTableItem &table_item, + share::ObImportTableTask &import_task) +{ + int ret = OB_SUCCESS; + ObSchemaGetterGuard guard; + const uint64_t src_tenant_id = import_job.get_src_tenant_id(); + const ObTableSchema *table_schema = NULL; + MTL_SWITCH(OB_SYS_TENANT_ID) { + if (OB_FAIL(ObImportTableUtil::get_tenant_schema_guard(*schema_service_, src_tenant_id, guard))) { + LOG_WARN("failed to get tenant schema guard", K(ret)); + } else if (OB_FAIL(guard.get_table_schema(src_tenant_id, table_item.database_name_, table_item.table_name_, false/*no index*/, table_schema))) { + LOG_WARN("failed to get table schema", K(ret), K(src_tenant_id), K(table_item)); + } else if (OB_ISNULL(table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table not exist", K(ret), K(src_tenant_id), K(table_item)); + int tmp_ret = OB_SUCCESS; + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), "import table not exist, %.*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 databuff printf", K(ret)); + } else { + import_job.get_result().set_result(false, comment); + } + } else if (OB_FAIL(fill_import_task_from_import_table_(import_job, guard, *table_schema, table_item, import_task))) { + LOG_WARN("failed to fill import task", K(ret), K(import_job), K(table_item)); + } + } + return ret; +} + +int ObImportTableTaskGenerator::fill_import_task_from_import_db_( + share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::ObImportDatabaseItem &db_item, + const share::ObImportTableItem &table_item, + const share::schema::ObTableSchema &table_schema, + share::ObImportTableTask &import_task) +{ + int ret = OB_SUCCESS; + share::ObImportDatabaseItem remap_db_item; + share::ObImportTableItem remap_table_item; + const share::ObImportRemapArg &remap_arg = import_job.get_import_arg().get_remap_table_arg(); + + if (OB_FAIL(remap_arg.get_remap_database(db_item, remap_db_item))) { + if (OB_ENTRY_NOT_EXIST == ret) { + // no remap, set target database name the same as source. + remap_db_item = db_item; + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to get remap database", K(ret), K(remap_arg), K(db_item)); + } + } + + + if (OB_SUCC(ret)) { + remap_table_item.mode_ = remap_db_item.mode_; + remap_table_item.database_name_ = remap_db_item.name_; + remap_table_item.table_name_ = table_item.table_name_; + } + + if (FAILEDx(fill_import_task_(import_job, guard, table_schema, table_item, remap_table_item, import_task))) { + LOG_WARN("failed to fill import task", K(ret), K(import_job), K(table_schema), K(table_item), K(remap_table_item)); + } + + return ret; +} + +int ObImportTableTaskGenerator::fill_import_task_from_import_table_( + share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + share::ObImportTableTask &import_task) +{ + int ret = OB_SUCCESS; + share::ObImportTableItem remap_table_item; + const share::ObImportRemapArg &remap_arg = import_job.get_import_arg().get_remap_table_arg(); + + if (OB_FAIL(remap_arg.get_remap_table(table_item, remap_table_item))) { + if (OB_ENTRY_NOT_EXIST == ret) { + // no remap, set target name the same as source. + remap_table_item = table_item; + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to get remap table", K(ret), K(remap_arg), K(table_item)); + } + } + + + if (FAILEDx(fill_import_task_(import_job, guard, table_schema, table_item, remap_table_item, import_task))) { + LOG_WARN("failed to fill import task", K(ret), K(import_job), K(table_schema), K(table_item), K(remap_table_item)); + } + + return ret; +} + +int ObImportTableTaskGenerator::fill_import_task_( + share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + const share::ObImportTableItem &remap_table_item, + share::ObImportTableTask &import_task) +{ + int ret = OB_SUCCESS; + if (!table_schema.is_user_table()) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("import not user table is not allowed"); + } else if (OB_FAIL(import_task.set_src_database(table_item.database_name_))) { + LOG_WARN("failed to set src database name", K(ret)); + } else if (OB_FAIL(import_task.set_src_table(table_item.table_name_))) { + LOG_WARN("failed to set src table name", K(ret)); + } else if (OB_FAIL(import_task.set_target_database(remap_table_item.database_name_))) { + LOG_WARN("failed to set target database name", K(ret)); + } else if (OB_FAIL(import_task.set_target_table(remap_table_item.table_name_))) { + LOG_WARN("failed to set target table name", K(ret)); + } else if (OB_FAIL(fill_common_para_(import_job, table_schema, import_task))) { + LOG_WARN("failed to fill task common para", K(ret)); + } else if (OB_FAIL(fill_tablegroup_(import_job, guard, table_schema, table_item, import_task))) { + LOG_WARN("failed to set tablegroup", K(ret)); + } else if (OB_FAIL(fill_tablespace_(import_job, guard, table_schema, table_item, import_task))) { + LOG_WARN("failed to set tablespace", K(ret)); + } else if (OB_FAIL(check_target_schema_(import_job, import_task))) { + LOG_WARN("failed to check target schema", K(ret), K(import_task)); + } + + return ret; +} + +int ObImportTableTaskGenerator::check_target_schema_( + share::ObImportTableJob &import_job, + const share::ObImportTableTask &task) +{ + int ret = OB_SUCCESS; + const uint64_t target_tenant_id = task.get_tenant_id(); + bool is_exist = true; + int tmp_ret = OB_SUCCESS; + if (OB_FAIL(ObImportTableUtil::check_database_schema_exist(*schema_service_, + target_tenant_id, + task.get_target_database(), + is_exist))) { + LOG_WARN("failed to check target database schema exist", K(ret), K(task)); + } else if (!is_exist) { + ret = OB_ERR_BAD_DATABASE;; + LOG_INFO("database not exist", K(ret), K(task.get_target_database())); + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), "import database %.*s not exist", + task.get_target_database().length(), task.get_target_database().ptr()))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + import_job.get_result().set_result(false, comment); + } + } else if (OB_FAIL(ObImportTableUtil::check_table_schema_exist(*schema_service_, + target_tenant_id, + task.get_target_database(), + task.get_target_table(), + is_exist))) { + LOG_WARN("failed to check target table schema exist", K(ret), K(task)); + } else if (is_exist) { + ret = OB_ERR_TABLE_EXIST; + LOG_INFO("target table exist", K(ret), K(task.get_target_table())); + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), "import table %.*s.%.*s exist", + task.get_target_database().length(), task.get_target_database().ptr(), + task.get_target_table().length(), task.get_target_table().ptr()))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + import_job.get_result().set_result(false, comment); + } + } + is_exist = false; + if (task.get_target_tablegroup().empty()) { + } else if (FAILEDx(ObImportTableUtil::check_tablegroup_exist(*schema_service_, + target_tenant_id, + task.get_target_tablegroup(), + is_exist))) { + LOG_WARN("failed to check tablegroup exist", K(ret)); + } else if (!is_exist) { + ret = OB_TABLEGROUP_NOT_EXIST; + LOG_WARN("tablegroup not exist", K(ret), K(task.get_target_tablegroup())); + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), "remap tablegroup %.*s not exist", + task.get_target_tablegroup().length(), task.get_target_tablegroup().ptr()))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + import_job.get_result().set_result(false, comment); + } + } + is_exist = false; + if (task.get_target_tablespace().empty()) { + } else if (FAILEDx(ObImportTableUtil::check_tablespace_exist(*schema_service_, + target_tenant_id, + task.get_target_tablespace(), + is_exist))) { + LOG_WARN("failed to check tablespace exist", K(ret)); + } else if (!is_exist) { + ret = OB_TABLESPACE_NOT_EXIST; + LOG_WARN("tablespace not exist", K(ret), K(task.get_target_tablespace())); + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), "remap tablespace %.*s not exist", + task.get_target_tablespace().length(), task.get_target_tablespace().ptr()))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + import_job.get_result().set_result(false, comment); + } + } + return ret; +} + +int ObImportTableTaskGenerator::fill_common_para_( + const share::ObImportTableJob &import_job, + const share::schema::ObTableSchema &table_schema, + share::ObImportTableTask &task) +{ + int ret = OB_SUCCESS; + int64_t task_id = 0; + if (OB_FAIL(ObDDLTask::fetch_new_task_id(*sql_proxy_, import_job.get_tenant_id(), task_id))) { + LOG_WARN("fail to fecth new ddl task id", K(ret)); + } else { + task.set_tenant_id(import_job.get_tenant_id()); + task.set_task_id(task_id); + task.set_job_id(import_job.get_job_id()); + task.set_src_tenant_id(import_job.get_src_tenant_id()); + task.set_table_column(table_schema.get_column_count()); + task.set_status(ObImportTableTaskStatus(ObImportTableTaskStatus::INIT)); + task.set_start_ts(ObTimeUtility::current_time()); + task.set_total_bytes(table_schema.get_tablet_size()); + //task.set_total_rows() + task.set_total_index_count(table_schema.get_simple_index_infos().count()); + task.set_total_constraint_count(table_schema.get_constraint_count()); + task.set_total_ref_constraint_count(table_schema.get_foreign_key_infos().count()); + task.set_total_trigger_count(table_schema.get_trigger_list().count()); + } + + return ret; +} + +int ObImportTableTaskGenerator::fill_tablespace_( + const share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + share::ObImportTableTask &task) +{ + int ret = OB_SUCCESS; + const uint64_t src_tenant_id = import_job.get_src_tenant_id(); + uint64_t tablespace_id = table_schema.get_tablespace_id(); + const schema::ObTablespaceSchema *schema = nullptr; + if (OB_INVALID_ID == tablespace_id) { + LOG_DEBUG("[RECOVER_TABLE]no tablespace", K(table_schema)); + } else if (OB_FAIL(guard.get_tablespace_schema(src_tenant_id, tablespace_id, schema))) { + LOG_WARN("failed to get tablesapce schema", K(ret), K(src_tenant_id), K(tablespace_id)); + } else if (OB_ISNULL(schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tablespace schema must not be null", K(ret), K(src_tenant_id), K(tablespace_id)); + } else if (OB_FAIL(task.set_src_tablespace(schema->get_tablespace_name_str()))) { + LOG_WARN("failed to set src table space", K(ret)); + } else { + share::ObImportTablespaceItem src_ts; + share::ObImportTablespaceItem target_ts; + const ObString &ts_name = schema->get_tablespace_name(); + src_ts.mode_ = table_item.mode_; + src_ts.name_ = ts_name; + + const share::ObImportRemapArg &remap_arg = import_job.get_import_arg().get_remap_table_arg(); + if (OB_FAIL(remap_arg.get_remap_tablespace(src_ts, target_ts))) { + if (OB_ENTRY_NOT_EXIST == ret && OB_FAIL(task.set_target_tablespace(ts_name))) { + // no remap, set the same as source tablespace + LOG_WARN("failed to set target tablespace", K(ret), K(ts_name)); + } + } else if (OB_FAIL(task.set_target_tablespace(target_ts.name_))) { + LOG_WARN("failed to set target tablespace", K(ret), K(target_ts)); + } + } + + return ret; +} + +int ObImportTableTaskGenerator::fill_tablegroup_( + const share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + share::ObImportTableTask &task) +{ + int ret = OB_SUCCESS; + const uint64_t src_tenant_id = import_job.get_src_tenant_id(); + uint64_t tablegroup_id = table_schema.get_tablegroup_id(); + const schema::ObSimpleTablegroupSchema *tablegroup_schema = nullptr; + if (OB_INVALID_ID == tablegroup_id) { + LOG_DEBUG("[RECOVER_TABLE]no tablegroup", K(table_schema)); + } else if (OB_FAIL(guard.get_tablegroup_schema(src_tenant_id, tablegroup_id, tablegroup_schema))) { + LOG_WARN("failed to get tablegroup scheam", K(ret)); + } else if (OB_ISNULL(tablegroup_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tablegroup schema must not be null", K(ret), K(src_tenant_id), K(tablegroup_id)); + } else if (OB_FAIL(task.set_src_tablegroup(tablegroup_schema->get_tablegroup_name()))) { + LOG_WARN("failed to set tablegroup name", K(ret)); + } else { + share::ObImportTablegroupItem src_tg; + share::ObImportTablegroupItem target_tg; + const ObString &tg_name = tablegroup_schema->get_tablegroup_name(); + src_tg.mode_ = table_item.mode_; + src_tg.name_ = tg_name; + + const share::ObImportRemapArg &remap_arg = import_job.get_import_arg().get_remap_table_arg(); + if (OB_FAIL(remap_arg.get_remap_tablegroup(src_tg, target_tg))) { + if (OB_ENTRY_NOT_EXIST == ret && OB_FAIL(task.set_target_tablegroup(tg_name))) { + // no remap, set the same as source tablegroup + LOG_WARN("failed to set target tablegroup", K(ret), K(tg_name)); + } + } else if (OB_FAIL(task.set_target_tablegroup(target_tg.name_))) { + LOG_WARN("failed to set target tablegroup", K(ret), K(target_tg)); + } + } + + return ret; +} \ No newline at end of file diff --git a/src/rootserver/restore/ob_import_table_task_generator.h b/src/rootserver/restore/ob_import_table_task_generator.h new file mode 100644 index 0000000000..09f40db1fd --- /dev/null +++ b/src/rootserver/restore/ob_import_table_task_generator.h @@ -0,0 +1,126 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_ROOTSERVICE_RECOVER_TABLE_TASK_GENERATOR_H +#define OCEANBASE_ROOTSERVICE_RECOVER_TABLE_TASK_GENERATOR_H + +#include "lib/ob_define.h" +#include "share/restore/ob_import_table_struct.h" +namespace oceanbase +{ + +namespace share +{ +namespace schema +{ +class ObMultiVersionSchemaService; +class ObSchemaGetterGuard; +class ObSimpleTableSchemaV2; +} +} + +namespace common +{ +class ObMySQLProxy; +class ObString; +} + +namespace obrpc +{ +class ObRecoverTableArg; +class ObPhysicalRestoreTenantArg; +} + +namespace rootserver +{ + +class ObImportTableTaskGenerator final +{ +public: + ObImportTableTaskGenerator() + : is_inited_(false), schema_service_(nullptr), sql_proxy_(nullptr) {} + ~ObImportTableTaskGenerator() {} + + int init( + share::schema::ObMultiVersionSchemaService &schema_service, + common::ObMySQLProxy &sql_proxy); + + int gen_import_task( + share::ObImportTableJob &import_job, + common::ObIArray &import_tasks); + + +private: + int gen_db_import_tasks_( + share::ObImportTableJob &import_job, + common::ObIArray &import_tasks); + int gen_one_db_import_tasks_( + share::ObImportTableJob &import_job, + const share::ObImportDatabaseItem &db_item, + common::ObIArray &import_tasks); + int gen_table_import_tasks_( + share::ObImportTableJob &import_job, + common::ObIArray &import_tasks); + int gen_table_import_task_( + share::ObImportTableJob &import_job, + const share::ObImportTableItem &item, + share::ObImportTableTask &import_task); + int fill_import_task_from_import_db_( + share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::ObImportDatabaseItem &db_item, + const share::ObImportTableItem &table_item, + const share::schema::ObTableSchema &table_schema, + share::ObImportTableTask &import_task); + int fill_import_task_from_import_table_( + share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + share::ObImportTableTask &import_task); + int fill_import_task_( + share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + const share::ObImportTableItem &remap_table_item, + share::ObImportTableTask &import_task); + int fill_common_para_( + const share::ObImportTableJob &import_job, + const share::schema::ObTableSchema &table_schema, + share::ObImportTableTask &task); + int fill_tablespace_( + const share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + share::ObImportTableTask &task); + int fill_tablegroup_( + const share::ObImportTableJob &import_job, + share::schema::ObSchemaGetterGuard &guard, + const share::schema::ObTableSchema &table_schema, + const share::ObImportTableItem &table_item, + share::ObImportTableTask &task); + int check_target_schema_( + share::ObImportTableJob &import_job, + const share::ObImportTableTask &task); +private: + bool is_inited_; + share::schema::ObMultiVersionSchemaService *schema_service_; + common::ObMySQLProxy *sql_proxy_; + DISALLOW_COPY_AND_ASSIGN(ObImportTableTaskGenerator); +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/rootserver/restore/ob_recover_table_initiator.cpp b/src/rootserver/restore/ob_recover_table_initiator.cpp new file mode 100644 index 0000000000..395526b551 --- /dev/null +++ b/src/rootserver/restore/ob_recover_table_initiator.cpp @@ -0,0 +1,560 @@ +/** + * 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 "ob_recover_table_initiator.h" +#include "lib/hash/ob_hashset.h" +#include "lib/charset/ob_charset.h" +#include "share/ob_rpc_struct.h" +#include "share/backup/ob_backup_data_table_operator.h" +#include "rootserver/ob_rs_event_history_table_operator.h" +#include "ob_restore_util.h" +#include "share/restore/ob_recover_table_persist_helper.h" +#include "sql/parser/parse_node.h" +#include "rootserver/ddl_task/ob_ddl_task.h" + +using namespace oceanbase; +using namespace share::schema; +using namespace common; +using namespace obrpc; +using namespace rootserver; +using namespace share; + +int ObRecoverTableInitiator::init( + share::schema::ObMultiVersionSchemaService *schema_service, common::ObMySQLProxy *sql_proxy) +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObRecoverTableInitiator init twice", K(ret)); + } else if (OB_ISNULL(schema_service) || OB_ISNULL(schema_service)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("schema service and sql prxoy must not be null", K(ret)); + } else { + schema_service_ = schema_service; + sql_proxy_ = sql_proxy; + is_inited_ = true; + } + return ret; +} + +int ObRecoverTableInitiator::initiate_recover_table(const obrpc::ObRecoverTableArg &arg) +{ + int ret = OB_SUCCESS; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTableInitiator is not init", K(ret)); + } else if (!arg.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid ObRecoverTableArg", K(ret), K(arg)); + } else if (OB_FAIL(check_before_initiate_(arg))) { + LOG_WARN("failed to check before initiate", K(ret)); + } else if (obrpc::ObRecoverTableArg::Action::INITIATE == arg.action_) { + if (OB_FAIL(start_recover_table_(arg))) { + LOG_WARN("failed to start recover table", K(ret), K(arg)); + } + } else if (obrpc::ObRecoverTableArg::Action::CANCEL == arg.action_) { + if (OB_FAIL(cancel_recover_table_(arg))) { + LOG_WARN("failed to cancel recover table", K(ret), K(arg)); + } + } + return ret; +} + +int ObRecoverTableInitiator::is_recover_job_exist(const uint64_t target_tenant_id, bool &is_exist) const +{ + int ret = OB_SUCCESS; + share::ObRecoverTablePersistHelper table_op; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTableInitiator is not init", K(ret)); + } else if (!is_user_tenant(target_tenant_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid target_tenant_id", K(ret), K(target_tenant_id)); + } else if (OB_FAIL(table_op.init(OB_SYS_TENANT_ID))) { + LOG_WARN("failed to init sys table op", K(ret)); + } else if (OB_FAIL(table_op.is_recover_table_job_exist(*sql_proxy_, target_tenant_id, is_exist))) { + LOG_WARN("failed to check recover table job exist", K(ret), K(target_tenant_id)); + } + return ret; +} + +int ObRecoverTableInitiator::start_recover_table_(const obrpc::ObRecoverTableArg &arg) +{ + int ret = OB_SUCCESS; + share::ObRecoverTableJob job; + ObPhysicalRestoreJob physical_restore_job; + if (OB_FALSE_IT(job.set_status(share::ObRecoverTableStatus::PREPARE))) { + } else if (OB_FAIL(job.set_target_tenant_name(arg.tenant_name_))) { + LOG_WARN("failed to set traget tenant name", K(ret)); + } else if (OB_FALSE_IT(job.set_target_tenant_id(arg.tenant_id_))) { + } else if (OB_FAIL(job.set_description(arg.restore_tenant_arg_.description_))) { + LOG_WARN("failed to set description", K(ret)); + } else if (OB_FAIL(fill_aux_tenant_restore_info_(arg, job, physical_restore_job))) { + LOG_WARN("failed to fill aux tenant resetore info", K(ret), K(arg)); + } else if (OB_FAIL(fill_recover_table_arg_(arg, job))) { + LOG_WARN("failed to fill recover table arg", K(ret)); + } else if (OB_FAIL(insert_sys_job_(job, physical_restore_job))) { + LOG_WARN("failed to insert sys recover table job", K(ret)); + } else { + LOG_INFO("initiate recover table succeed", K(ret), K(job)); + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = insert_user_job_(job))) { + LOG_WARN("failed to insert user job", K(ret), K(job)); + } + } + uint64_t tenant_id = arg.tenant_id_; + int64_t job_id = job.get_job_id(); + share::ObTaskId trace_id(*ObCurTraceId::get_trace_id()); + ROOTSERVICE_EVENT_ADD("recover_table", "start_recover_table", K(tenant_id), K(job_id), K(ret), K(trace_id)); + return ret; +} +int ObRecoverTableInitiator::cancel_recover_table_(const obrpc::ObRecoverTableArg &arg) +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + share::ObRecoverTablePersistHelper recover_helper; + share::ObImportTableJobPersistHelper import_helper; + uint64_t exec_tenant_id = gen_meta_tenant_id(arg.tenant_id_); + if (OB_FAIL(recover_helper.init(arg.tenant_id_))) { + LOG_WARN("failed to init helper", K(ret), K(arg)); + } else if (OB_FAIL(import_helper.init(arg.tenant_id_))) { + LOG_WARN("failed to init helper", K(ret), K(arg)); + } else if (OB_FAIL(trans.start(sql_proxy_, exec_tenant_id))) { + LOG_WARN("failed to start trans", K(ret), K(exec_tenant_id)); + } else if (OB_FAIL(import_helper.force_cancel_import_job(trans))) { + LOG_WARN("failed to force cancel import job", K(ret), K(arg)); + } else if (OB_FAIL(recover_helper.force_cancel_recover_job(trans))) { + LOG_WARN("failed to force cancel recover job", K(ret), K(arg)); + } + if (trans.is_started()) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(trans.end(OB_SUCC(ret)))) { + ret = OB_SUCC(ret) ? tmp_ret : ret; + LOG_WARN("failed to end trans", K(ret)); + } + } + + ROOTSERVICE_EVENT_ADD("recover_table", "cancel_recover_table", "tenant_id", arg.tenant_id_, "result", ret); + return ret; +} + +int ObRecoverTableInitiator::insert_user_job_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + int64_t job_id = -1; + const int64_t initiator_job_id = job.get_job_id(); // sys job id + const uint64_t exec_tenant_id = gen_meta_tenant_id(job.get_target_tenant_id()); + if (OB_FAIL(trans.start(sql_proxy_, exec_tenant_id))) { + LOG_WARN("failed to start trans", K(ret)); + } else { + share::ObRecoverTablePersistHelper helper; + if (OB_FAIL(ObLSBackupInfoOperator::get_next_job_id(trans, exec_tenant_id, job_id))) { + LOG_WARN("failed to get next job_id", K(ret)); + } else if (OB_FALSE_IT(job.set_tenant_id(job.get_target_tenant_id()))) { + } else if (OB_FALSE_IT(job.set_initiator_tenant_id(OB_SYS_TENANT_ID))) { + } else if (OB_FALSE_IT(job.set_job_id(job_id))) { + } else if (OB_FALSE_IT(job.set_initiator_job_id(initiator_job_id))) { + } else if (OB_FALSE_IT(job.set_start_ts(ObTimeUtility::current_time()))) { + } else if (OB_FAIL(helper.init(job.get_tenant_id()))) { + LOG_WARN("failed to init table op", K(ret), "tenant_id", job.get_tenant_id()); + } else if (OB_FAIL(helper.insert_recover_table_job(trans, job))) { + LOG_WARN("failed to insert initial recover table job", K(ret), K(job)); + } + if (OB_SUCC(ret)) { + if (OB_FAIL(trans.end(true))) { + LOG_WARN("failed to commit trans", K(ret)); + } + } else { + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = trans.end(false))) { + LOG_WARN("failed to rollback trans", K(tmp_ret)); + } + } + } + return ret; +} + +int ObRecoverTableInitiator::insert_sys_job_( + share::ObRecoverTableJob &job, share::ObPhysicalRestoreJob &physical_restore_job) +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + int64_t job_id = -1; + if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID))) { + LOG_WARN("failed to start trans", K(ret)); + } else { + share::ObRecoverTablePersistHelper helper; + if (OB_FAIL(ObLSBackupInfoOperator::get_next_job_id(trans, OB_SYS_TENANT_ID, job_id))) { + LOG_WARN("failed to get next job_id", K(ret)); + } else if (OB_FALSE_IT(job.set_tenant_id(OB_SYS_TENANT_ID))) { + } else if (OB_FALSE_IT(job.set_initiator_tenant_id(OB_SYS_TENANT_ID))) { + } else if (OB_FALSE_IT(job.set_job_id(job_id))) { + } else if (OB_FALSE_IT(job.set_initiator_job_id(0/*sys job default value*/))) { + } else if (OB_FALSE_IT(job.set_start_ts(ObTimeUtility::current_time()))) { + } else if (OB_FAIL(helper.init(OB_SYS_TENANT_ID))) { + LOG_WARN("failed to init sys table op", K(ret)); + } else if (OB_FAIL(helper.insert_recover_table_job(trans, job))) { + LOG_WARN("failed to insert initital recover table job", K(ret), K(job)); + } + + if (FAILEDx(RS_JOB_CREATE_EXT(job_id, RESTORE_TENANT, trans, "sql_text", "restore aux tenant"))) { + LOG_WARN("failed to get job id", K(ret)); + } else if (OB_FALSE_IT(physical_restore_job.init_restore_key(OB_SYS_TENANT_ID, job_id))) { + } else if (OB_FALSE_IT(physical_restore_job.set_initiator_job_id(job.get_job_id()))) { + } else if (OB_FALSE_IT(physical_restore_job.set_initiator_tenant_id(OB_SYS_TENANT_ID))) { + } else if (OB_FAIL(ObRestoreUtil::record_physical_restore_job(trans, physical_restore_job))) { + LOG_WARN("failed to record physical restore job", K(ret)); + } + if (OB_SUCC(ret)) { + if (OB_FAIL(trans.end(true))) { + LOG_WARN("failed to commit trans", K(ret)); + } + } else { + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = trans.end(false))) { + LOG_WARN("failed to rollback trans", K(tmp_ret)); + } + } + } + return ret; +} + +int ObRecoverTableInitiator::check_before_initiate_(const obrpc::ObRecoverTableArg &arg) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard guard; + uint64_t target_tenant_id = 0; + if (!is_user_tenant(arg.tenant_id_)) { + ret = OB_INVALID_ARGUMENT; + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "TENANT, it must be user tenant"); + LOG_WARN("invlaid tenant id, it must be user tenant", K(ret), K(arg.tenant_id_)); + } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(OB_SYS_TENANT_ID, guard))) { + LOG_WARN("failed to get sys schema guard", K(ret)); + } else if (OB_FAIL(guard.get_tenant_id(arg.tenant_name_, target_tenant_id))) { + LOG_WARN("failed to get tenant id", K(ret), K(arg.tenant_name_)); + } else if (arg.tenant_id_ != target_tenant_id) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("arg tenant id and tenant name must be couple", K(ret), K(arg)); + } + return ret; +} + +int ObRecoverTableInitiator::fill_aux_tenant_name_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + char aux_tenant_name[OB_MAX_TENANT_NAME_LENGTH] = ""; + if (OB_FAIL(databuff_printf(aux_tenant_name, OB_MAX_TENANT_NAME_LENGTH, "AUX_RECOVER$%ld", ObTimeUtility::current_time()))) { + LOG_WARN("failed to generate aux tenant name", K(ret)); + } else if (OB_FAIL(job.set_aux_tenant_name(ObString(aux_tenant_name)))) { + LOG_WARN("failed to set aux tenant name", K(ret), K(aux_tenant_name)); + } + return ret; +} + +int ObRecoverTableInitiator::fill_aux_tenant_restore_info_( + const obrpc::ObRecoverTableArg &arg, share::ObRecoverTableJob &job, share::ObPhysicalRestoreJob &physical_restore_job) +{ + int ret = OB_SUCCESS; + obrpc::ObPhysicalRestoreTenantArg tenant_restore_arg; + if (OB_FAIL(fill_aux_tenant_name_(job))) { + LOG_WARN("failed to fill aux tenant name", K(ret)); + } else if (OB_FAIL(tenant_restore_arg.assign(arg.restore_tenant_arg_))) { + LOG_WARN("failed to assign tenant restore arg", K(ret), K(arg.restore_tenant_arg_)); + } else if (OB_FALSE_IT(tenant_restore_arg.tenant_name_ = job.get_aux_tenant_name())) { + } else if (OB_FAIL(ObRestoreUtil::fill_physical_restore_job(1/*fake job id*/, tenant_restore_arg, physical_restore_job))) { + LOG_WARN("failed to fill physical restore job", K(ret), K(tenant_restore_arg)); + } else if (OB_FALSE_IT(job.set_restore_scn(physical_restore_job.get_restore_scn()))) { + } else if (OB_FAIL(job.set_restore_option(physical_restore_job.get_restore_option()))) { + LOG_WARN("failed to set restore option", K(ret)); + } else if (OB_FAIL(job.set_backup_dest(physical_restore_job.get_backup_dest()))) { + LOG_WARN("failed to set backup dest", K(ret)); + } else if (OB_FAIL(job.set_external_kms_info(physical_restore_job.get_kms_info()))) { + LOG_WARN("failed to set kms info", K(ret)); + } else if (OB_FAIL(job.set_backup_passwd(physical_restore_job.get_passwd_array()))) { + LOG_WARN("failed to set backup passwd", K(ret)); + } else if (OB_FAIL(job.get_multi_restore_path_list().assign(physical_restore_job.get_multi_restore_path_list()))) { + LOG_WARN("faield to assign multi restore path", K(ret)); + } + return ret; +} + +int ObRecoverTableInitiator::fill_recover_database( + const share::ObImportArg &import_arg, + share::ObImportTableArg &import_table_arg) +{ + //TODO(chongrong.th) move duplicate item checking logic to ObImportArg internal later. + int ret = OB_SUCCESS; + const share::ObImportDatabaseArray &db_array = import_arg.get_import_database_array(); + ARRAY_FOREACH(db_array.get_items(), i) { + const share::ObImportDatabaseItem db_item = db_array.get_items().at(i); + if (OB_FAIL(import_table_arg.add_database(db_item))) { + LOG_WARN("failed to add database", K(ret)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("succeed fill recover database", K(import_arg), K(db_array), K(import_table_arg.get_import_database_array())); + } + return ret; +} + +int ObRecoverTableInitiator::fill_recover_table( + const share::ObImportArg &import_arg, + share::ObImportTableArg &import_table_arg) +{ + int ret = OB_SUCCESS; + const share::ObImportTableArray &table_array = import_arg.get_import_table_array(); + bool is_dup = false; + ObSqlString dup_item_str; + ARRAY_FOREACH(table_array.get_items(), i) { + const share::ObImportTableItem table_item = table_array.get_items().at(i); + share::ObImportDatabaseItem db_item(table_item.mode_, table_item.database_name_.ptr(), table_item.database_name_.length()); + if (OB_FAIL(import_table_arg.check_database_dup(db_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check database dup", K(ret)); + } else if (is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("duplicate database", K(table_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, dup_item_str.ptr()); + } else if (OB_FAIL(import_table_arg.add_table(table_item))) { + LOG_WARN("failed to add table", K(ret)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("succeed fill recover table", K(import_arg), K(table_array), K(import_table_arg.get_import_table_array())); + } + return ret; +} + +int ObRecoverTableInitiator::fill_recover_partition( + const share::ObImportArg &import_arg, + share::ObImportTableArg &import_table_arg) +{ + int ret = OB_SUCCESS; + bool is_dup = true; + ObSqlString dup_item_str; + const share::ObImportPartitionArray &partition_array = import_arg.get_import_partition_array(); + ARRAY_FOREACH(partition_array.get_items(), i) { + const share::ObImportPartitionItem partition_item = partition_array.get_items().at(i); + share::ObImportDatabaseItem db_item(partition_item.mode_, + partition_item.database_name_.ptr(), + partition_item.database_name_.length()); + share::ObImportTableItem table_item(partition_item.mode_, + partition_item.database_name_.ptr(), + partition_item.database_name_.length(), + partition_item.table_name_.ptr(), + partition_item.table_name_.length()); + if (OB_FAIL(import_table_arg.check_database_dup(db_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check database dup", K(ret)); + } else if (is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("duplicate database", K(table_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, dup_item_str.ptr()); + } else if (OB_FAIL(import_table_arg.check_table_dup(table_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check table dup", K(ret)); + } else if (is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("duplicate table", K(table_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, dup_item_str.ptr()); + } else if (OB_FAIL(import_table_arg.add_partition(partition_item))) { + LOG_WARN("failed to add partition", K(ret)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("succeed fill recover partition", K(import_arg), K(partition_array), K(import_table_arg.get_import_partition_array())); + } + return ret; +} + +int ObRecoverTableInitiator::fill_remap_database( + const share::ObImportArg &import_arg, + const share::ObImportTableArg &import_table_arg, + share::ObImportRemapArg &import_remap_arg) +{ + int ret = OB_SUCCESS; + bool is_dup = true; + ObSqlString dup_item_str; + const share::ObRemapDatabaseArray &remap_db_array = import_arg.get_remap_database_array(); + ARRAY_FOREACH(remap_db_array.get_remap_items(), i) { + const share::ObRemapDatabaseItem remap_db_item = remap_db_array.get_remap_items().at(i); + const share::ObImportDatabaseItem src_db_item(remap_db_item.src_.mode_, + remap_db_item.src_.name_.ptr(), + remap_db_item.src_.name_.length()); + const share::ObImportDatabaseItem target_db_item(remap_db_item.target_.mode_, + remap_db_item.target_.name_.ptr(), + remap_db_item.target_.name_.length()); + if (OB_FAIL(import_table_arg.check_database_dup(src_db_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check database dup", K(ret)); + } else if (!is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("remap not exist database", K(src_db_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "remap not exist recover database"); + } else if (OB_FAIL(import_table_arg.check_database_dup(target_db_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check dup", K(ret)); + } else if (is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("remap exist database", K(src_db_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, dup_item_str.ptr()); + } else if (OB_FAIL(import_remap_arg.add_remap_database(remap_db_item))) { + LOG_WARN("failed to add database", K(ret)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("succeed fill remap database", K(import_arg), K(remap_db_array), K(import_remap_arg.get_remap_database_array())); + } + return ret; +} + +int ObRecoverTableInitiator::fill_remap_table( + const share::ObImportArg &import_arg, + const share::ObImportTableArg &import_table_arg, + share::ObImportRemapArg &import_remap_arg) +{ + int ret = OB_SUCCESS; + bool is_dup = true; + ObSqlString dup_item_str; + const share::ObRemapTableArray &remap_table_array = import_arg.get_remap_table_array(); + ARRAY_FOREACH(remap_table_array.get_remap_items(), i) { + const share::ObRemapTableItem remap_table_item = remap_table_array.get_remap_items().at(i); + const share::ObImportTableItem src_table_item(remap_table_item.src_.mode_, + remap_table_item.src_.database_name_.ptr(), + remap_table_item.src_.database_name_.length(), + remap_table_item.src_.table_name_.ptr(), + remap_table_item.src_.table_name_.length()); + const share::ObImportTableItem target_table_item(remap_table_item.target_.mode_, + remap_table_item.target_.database_name_.ptr(), + remap_table_item.target_.database_name_.length(), + remap_table_item.target_.table_name_.ptr(), + remap_table_item.target_.table_name_.length()); + if (OB_FAIL(import_table_arg.check_table_dup(src_table_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check table dup", K(ret)); + } else if (!is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("remap not exist table", K(src_table_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "remap not exist recover table"); + } else if (OB_FAIL(import_table_arg.check_table_dup(target_table_item, is_dup, dup_item_str))) { + } else if (is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("remap exist table", K(target_table_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, dup_item_str.ptr()); + } else if (OB_FAIL(import_remap_arg.add_remap_table(remap_table_item))) { + LOG_WARN("failed to add remap table", K(ret)); + } + } + if (OB_SUCC(ret)) { + LOG_INFO("succeed fill remap table", K(import_arg), K(remap_table_array), K(import_remap_arg.get_remap_table_array())); + } + return ret; +} + +int ObRecoverTableInitiator::fill_remap_partition( + const share::ObImportArg &import_arg, + const share::ObImportTableArg &import_table_arg, + share::ObImportRemapArg &import_remap_arg) +{ + int ret = OB_SUCCESS; + bool is_dup = true; + ObSqlString dup_item_str; + const share::ObRemapPartitionArray &remap_partition_array = import_arg.get_remap_partition_array(); + ARRAY_FOREACH(remap_partition_array.get_remap_items(), i) { + const share::ObRemapPartitionItem remap_part_item = remap_partition_array.get_remap_items().at(i); + const share::ObImportPartitionItem src_part_item(remap_part_item.src_.mode_, + remap_part_item.src_.database_name_.ptr(), + remap_part_item.src_.database_name_.length(), + remap_part_item.src_.table_name_.ptr(), + remap_part_item.src_.table_name_.length(), + remap_part_item.src_.partition_name_.ptr(), + remap_part_item.src_.partition_name_.length()); + const share::ObImportTableItem target_table_item(remap_part_item.target_.mode_, + remap_part_item.target_.database_name_.ptr(), + remap_part_item.target_.database_name_.length(), + remap_part_item.target_.table_name_.ptr(), + remap_part_item.target_.table_name_.length()); + if (OB_FAIL(import_table_arg.check_partion_dup(src_part_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check dup", K(ret)); + } else if (!is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("remap not exist partition", K(src_part_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "remap not exist recover partition"); + } else if (OB_FAIL(import_table_arg.check_table_dup(target_table_item, is_dup, dup_item_str))) { + LOG_WARN("failed to check dup", K(ret)); + } else if (is_dup) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("remap exist partition", K(target_table_item)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, dup_item_str.ptr()); + } else if (OB_FAIL(import_remap_arg.add_remap_parition(remap_part_item))) { + LOG_WARN("failed to add remap partition", K(ret)); + } + } + return ret; +} + +int ObRecoverTableInitiator::fill_remap_tablespace( + const share::ObImportArg &import_arg, + share::ObImportRemapArg &import_remap_arg) +{ + int ret = OB_SUCCESS; + const share::ObRemapTablespaceArray &remap_tablespace_array = import_arg.get_remap_tablespace_array(); + ARRAY_FOREACH(remap_tablespace_array.get_remap_items(), i) { + const share::ObRemapTablespaceItem remap_tablespace_item = remap_tablespace_array.get_remap_items().at(i); + if (OB_FAIL(import_remap_arg.add_remap_tablespace(remap_tablespace_item))) { + LOG_WARN("failed to add tablespace", K(ret)); + } + } + return ret; +} + +int ObRecoverTableInitiator::fill_remap_tablegroup( + const share::ObImportArg &import_arg, + share::ObImportRemapArg &import_remap_arg) +{ + int ret = OB_SUCCESS; + const share::ObRemapTablegroupArray &remap_tablegroup_array = import_arg.get_remap_tablegroup_array(); + ARRAY_FOREACH(remap_tablegroup_array.get_remap_items(), i) { + const share::ObRemapTablegroupItem remap_tablegroup_item = remap_tablegroup_array.get_remap_items().at(i); + if (OB_FAIL(import_remap_arg.add_remap_tablegroup(remap_tablegroup_item))) { + LOG_WARN("failed to add tablespace", K(ret)); + } + } + return ret; +} + + +int ObRecoverTableInitiator::fill_recover_table_arg_( + const obrpc::ObRecoverTableArg &arg, share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + share::ObImportTableArg &import_table_arg = job.get_import_arg().get_import_table_arg(); + share::ObImportRemapArg &import_remap_arg = job.get_import_arg().get_remap_table_arg(); + LOG_INFO("succeed fill arg", K(arg), K(import_table_arg), K(import_remap_arg)); + if (arg.import_arg_.get_import_table_arg().is_import_all()) { + import_table_arg.set_import_all(); + } else if (OB_FAIL(fill_recover_database(arg.import_arg_, import_table_arg))) { + LOG_WARN("failed to recover database", K(ret), K(arg.import_arg_)); + } else if (OB_FAIL(fill_recover_table(arg.import_arg_, import_table_arg))) { + LOG_WARN("failed to recover table", K(ret), K(arg.import_arg_)); + } else if (OB_FAIL(fill_recover_partition(arg.import_arg_, import_table_arg))) { + LOG_WARN("failed to recover partition", K(ret), K(arg.import_arg_)); + } else if (OB_FAIL(fill_remap_database(arg.import_arg_, import_table_arg, import_remap_arg))) { + LOG_WARN("failed to remap database", K(ret), K(arg.import_arg_)); + } else if (OB_FAIL(fill_remap_table(arg.import_arg_, import_table_arg, import_remap_arg))) { + LOG_WARN("failed to remap table", K(ret), K(arg.import_arg_)); + } else if (OB_FAIL(fill_remap_partition(arg.import_arg_, import_table_arg, import_remap_arg))) { + LOG_WARN("failed to remap partition", K(ret), K(arg.import_arg_)); + } else if (OB_FAIL(fill_remap_tablespace(arg.import_arg_, import_remap_arg))) { + LOG_WARN("failed to remap tablespace", K(ret), K(arg.import_arg_)); + } else if (OB_FAIL(fill_remap_tablegroup(arg.import_arg_, import_remap_arg))) { + LOG_WARN("failed to remap tablegroup", K(ret), K(arg.import_arg_)); + } + return ret; +} \ No newline at end of file diff --git a/src/rootserver/restore/ob_recover_table_initiator.h b/src/rootserver/restore/ob_recover_table_initiator.h new file mode 100644 index 0000000000..4463e3b484 --- /dev/null +++ b/src/rootserver/restore/ob_recover_table_initiator.h @@ -0,0 +1,98 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_ROOTSERVICE_RECOVER_TABLE_INITIATOR_H +#define OCEANBASE_ROOTSERVICE_RECOVER_TABLE_INITIATOR_H + +#include "lib/ob_define.h" +#include "share/restore/ob_import_table_struct.h" +#include "rootserver/restore/ob_import_table_task_generator.h" +namespace oceanbase +{ + +namespace share +{ +struct ObPhysicalRestoreJob; +namespace schema +{ +class ObMultiVersionSchemaService; +class ObSchemaGetterGuard; +class ObSimpleTableSchemaV2; +} +} + +namespace common +{ +class ObMySQLProxy; +class ObString; +} + +namespace obrpc +{ +class ObRecoverTableArg; +class ObPhysicalRestoreTenantArg; +} + +namespace rootserver +{ +class ObRecoverTableInitiator final +{ +public: + ObRecoverTableInitiator() + : is_inited_(false), schema_service_(nullptr), sql_proxy_(nullptr) {} + ~ObRecoverTableInitiator() {} + + int init(share::schema::ObMultiVersionSchemaService *schema_service, common::ObMySQLProxy *sql_proxy); + int initiate_recover_table(const obrpc::ObRecoverTableArg &arg); + int is_recover_job_exist(const uint64_t tenant_id, bool &is_exist) const; + +private: + int start_recover_table_(const obrpc::ObRecoverTableArg &arg); + int cancel_recover_table_(const obrpc::ObRecoverTableArg &arg); + int check_before_initiate_(const obrpc::ObRecoverTableArg &arg); + int fill_aux_tenant_name_(share::ObRecoverTableJob &job); + int fill_aux_tenant_restore_info_( + const obrpc::ObRecoverTableArg &arg, + share::ObRecoverTableJob &job, + share::ObPhysicalRestoreJob &physical_restore_job); + int fill_recover_table_arg_( + const obrpc::ObRecoverTableArg &arg, share::ObRecoverTableJob &job); + int insert_sys_job_(share::ObRecoverTableJob &job, share::ObPhysicalRestoreJob &physical_restore_job); + int insert_user_job_(share::ObRecoverTableJob &job); + int fill_recover_database(const share::ObImportArg &import_arg, share::ObImportTableArg &import_table_arg); + int fill_recover_table(const share::ObImportArg &import_arg, share::ObImportTableArg &import_table_arg); + int fill_recover_partition(const share::ObImportArg &import_arg, share::ObImportTableArg &import_table_arg); + int fill_remap_database(const share::ObImportArg &import_arg, + const share::ObImportTableArg &import_table_arg, + share::ObImportRemapArg &import_remap_arg); + int fill_remap_table(const share::ObImportArg &import_arg, + const share::ObImportTableArg &import_table_arg, + share::ObImportRemapArg &import_remap_arg); + int fill_remap_partition(const share::ObImportArg &import_arg, + const share::ObImportTableArg &import_table_arg, + share::ObImportRemapArg &import_remap_arg); + int fill_remap_tablespace(const share::ObImportArg &import_arg, + share::ObImportRemapArg &import_remap_arg); + int fill_remap_tablegroup(const share::ObImportArg &import_arg, + share::ObImportRemapArg &import_remap_arg); + +private: + bool is_inited_; + share::schema::ObMultiVersionSchemaService *schema_service_; + common::ObMySQLProxy *sql_proxy_; + DISALLOW_COPY_AND_ASSIGN(ObRecoverTableInitiator); +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/rootserver/restore/ob_recover_table_job_scheduler.cpp b/src/rootserver/restore/ob_recover_table_job_scheduler.cpp new file mode 100644 index 0000000000..012d5122ee --- /dev/null +++ b/src/rootserver/restore/ob_recover_table_job_scheduler.cpp @@ -0,0 +1,755 @@ +/** + * 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 "ob_recover_table_job_scheduler.h" +#include "rootserver/ob_rs_event_history_table_operator.h" +#include "rootserver/restore/ob_recover_table_initiator.h" +#include "share/backup/ob_backup_data_table_operator.h" +#include "share/ob_primary_standby_service.h" +#include "share/location_cache/ob_location_service.h" +#include "share/restore/ob_physical_restore_table_operator.h" +#include "share/restore/ob_import_util.h" + +using namespace oceanbase; +using namespace rootserver; +using namespace share; + +void ObRecoverTableJobScheduler::reset() +{ + rs_rpc_proxy_ = nullptr; + sql_proxy_ = nullptr; + schema_service_ = nullptr; + srv_rpc_proxy_ = nullptr; + is_inited_ = false; + tenant_id_ = OB_INVALID_TENANT_ID; +} + +int ObRecoverTableJobScheduler::init(share::schema::ObMultiVersionSchemaService &schema_service, + common::ObMySQLProxy &sql_proxy, obrpc::ObCommonRpcProxy &rs_rpc_proxy, obrpc::ObSrvRpcProxy &srv_rpc_proxy) +{ + int ret = OB_SUCCESS; + const uint64_t tenant_id = gen_user_tenant_id(MTL_ID()); + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("ObRecoverTableJobScheduler init twice", K(ret)); + } else if (OB_FAIL(helper_.init(tenant_id))) { + LOG_WARN("failed to init table op", K(ret), K(tenant_id)); + } else { + schema_service_ = &schema_service; + sql_proxy_ = &sql_proxy; + rs_rpc_proxy_ = &rs_rpc_proxy; + srv_rpc_proxy_ = &srv_rpc_proxy; + tenant_id_ = tenant_id; + is_inited_ = true; + } + return ret; +} + +void ObRecoverTableJobScheduler::do_work() +{ + int ret = OB_SUCCESS; + ObArray jobs; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("not init ObSysRecoverTableJobScheduler", K(ret)); + } else if (OB_FAIL(check_compatible_())) { + LOG_WARN("check compatible failed", K(ret)); + } else if (OB_FAIL(helper_.get_all_recover_table_job(*sql_proxy_, jobs))) { + LOG_WARN("failed to get recover all recover table job", K(ret)); + } else { + ARRAY_FOREACH(jobs, i) { + ObRecoverTableJob &job = jobs.at(i); + if (!job.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("recover table job is not valid", K(ret), K(job)); + } else if (is_sys_tenant(job.get_tenant_id())) { + sys_process_(job); + } else if (is_user_tenant(job.get_tenant_id())) { + user_process_(job); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid tenant", K(ret), K(job)); + } + } + } + +} + +int ObRecoverTableJobScheduler::check_compatible_() const +{ + int ret = OB_SUCCESS; + uint64_t data_version = 0; + if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id_, data_version))) { + LOG_WARN("fail to get data version", K(ret), K_(tenant_id)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("min data version is smaller than v4.2.1", K(ret), K_(tenant_id), K(data_version)); + } else if (is_sys_tenant(tenant_id_)) { + } else if (OB_FAIL(GET_MIN_DATA_VERSION(gen_meta_tenant_id(tenant_id_), data_version))) { + LOG_WARN("fail to get data version", K(ret), "tenant_id", gen_meta_tenant_id(tenant_id_)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("min data version is smaller than v4.2.1", K(ret), K_(tenant_id), K(data_version)); + } + + return ret; +} + +int ObRecoverTableJobScheduler::try_advance_status_(share::ObRecoverTableJob &job, const int err_code) +{ + int ret = OB_SUCCESS; + share::ObRecoverTableStatus next_status; + const uint64_t tenant_id = job.get_tenant_id(); + const int64_t job_id = job.get_job_id(); + bool need_advance_status = true; + if (err_code != OB_SUCCESS) { + if (ObImportTableUtil::can_retrieable_err(err_code)) { + need_advance_status = false; + } else { + share::ObTaskId trace_id(*ObCurTraceId::get_trace_id()); + next_status = ObRecoverTableStatus::FAILED; + if (job.get_result().is_comment_setted()) { + } else if (OB_FAIL(job.get_result().set_result( + err_code, trace_id, GCONF.self_addr_))) { + LOG_WARN("failed to set result", K(ret)); + } + LOG_WARN("[RECOVER_TABLE]recover table job failed", K(err_code), K(job)); + ROOTSERVICE_EVENT_ADD("recover_table", "recover_table_failed", K(tenant_id), K(job_id), K(err_code), K(trace_id)); + } + } else if (job.get_tenant_id() == OB_SYS_TENANT_ID) { + next_status = ObRecoverTableStatus::get_sys_next_status(job.get_status()); + } else { + next_status = ObRecoverTableStatus::get_user_next_status(job.get_status()); + } + if (next_status.is_finish()) { + job.set_end_ts(ObTimeUtility::current_time()); + } + if (OB_FAIL(ret)) { + } else if (need_advance_status && OB_FAIL(helper_.advance_status(*sql_proxy_, job, next_status))) { + LOG_WARN("failed to advance statsu", K(ret), K(job), K(next_status)); + } else { + ROOTSERVICE_EVENT_ADD("recover_table", "advance_status", K(tenant_id), K(job_id), K(next_status)); + } + return ret; +} + +void ObRecoverTableJobScheduler::sys_process_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + LOG_INFO("ready to schedule sys recover table job", K(job)); + uint64_t data_version = 0; + const uint64_t target_tenant_id = job.get_target_tenant_id(); + // check data version + if (OB_FAIL(GET_MIN_DATA_VERSION(target_tenant_id, data_version))) { + LOG_WARN("fail to get data version", K(ret), K(target_tenant_id)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("min data version is smaller than v4.2.1", K(ret), K(target_tenant_id), K(data_version)); + } else if (OB_FAIL(GET_MIN_DATA_VERSION(gen_meta_tenant_id(target_tenant_id), data_version))) { + LOG_WARN("fail to get data version", K(ret), "target_tenant_id", gen_meta_tenant_id(target_tenant_id)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("min data version is smaller than v4.2.1", K(ret), K(target_tenant_id), K(data_version)); + } else { + switch(job.get_status()) { + case ObRecoverTableStatus::Status::PREPARE: { + if (OB_FAIL(sys_prepare_(job))) { + LOG_WARN("failed to do sys prepare work", K(ret), K(job)); + } + break; + } + case ObRecoverTableStatus::Status::RECOVERING: { + if (OB_FAIL(recovering_(job))) { + LOG_WARN("failed to do sys recovering work", K(ret), K(job)); + } + break; + } + case ObRecoverTableStatus::Status::COMPLETED: + case ObRecoverTableStatus::Status::FAILED: { + if (OB_FAIL(sys_finish_(job))) { + LOG_WARN("failed to do sys finish work", K(ret), K(job)); + } + break; + } + default: { + ret = OB_ERR_SYS; + LOG_WARN("invalid sys recover job status", K(ret), K(job)); + break; + } + } + } +} + +int ObRecoverTableJobScheduler::sys_prepare_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + ObRecoverTableJob target_job; + share::ObRecoverTablePersistHelper helper; + DEBUG_SYNC(BEFORE_INSERT_UERR_RECOVER_TABLE_JOB); + if (OB_FAIL(helper.init(job.get_target_tenant_id()))) { + LOG_WARN("failed to init recover table persist helper", K(ret)); + } else if (OB_FAIL(helper.get_recover_table_job_by_initiator(*sql_proxy_, job, target_job))) { + if (OB_ENTRY_NOT_EXIST == ret) { + if (OB_FAIL(helper.get_recover_table_job_history_by_initiator(*sql_proxy_, job, target_job))) { + if (OB_ENTRY_NOT_EXIST == ret) { + if (OB_FAIL(insert_user_job_(job, helper))) { + LOG_WARN("failed to insert user job", K(ret), K(job)); + } else { + ROOTSERVICE_EVENT_ADD("recover_table", "insert_user_job", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id()); + } + } else { + LOG_WARN("failed to get target tenant recover table job history", K(ret), K(job)); + } + } + } else { + LOG_WARN("failed to get target tenant recover table job", K(ret), K(job)); + } + } +#ifdef ERRSIM + ret = OB_E(EventTable::EN_INSERT_USER_RECOVER_JOB_FAILED) OB_SUCCESS; + if (OB_FAIL(ret)) { + ROOTSERVICE_EVENT_ADD("recover_table_errsim", "insert_user_job_failed"); + } +#endif + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = try_advance_status_(job, ret))) { + LOG_INFO("failed to advance status", K(tmp_ret), K(ret), K(job)); + } + return ret; +} + +int ObRecoverTableJobScheduler::insert_user_job_( + const share::ObRecoverTableJob &job, share::ObRecoverTablePersistHelper &helper) +{ + int ret = OB_SUCCESS; + ObRecoverTableJob target_job; + ObMySQLTransaction trans; + uint64_t meta_tenant_id = gen_meta_tenant_id(job.get_target_tenant_id()); + if (OB_FAIL(target_job.assign(job))) { + LOG_WARN("failed to assign target job", K(ret)); + } else { + target_job.set_tenant_id(job.get_target_tenant_id()); + target_job.set_initiator_tenant_id(job.get_tenant_id()); + target_job.set_initiator_job_id(job.get_job_id()); + target_job.set_target_tenant_id(target_job.get_tenant_id()); + } + if (FAILEDx(trans.start(sql_proxy_, meta_tenant_id))) { + LOG_WARN("failed to start trans", K(ret)); + } else { + int64_t job_id = 0; + if (OB_FAIL(ObLSBackupInfoOperator::get_next_job_id(trans, target_job.get_tenant_id(), job_id))) { + LOG_WARN("failed to get next job id", K(ret), "tenant_id", target_job.get_tenant_id()); + } else if (OB_FALSE_IT(target_job.set_job_id(job_id))) { + } else if (OB_FAIL(helper.insert_recover_table_job(trans, target_job))) { + LOG_WARN("failed to insert initial recover table job", K(ret)); + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(trans.end(true))) { + LOG_WARN("failed to commit trans", K(ret)); + } + } else { + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = trans.end(false))) { + LOG_WARN("failed to rollback", K(ret), K(tmp_ret)); + } + } + } + + return ret; +} + +int ObRecoverTableJobScheduler::recovering_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + share::ObRecoverTablePersistHelper helper; + ObRecoverTableJob target_job; + bool user_job_finish = true; + bool user_tenant_not_exist = false; + int tmp_ret = OB_SUCCESS; + DEBUG_SYNC(BEFORE_RECOVER_UESR_RECOVER_TABLE_JOB); + if (OB_FAIL(helper.init(job.get_target_tenant_id()))) { + LOG_WARN("failed to init recover table persist helper", K(ret)); + } else if (OB_FAIL(helper.get_recover_table_job_history_by_initiator(*sql_proxy_, job, target_job))) { + if (OB_ENTRY_NOT_EXIST == ret) { + user_job_finish = false; + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to get target tenant recover table job history", K(ret), K(job)); + } + } else { + ROOTSERVICE_EVENT_ADD("recover_table", "sys_wait_user_recover_finish", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id()); + job.set_result(target_job.get_result()); + } + if (OB_FAIL(ret)) { + schema::ObSchemaGetterGuard guard; + if (OB_TMP_FAIL(ObImportTableUtil::get_tenant_schema_guard(*schema_service_, job.get_target_tenant_id(), guard))) { + if (OB_TENANT_NOT_EXIST == tmp_ret) { + user_tenant_not_exist = true; + } + LOG_WARN("failed to get tenant schema guard", K(tmp_ret)); + } + } + + if ((OB_FAIL(ret) && user_tenant_not_exist) || (OB_SUCC(ret) && user_job_finish)) { + if (OB_SUCC(ret) && !job.get_result().is_succeed()) { + ret = OB_LS_RESTORE_FAILED; + } + job.set_end_ts(ObTimeUtility::current_time()); + if (OB_SUCCESS != (tmp_ret = try_advance_status_(job, ret))) { + LOG_INFO("failed to advance status", K(tmp_ret), K(ret), K(job)); + } + } + return ret; +} + +int ObRecoverTableJobScheduler::sys_finish_(const share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + bool drop_aux_tenant = GCONF._auto_drop_recovering_auxiliary_tenant; + if (drop_aux_tenant && OB_FAIL(drop_aux_tenant_(job))) { + LOG_WARN("failed ot drop aux tenant", K(ret)); + } else if (OB_FAIL(trans.start(sql_proxy_, OB_SYS_TENANT_ID))) { + LOG_WARN("failed to start trans", K(ret)); + } else { + if (OB_FAIL(helper_.insert_recover_table_job_history(trans, job))) { + LOG_WARN("failed to insert recover table job history", K(ret), K(job)); + } else if (OB_FAIL(helper_.delete_recover_table_job(trans, job))) { + LOG_WARN("failed to delete recover table job", K(ret), K(job)); + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(trans.end(true))) { + LOG_WARN("failed to commit", K(ret)); + } else { + ROOTSERVICE_EVENT_ADD("recover_table", "sys_recover_finish", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id()); + } + } else { + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = trans.end(false))) { + LOG_WARN("failed to roll back", K(tmp_ret), K(ret)); + } + } + } + return ret; +} + +int ObRecoverTableJobScheduler::drop_aux_tenant_(const share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + obrpc::ObDropTenantArg drop_tenant_arg; + drop_tenant_arg.exec_tenant_id_ = OB_SYS_TENANT_ID; + drop_tenant_arg.if_exist_ = false; + drop_tenant_arg.force_drop_ = true; + drop_tenant_arg.delay_to_drop_ = false; + drop_tenant_arg.open_recyclebin_ = false; + drop_tenant_arg.tenant_name_ = job.get_aux_tenant_name(); + drop_tenant_arg.drop_only_in_restore_ = false; + common::ObAddr rs_addr; + if (OB_ISNULL(GCTX.rs_rpc_proxy_) || OB_ISNULL(GCTX.rs_mgr_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rootserver rpc proxy or rs mgr must not be NULL", K(ret), K(GCTX)); + } else if (OB_FAIL(GCTX.rs_mgr_->get_master_root_server(rs_addr))) { + LOG_WARN("failed to get rootservice address", K(ret)); + } else if (OB_FAIL(GCTX.rs_rpc_proxy_->to(rs_addr).drop_tenant(drop_tenant_arg))) { + if (OB_TENANT_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to drop tenant", K(ret), K(drop_tenant_arg)); + } + } else { + LOG_INFO("[RECOVER_TABLE]drop aux tenant succeed", K(job)); + ROOTSERVICE_EVENT_ADD("recover_table", "drop_aux_tenant", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id(), + "aux_tenant_name", job.get_aux_tenant_name()); + } + return ret; +} + +void ObRecoverTableJobScheduler::user_process_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + LOG_INFO("ready to schedule user recover table job", K(job)); + switch(job.get_status()) { + case ObRecoverTableStatus::Status::PREPARE: { + if (OB_FAIL(user_prepare_(job))) { + LOG_WARN("failed to do user prepare work", K(ret), K(job)); + } + break; + } + case ObRecoverTableStatus::Status::RESTORE_AUX_TENANT: { + if (OB_FAIL(restore_aux_tenant_(job))) { + LOG_WARN("failed to do user restore aux tenant work", K(ret), K(job)); + } + break; + } + case ObRecoverTableStatus::Status::GEN_IMPORT_JOB: { + if (OB_FAIL(gen_import_job_(job))) { + LOG_WARN("failed to do user import work", K(ret), K(job)); + } + break; + } + case ObRecoverTableStatus::Status::CANCELING: { + if (OB_FAIL(canceling_(job))) { + LOG_WARN("failed to do user canceling", K(ret), K(job)); + } + break; + } + case ObRecoverTableStatus::Status::IMPORTING: { + if (OB_FAIL(importing_(job))) { + LOG_WARN("failed to do user importing work", K(ret), K(job)); + } + break; + } + case ObRecoverTableStatus::Status::COMPLETED: + case ObRecoverTableStatus::Status::FAILED: { + if (OB_FAIL(user_finish_(job))) { + LOG_WARN("failed to do user finish work", K(ret), K(job)); + } + break; + } + default: { + ret = OB_ERR_SYS; + LOG_WARN("invalid sys recover job status", K(ret), K(job)); + break; + } + } +} + +int ObRecoverTableJobScheduler::canceling_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + share::ObImportTableJobPersistHelper helper; + share::ObImportTableJob import_job; + bool cancel_import_job_finish = false; + if (OB_FAIL(helper.init(job.get_tenant_id()))) { + LOG_WARN("failed to init helper", K(ret)); + } else if (OB_FAIL(helper.get_import_table_job_by_initiator( + *sql_proxy_, job.get_tenant_id(), job.get_job_id(), import_job))) { + if (OB_ENTRY_NOT_EXIST == ret) { + cancel_import_job_finish = true; + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to get import table job by initiator", K(ret)); + } + } + + if (OB_SUCC(ret) && cancel_import_job_finish) { + share::ObTaskId trace_id(*ObCurTraceId::get_trace_id()); + job.get_result().set_result(OB_CANCELED, trace_id, GCONF.self_addr_); + if (OB_FAIL(try_advance_status_(job, OB_CANCELED))) { + LOG_WARN("failed to advance status", K(ret)); + } else { + LOG_INFO("[RECOVER_TABLE]cancel recover table job finish", K(job), K(import_job)); + ROOTSERVICE_EVENT_ADD("recover_table", "cancel recover job finish", + "tenant_id", job.get_tenant_id(), + "recover_job_id", job.get_job_id()); + } + } + return ret; +} + +int ObRecoverTableJobScheduler::user_prepare_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(try_advance_status_(job, ret))) { + LOG_WARN("failed to advance status", K(ret)); + } + return ret; +} + +int ObRecoverTableJobScheduler::restore_aux_tenant_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + ObRestorePersistHelper restore_helper; + ObHisRestoreJobPersistInfo restore_history_info; + bool aux_tenant_restore_finish = true; + int tmp_ret = OB_SUCCESS; + DEBUG_SYNC(BEFORE_RESTORE_AUX_TENANT); + if (OB_FAIL(restore_helper.init(OB_SYS_TENANT_ID))) { + LOG_WARN("failed to init retore helper", K(ret)); + } else if (OB_FAIL(restore_helper.get_restore_job_history( + *sql_proxy_, job.get_initiator_job_id(), job.get_initiator_tenant_id(), restore_history_info))) { + if (OB_ENTRY_NOT_EXIST == ret) { + aux_tenant_restore_finish = false; + ret = OB_SUCCESS; + if (REACH_TIME_INTERVAL(60 * 1000 * 1000)) { + LOG_INFO("[RECOVER_TABLE]aux tenant restore not finish, wait later", K(job)); + } + } else { + LOG_WARN("failed to get restore job history", K(ret), + "initiator_job_id", job.get_job_id(), "initiator_tenant_id", job.get_tenant_id()); + } + } else { + LOG_INFO("[RECOVER_TABLE]aux tenant restore finish", K(restore_history_info), K(job)); + ROOTSERVICE_EVENT_ADD("recover_table", "restore_aux_tenant_finish", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id(), + "aux_tenant_name", job.get_aux_tenant_name()); + const uint64_t aux_tenant_id = restore_history_info.restore_tenant_id_; + schema::ObSchemaGetterGuard guard; + schema::ObTenantStatus status; + if (!restore_history_info.is_restore_success()) { + ret = OB_LS_RESTORE_FAILED; // TODO(chongrong.th) adjust error code to restore tenant failed later. + LOG_WARN("[RECOVER_TABLE]restore aux tenant failed", K(ret), K(restore_history_info), K(job)); + job.get_result().set_result(false, restore_history_info.comment_); + } else if (OB_FAIL(check_aux_tenant_(job, aux_tenant_id))) { + LOG_WARN("failed to check aux tenant", K(ret), K(aux_tenant_id)); + } + + int tmp_ret = OB_SUCCESS; + if (OB_SUCCESS != (tmp_ret = try_advance_status_(job, ret))) { + LOG_WARN("failed to advance status", K(tmp_ret), K(ret)); + } + } + return ret; +} + +int ObRecoverTableJobScheduler::check_aux_tenant_(share::ObRecoverTableJob &job, const uint64_t aux_tenant_id) +{ + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + schema::ObTenantStatus status; + schema::ObSchemaGetterGuard aux_tenant_guard; + schema::ObSchemaGetterGuard recover_tenant_guard; + bool is_compatible = true; + if (OB_FAIL(schema_service_->get_tenant_schema_guard(aux_tenant_id, aux_tenant_guard))) { + if (OB_TENANT_NOT_EXIST) { + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), + "aux tenant %.*s has been dropped", job.get_aux_tenant_name().length(), job.get_aux_tenant_name().ptr()))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + job.get_result().set_result(false, comment); + } + } + LOG_WARN("failed to get tenant schema guard", K(ret), K(aux_tenant_id)); + } else if (OB_FAIL(aux_tenant_guard.get_tenant_status(aux_tenant_id, status))) { + LOG_WARN("failed to get tenant status", K(ret), K(aux_tenant_id)); + } else if (schema::ObTenantStatus::TENANT_STATUS_NORMAL != status) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("aux tenant status is not normal", K(ret), K(aux_tenant_id), K(status)); + } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(job.get_tenant_id(), recover_tenant_guard))) { + LOG_WARN("failed to get tenant schema guard", K(ret), "tenant_id", job.get_tenant_id()); + } else if (OB_FAIL(check_tenant_compatibility(aux_tenant_guard, recover_tenant_guard, is_compatible))) { + LOG_WARN("failed to get check tenant compatibility", K(ret)); + } else if (!is_compatible) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("recover from different compatibility tenant is not supported", K(ret)); + if (OB_TMP_FAIL(job.get_result().set_result(false, "recover from different compatibility tenant is not supported"))) { + LOG_WARN("failed to set result", K(ret), K(tmp_ret)); + } + } else if (OB_FAIL(check_case_sensitive_compatibility(aux_tenant_guard, recover_tenant_guard, is_compatible))) { + LOG_WARN("failed to check case sensitive compatibility", K(ret)); + } else if (!is_compatible) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("recover from different case sensitive compatibility tenant is not supported", K(ret)); + if (OB_TMP_FAIL(job.get_result().set_result(false, "recover from different case sensitive compatibility tenant is not supported"))) { + LOG_WARN("failed to set result", K(ret), K(tmp_ret)); + } + } + return ret; +} + +int ObRecoverTableJobScheduler::check_tenant_compatibility( + share::schema::ObSchemaGetterGuard &aux_tenant_guard, + share::schema::ObSchemaGetterGuard &recover_tenant_guard, + bool &is_compatible) +{ + int ret = OB_SUCCESS; + lib::Worker::CompatMode aux_compat_mode; + lib::Worker::CompatMode recover_compat_mode; + is_compatible = false; + const uint64_t aux_tenant_id = aux_tenant_guard.get_tenant_id(); + const uint64_t recover_tenant_id = recover_tenant_guard.get_tenant_id(); + if (OB_FAIL(aux_tenant_guard.get_tenant_compat_mode(aux_tenant_id, aux_compat_mode))) { + LOG_WARN("failed to get tenant compat mode", K(ret), K(aux_tenant_id)); + } else if (OB_FAIL(recover_tenant_guard.get_tenant_compat_mode(recover_tenant_id, recover_compat_mode))) { + LOG_WARN("failed to get tenant compat mode", K(ret), K(recover_tenant_id)); + } else { + is_compatible = aux_compat_mode == recover_compat_mode; + if (!is_compatible) { + LOG_WARN("[RECOVER_TABLE]tenant compat mode is different", K(is_compatible), + K(aux_tenant_id), + K(aux_compat_mode), + K(recover_tenant_id), + K(recover_compat_mode)); + } + } + return ret; +} + +int ObRecoverTableJobScheduler::check_case_sensitive_compatibility( + share::schema::ObSchemaGetterGuard &aux_tenant_guard, + share::schema::ObSchemaGetterGuard &recover_tenant_guard, + bool &is_compatible) +{ + int ret = OB_SUCCESS; + common::ObNameCaseMode aux_mode; + common::ObNameCaseMode recover_mode; + is_compatible = false; + const uint64_t aux_tenant_id = aux_tenant_guard.get_tenant_id(); + const uint64_t recover_tenant_id = recover_tenant_guard.get_tenant_id(); + if (OB_FAIL(aux_tenant_guard.get_tenant_name_case_mode(aux_tenant_id, aux_mode))) { + LOG_WARN("failed to get tenant name case mode", K(ret), K(aux_tenant_id)); + } else if (OB_FAIL(recover_tenant_guard.get_tenant_name_case_mode(recover_tenant_id, recover_mode))) { + LOG_WARN("failed to get tenant name case mode", K(ret), K(recover_tenant_id)); + } else { + is_compatible = aux_mode == recover_mode; + if (!is_compatible) { + LOG_WARN("[RECOVER_TABLE]tenant name case mode is different", K(is_compatible), + K(aux_tenant_id), + K(aux_mode), + K(recover_tenant_id), + K(recover_mode)); + } + } + return ret; +} + +int ObRecoverTableJobScheduler::gen_import_job_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + LOG_INFO("[RECOVER_TABLE]generate import table job", K(job)); + share::ObImportTableJobPersistHelper import_helper; + share::ObImportTableJob import_job; + import_job.set_tenant_id(job.get_tenant_id()); + import_job.set_job_id(job.get_job_id()); + import_job.set_initiator_job_id(job.get_job_id()); + import_job.set_initiator_tenant_id(job.get_tenant_id()); + import_job.set_start_ts(ObTimeUtility::current_time()); + import_job.set_status(ObImportTableJobStatus::INIT); + int tmp_ret = OB_SUCCESS; + schema::ObSchemaGetterGuard guard; + uint64_t tenant_id = OB_INVALID_TENANT_ID; + ObMySQLTransaction trans; + int64_t job_id = 0; + if (OB_FAIL(import_job.set_src_tenant_name(job.get_aux_tenant_name()))) { + LOG_WARN("failed to set src tenant name", K(ret)); + } else if (OB_FAIL(schema_service_->get_tenant_schema_guard(OB_SYS_TENANT_ID, guard))) { + LOG_WARN("failed to get schema guard", K(ret)); + } else if (OB_FAIL(guard.get_tenant_id(job.get_aux_tenant_name(), tenant_id))) { + if (OB_ERR_INVALID_TENANT_NAME == ret) { + ObImportResult::Comment comment; + if (OB_TMP_FAIL(databuff_printf(comment.ptr(), comment.capacity(), + "aux tenant %.*s has been dropped", job.get_aux_tenant_name().length(), job.get_aux_tenant_name().ptr()))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + job.get_result().set_result(false, comment); + } + } + LOG_WARN("failed to get tenant id", K(ret), "aux_tenant_name", job.get_aux_tenant_name()); + } else if (OB_FALSE_IT(import_job.set_src_tenant_id(tenant_id))) { + } else if (OB_FAIL(import_job.get_import_arg().assign(job.get_import_arg()))) { + LOG_WARN("failed to assign import arg", K(ret)); + } else if (OB_FAIL(import_helper.init(import_job.get_tenant_id()))) { + LOG_WARN("failed to init import job", K(ret)); + } else if (OB_FAIL(trans.start(sql_proxy_, gen_meta_tenant_id(job.get_tenant_id())))) { + LOG_WARN("failed to start trans", K(ret)); + } else if (OB_FAIL(ObLSBackupInfoOperator::get_next_job_id(trans, job.get_tenant_id(), job_id))) { + LOG_WARN("failed to get next job id", K(ret)); + } else if (OB_FALSE_IT(import_job.set_job_id(job_id))) { + } else if (OB_FAIL(import_helper.insert_import_table_job(trans, import_job))) { + LOG_WARN("failed to insert into improt table job", K(ret)); + } + + if (trans.is_started()) { + if (OB_TMP_FAIL(trans.end(OB_SUCC(ret)))) { + ret = OB_SUCC(ret) ? tmp_ret : ret; + LOG_WARN("failed to commit trans", K(ret), K(tmp_ret)); + } + if (OB_SUCC(ret)) { + LOG_INFO("[RECOVER_TABLE]succeed generate import job", K(job), K(import_job)); + ROOTSERVICE_EVENT_ADD("recover_table", "generate_import_job", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id(), + "import_job_id", import_job.get_job_id()); + } + } + + if (OB_TMP_FAIL(try_advance_status_(job, ret))) { + LOG_WARN("failed to advance status", K(tmp_ret), K(ret)); + } + return ret; +} + +int ObRecoverTableJobScheduler::importing_(share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + share::ObImportTableJobPersistHelper helper; + share::ObImportTableJob import_job; + if (OB_FAIL(helper.init(job.get_tenant_id()))) { + LOG_WARN("failed to init helper", K(ret)); + } else if (OB_FAIL(helper.get_import_table_job_history_by_initiator( + *sql_proxy_, job.get_tenant_id(), job.get_job_id(), import_job))) { + if (OB_ENTRY_NOT_EXIST == ret) { + if (REACH_TIME_INTERVAL(60 * 1000 * 1000)) { + LOG_INFO("[RECOVER_TABLE]import table is not finish, wait later", K(job)); + } + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to get recover table job histroy by initiator", K(ret)); + } + } else if (OB_FALSE_IT(job.set_end_ts(ObTimeUtility::current_time()))) { + } else if (OB_FALSE_IT(job.set_result(import_job.get_result()))) { + } else if (!job.get_result().is_succeed() && OB_FALSE_IT(ret = OB_LS_RESTORE_FAILED)) { + } else if (OB_FAIL(try_advance_status_(job, ret))) { + LOG_WARN("failed to advance status", K(ret)); + } else { + LOG_INFO("[RECOVER_TABLE]import table job finish", K(job), K(import_job)); + ROOTSERVICE_EVENT_ADD("recover_table", "import job finish", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id(), + "import_job_id", import_job.get_job_id()); + } + return ret; +} + +int ObRecoverTableJobScheduler::user_finish_(const share::ObRecoverTableJob &job) +{ + int ret = OB_SUCCESS; + ObMySQLTransaction trans; + const uint64_t tenant_id = job.get_tenant_id(); + const int64_t job_id = job.get_job_id(); + if (OB_FAIL(trans.start(sql_proxy_, gen_meta_tenant_id(tenant_id)))) { + LOG_WARN("failed to start trans", K(ret)); + } else if (OB_FAIL(helper_.insert_recover_table_job_history(trans, job))) { + LOG_WARN("failed to insert recover table job history", K(ret), K(job)); + } else if (OB_FAIL(helper_.delete_recover_table_job(trans, job))) { + LOG_WARN("failed to delete recover table job", K(ret), K(job)); + } + + if (trans.is_started()) { + int tmp_ret = OB_SUCCESS; + if (OB_TMP_FAIL(trans.end(OB_SUCC(ret)))) { + ret = OB_SUCC(ret) ? tmp_ret : ret; + LOG_WARN("end trans failed", K(ret), K(tmp_ret)); + } + if (OB_SUCC(ret)) { + LOG_INFO("[RECOVER_TABLE] recover table finish", K(job)); + ROOTSERVICE_EVENT_ADD("recover_table", "recover table job finish", + "tenant_id", job.get_tenant_id(), + "job_id", job.get_job_id()); + } + } + return ret; +} diff --git a/src/rootserver/restore/ob_recover_table_job_scheduler.h b/src/rootserver/restore/ob_recover_table_job_scheduler.h new file mode 100644 index 0000000000..d6002c444f --- /dev/null +++ b/src/rootserver/restore/ob_recover_table_job_scheduler.h @@ -0,0 +1,99 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_ROOTSERVER_RECOVER_TABLE_JOB_SCHEDULER_H +#define OCEANBASE_ROOTSERVER_RECOVER_TABLE_JOB_SCHEDULER_H + +#include "share/restore/ob_recover_table_persist_helper.h" +namespace oceanbase +{ + +namespace obrpc +{ +class ObCommonRpcProxy; +class ObSrvRpcProxy; +} + +namespace share +{ +class ObLocationService; +namespace schema +{ +class ObMultiVersionSchemaService; +} +} + +namespace common +{ +class ObMySQLProxy; +class ObString; +class ObMySQLTransaction; +class ObISQLClient; +} + +namespace rootserver +{ +class ObRecoverTableJobScheduler +{ +public: + ObRecoverTableJobScheduler() + : is_inited_(false), schema_service_(nullptr), sql_proxy_(nullptr), rs_rpc_proxy_(nullptr), + srv_rpc_proxy_(nullptr), helper_() {} + virtual ~ObRecoverTableJobScheduler() {} + void reset(); + int init(share::schema::ObMultiVersionSchemaService &schema_service, + common::ObMySQLProxy &sql_proxy, obrpc::ObCommonRpcProxy &rs_rpc_proxy, obrpc::ObSrvRpcProxy &srv_rpc_proxy); + void do_work(); + +private: + int try_advance_status_(share::ObRecoverTableJob &job, const int err_code); + + void sys_process_(share::ObRecoverTableJob &job); + int sys_prepare_(share::ObRecoverTableJob &job); + int insert_user_job_(const share::ObRecoverTableJob &job, share::ObRecoverTablePersistHelper &helper); + int recovering_(share::ObRecoverTableJob &job); + int sys_finish_(const share::ObRecoverTableJob &job); + int drop_aux_tenant_(const share::ObRecoverTableJob &job); + + void user_process_(share::ObRecoverTableJob &job); + int user_prepare_(share::ObRecoverTableJob &job); + int restore_aux_tenant_(share::ObRecoverTableJob &job); + int check_aux_tenant_(share::ObRecoverTableJob &job, const uint64_t aux_tenant_id); + int check_tenant_compatibility( + share::schema::ObSchemaGetterGuard &aux_tenant_guard, + share::schema::ObSchemaGetterGuard &recover_tenant_guard, + bool &is_compatible); + int check_case_sensitive_compatibility( + share::schema::ObSchemaGetterGuard &aux_tenant_guard, + share::schema::ObSchemaGetterGuard &recover_tenant_guard, + bool &is_compatible); + + int gen_import_job_(share::ObRecoverTableJob &job); + int importing_(share::ObRecoverTableJob &job); + int canceling_(share::ObRecoverTableJob &job); + int user_finish_(const share::ObRecoverTableJob &job); + int check_compatible_() const; +private: + bool is_inited_; + uint64_t tenant_id_; + share::schema::ObMultiVersionSchemaService *schema_service_; + common::ObMySQLProxy *sql_proxy_; + obrpc::ObCommonRpcProxy *rs_rpc_proxy_; + obrpc::ObSrvRpcProxy *srv_rpc_proxy_; + share::ObRecoverTablePersistHelper helper_; + DISALLOW_COPY_AND_ASSIGN(ObRecoverTableJobScheduler); +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/rootserver/restore/ob_restore_scheduler.cpp b/src/rootserver/restore/ob_restore_scheduler.cpp index ded9843245..c75337b047 100644 --- a/src/rootserver/restore/ob_restore_scheduler.cpp +++ b/src/rootserver/restore/ob_restore_scheduler.cpp @@ -37,6 +37,7 @@ #include "share/ob_primary_standby_service.h" #include "logservice/palf/log_define.h"//scn #include "share/scn.h" +#include "ob_restore_service.h" #ifdef OB_BUILD_TDE_SECURITY #include "share/ob_master_key_getter.h" #endif @@ -51,30 +52,21 @@ using namespace share::schema; using namespace obrpc; using namespace palf; -ObRestoreService::ObRestoreService() +ObRestoreScheduler::ObRestoreScheduler() : inited_(false), schema_service_(NULL), sql_proxy_(NULL), rpc_proxy_(NULL), srv_rpc_proxy_(NULL), lst_operator_(NULL), - upgrade_processors_(), self_addr_(), + restore_service_(nullptr), self_addr_(), tenant_id_(OB_INVALID_TENANT_ID), idle_time_us_(1) - { } -ObRestoreService::~ObRestoreService() +ObRestoreScheduler::~ObRestoreScheduler() { - if (!has_set_stop()) { - stop(); - wait(); - } } -void ObRestoreService::destroy() -{ - ObTenantThreadHelper::destroy(); - inited_ = false; -} -int ObRestoreService::init() + +int ObRestoreScheduler::init(ObRestoreService &restore_service) { int ret = OB_SUCCESS; if (inited_) { @@ -86,41 +78,20 @@ int ObRestoreService::init() ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", KR(ret), KP(GCTX.schema_service_), KP(GCTX.sql_proxy_), KP(GCTX.rs_rpc_proxy_), KP(GCTX.srv_rpc_proxy_), KP(GCTX.lst_operator_)); - } else if (OB_FAIL(ObTenantThreadHelper::create("REST_SER", lib::TGDefIDs::SimpleLSService, *this))) { - LOG_WARN("failed to create thread", KR(ret)); - } else if (OB_FAIL(ObTenantThreadHelper::start())) { - LOG_WARN("fail to start thread", KR(ret)); - } else if (OB_FAIL(upgrade_processors_.init( - ObBaseUpgradeProcessor::UPGRADE_MODE_PHYSICAL_RESTORE, - *GCTX.sql_proxy_, *GCTX.srv_rpc_proxy_, *GCTX.rs_rpc_proxy_, *GCTX.schema_service_, *this))) { - LOG_WARN("fail to init upgrade processors", KR(ret)); } else { schema_service_ = GCTX.schema_service_; sql_proxy_ = GCTX.sql_proxy_; rpc_proxy_ = GCTX.rs_rpc_proxy_; srv_rpc_proxy_ = GCTX.srv_rpc_proxy_; lst_operator_ = GCTX.lst_operator_; + restore_service_ = &restore_service; tenant_id_ = is_sys_tenant(MTL_ID()) ? MTL_ID() : gen_user_tenant_id(MTL_ID()); self_addr_ = GCTX.self_addr(); inited_ = true; } return ret; } - -int ObRestoreService::idle() -{ - int ret = OB_SUCCESS; - if (!inited_) { - ret = OB_NOT_INIT; - LOG_WARN("not init", K(ret)); - } else { - ObTenantThreadHelper::idle(idle_time_us_); - idle_time_us_ = GCONF._restore_idle_time; - } - return ret; -} - -void ObRestoreService::do_work() +void ObRestoreScheduler::do_work() { LOG_INFO("[RESTORE] restore scheduler start"); int ret = OB_SUCCESS; @@ -128,61 +99,37 @@ void ObRestoreService::do_work() ret = OB_NOT_INIT; LOG_WARN("not inited", K(ret)); } else { - ObRSThreadFlag rs_work; - // avoid using default idle time when observer restarts. idle_time_us_ = GCONF._restore_idle_time; - const uint64_t tenant_id = MTL_ID(); - while (!has_set_stop()) { - { - ObCurTraceId::init(GCTX.self_addr()); - LOG_INFO("[RESTORE] try process restore job"); - ObArray job_infos; - ObPhysicalRestoreTableOperator restore_op; - share::schema::ObSchemaGetterGuard schema_guard; - const share::schema::ObTenantSchema *tenant_schema = NULL; - if (OB_ISNULL(GCTX.schema_service_)) { + ObCurTraceId::init(GCTX.self_addr()); + LOG_INFO("[RESTORE] try process restore job"); + ObArray job_infos; + ObPhysicalRestoreTableOperator restore_op; + if (OB_FAIL(restore_op.init(sql_proxy_, tenant_id_))) { + LOG_WARN("fail init", K(ret), K(tenant_id_)); + } else if (OB_FAIL(restore_op.get_jobs(job_infos))) { + LOG_WARN("fail to get jobs", KR(ret), K(tenant_id_)); + } else { + FOREACH_CNT_X(job_info, job_infos, !restore_service_->has_set_stop()) { // ignore ret + if (OB_ISNULL(job_info)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected error", KR(ret)); - } else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard( - OB_SYS_TENANT_ID, schema_guard))) { - LOG_WARN("fail to get schema guard", KR(ret)); - } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { - LOG_WARN("failed to get tenant ids", 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_normal()) { - //tenant not normal, maybe meta or sys tenant - //while meta tenant not ready, cannot process tenant restore job - } else if (OB_FAIL(restore_op.init(sql_proxy_, tenant_id_))) { - LOG_WARN("fail init", K(ret), K(tenant_id_)); - } else if (OB_FAIL(restore_op.get_jobs(job_infos))) { - LOG_WARN("fail to get jobs", KR(ret), K(tenant_id_)); - } else { - FOREACH_CNT_X(job_info, job_infos, !has_set_stop()) { // ignore ret - if (OB_ISNULL(job_info)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("job info is null", K(ret)); - } else if (is_sys_tenant(tenant_id_)) { - if (OB_FAIL(process_sys_restore_job(*job_info))) { - LOG_WARN("failed to process sys restore job", KR(ret), KPC(job_info)); - } - } else if (OB_FAIL(process_restore_job(*job_info))) { - LOG_WARN("fail to process restore job", K(ret), KPC(job_info)); - } + LOG_WARN("job info is null", K(ret)); + } else if (is_sys_tenant(tenant_id_)) { + if (OB_FAIL(process_sys_restore_job(*job_info))) { + LOG_WARN("failed to process sys restore job", KR(ret), KPC(job_info)); } + } else if (OB_FAIL(process_restore_job(*job_info))) { + LOG_WARN("fail to process restore job", K(ret), KPC(job_info)); } - }//for schema guard, must be free - // retry until stopped, reset ret to OB_SUCCESS - ret = OB_SUCCESS; - idle(); + } } + ret = OB_SUCCESS; + restore_service_->idle(); } LOG_INFO("[RESTORE] restore scheduler quit"); return; } -int ObRestoreService::process_sys_restore_job(const ObPhysicalRestoreJob &job) +int ObRestoreScheduler::process_sys_restore_job(const ObPhysicalRestoreJob &job) { int ret = OB_SUCCESS; if (!inited_) { @@ -222,7 +169,7 @@ int ObRestoreService::process_sys_restore_job(const ObPhysicalRestoreJob &job) } -int ObRestoreService::process_restore_job(const ObPhysicalRestoreJob &job) +int ObRestoreScheduler::process_restore_job(const ObPhysicalRestoreJob &job) { int ret = OB_SUCCESS; if (!inited_) { @@ -269,7 +216,7 @@ int ObRestoreService::process_restore_job(const ObPhysicalRestoreJob &job) } // restore_tenant is not reentrant -int ObRestoreService::restore_tenant(const ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_tenant(const ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; ObCreateTenantArg arg; @@ -281,7 +228,7 @@ int ObRestoreService::restore_tenant(const ObPhysicalRestoreJob &job_info) if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not inited", K(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_INVALID_TENANT_ID != job_info.get_tenant_id()) { // restore_tenant can only be executed once. @@ -315,7 +262,7 @@ int ObRestoreService::restore_tenant(const ObPhysicalRestoreJob &job_info) return ret; } -int ObRestoreService::fill_create_tenant_arg( +int ObRestoreScheduler::fill_create_tenant_arg( const ObPhysicalRestoreJob &job, const ObSqlString &pool_list, ObCreateTenantArg &arg) @@ -325,7 +272,7 @@ int ObRestoreService::fill_create_tenant_arg( if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not inited", K(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_FAIL(schema_service_->get_tenant_schema_guard( OB_SYS_TENANT_ID, schema_guard))) { @@ -379,7 +326,7 @@ int ObRestoreService::fill_create_tenant_arg( return ret; } -int ObRestoreService::assign_pool_list( +int ObRestoreScheduler::assign_pool_list( const char *str, common::ObIArray &pool_list) { @@ -400,7 +347,7 @@ int ObRestoreService::assign_pool_list( return ret; } -int ObRestoreService::check_locality_valid( +int ObRestoreScheduler::check_locality_valid( const share::schema::ZoneLocalityIArray &locality) { int ret = OB_SUCCESS; @@ -430,7 +377,7 @@ int ObRestoreService::check_locality_valid( } -int ObRestoreService::check_tenant_can_restore_(const uint64_t tenant_id) +int ObRestoreScheduler::check_tenant_can_restore_(const uint64_t tenant_id) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!inited_)) { @@ -439,7 +386,7 @@ int ObRestoreService::check_tenant_can_restore_(const uint64_t tenant_id) } else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("tenant id invalid", KR(ret), K(tenant_id)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", KR(ret)); } else if (GCONF.in_upgrade_mode()) { // 2. check in upgrade mode @@ -452,7 +399,7 @@ int ObRestoreService::check_tenant_can_restore_(const uint64_t tenant_id) } //restore pre :modify parameters -int ObRestoreService::restore_pre(const ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_pre(const ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; if (!inited_) { @@ -462,7 +409,7 @@ int ObRestoreService::restore_pre(const ObPhysicalRestoreJob &job_info) || OB_SYS_TENANT_ID == tenant_id_) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant id", K(ret), K(tenant_id_)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_FAIL(restore_root_key(job_info))) { LOG_WARN("fail to restore root key", K(ret)); @@ -481,7 +428,7 @@ int ObRestoreService::restore_pre(const ObPhysicalRestoreJob &job_info) return ret; } -int ObRestoreService::fill_restore_statistics(const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::fill_restore_statistics(const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; ObRestoreProgressPersistInfo restore_progress_info; @@ -520,7 +467,7 @@ int ObRestoreService::fill_restore_statistics(const share::ObPhysicalRestoreJob return ret; } -int ObRestoreService::convert_parameters( +int ObRestoreScheduler::convert_parameters( const ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; @@ -533,7 +480,7 @@ int ObRestoreService::convert_parameters( || OB_SYS_TENANT_ID == tenant_id) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant id", K(ret), K(tenant_id)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (job_info.get_kms_dest().empty()) { // do nothing @@ -573,7 +520,7 @@ int ObRestoreService::convert_parameters( return ret; } -int ObRestoreService::restore_root_key(const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_root_key(const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; #ifdef OB_BUILD_TDE_SECURITY @@ -627,7 +574,7 @@ int ObRestoreService::restore_root_key(const share::ObPhysicalRestoreJob &job_in return ret; } -int ObRestoreService::restore_keystore(const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_keystore(const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; #ifdef OB_BUILD_TDE_SECURITY @@ -674,7 +621,7 @@ int ObRestoreService::restore_keystore(const share::ObPhysicalRestoreJob &job_in return ret; } -int ObRestoreService::post_check(const ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::post_check(const ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; @@ -689,7 +636,7 @@ int ObRestoreService::post_check(const ObPhysicalRestoreJob &job_info) || OB_SYS_TENANT_ID == tenant_id_) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant id", K(ret), K(tenant_id_)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_FAIL(ObAllTenantInfoProxy::load_tenant_info(tenant_id_, sql_proxy_, false, /*for_update*/all_tenant_info))) { @@ -742,14 +689,14 @@ int ObRestoreService::post_check(const ObPhysicalRestoreJob &job_info) return ret; } -int ObRestoreService::restore_finish(const ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_finish(const ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_FAIL(ObRestoreUtil::recycle_restore_job(tenant_id_, *sql_proxy_, job_info))) { @@ -763,7 +710,7 @@ int ObRestoreService::restore_finish(const ObPhysicalRestoreJob &job_info) return ret; } -int ObRestoreService::tenant_restore_finish(const ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::tenant_restore_finish(const ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; ObHisRestoreJobPersistInfo history_info; @@ -771,7 +718,7 @@ int ObRestoreService::tenant_restore_finish(const ObPhysicalRestoreJob &job_info if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_FAIL(try_get_tenant_restore_history_(job_info, history_info, restore_tenant_exist))) { LOG_WARN("failed to get user tenant restory info", KR(ret), K(job_info)); @@ -793,18 +740,7 @@ int ObRestoreService::tenant_restore_finish(const ObPhysicalRestoreJob &job_info return ret; } - -int ObRestoreService::check_stop() const -{ - int ret = OB_SUCCESS; - if (has_set_stop()) { - ret = OB_CANCELED; - LOG_WARN("restore scheduler stopped", K(ret)); - } - return ret; -} - -int ObRestoreService::try_get_tenant_restore_history_( +int ObRestoreScheduler::try_get_tenant_restore_history_( const ObPhysicalRestoreJob &job_info, ObHisRestoreJobPersistInfo &history_info, bool &restore_tenant_exist) @@ -818,7 +754,7 @@ int ObRestoreService::try_get_tenant_restore_history_( if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not inited", KR(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", KR(ret)); } else if (OB_INVALID_TENANT_ID == restore_tenant_id) { //maybe failed to create tenant @@ -875,7 +811,7 @@ int ObRestoreService::try_get_tenant_restore_history_( * 2. Physical restore jobs will be recycled asynchronously when restore tenant has been dropped. * 3. Physical restore jobs will be used to avoid duplicate tenant_name when tenant is creating. */ -int ObRestoreService::try_recycle_job(const ObPhysicalRestoreJob &job) +int ObRestoreScheduler::try_recycle_job(const ObPhysicalRestoreJob &job) { int ret = OB_SUCCESS; ObSchemaGetterGuard schema_guard; @@ -913,7 +849,7 @@ int ObRestoreService::try_recycle_job(const ObPhysicalRestoreJob &job) return ret; } -int ObRestoreService::try_update_job_status( +int ObRestoreScheduler::try_update_job_status( common::ObISQLClient &sql_client, int return_ret, const ObPhysicalRestoreJob &job, @@ -924,7 +860,7 @@ int ObRestoreService::try_update_job_status( if (!inited_) { ret = OB_NOT_INIT; LOG_WARN("not inited", K(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_FAIL(restore_op.init(&sql_client, tenant_id_))) { LOG_WARN("fail init", K(ret), K(tenant_id_)); @@ -949,7 +885,7 @@ int ObRestoreService::try_update_job_status( return ret; } -void ObRestoreService::record_rs_event( +void ObRestoreScheduler::record_rs_event( const ObPhysicalRestoreJob &job, const PhysicalRestoreStatus status) { @@ -961,7 +897,7 @@ void ObRestoreService::record_rs_event( "status", status_str); } -PhysicalRestoreStatus ObRestoreService::get_sys_next_status( +PhysicalRestoreStatus ObRestoreScheduler::get_sys_next_status( PhysicalRestoreStatus current_status) { PhysicalRestoreStatus next_status = PHYSICAL_RESTORE_MAX_STATUS; @@ -983,7 +919,7 @@ PhysicalRestoreStatus ObRestoreService::get_sys_next_status( -PhysicalRestoreStatus ObRestoreService::get_next_status( +PhysicalRestoreStatus ObRestoreScheduler::get_next_status( int return_ret, PhysicalRestoreStatus current_status) { @@ -1026,7 +962,7 @@ PhysicalRestoreStatus ObRestoreService::get_next_status( return next_status; } -int ObRestoreService::restore_upgrade(const ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_upgrade(const ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; DEBUG_SYNC(BEFORE_PHYSICAL_RESTORE_UPGRADE_PRE); @@ -1037,7 +973,7 @@ int ObRestoreService::restore_upgrade(const ObPhysicalRestoreJob &job_info) || OB_SYS_TENANT_ID == tenant_id_) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid tenant id", KR(ret), K(tenant_id_)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", KR(ret)); } else { if (OB_SUCC(ret)) { @@ -1051,7 +987,7 @@ int ObRestoreService::restore_upgrade(const ObPhysicalRestoreJob &job_info) return ret; } -int ObRestoreService::restore_init_ls(const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_init_ls(const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; DEBUG_SYNC(BEFORE_PHYSICAL_RESTORE_INIT_LS); @@ -1133,7 +1069,7 @@ int ObRestoreService::restore_init_ls(const share::ObPhysicalRestoreJob &job_inf return ret; } -int ObRestoreService::set_restore_to_target_scn_( +int ObRestoreScheduler::set_restore_to_target_scn_( common::ObMySQLTransaction &trans, const share::ObPhysicalRestoreJob &job_info, const share::SCN &scn) { int ret = OB_SUCCESS; @@ -1165,7 +1101,7 @@ int ObRestoreService::set_restore_to_target_scn_( } return ret; } -int ObRestoreService::create_all_ls_( +int ObRestoreScheduler::create_all_ls_( const share::schema::ObTenantSchema &tenant_schema, const common::ObIArray &ls_attr_array) { @@ -1175,7 +1111,7 @@ int ObRestoreService::create_all_ls_( if (OB_UNLIKELY(!inited_)) { ret = OB_NOT_INIT; LOG_WARN("not inited", KR(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", KR(ret)); } else if (OB_ISNULL(sql_proxy_)) { ret = OB_ERR_UNEXPECTED; @@ -1222,14 +1158,14 @@ int ObRestoreService::create_all_ls_( return ret; } -int ObRestoreService::wait_all_ls_created_(const share::schema::ObTenantSchema &tenant_schema, +int ObRestoreScheduler::wait_all_ls_created_(const share::schema::ObTenantSchema &tenant_schema, const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!inited_)) { ret = OB_NOT_INIT; LOG_WARN("not inited", KR(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", KR(ret)); } else if (OB_UNLIKELY(!tenant_schema.is_valid())) { ret = OB_INVALID_ARGUMENT; @@ -1274,7 +1210,7 @@ int ObRestoreService::wait_all_ls_created_(const share::schema::ObTenantSchema & return ret; } -int ObRestoreService::finish_create_ls_( +int ObRestoreScheduler::finish_create_ls_( const share::schema::ObTenantSchema &tenant_schema, const common::ObIArray &ls_attr_array) { @@ -1282,7 +1218,7 @@ int ObRestoreService::finish_create_ls_( if (OB_UNLIKELY(!inited_)) { ret = OB_NOT_INIT; LOG_WARN("not inited", KR(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", KR(ret)); } else if (OB_UNLIKELY(!tenant_schema.is_valid())) { ret = OB_INVALID_ARGUMENT; @@ -1341,7 +1277,7 @@ int ObRestoreService::finish_create_ls_( return ret; } -int ObRestoreService::restore_wait_to_consistent_scn(const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_wait_to_consistent_scn(const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; TenantRestoreStatus tenant_restore_status; @@ -1401,7 +1337,7 @@ int ObRestoreService::restore_wait_to_consistent_scn(const share::ObPhysicalRest -int ObRestoreService::check_tenant_replay_to_consistent_scn(const uint64_t tenant_id, const share::SCN &scn, bool &is_replay_finish) +int ObRestoreScheduler::check_tenant_replay_to_consistent_scn(const uint64_t tenant_id, const share::SCN &scn, bool &is_replay_finish) { int ret = OB_SUCCESS; ObAllTenantInfo tenant_info; @@ -1417,7 +1353,7 @@ int ObRestoreService::check_tenant_replay_to_consistent_scn(const uint64_t tenan return ret; } -int ObRestoreService::restore_wait_ls_finish(const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_wait_ls_finish(const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; DEBUG_SYNC(BEFORE_PHYSICAL_RESTORE_WAIT_LS_FINISH); @@ -1459,7 +1395,7 @@ int ObRestoreService::restore_wait_ls_finish(const share::ObPhysicalRestoreJob & return ret; } -int ObRestoreService::check_all_ls_restore_finish_( +int ObRestoreScheduler::check_all_ls_restore_finish_( const uint64_t tenant_id, TenantRestoreStatus &tenant_restore_status) { @@ -1520,7 +1456,7 @@ int ObRestoreService::check_all_ls_restore_finish_( return ret; } -int ObRestoreService::check_all_ls_restore_to_consistent_scn_finish_( +int ObRestoreScheduler::check_all_ls_restore_to_consistent_scn_finish_( const uint64_t tenant_id, TenantRestoreStatus &tenant_restore_status) { @@ -1549,7 +1485,7 @@ int ObRestoreService::check_all_ls_restore_to_consistent_scn_finish_( return ret; } -int ObRestoreService::restore_wait_tenant_finish(const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::restore_wait_tenant_finish(const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; DEBUG_SYNC(BEFORE_WAIT_RESTORE_TENANT_FINISH); @@ -1560,7 +1496,7 @@ int ObRestoreService::restore_wait_tenant_finish(const share::ObPhysicalRestoreJ if (OB_UNLIKELY(!inited_)) { ret = OB_NOT_INIT; LOG_WARN("not inited", KR(ret)); - } else if (OB_FAIL(check_stop())) { + } else if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", KR(ret)); } else if (OB_FAIL(restore_op.init(sql_proxy_, tenant_id_))) { LOG_WARN("fail init", K(ret), K(tenant_id_)); @@ -1588,7 +1524,7 @@ int ObRestoreService::restore_wait_tenant_finish(const share::ObPhysicalRestoreJ obrpc::ObCreateTenantEndArg arg; arg.tenant_id_ = tenant_id; arg.exec_tenant_id_ = OB_SYS_TENANT_ID; - if (OB_FAIL(check_stop())) { + if (OB_FAIL(restore_service_->check_stop())) { LOG_WARN("restore scheduler stopped", K(ret)); } else if (OB_FAIL(rpc_proxy_->timeout(DEFAULT_TIMEOUT) .create_tenant_end(arg))) { @@ -1619,7 +1555,7 @@ int ObRestoreService::restore_wait_tenant_finish(const share::ObPhysicalRestoreJ return ret; } -int ObRestoreService::reset_schema_status(const uint64_t tenant_id, common::ObMySQLProxy *sql_proxy) +int ObRestoreScheduler::reset_schema_status(const uint64_t tenant_id, common::ObMySQLProxy *sql_proxy) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!is_user_tenant(tenant_id))) { @@ -1640,7 +1576,7 @@ int ObRestoreService::reset_schema_status(const uint64_t tenant_id, common::ObMy return ret; } -int ObRestoreService::may_update_restore_concurrency_(const uint64_t new_tenant_id, const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::may_update_restore_concurrency_(const uint64_t new_tenant_id, const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; double cpu_count = 0; @@ -1674,7 +1610,7 @@ int ObRestoreService::may_update_restore_concurrency_(const uint64_t new_tenant_ return ret; } -int ObRestoreService::reset_restore_concurrency_(const uint64_t new_tenant_id, const share::ObPhysicalRestoreJob &job_info) +int ObRestoreScheduler::reset_restore_concurrency_(const uint64_t new_tenant_id, const share::ObPhysicalRestoreJob &job_info) { int ret = OB_SUCCESS; const int64_t concurrency = 0; @@ -1688,7 +1624,7 @@ int ObRestoreService::reset_restore_concurrency_(const uint64_t new_tenant_id, c return ret; } -int ObRestoreService::update_restore_concurrency_(const common::ObString &tenant_name, +int ObRestoreScheduler::update_restore_concurrency_(const common::ObString &tenant_name, const uint64_t tenant_id, const int64_t concurrency) { int ret = OB_SUCCESS; diff --git a/src/rootserver/restore/ob_restore_scheduler.h b/src/rootserver/restore/ob_restore_scheduler.h index 24b9102666..19c4188780 100644 --- a/src/rootserver/restore/ob_restore_scheduler.h +++ b/src/rootserver/restore/ob_restore_scheduler.h @@ -14,11 +14,8 @@ #define OCEANBASE_ROOTSERVER_OB_RESTORE_SCHEDULER_H_ #include "rootserver/restore/ob_restore_util.h" -#include "rootserver/ob_tenant_thread_helper.h"//ObTenantThreadHelper #include "share/backup/ob_backup_struct.h" #include "share/ob_rpc_struct.h" -#include "share/ob_common_rpc_proxy.h" -#include "share/ob_rpc_struct.h" #include "share/ob_upgrade_utils.h" namespace oceanbase @@ -32,32 +29,20 @@ struct ObHisRestoreJobPersistInfo; } namespace rootserver { +class ObRestoreService; // Running in a single thread. // schedule restore job, register to sys ls of meta tenant -class ObRestoreService : public ObTenantThreadHelper, - public logservice::ObICheckpointSubHandler, public logservice::ObIReplaySubHandler, - public share::ObCheckStopProvider +class ObRestoreScheduler { public: static const int64_t MAX_RESTORE_TASK_CNT = 10000; public: - ObRestoreService(); - virtual ~ObRestoreService(); - int init(); - virtual void do_work() override; + ObRestoreScheduler(); + virtual ~ObRestoreScheduler(); + int init(ObRestoreService &restore_service); + void do_work(); void destroy(); - DEFINE_MTL_FUNC(ObRestoreService) public: - virtual share::SCN get_rec_scn() override { return share::SCN::max_scn();} - virtual int flush(share::SCN &rec_scn) override { return OB_SUCCESS; } - int replay(const void *buffer, const int64_t nbytes, const palf::LSN &lsn, const share::SCN &scn) override - { - UNUSED(buffer); - UNUSED(nbytes); - UNUSED(lsn); - UNUSED(scn); - return OB_SUCCESS; - } enum TenantRestoreStatus { IN_PROGRESS = 0, @@ -84,9 +69,6 @@ public: common::ObIArray &pool_list); private: - int idle(); - int check_stop() const override; - int process_restore_job(const share::ObPhysicalRestoreJob &job); int process_sys_restore_job(const share::ObPhysicalRestoreJob &job); int try_recycle_job(const share::ObPhysicalRestoreJob &job); @@ -149,13 +131,16 @@ private: obrpc::ObCommonRpcProxy *rpc_proxy_; obrpc::ObSrvRpcProxy *srv_rpc_proxy_; share::ObLSTableOperator *lst_operator_; - share::ObUpgradeProcesserSet upgrade_processors_; + ObRestoreService *restore_service_; common::ObAddr self_addr_; uint64_t tenant_id_; int64_t idle_time_us_; - DISALLOW_COPY_AND_ASSIGN(ObRestoreService); + DISALLOW_COPY_AND_ASSIGN(ObRestoreScheduler); }; + + + } // end namespace rootserver } // end namespace oceanbase diff --git a/src/rootserver/restore/ob_restore_service.cpp b/src/rootserver/restore/ob_restore_service.cpp new file mode 100644 index 0000000000..dc5b4ca8fc --- /dev/null +++ b/src/rootserver/restore/ob_restore_service.cpp @@ -0,0 +1,157 @@ +/** + * 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 "ob_restore_service.h" +#include "ob_recover_table_job_scheduler.h" +#include "share/restore/ob_import_table_struct.h" +#include "share/restore/ob_recover_table_persist_helper.h" + +using namespace oceanbase; +using namespace rootserver; +using namespace share; + + +ObRestoreService::ObRestoreService() + : inited_(false), + schema_service_(NULL), + sql_proxy_(NULL), + rpc_proxy_(NULL), + srv_rpc_proxy_(NULL), + self_addr_(), + tenant_id_(OB_INVALID_TENANT_ID), + idle_time_us_(1), + restore_scheduler_(), + recover_table_scheduler_() + +{ +} + +ObRestoreService::~ObRestoreService() +{ + if (!has_set_stop()) { + stop(); + wait(); + } +} + +void ObRestoreService::destroy() +{ + ObTenantThreadHelper::destroy(); + inited_ = false; +} + +int ObRestoreService::init() +{ + int ret = OB_SUCCESS; + if (inited_) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", KR(ret)); + } else if (OB_ISNULL(GCTX.schema_service_) || OB_ISNULL(GCTX.sql_proxy_) + || OB_ISNULL(GCTX.rs_rpc_proxy_) || OB_ISNULL(GCTX.srv_rpc_proxy_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), KP(GCTX.schema_service_), KP(GCTX.sql_proxy_), + KP(GCTX.rs_rpc_proxy_), KP(GCTX.srv_rpc_proxy_), KP(GCTX.lst_operator_)); + } else if (OB_FAIL(ObTenantThreadHelper::create("REST_SER", lib::TGDefIDs::SimpleLSService, *this))) { + LOG_WARN("failed to create thread", KR(ret)); + } else if (OB_FAIL(ObTenantThreadHelper::start())) { + LOG_WARN("fail to start thread", KR(ret)); + } else if (OB_FAIL(restore_scheduler_.init(*this))) { + LOG_WARN("failed to init restore scheduler", K(ret)); + } else if (OB_FAIL(recover_table_scheduler_.init( + *GCTX.schema_service_, *GCTX.sql_proxy_, *GCTX.rs_rpc_proxy_, *GCTX.srv_rpc_proxy_))) { + LOG_WARN("failed to init recover table scheduler", K(ret)); + } else if (OB_FAIL(import_table_scheduler_.init(*GCTX.schema_service_, *GCTX.sql_proxy_))) { + LOG_WARN("failed to init import table scheduler", K(ret)); + } else { + schema_service_ = GCTX.schema_service_; + sql_proxy_ = GCTX.sql_proxy_; + rpc_proxy_ = GCTX.rs_rpc_proxy_; + srv_rpc_proxy_ = GCTX.srv_rpc_proxy_; + tenant_id_ = is_sys_tenant(MTL_ID()) ? MTL_ID() : gen_user_tenant_id(MTL_ID()); + self_addr_ = GCTX.self_addr(); + inited_ = true; + } + return ret; +} + +int ObRestoreService::idle() +{ + int ret = OB_SUCCESS; + if (!inited_) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else { + ObTenantThreadHelper::idle(idle_time_us_); + idle_time_us_ = GCONF._restore_idle_time; + } + return ret; +} + +int ObRestoreService::check_stop() const +{ + int ret = OB_SUCCESS; + if (has_set_stop()) { + ret = OB_CANCELED; + LOG_WARN("restore service stopped", K(ret)); + } + return ret; +} + +void ObRestoreService::do_work() +{ + LOG_INFO("[RESTORE] restore service start"); + int ret = OB_SUCCESS; + if (!inited_) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", K(ret)); + } else { + ObRSThreadFlag rs_work; + // avoid using default idle time when observer restarts. + idle_time_us_ = GCONF._restore_idle_time; + const uint64_t tenant_id = MTL_ID(); + while (!has_set_stop()) { + { + ObCurTraceId::init(GCTX.self_addr()); + ObArray job_infos; + share::schema::ObSchemaGetterGuard schema_guard; + const share::schema::ObTenantSchema *tenant_schema = NULL; + if (OB_ISNULL(GCTX.schema_service_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error", KR(ret)); + } else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard( + OB_SYS_TENANT_ID, schema_guard))) { + LOG_WARN("fail to get schema guard", KR(ret)); + } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { + LOG_WARN("failed to get tenant ids", 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_normal()) { + //tenant not normal, maybe meta or sys tenant + //while meta tenant not ready, cannot process tenant restore job + } else { + restore_scheduler_.do_work(); + recover_table_scheduler_.do_work(); + import_table_scheduler_.do_work(); + idle_time_us_ = 10; + } + }//for schema guard, must be free + // retry until stopped, reset ret to OB_SUCCESS + ret = OB_SUCCESS; + idle(); + } + } + LOG_INFO("[RESTORE] restore service quit"); + return; +} \ No newline at end of file diff --git a/src/rootserver/restore/ob_restore_service.h b/src/rootserver/restore/ob_restore_service.h new file mode 100644 index 0000000000..8212215543 --- /dev/null +++ b/src/rootserver/restore/ob_restore_service.h @@ -0,0 +1,94 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_ROOTSERVER_RECOVER_TABLE_SERVICE_H +#define OCEANBASE_ROOTSERVER_RECOVER_TABLE_SERVICE_H +#include "ob_restore_scheduler.h" +#include "ob_recover_table_job_scheduler.h" +#include "ob_import_table_job_scheduler.h" +#include "rootserver/ob_tenant_thread_helper.h"//ObTenantThreadHelper +#include "share/ob_check_stop_provider.h" +#include "share/ob_common_rpc_proxy.h" +#include "share/scn.h" + +namespace oceanbase +{ + +namespace share +{ +class ObLocationService; +namespace schema +{ +class ObMultiVersionSchemaService; +} + +struct ObRecoverTableJob; +} + +namespace common +{ +class ObMySQLProxy; +} + +namespace rootserver +{ + +// Running in a single thread. +// schedule restore job, register to sys ls of meta tenant +class ObRestoreService : public ObTenantThreadHelper, + public logservice::ObICheckpointSubHandler, public logservice::ObIReplaySubHandler, + public share::ObCheckStopProvider +{ +public: + static const int64_t MAX_RESTORE_TASK_CNT = 10000; +public: + ObRestoreService(); + virtual ~ObRestoreService(); + int init(); + virtual void do_work() override; + void destroy(); + DEFINE_MTL_FUNC(ObRestoreService) +public: + virtual share::SCN get_rec_scn() override { return share::SCN::max_scn();} + virtual int flush(share::SCN &rec_scn) override { return OB_SUCCESS; } + int replay(const void *buffer, const int64_t nbytes, const palf::LSN &lsn, const share::SCN &scn) override + { + UNUSED(buffer); + UNUSED(nbytes); + UNUSED(lsn); + UNUSED(scn); + return OB_SUCCESS; + } + +public: + int idle(); + int check_stop() const override; + +private: + bool inited_; + share::schema::ObMultiVersionSchemaService *schema_service_; + common::ObMySQLProxy *sql_proxy_; + obrpc::ObCommonRpcProxy *rpc_proxy_; + obrpc::ObSrvRpcProxy *srv_rpc_proxy_; + common::ObAddr self_addr_; + uint64_t tenant_id_; + int64_t idle_time_us_; + ObRestoreScheduler restore_scheduler_; + ObRecoverTableJobScheduler recover_table_scheduler_; + ObImportTableJobScheduler import_table_scheduler_; + DISALLOW_COPY_AND_ASSIGN(ObRestoreService); +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/rootserver/restore/ob_restore_util.cpp b/src/rootserver/restore/ob_restore_util.cpp index 30e141a927..164ab4c021 100644 --- a/src/rootserver/restore/ob_restore_util.cpp +++ b/src/rootserver/restore/ob_restore_util.cpp @@ -53,7 +53,8 @@ int ObRestoreUtil::fill_physical_restore_job( job.init_restore_key(OB_SYS_TENANT_ID, job_id); job.set_status(PhysicalRestoreStatus::PHYSICAL_RESTORE_CREATE_TENANT); job.set_tenant_name(arg.tenant_name_); - job.set_initiator_tenant_id(OB_SYS_TENANT_ID); + job.set_initiator_job_id(arg.initiator_job_id_); + job.set_initiator_tenant_id(arg.initiator_tenant_id_); if (OB_FAIL(job.set_description(arg.description_))) { LOG_WARN("fail to set description", K(ret)); } @@ -292,11 +293,12 @@ int ObRestoreUtil::fill_compat_backup_path( ObArenaAllocator allocator; ObArray tenant_path_array; ObArray backup_set_list; - ObArray backup_piece_list; + ObArray backup_piece_list; ObArray log_path_list; ObString tenant_dest_list; int64_t last_backup_set_idx = -1; bool restore_using_compl_log = false; + share::SCN restore_scn; if (!arg.multi_uri_.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid args", K(ret), K(arg)); @@ -308,8 +310,11 @@ int ObRestoreUtil::fill_compat_backup_path( LOG_WARN("failed to copy backup dest", K(ret), K(arg)); } else if (OB_FAIL(check_restore_using_complement_log_(tenant_path_array, restore_using_compl_log))) { LOG_WARN("failed to check only contain backup set", K(ret), K(tenant_path_array)); - } else if (OB_FAIL(fill_restore_scn_(arg, tenant_path_array, restore_using_compl_log, job))) { + } else if (OB_FAIL(fill_restore_scn_( + arg.restore_scn_, arg.restore_timestamp_, arg.with_restore_scn_, tenant_path_array, arg.passwd_array_, + restore_using_compl_log, restore_scn))) { LOG_WARN("fail to fill restore scn", K(ret), K(arg), K(tenant_path_array)); + } else if (OB_FALSE_IT(job.set_restore_scn(restore_scn))) { } else if (OB_FAIL(get_restore_source(restore_using_compl_log, tenant_path_array, arg.passwd_array_, job.get_restore_scn(), backup_set_list, backup_piece_list, log_path_list))) { LOG_WARN("fail to get restore source", K(ret), K(tenant_path_array), K(arg)); @@ -322,22 +327,26 @@ int ObRestoreUtil::fill_compat_backup_path( } else if (OB_FAIL(do_fill_backup_info_(backup_set_list.at(last_backup_set_idx).backup_set_path_, job))) { LOG_WARN("fail to do fill backup info"); } - return ret; } -int ObRestoreUtil::fill_restore_scn_(const obrpc::ObPhysicalRestoreTenantArg &arg, - const ObIArray &tenant_path_array, const bool restore_using_compl_log, - share::ObPhysicalRestoreJob &job) +int ObRestoreUtil::fill_restore_scn_( + const share::SCN &src_scn, + const ObString ×tamp, + const bool with_restore_scn, + const ObIArray &tenant_path_array, + const common::ObString &passwd, + const bool restore_using_compl_log, + share::SCN &restore_scn) { int ret = OB_SUCCESS; - if (!arg.is_valid() || tenant_path_array.empty()) { + if (tenant_path_array.empty()) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(arg), K(tenant_path_array)); - } else if (arg.with_restore_scn_) { + LOG_WARN("invalid argument", K(ret), K(tenant_path_array)); + } else if (with_restore_scn) { // restore scn which is specified by user - job.set_restore_scn(arg.restore_scn_); - } else if (!arg.with_restore_scn_) { + restore_scn = src_scn; + } else if (!with_restore_scn) { if (restore_using_compl_log) { SCN min_restore_scn = SCN::min_scn(); ARRAY_FOREACH_X(tenant_path_array, i, cnt, OB_SUCC(ret)) { @@ -354,7 +363,7 @@ int ObRestoreUtil::fill_restore_scn_(const obrpc::ObPhysicalRestoreTenantArg &ar LOG_WARN("failed to read format file", K(ret), K(store)); } else if (ObBackupDestType::DEST_TYPE_BACKUP_DATA != format_desc.dest_type_) { LOG_INFO("skip log dir", K(tenant_path), K(format_desc)); - } else if (OB_FAIL(store.get_max_backup_set_file_info(arg.passwd_array_, backup_set_file_desc))) { + } else if (OB_FAIL(store.get_max_backup_set_file_info(passwd, backup_set_file_desc))) { LOG_WARN("fail to get backup set array", K(ret)); } else { min_restore_scn = backup_set_file_desc.min_restore_scn_; @@ -365,18 +374,15 @@ int ObRestoreUtil::fill_restore_scn_(const obrpc::ObPhysicalRestoreTenantArg &ar ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid min restore scn, do not find available backup tenant path to restore", K(ret), K(tenant_path_array)); } else { - job.set_restore_scn(min_restore_scn); + restore_scn = min_restore_scn; } } - } else if (!arg.restore_timestamp_.empty()) { - SCN restore_scn = SCN::min_scn(); + } else if (!timestamp.empty()) { common::ObTimeZoneInfoWrap time_zone_wrap; if (OB_FAIL(get_backup_sys_time_zone_(tenant_path_array, time_zone_wrap))) { LOG_WARN("failed to get backup sys time zone", K(ret), K(tenant_path_array)); - } else if (OB_FAIL(convert_restore_timestamp_to_scn_(arg.restore_timestamp_, time_zone_wrap, restore_scn))) { + } else if (OB_FAIL(convert_restore_timestamp_to_scn_(timestamp, time_zone_wrap, restore_scn))) { LOG_WARN("failed to convert restore timestamp to scn", K(ret)); - } else { - job.set_restore_scn(restore_scn); } } else { int64_t round_id = 0; @@ -408,7 +414,7 @@ int ObRestoreUtil::fill_restore_scn_(const obrpc::ObPhysicalRestoreTenantArg &ar ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid max checkpoint scn, no archvie tenant path", K(ret), K(tenant_path_array)); } else { - job.set_restore_scn(max_checkpoint_scn); + restore_scn = max_checkpoint_scn; } } } @@ -458,7 +464,7 @@ int ObRestoreUtil::get_restore_source( const common::ObString &passwd_array, const SCN &restore_scn, ObIArray &backup_set_list, - ObIArray &backup_piece_list, + ObIArray &backup_piece_list, ObIArray &log_path_list) { int ret = OB_SUCCESS; @@ -552,8 +558,8 @@ int ObRestoreUtil::get_restore_backup_set_array_( int ObRestoreUtil::get_restore_backup_piece_list_( const ObBackupDest &dest, - const ObArray &piece_array, - ObIArray &backup_piece_list) + const ObArray &piece_array, + ObIArray &backup_piece_list) { int ret = OB_SUCCESS; if (!dest.is_valid()) { @@ -561,12 +567,15 @@ int ObRestoreUtil::get_restore_backup_piece_list_( LOG_WARN("dest is invalid", K(ret), K(dest)); } else { for (int64_t j = 0; OB_SUCC(ret) && j < piece_array.count(); ++j) { - const share::ObBackupPath &piece_path = piece_array.at(j); - ObBackupPiecePath backup_piece_path; + const share::ObRestoreLogPieceBriefInfo &piece_path = piece_array.at(j); + ObRestoreLogPieceBriefInfo backup_piece_path; + backup_piece_path.piece_id_ = piece_path.piece_id_; + backup_piece_path.start_scn_ = piece_path.start_scn_; + backup_piece_path.checkpoint_scn_ = piece_path.checkpoint_scn_; ObBackupDest piece_dest; - if (OB_FAIL(piece_dest.set(piece_path.get_ptr(), dest.get_storage_info()))) { + if (OB_FAIL(piece_dest.set(piece_path.piece_path_.ptr(), dest.get_storage_info()))) { LOG_WARN("fail to set piece dest", K(ret), K(piece_path), K(dest)); - } else if (OB_FAIL(piece_dest.get_backup_dest_str(backup_piece_path.ptr(), backup_piece_path.capacity()))) { + } else if (OB_FAIL(piece_dest.get_backup_dest_str(backup_piece_path.piece_path_.ptr(), backup_piece_path.piece_path_.capacity()))) { LOG_WARN("fail to get piece dest str", K(ret), K(piece_dest)); } else if (OB_FAIL(backup_piece_list.push_back(backup_piece_path))) { LOG_WARN("fail to push backup piece list", K(ret)); @@ -580,7 +589,7 @@ int ObRestoreUtil::get_restore_backup_piece_list_( int ObRestoreUtil::get_restore_backup_piece_list_( const ObBackupDest &dest, const ObArray &piece_array, - ObIArray &backup_piece_list) + ObIArray &backup_piece_list) { int ret = OB_SUCCESS; if (!dest.is_valid()) { @@ -589,15 +598,16 @@ int ObRestoreUtil::get_restore_backup_piece_list_( } else { for (int64_t j = 0; OB_SUCC(ret) && j < piece_array.count(); ++j) { const share::ObPieceKey &piece_key = piece_array.at(j); + ObRestoreLogPieceBriefInfo backup_piece_path; + backup_piece_path.piece_id_ = piece_key.piece_id_; ObBackupPath backup_path; - ObBackupPiecePath backup_piece_path; ObBackupDest piece_dest; if (OB_FAIL(ObArchivePathUtil::get_piece_dir_path(dest, piece_key.dest_id_, piece_key.round_id_, piece_key.piece_id_, backup_path))) { LOG_WARN("failed to get piece dir path", K(ret), K(dest), K(piece_key)); } else if (OB_FAIL(piece_dest.set(backup_path.get_ptr(), dest.get_storage_info()))) { LOG_WARN("fail to set piece dest", K(ret), K(backup_path), K(dest)); - } else if (OB_FAIL(piece_dest.get_backup_dest_str(backup_piece_path.ptr(), backup_piece_path.capacity()))) { + } else if (OB_FAIL(piece_dest.get_backup_dest_str(backup_piece_path.piece_path_.ptr(), backup_piece_path.piece_path_.capacity()))) { LOG_WARN("fail to get piece dest str", K(ret), K(piece_dest)); } else if (OB_FAIL(backup_piece_list.push_back(backup_piece_path))) { LOG_WARN("fail to push backup piece list", K(ret)); @@ -628,11 +638,11 @@ int ObRestoreUtil::get_restore_log_piece_array_( const ObIArray &tenant_path_array, const SCN &restore_start_scn, const SCN &restore_end_scn, - ObIArray &backup_piece_list, + ObIArray &backup_piece_list, ObIArray &log_path_list) { int ret = OB_SUCCESS; - ObArray piece_array; + ObArray piece_array; if (tenant_path_array.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invaldi argument", K(ret), K(tenant_path_array)); @@ -667,7 +677,7 @@ int ObRestoreUtil::get_restore_log_array_for_complement_log_( const ObIArray &backup_set_list, const share::SCN &restore_start_scn, const share::SCN &restore_end_scn, - ObIArray &backup_piece_list, + ObIArray &backup_piece_list, ObIArray &log_path_list) { int ret = OB_SUCCESS; @@ -701,7 +711,7 @@ int ObRestoreUtil::get_restore_log_array_for_complement_log_( int ObRestoreUtil::do_fill_backup_path_( const ObIArray &backup_set_list, - const ObIArray &backup_piece_list, + const ObIArray &backup_piece_list, const ObIArray &log_path_list, share::ObPhysicalRestoreJob &job) { @@ -709,8 +719,17 @@ int ObRestoreUtil::do_fill_backup_path_( if (backup_set_list.empty() || backup_piece_list.empty()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(backup_set_list), K(backup_piece_list)); - } else if (OB_FAIL(job.get_multi_restore_path_list().set(backup_set_list, backup_piece_list, log_path_list))) { - LOG_WARN("failed to set mutli restore path list", KR(ret)); + } else { + ObArray backup_piece_path_list; + for (int64_t i = 0; OB_SUCC(ret) && i < backup_piece_list.count(); ++i) { + if (OB_FAIL(backup_piece_path_list.push_back(backup_piece_list.at(i).piece_path_))) { + LOG_WARN("failed to push backup piece", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(job.get_multi_restore_path_list().set(backup_set_list, backup_piece_path_list, log_path_list))) { + LOG_WARN("failed to set mutli restore path list", KR(ret)); + } } return ret; } diff --git a/src/rootserver/restore/ob_restore_util.h b/src/rootserver/restore/ob_restore_util.h index 38ca8f3e5f..35d5300adc 100644 --- a/src/rootserver/restore/ob_restore_util.h +++ b/src/rootserver/restore/ob_restore_util.h @@ -57,7 +57,7 @@ public: const common::ObString &passwd_array, const share::SCN &restore_scn, ObIArray &backup_set_list, - ObIArray &backup_piece_list, + ObIArray &backup_piece_list, ObIArray &log_path_list); static int insert_user_tenant_restore_job( common::ObISQLClient &sql_client, @@ -73,6 +73,17 @@ public: palf::PalfBaseInfo &palf_base_info); static int check_physical_restore_finish(common::ObISQLClient &proxy, const int64_t job_id, bool &is_finish, bool &is_failed); static int get_restore_tenant_cpu_count(common::ObMySQLProxy &proxy, const uint64_t tenant_id, double &cpu_count); + static int fill_restore_scn_( + const share::SCN &src_scn, + const ObString ×tamp, + const bool with_restore_scn, + const ObIArray &tenant_path_array, + const common::ObString &passwd, + const bool restore_using_compl_log, + share::SCN &restore_scn); + static int check_restore_using_complement_log_( + const ObIArray &tenant_path_array, + bool &only_contain_backup_set); private: static int fill_backup_info_( const obrpc::ObPhysicalRestoreTenantArg &arg, @@ -83,9 +94,6 @@ private: static int fill_compat_backup_path( const obrpc::ObPhysicalRestoreTenantArg &arg, share::ObPhysicalRestoreJob &job); - static int check_restore_using_complement_log_( - const ObIArray &tenant_path_array, - bool &only_contain_backup_set); static int get_restore_backup_set_array_( const ObIArray &tenant_path_array, const common::ObString &passwd_array, @@ -96,28 +104,28 @@ private: const ObIArray &tenant_path_array, const share::SCN &restore_start_scn, const share::SCN &restore_end_scn, - ObIArray &backup_piece_list, + ObIArray &backup_piece_list, ObIArray &log_path_list); static int get_restore_log_array_for_complement_log_( const ObIArray &backup_set_list, const share::SCN &restore_start_scn, const share::SCN &restore_end_scn, - ObIArray &backup_piece_list, + ObIArray &backup_piece_list, ObIArray &log_path_list); static int get_restore_backup_piece_list_( const share::ObBackupDest &dest, const ObArray &piece_array, - ObIArray &backup_piece_list); + ObIArray &backup_piece_list); static int get_restore_backup_piece_list_( const share::ObBackupDest &dest, - const ObArray &piece_array, - ObIArray &backup_piece_list); + const ObArray &piece_array, + ObIArray &backup_piece_list); static int get_restore_log_path_list_( const share::ObBackupDest &dest, ObIArray &log_path_list); static int do_fill_backup_path_( const ObIArray &backup_set_list, - const ObIArray &backup_piece_list, + const ObIArray &backup_piece_list, const ObIArray &log_path_list, share::ObPhysicalRestoreJob &job); static int do_fill_backup_info_( @@ -135,11 +143,6 @@ private: const ObArray &original_dest_list, common::ObArenaAllocator &allocator, common::ObString &encrypt_dest_str); - static int fill_restore_scn_( - const obrpc::ObPhysicalRestoreTenantArg &arg, - const ObIArray &tenant_path_array, - const bool restore_using_compl_log, - share::ObPhysicalRestoreJob &job); static int fill_encrypt_info_( const obrpc::ObPhysicalRestoreTenantArg &arg, share::ObPhysicalRestoreJob &job); diff --git a/src/share/CMakeLists.txt b/src/share/CMakeLists.txt index 4b6bcf56ae..41c4e3f068 100755 --- a/src/share/CMakeLists.txt +++ b/src/share/CMakeLists.txt @@ -214,6 +214,9 @@ ob_set_subtarget(ob_share common_mixed ratelimit/ob_rl_struct.cpp rc/ob_context.cpp rc/ob_tenant_base.cpp + restore/ob_import_table_struct.cpp + restore/ob_import_util.cpp + restore/ob_recover_table_util.cpp restore/ob_restore_args.cpp restore/ob_restore_uri_parser.cpp restore/ob_ls_restore_status.cpp @@ -223,7 +226,17 @@ ob_set_subtarget(ob_share common_mixed restore/ob_log_restore_source.cpp restore/ob_log_restore_source_mgr.cpp restore/ob_restore_table_operator.cpp + restore/ob_recover_table_persist_helper.cpp restore/ob_restore_persist_helper.cpp + restore/ob_import_table_persist_helper.cpp + restore/ob_import_item_format_provider.cpp + restore/ob_import_schema_item.hpp + restore/ob_import_partition_item.cpp + restore/ob_import_table_item.cpp + restore/ob_remap_schema_item.hpp + restore/ob_import_remap_arg.cpp + restore/ob_import_table_arg.cpp + restore/ob_import_arg.cpp sequence/ob_sequence_cache.cpp sequence/ob_sequence_ddl_proxy.cpp sequence/ob_sequence_dml_proxy.cpp diff --git a/src/share/backup/ob_archive_store.cpp b/src/share/backup/ob_archive_store.cpp index b77728f3c9..5fda8195e4 100644 --- a/src/share/backup/ob_archive_store.cpp +++ b/src/share/backup/ob_archive_store.cpp @@ -1236,7 +1236,7 @@ int ObArchiveStore::get_whole_piece_info(const int64_t dest_id, const int64_t ro // Get pieces needed in the specific interval indicated by 'start_scn' and 'end_scn'. -int ObArchiveStore::get_piece_paths_in_range(const SCN &start_scn, const SCN &end_scn, ObIArray &pieces) +int ObArchiveStore::get_piece_paths_in_range(const SCN &start_scn, const SCN &end_scn, ObIArray &pieces) { int ret = OB_SUCCESS; ObArray piece_keys; @@ -1295,7 +1295,7 @@ int ObArchiveStore::get_piece_paths_in_range(const SCN &start_scn, const SCN &en ++i; continue; } - + ObRestoreLogPieceBriefInfo piece_brief_info; if (cur.start_scn_ >= end_scn) { // this piece may be required for restore, consider the following case. // Piece#1 : <2022-06-01 06:00:00, 2022-06-02 05:00:00, 2022-06-02 06:00:00> @@ -1308,8 +1308,13 @@ int ObArchiveStore::get_piece_paths_in_range(const SCN &start_scn, const SCN &en if (prev.end_scn_ == cur.start_scn_ && prev.checkpoint_scn_ < end_scn) { if (OB_FAIL(ObArchivePathUtil::get_piece_dir_path(dest, cur.key_.dest_id_, cur.key_.round_id_, cur.key_.piece_id_, piece_path))) { LOG_WARN("failed to get piece path", K(ret), K(dest), K(cur)); - } else if (OB_FAIL(pieces.push_back(piece_path))) { - LOG_WARN("fail to push back path", K(ret), K(piece_path)); + } else if (OB_FAIL(piece_brief_info.piece_path_.assign(piece_path.get_obstr()))) { + LOG_WARN("failed to assign piece path", K(ret)); + } else if (OB_FALSE_IT(piece_brief_info.piece_id_ = cur.key_.piece_id_)) { + } else if (OB_FALSE_IT(piece_brief_info.start_scn_ = cur.start_scn_)) { + } else if (OB_FALSE_IT(piece_brief_info.checkpoint_scn_ = cur.checkpoint_scn_)) { + } else if (OB_FAIL(pieces.push_back(piece_brief_info))) { + LOG_WARN("fail to push back path", K(ret), K(piece_brief_info)); } else { last_piece_idx = i; } @@ -1317,14 +1322,18 @@ int ObArchiveStore::get_piece_paths_in_range(const SCN &start_scn, const SCN &en } break; } - if (pieces.empty()) { // this piece may be used to restore. if (cur.start_scn_ <= start_scn) { if (OB_FAIL(ObArchivePathUtil::get_piece_dir_path(dest, cur.key_.dest_id_, cur.key_.round_id_, cur.key_.piece_id_, piece_path))) { LOG_WARN("failed to get piece path", K(ret), K(dest), K(cur)); - } else if (OB_FAIL(pieces.push_back(piece_path))) { - LOG_WARN("fail to push back path", K(ret), K(piece_path)); + } else if (OB_FAIL(piece_brief_info.piece_path_.assign(piece_path.get_obstr()))) { + LOG_WARN("failed to assign piece path", K(ret)); + } else if (OB_FALSE_IT(piece_brief_info.piece_id_ = cur.key_.piece_id_)) { + } else if (OB_FALSE_IT(piece_brief_info.start_scn_ = cur.start_scn_)) { + } else if (OB_FALSE_IT(piece_brief_info.checkpoint_scn_ = cur.checkpoint_scn_)) { + } else if (OB_FAIL(pieces.push_back(piece_brief_info))) { + LOG_WARN("fail to push back path", K(ret), K(piece_brief_info)); } else { last_piece_idx = i; ++i; @@ -1353,7 +1362,12 @@ int ObArchiveStore::get_piece_paths_in_range(const SCN &start_scn, const SCN &en LOG_INFO("pieces are not continous", K(prev), K(cur), K(start_scn), K(end_scn)); } else if (OB_FAIL(ObArchivePathUtil::get_piece_dir_path(dest, cur.key_.dest_id_, cur.key_.round_id_, cur.key_.piece_id_, piece_path))) { LOG_WARN("failed to get piece path", K(ret), K(dest), K(cur)); - } else if (OB_FAIL(pieces.push_back(piece_path))) { + } else if (OB_FAIL(piece_brief_info.piece_path_.assign(piece_path.get_obstr()))) { + LOG_WARN("failed to assign piece path", K(ret)); + } else if (OB_FALSE_IT(piece_brief_info.piece_id_ = cur.key_.piece_id_)) { + } else if (OB_FALSE_IT(piece_brief_info.start_scn_ = cur.start_scn_)) { + } else if (OB_FALSE_IT(piece_brief_info.checkpoint_scn_ = cur.checkpoint_scn_)) { + } else if (OB_FAIL(pieces.push_back(piece_brief_info))) { LOG_WARN("fail to push back path", K(ret), K(piece_path)); } else { last_piece_idx = i; diff --git a/src/share/backup/ob_archive_store.h b/src/share/backup/ob_archive_store.h index 9cdaa22933..574e782b9a 100644 --- a/src/share/backup/ob_archive_store.h +++ b/src/share/backup/ob_archive_store.h @@ -406,7 +406,7 @@ public: // Get pieces needed in the specific interval indicated by 'start_scn' and 'end_scn'. // Return OB_ENTRY_NOT_EXIST if cannot find enough pieces. - int get_piece_paths_in_range(const SCN &start_scn, const SCN &end_scn, ObIArray &pieces); + int get_piece_paths_in_range(const SCN &start_scn, const SCN &end_scn, ObIArray &pieces); // Get archive file range in one piece // return OB_ENTRY_NOT_EXIST if no file exist diff --git a/src/share/backup/ob_backup_struct.cpp b/src/share/backup/ob_backup_struct.cpp index a3888ed238..86f84744b5 100755 --- a/src/share/backup/ob_backup_struct.cpp +++ b/src/share/backup/ob_backup_struct.cpp @@ -1726,7 +1726,7 @@ int ObBackupInfoStatus::set_info_backup_status( /* ObBackupSetDesc */ -ObBackupSetDesc::ObBackupSetDesc() : backup_set_id_(-1), backup_type_() +ObBackupSetDesc::ObBackupSetDesc() : backup_set_id_(-1), backup_type_(), min_restore_scn_(), total_bytes_(0) {} bool ObBackupSetDesc::is_valid() const @@ -1743,6 +1743,8 @@ int ObBackupSetDesc::assign(const ObBackupSetDesc &that) } else { backup_set_id_ = that.backup_set_id_; backup_type_.type_ = that.backup_type_.type_; + min_restore_scn_ = that.min_restore_scn_; + total_bytes_ = that.total_bytes_; } return ret; } @@ -3104,7 +3106,8 @@ const char *ObHAResultInfo::get_failed_type_str() const "RESTORE_DATA", "RESTORE_CLOG", "BACKUP_DATA", - "BACKUP_CLEAN" + "BACKUP_CLEAN", + "RECOVER_TABLE" }; if (type_ < ROOT_SERVICE || type_ >= MAX_FAILED_TYPE) { LOG_ERROR_RET(OB_ERR_UNEXPECTED, "invalid failed type", K(type_)); @@ -4261,6 +4264,44 @@ int share::backup_scn_to_time_tag(const SCN &scn, char *buf, const int64_t buf_l return ret; } +int ObRestoreBackupSetBriefInfo::get_restore_backup_set_brief_info_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + char *str_buf = NULL; + int64_t str_buf_len = 128; + if (str_buf_len > OB_MAX_LONGTEXT_LENGTH + 1) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("format str is too long", KR(ret), K(str_buf_len)); + } else if (OB_ISNULL(str_buf = static_cast(allocator.alloc(str_buf_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc buf", KR(ret), K(str_buf_len)); + } else if (OB_FALSE_IT(MEMSET(str_buf, '\0', str_buf_len))) { + } else { + // type: FULL min_restore_scn: 2022-05-31 12:00:00 size: 22 G + // path: file:///test_backup_dest + int64_t pos = 0; + const char *type_str = backup_set_desc_.backup_type_.is_full_backup() ? OB_STR_FULL_BACKUP : OB_STR_INC_BACKUP; + char scn_display_buf[OB_MAX_TIME_STR_LENGTH] = ""; + if (OB_FALSE_IT(pos = 0)) { + } else if (OB_FAIL(backup_scn_to_time_tag(backup_set_desc_.min_restore_scn_, scn_display_buf, OB_MAX_TIME_STR_LENGTH, pos))) { + LOG_WARN("failed to backup scn to time tag", K(ret)); + } else if (OB_FAIL(databuff_printf(str_buf, str_buf_len, pos, "type: %s, min_restore_scn_display: %s, size: %s.", + type_str, scn_display_buf, to_cstring(ObSizeLiteralPrettyPrinter(backup_set_desc_.total_bytes_))))) { + LOG_WARN("failed to databuff print", K(ret), KPC(this)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_ISNULL(str_buf) || str_buf_len <= 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format str", KR(ret), K(str_buf), K(str_buf_len)); + } else { + str.assign_ptr(str_buf, STRLEN(str_buf)); + LOG_DEBUG("get log path list str", KR(ret), K(str)); + } + return ret; +} + int ObRestoreBackupSetBriefInfo::assign(const ObRestoreBackupSetBriefInfo &that) { int ret = OB_SUCCESS; @@ -4310,3 +4351,59 @@ int ObBackupSkippedType::parse_from_str(const ObString &str) } return ret; } + +int ObRestoreLogPieceBriefInfo::get_restore_log_piece_brief_info_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + char *str_buf = NULL; + int64_t str_buf_len = 128; + if (str_buf_len > OB_MAX_LONGTEXT_LENGTH + 1) { + ret = OB_SIZE_OVERFLOW; + LOG_WARN("format str is too long", KR(ret), K(str_buf_len)); + } else if (OB_ISNULL(str_buf = static_cast(allocator.alloc(str_buf_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc buf", KR(ret), K(str_buf_len)); + } else if (OB_FALSE_IT(MEMSET(str_buf, '\0', str_buf_len))) { + } else { + // start_scn_display: 2022-05-31 12:00:00 checkpoint_scn_display: 2022-05-32 12:00:00 + // path: file:///test_archive_dest + int64_t pos = 0; + char buf1[OB_MAX_TIME_STR_LENGTH] = ""; + char buf2[OB_MAX_TIME_STR_LENGTH] = ""; + if (OB_FAIL(backup_scn_to_time_tag(start_scn_, buf1, sizeof(buf1), pos))) { + LOG_WARN("failed to backup scn to time", K(ret), K(start_scn_)); + } else if (OB_FALSE_IT(pos = 0)) { + } else if (OB_FAIL(backup_scn_to_time_tag(checkpoint_scn_, buf2, sizeof(buf2), pos))) { + LOG_WARN("failed to backup scn to time", K(ret), K(checkpoint_scn_)); + } else if (OB_FAIL(databuff_printf(str_buf, str_buf_len, "start_scn_display: %s, checkpoint_scn_display: %s.", buf1, buf2))) { + LOG_WARN("failed to databuff print", K(ret), KPC(this)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_ISNULL(str_buf) || str_buf_len <= 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format str", KR(ret), K(str_buf), K(str_buf_len)); + } else { + + str.assign_ptr(str_buf, STRLEN(str_buf)); + LOG_DEBUG("get log path list str", KR(ret), K(str)); + } + return ret; +} + +int ObRestoreLogPieceBriefInfo::assign(const ObRestoreLogPieceBriefInfo &that) +{ + int ret = OB_SUCCESS; + if (!that.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(that)); + } else if (OB_FAIL(piece_path_.assign(that.piece_path_))) { + LOG_WARN("fail to assign backup set path", K(ret), K(that)); + } else { + piece_id_ = that.piece_id_; + start_scn_ = that.start_scn_; + checkpoint_scn_ = that.checkpoint_scn_; + } + return ret; +} diff --git a/src/share/backup/ob_backup_struct.h b/src/share/backup/ob_backup_struct.h index fdc3dccd2c..28a07a4613 100755 --- a/src/share/backup/ob_backup_struct.h +++ b/src/share/backup/ob_backup_struct.h @@ -345,6 +345,7 @@ const char *const OB_BACKUP_DECRYPTION_PASSWD_ARRAY_SESSION_STR = "__ob_backup_d const char *const OB_RESTORE_SOURCE_NAME_SESSION_STR = "__ob_restore_source_name__"; const char *const OB_RESTORE_PREVIEW_TENANT_ID_SESSION_STR = "__ob_restore_preview_tenant_id__"; const char *const OB_RESTORE_PREVIEW_BACKUP_DEST_SESSION_STR = "__ob_restore_preview_backup_dest__"; +const char *const OB_RESTORE_PREVIEW_SCN_SESSION_STR = "__ob_restore_preview_timestamp__"; const char *const OB_RESTORE_PREVIEW_TIMESTAMP_SESSION_STR = "__ob_restore_preview_timestamp__"; const char *const OB_RESTORE_PREVIEW_BACKUP_CLUSTER_NAME_SESSION_STR = "__ob_restore_preview_backup_cluster_name__"; const char *const OB_RESTORE_PREVIEW_BACKUP_CLUSTER_ID_SESSION_STR = "__ob_restore_preview_backup_cluster_id__"; @@ -407,6 +408,10 @@ const char *const OB_BACKUP_SUFFIX=".obbak"; const char *const OB_ARCHIVE_SUFFIX=".obarc"; const char *const OB_STR_MIN_RESTORE_SCN_DISPLAY = "min_restore_scn_display"; const char *const OB_STR_CHECKPOINT_FILE_NAME = "checkpoint_info"; +const char *const OB_STR_SRC_TENANT_NAME = "src_tenant_name"; +const char *const OB_STR_AUX_TENANT_NAME = "aux_tenant_name"; +const char *const OB_STR_TARGET_TENANT_NAME = "target_tenant_name"; +const char *const OB_STR_TARGET_TENANT_ID = "target_tenant_id"; enum ObBackupFileType { @@ -553,24 +558,43 @@ struct ObBackupSetDesc { bool operator==(const ObBackupSetDesc &other) const; void reset(); - TO_STRING_KV(K_(backup_set_id), K_(backup_type)); + TO_STRING_KV(K_(backup_set_id), K_(backup_type), K_(min_restore_scn), K_(total_bytes)); int64_t backup_set_id_; ObBackupType backup_type_; // FULL OR INC + share::SCN min_restore_scn_; + int64_t total_bytes_; }; struct ObRestoreBackupSetBriefInfo final { public: - ObRestoreBackupSetBriefInfo(): backup_set_path_(), backup_set_desc_() {} + ObRestoreBackupSetBriefInfo(): backup_set_path_(), backup_set_desc_(){} ~ObRestoreBackupSetBriefInfo() {} void reset() { backup_set_path_.reset(); } bool is_valid() const { return !backup_set_path_.is_empty(); } int assign(const ObRestoreBackupSetBriefInfo &that); + int get_restore_backup_set_brief_info_str(common::ObIAllocator &allocator, common::ObString &str) const; TO_STRING_KV(K_(backup_set_path), K_(backup_set_desc)); share::ObBackupSetPath backup_set_path_; share::ObBackupSetDesc backup_set_desc_; }; +struct ObRestoreLogPieceBriefInfo final +{ +public: + ObRestoreLogPieceBriefInfo(): piece_path_(), piece_id_(0), start_scn_(), checkpoint_scn_() {} + ~ObRestoreLogPieceBriefInfo() {} + void reset() { piece_path_.reset(); } + bool is_valid() const { return !piece_path_.is_empty(); } + int get_restore_log_piece_brief_info_str(common::ObIAllocator &allocator, common::ObString &str) const; + int assign(const ObRestoreLogPieceBriefInfo &that); + TO_STRING_KV(K_(piece_path), K_(piece_id), K_(start_scn), K_(checkpoint_scn)); + share::ObBackupPiecePath piece_path_; + int64_t piece_id_; + share::SCN start_scn_; + share::SCN checkpoint_scn_; +}; + class ObBackupStorageInfo; class ObPhysicalRestoreBackupDestList { @@ -1700,4 +1724,4 @@ inline uint64_t trans_scn_to_second(const SCN &scn) { return scn.convert_to_ts() }//share }//oceanbase -#endif /* OCEANBASE_SHARE_BACKUP_OB_BACKUP_STRUCT_H_ */ +#endif /* OCEANBASE_SHARE_BACKUP_OB_BACKUP_STRUCT_H_ */ \ No newline at end of file diff --git a/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp b/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp index 615cbe429f..dd36edec1c 100644 --- a/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp +++ b/src/share/inner_table/ob_inner_table_schema.11101_11150.cpp @@ -169,6 +169,110 @@ int ObInnerTableSchema::all_virtual_tablet_encrypt_info_schema(ObTableSchema &ta return ret; } +int ObInnerTableSchema::tenant_virtual_show_restore_preview_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_TENANT_VIRTUAL_SHOW_RESTORE_PREVIEW_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_TENANT_VIRTUAL_SHOW_RESTORE_PREVIEW_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_type", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + 20, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_dest", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_BACKUP_DEST_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_BACKUP_DEST_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_HASH); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_virtual_master_key_version_info_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.12401_12450.cpp b/src/share/inner_table/ob_inner_table_schema.12401_12450.cpp index 019c78f08e..ce6b220ae3 100644 --- a/src/share/inner_table/ob_inner_table_schema.12401_12450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.12401_12450.cpp @@ -1988,6 +1988,4110 @@ int ObInnerTableSchema::all_virtual_tenant_scheduler_job_class_schema(ObTableSch return ret; } +int ObInnerTableSchema::all_virtual_recover_table_job_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("aux_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_scn", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_option", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_dest", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_set_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_piece_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj backup_passwd_default; + backup_passwd_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("backup_passwd", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + backup_passwd_default, + backup_passwd_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj external_kms_info_default; + external_kms_info_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("external_kms_info", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + external_kms_info_default, + external_kms_info_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_recover_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("aux_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_scn", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_option", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_dest", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_set_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_piece_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj backup_passwd_default; + backup_passwd_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("backup_passwd", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + backup_passwd_default, + backup_passwd_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj external_kms_info_default; + external_kms_info_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("external_kms_info", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + external_kms_info_default, + external_kms_info_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_job_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_task_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("task_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj src_tablespace_default; + src_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablespace_default, + src_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_tablegroup_default; + src_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablegroup_default, + src_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_database_default; + src_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_database_default, + src_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_table_default; + src_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_table_default, + src_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_partition_default; + src_partition_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_partition", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_PARTITION_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_partition_default, + src_partition_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablespace_default; + target_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablespace_default, + target_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablegroup_default; + target_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablegroup_default, + target_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_database_default; + target_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_database_default, + target_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_table_default; + target_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_table_default, + target_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("table_column", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("completion_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("cumulative_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_task_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("task_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA_TS("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(ObPreciseDateTime), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false, //is_autoincrement + false); //is_on_update_for_timestamp + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj src_tablespace_default; + src_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablespace_default, + src_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_tablegroup_default; + src_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablegroup_default, + src_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_database_default; + src_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_database_default, + src_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_table_default; + src_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_table_default, + src_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_partition_default; + src_partition_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_partition", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_PARTITION_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_partition_default, + src_partition_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablespace_default; + target_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablespace_default, + target_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablegroup_default; + target_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablegroup_default, + target_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_database_default; + target_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_database_default, + target_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_table_default; + target_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_table_default, + target_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("table_column", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("completion_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("cumulative_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.15401_15450.cpp b/src/share/inner_table/ob_inner_table_schema.15401_15450.cpp index 489c77b54d..1a516ee931 100644 --- a/src/share/inner_table/ob_inner_table_schema.15401_15450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.15401_15450.cpp @@ -681,6 +681,3690 @@ int ObInnerTableSchema::all_virtual_tenant_scheduler_job_class_real_agent_ora_sc return ret; } +int ObInnerTableSchema::all_virtual_recover_table_job_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TENANT_ID", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("JOB_ID", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_JOB_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("START_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("END_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("STATUS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("AUX_TENANT_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TENANT_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORT_ALL", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESTORE_SCN", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESTORE_OPTION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_DEST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_SET_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_PIECE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_PASSWD", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("EXTERNAL_KMS_INFO", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESULT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMMENT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DESCRIPTION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_recover_table_job_history_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TENANT_ID", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("JOB_ID", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_JOB_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("START_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("END_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("STATUS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("AUX_TENANT_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TENANT_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORT_ALL", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESTORE_SCN", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESTORE_OPTION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_DEST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_SET_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_PIECE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("BACKUP_PASSWD", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("EXTERNAL_KMS_INFO", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESULT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMMENT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DESCRIPTION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_job_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TENANT_ID", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("JOB_ID", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_JOB_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("START_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("END_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TENANT_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("STATUS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORT_ALL", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_TABLE_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FINISHED_TABLE_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_TABLE_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FINISHED_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESULT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMMENT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DESCRIPTION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_job_history_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TENANT_ID", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("JOB_ID", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("INITIATOR_JOB_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("START_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("END_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TENANT_NAME", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("STATUS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORT_ALL", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_DB_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_PARTITION_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLEGROUP_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("HEX_REMAP_TABLESPACE_LIST", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_TABLE_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FINISHED_TABLE_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_TABLE_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FINISHED_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESULT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMMENT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("DESCRIPTION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_task_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TENANT_ID", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TASK_ID", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("JOB_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TABLESPACE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TABLEGROUP", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_DATABASE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TABLE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_PARTITION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_PARTITION_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TABLESPACE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TABLEGROUP", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_DATABASE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TABLE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TABLE_COLUMN", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("STATUS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("START_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMPLETION_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("CUMULATIVE_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_ROWS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_ROWS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_INDEX_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_INDEX_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_INDEX_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_REF_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_REF_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_REF_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_TRIGGER_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_TRIGGER_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_TRIGGER_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESULT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMMENT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_virtual_import_table_task_history_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(VIRTUAL_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TENANT_ID", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TASK_ID", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampLTZType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("JOB_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TENANT_ID", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TABLESPACE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TABLEGROUP", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_DATABASE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_TABLE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("SRC_PARTITION", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_PARTITION_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TABLESPACE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TABLEGROUP", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_DATABASE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TARGET_TABLE", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TABLE_COLUMN", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("STATUS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("START_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMPLETION_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("CUMULATIVE_TS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_ROWS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_BYTES", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_ROWS", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_INDEX_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_INDEX_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_INDEX_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_REF_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_REF_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_REF_CONSTRAINT_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("TOTAL_TRIGGER_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("IMPORTED_TRIGGER_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("FAILED_TRIGGER_COUNT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObNumberType, //column_type + CS_TYPE_INVALID, //column_collation_type + 38, //column_length + 38, //column_precision + 0, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("RESULT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_UTF8MB4_BIN, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + 2, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("COMMENT", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::all_virtual_ls_info_ora_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.21451_21500.cpp b/src/share/inner_table/ob_inner_table_schema.21451_21500.cpp index 641a80b4b9..bb78d10fa1 100644 --- a/src/share/inner_table/ob_inner_table_schema.21451_21500.cpp +++ b/src/share/inner_table/ob_inner_table_schema.21451_21500.cpp @@ -25,6 +25,606 @@ using namespace common; namespace share { +int ObInnerTableSchema::cdb_ob_recover_table_jobs_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_CDB_OB_RECOVER_TABLE_JOBS_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_CDB_OB_RECOVER_TABLE_JOBS_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID, JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, CASE WHEN END_TS = 0 THEN NULL ELSE USEC_TO_TIME(END_TS) END AS FINISH_TIMESTAMP, STATUS, AUX_TENANT_NAME, TARGET_TENANT_NAME, IMPORT_ALL, DB_LIST, TABLE_LIST, RESTORE_SCN, CASE WHEN RESTORE_SCN = 0 THEN NULL ELSE SCN_TO_TIMESTAMP(RESTORE_SCN) END AS RESTORE_SCN_DISPLAY, RESTORE_OPTION, BACKUP_DEST, BACKUP_SET_LIST, BACKUP_PIECE_LIST, BACKUP_PASSWD, EXTERNAL_KMS_INFO, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB; )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_recover_table_jobs_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_RECOVER_TABLE_JOBS_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_RECOVER_TABLE_JOBS_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, CASE WHEN END_TS = 0 THEN NULL ELSE USEC_TO_TIME(END_TS) END AS FINISH_TIMESTAMP, STATUS, AUX_TENANT_NAME, TARGET_TENANT_NAME, IMPORT_ALL, DB_LIST, TABLE_LIST, RESTORE_SCN, CASE WHEN RESTORE_SCN = 0 THEN NULL ELSE SCN_TO_TIMESTAMP(RESTORE_SCN) END AS RESTORE_SCN_DISPLAY, RESTORE_OPTION, BACKUP_DEST, BACKUP_SET_LIST, BACKUP_PIECE_LIST, BACKUP_PASSWD, EXTERNAL_KMS_INFO, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::cdb_ob_recover_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_CDB_OB_RECOVER_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_CDB_OB_RECOVER_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID, JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, STATUS, AUX_TENANT_NAME, TARGET_TENANT_NAME, IMPORT_ALL, DB_LIST, TABLE_LIST, RESTORE_SCN, CASE WHEN RESTORE_SCN = 0 THEN NULL ELSE SCN_TO_TIMESTAMP(RESTORE_SCN) END AS RESTORE_SCN_DISPLAY, RESTORE_OPTION, BACKUP_DEST, BACKUP_SET_LIST, BACKUP_PIECE_LIST, BACKUP_PASSWD, EXTERNAL_KMS_INFO, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY; )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_recover_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, STATUS, AUX_TENANT_NAME, TARGET_TENANT_NAME, IMPORT_ALL, DB_LIST, TABLE_LIST, RESTORE_SCN, CASE WHEN RESTORE_SCN = 0 THEN NULL ELSE SCN_TO_TIMESTAMP(RESTORE_SCN) END AS RESTORE_SCN_DISPLAY, RESTORE_OPTION, BACKUP_DEST, BACKUP_SET_LIST, BACKUP_PIECE_LIST, BACKUP_PASSWD, EXTERNAL_KMS_INFO, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::cdb_ob_import_table_jobs_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_CDB_OB_IMPORT_TABLE_JOBS_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_CDB_OB_IMPORT_TABLE_JOBS_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID, JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, CASE WHEN END_TS = 0 THEN NULL ELSE USEC_TO_TIME(END_TS) END AS FINISH_TIMESTAMP, SRC_TENANT_NAME, SRC_TENANT_ID, STATUS, IMPORT_ALL, DB_LIST, TABLE_LIST, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, TOTAL_TABLE_COUNT, FINISHED_TABLE_COUNT, FAILED_TABLE_COUNT, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB; )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_jobs_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_JOBS_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_JOBS_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, CASE WHEN END_TS = 0 THEN NULL ELSE USEC_TO_TIME(END_TS) END AS FINISH_TIMESTAMP, SRC_TENANT_NAME, SRC_TENANT_ID, STATUS, IMPORT_ALL, DB_LIST, TABLE_LIST, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, TOTAL_TABLE_COUNT, FINISHED_TABLE_COUNT, FAILED_TABLE_COUNT, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::cdb_ob_import_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_CDB_OB_IMPORT_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_CDB_OB_IMPORT_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID, JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, SRC_TENANT_NAME, SRC_TENANT_ID, STATUS, IMPORT_ALL, DB_LIST, TABLE_LIST, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, TOTAL_TABLE_COUNT, FINISHED_TABLE_COUNT, FAILED_TABLE_COUNT, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY; )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, SRC_TENANT_NAME, SRC_TENANT_ID, STATUS, IMPORT_ALL, DB_LIST, TABLE_LIST, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, TOTAL_TABLE_COUNT, FINISHED_TABLE_COUNT, FAILED_TABLE_COUNT, RESULT, COMMENT, DESCRIPTION FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::cdb_ob_import_table_tasks_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_CDB_OB_IMPORT_TABLE_TASKS_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_CDB_OB_IMPORT_TABLE_TASKS_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID, TASK_ID, JOB_ID, SRC_TENANT_ID, SRC_TABLESPACE, SRC_TABLEGROUP, SRC_DATABASE, SRC_TABLE, SRC_PARTITION, TARGET_TABLESPACE, TARGET_TABLEGROUP, TARGET_DATABASE, TARGET_TABLE, TABLE_COLUMN, STATUS, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, CUMULATIVE_TS, TOTAL_INDEX_COUNT, IMPORTED_INDEX_COUNT, FAILED_INDEX_COUNT, TOTAL_CONSTRAINT_COUNT, IMPORTED_CONSTRAINT_COUNT, FAILED_CONSTRAINT_COUNT, TOTAL_REF_CONSTRAINT_COUNT, IMPORTED_REF_CONSTRAINT_COUNT, FAILED_REF_CONSTRAINT_COUNT, RESULT, COMMENT FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK; )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_tasks_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_TASKS_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_TASKS_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TASK_ID, JOB_ID, SRC_TENANT_ID, SRC_TABLESPACE, SRC_TABLEGROUP, SRC_DATABASE, SRC_TABLE, SRC_PARTITION, TARGET_TABLESPACE, TARGET_TABLEGROUP, TARGET_DATABASE, TARGET_TABLE, TABLE_COLUMN, STATUS, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, CUMULATIVE_TS, TOTAL_INDEX_COUNT, IMPORTED_INDEX_COUNT, FAILED_INDEX_COUNT, TOTAL_CONSTRAINT_COUNT, IMPORTED_CONSTRAINT_COUNT, FAILED_CONSTRAINT_COUNT, TOTAL_REF_CONSTRAINT_COUNT, IMPORTED_REF_CONSTRAINT_COUNT, FAILED_REF_CONSTRAINT_COUNT, RESULT, COMMENT FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::cdb_ob_import_table_task_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_CDB_OB_IMPORT_TABLE_TASK_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_CDB_OB_IMPORT_TABLE_TASK_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID, TASK_ID, JOB_ID, SRC_TENANT_ID, SRC_TABLESPACE, SRC_TABLEGROUP, SRC_DATABASE, SRC_TABLE, SRC_PARTITION, TARGET_TABLESPACE, TARGET_TABLEGROUP, TARGET_DATABASE, TARGET_TABLE, TABLE_COLUMN, STATUS, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, CUMULATIVE_TS, TOTAL_INDEX_COUNT, IMPORTED_INDEX_COUNT, FAILED_INDEX_COUNT, TOTAL_CONSTRAINT_COUNT, IMPORTED_CONSTRAINT_COUNT, FAILED_CONSTRAINT_COUNT, TOTAL_REF_CONSTRAINT_COUNT, IMPORTED_REF_CONSTRAINT_COUNT, FAILED_REF_CONSTRAINT_COUNT, RESULT, COMMENT FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY; )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_task_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TASK_ID, JOB_ID, SRC_TENANT_ID, SRC_TABLESPACE, SRC_TABLEGROUP, SRC_DATABASE, SRC_TABLE, SRC_PARTITION, TARGET_TABLESPACE, TARGET_TABLEGROUP, TARGET_DATABASE, TARGET_TABLE, TABLE_COLUMN, STATUS, USEC_TO_TIME(START_TS) AS START_TIMESTAMP, USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, CUMULATIVE_TS, TOTAL_INDEX_COUNT, IMPORTED_INDEX_COUNT, FAILED_INDEX_COUNT, TOTAL_CONSTRAINT_COUNT, IMPORTED_CONSTRAINT_COUNT, FAILED_CONSTRAINT_COUNT, TOTAL_REF_CONSTRAINT_COUNT, IMPORTED_REF_CONSTRAINT_COUNT, FAILED_REF_CONSTRAINT_COUNT, RESULT, COMMENT FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + int ObInnerTableSchema::gv_ob_tenant_runtime_info_schema(ObTableSchema &table_schema) { int ret = OB_SUCCESS; diff --git a/src/share/inner_table/ob_inner_table_schema.25251_25300.cpp b/src/share/inner_table/ob_inner_table_schema.25251_25300.cpp index bd1e751445..33156050fb 100644 --- a/src/share/inner_table/ob_inner_table_schema.25251_25300.cpp +++ b/src/share/inner_table/ob_inner_table_schema.25251_25300.cpp @@ -325,6 +325,306 @@ int ObInnerTableSchema::dba_scheduler_job_classes_schema(ObTableSchema &table_sc return ret; } +int ObInnerTableSchema::dba_ob_recover_table_jobs_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_RECOVER_TABLE_JOBS_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_RECOVER_TABLE_JOBS_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, CASE WHEN END_TS = 0 THEN NULL ELSE TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') END AS FINISH_TIMESTAMP, STATUS, AUX_TENANT_NAME, TARGET_TENANT_NAME, IMPORT_ALL, DB_LIST, TABLE_LIST, RESTORE_SCN, CASE WHEN RESTORE_SCN = 0 THEN NULL ELSE SCN_TO_TIMESTAMP(RESTORE_SCN) END AS RESTORE_SCN_DISPLAY, RESTORE_OPTION, BACKUP_DEST, BACKUP_SET_LIST, BACKUP_PIECE_LIST, BACKUP_PASSWD, EXTERNAL_KMS_INFO, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, RESULT, "COMMENT", DESCRIPTION FROM SYS.ALL_VIRTUAL_RECOVER_TABLE_JOB WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_recover_table_job_history_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS FINISH_TIMESTAMP, STATUS, AUX_TENANT_NAME, TARGET_TENANT_NAME, IMPORT_ALL, DB_LIST, TABLE_LIST, RESTORE_SCN, CASE WHEN RESTORE_SCN = 0 THEN NULL ELSE SCN_TO_TIMESTAMP(RESTORE_SCN) END AS RESTORE_SCN_DISPLAY, RESTORE_OPTION, BACKUP_DEST, BACKUP_SET_LIST, BACKUP_PIECE_LIST, BACKUP_PASSWD, EXTERNAL_KMS_INFO, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, RESULT, "COMMENT", DESCRIPTION FROM SYS.ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_jobs_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_JOBS_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_JOBS_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, CASE WHEN END_TS = 0 THEN NULL ELSE TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') END AS FINISH_TIMESTAMP, SRC_TENANT_NAME, SRC_TENANT_ID, STATUS, IMPORT_ALL, DB_LIST, TABLE_LIST, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, TOTAL_TABLE_COUNT, FINISHED_TABLE_COUNT, FAILED_TABLE_COUNT, RESULT, "COMMENT", DESCRIPTION FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_JOB WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_job_history_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT JOB_ID, INITIATOR_TENANT_ID, INITIATOR_JOB_ID, TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS FINISH_TIMESTAMP, SRC_TENANT_NAME, SRC_TENANT_ID, STATUS, IMPORT_ALL, DB_LIST, TABLE_LIST, REMAP_DB_LIST, REMAP_TABLE_LIST, REMAP_TABLEGROUP_LIST, REMAP_TABLESPACE_LIST, TOTAL_TABLE_COUNT, FINISHED_TABLE_COUNT, FAILED_TABLE_COUNT, RESULT, "COMMENT", DESCRIPTION FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_tasks_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_TASKS_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_TASKS_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TASK_ID, JOB_ID, SRC_TENANT_ID, SRC_TABLESPACE, SRC_TABLEGROUP, SRC_DATABASE, SRC_TABLE, SRC_PARTITION, TARGET_TABLESPACE, TARGET_TABLEGROUP, TARGET_DATABASE, TARGET_TABLE, TABLE_COLUMN, STATUS, TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, TO_CHAR(COMPLETION_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS COMPLETION_TIMESTAMP, CUMULATIVE_TS, TOTAL_INDEX_COUNT, IMPORTED_INDEX_COUNT, FAILED_INDEX_COUNT, TOTAL_CONSTRAINT_COUNT, IMPORTED_CONSTRAINT_COUNT, FAILED_CONSTRAINT_COUNT, TOTAL_REF_CONSTRAINT_COUNT, IMPORTED_REF_CONSTRAINT_COUNT, FAILED_REF_CONSTRAINT_COUNT, RESULT, "COMMENT" FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_TASK WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::dba_ob_import_table_task_history_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TASK_ID, JOB_ID, SRC_TENANT_ID, SRC_TABLESPACE, SRC_TABLEGROUP, SRC_DATABASE, SRC_TABLE, SRC_PARTITION, TARGET_TABLESPACE, TARGET_TABLEGROUP, TARGET_DATABASE, TARGET_TABLE, TABLE_COLUMN, STATUS, TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, TO_CHAR(COMPLETION_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS COMPLETION_TIMESTAMP, CUMULATIVE_TS, TOTAL_INDEX_COUNT, IMPORTED_INDEX_COUNT, FAILED_INDEX_COUNT, TOTAL_CONSTRAINT_COUNT, IMPORTED_CONSTRAINT_COUNT, FAILED_CONSTRAINT_COUNT, TOTAL_REF_CONSTRAINT_COUNT, IMPORTED_REF_CONSTRAINT_COUNT, FAILED_REF_CONSTRAINT_COUNT, RESULT, "COMMENT" FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.451_500.cpp b/src/share/inner_table/ob_inner_table_schema.451_500.cpp index 57aeac832d..61261a1eef 100644 --- a/src/share/inner_table/ob_inner_table_schema.451_500.cpp +++ b/src/share/inner_table/ob_inner_table_schema.451_500.cpp @@ -2746,6 +2746,4206 @@ int ObInnerTableSchema::all_tenant_scheduler_job_class_schema(ObTableSchema &tab return ret; } +int ObInnerTableSchema::all_recover_table_job_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RECOVER_TABLE_JOB_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RECOVER_TABLE_JOB_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ObObj gmt_create_default; + ObObj gmt_create_default_null; + + gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_create_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + false, //is_on_update_for_timestamp + gmt_create_default_null, + gmt_create_default) + } + + if (OB_SUCC(ret)) { + ObObj gmt_modified_default; + ObObj gmt_modified_default_null; + + gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_modified_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + true, //is_on_update_for_timestamp + gmt_modified_default_null, + gmt_modified_default) + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("aux_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_scn", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_option", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_dest", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_set_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_piece_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj backup_passwd_default; + backup_passwd_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("backup_passwd", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + backup_passwd_default, + backup_passwd_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj external_kms_info_default; + external_kms_info_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("external_kms_info", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + external_kms_info_default, + external_kms_info_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RECOVER_TABLE_JOB_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_recover_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RECOVER_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ObObj gmt_create_default; + ObObj gmt_create_default_null; + + gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_create_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + false, //is_on_update_for_timestamp + gmt_create_default_null, + gmt_create_default) + } + + if (OB_SUCC(ret)) { + ObObj gmt_modified_default; + ObObj gmt_modified_default_null; + + gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_modified_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + true, //is_on_update_for_timestamp + gmt_modified_default_null, + gmt_modified_default) + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("aux_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("target_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_scn", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("restore_option", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_dest", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_set_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("backup_piece_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj backup_passwd_default; + backup_passwd_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("backup_passwd", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + backup_passwd_default, + backup_passwd_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj external_kms_info_default; + external_kms_info_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("external_kms_info", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + external_kms_info_default, + external_kms_info_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_job_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_JOB_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_JOB_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ObObj gmt_create_default; + ObObj gmt_create_default_null; + + gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_create_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + false, //is_on_update_for_timestamp + gmt_create_default_null, + gmt_create_default) + } + + if (OB_SUCC(ret)) { + ObObj gmt_modified_default; + ObObj gmt_modified_default_null; + + gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_modified_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + true, //is_on_update_for_timestamp + gmt_modified_default_null, + gmt_modified_default) + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_JOB_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_job_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_JOB_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ObObj gmt_create_default; + ObObj gmt_create_default_null; + + gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_create_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + false, //is_on_update_for_timestamp + gmt_create_default_null, + gmt_create_default) + } + + if (OB_SUCC(ret)) { + ObObj gmt_modified_default; + ObObj gmt_modified_default_null; + + gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_modified_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + true, //is_on_update_for_timestamp + gmt_modified_default_null, + gmt_modified_default) + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("initiator_job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("end_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_name", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TENANT_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("import_all", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTinyIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + 1, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj db_list_default; + db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + db_list_default, + db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_db_list_default; + hex_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_db_list_default, + hex_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj table_list_default; + table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + table_list_default, + table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_table_list_default; + hex_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_table_list_default, + hex_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj partition_list_default; + partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + partition_list_default, + partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_partition_list_default; + hex_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_partition_list_default, + hex_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_db_list_default; + remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_db_list_default, + remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_db_list_default; + hex_remap_db_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_db_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_db_list_default, + hex_remap_db_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_table_list_default; + remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_table_list_default, + remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_table_list_default; + hex_remap_table_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_table_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_table_list_default, + hex_remap_table_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_partition_list_default; + remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_partition_list_default, + remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_partition_list_default; + hex_remap_partition_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_partition_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_partition_list_default, + hex_remap_partition_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablegroup_list_default; + remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablegroup_list_default, + remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablegroup_list_default; + hex_remap_tablegroup_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablegroup_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablegroup_list_default, + hex_remap_tablegroup_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj remap_tablespace_list_default; + remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + remap_tablespace_list_default, + remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj hex_remap_tablespace_list_default; + hex_remap_tablespace_list_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("hex_remap_tablespace_list", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + hex_remap_tablespace_list_default, + hex_remap_tablespace_list_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_table_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("finished_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj description_default; + description_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("description", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + description_default, + description_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_task_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_TASK_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_TASK_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ObObj gmt_create_default; + ObObj gmt_create_default_null; + + gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_create_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + false, //is_on_update_for_timestamp + gmt_create_default_null, + gmt_create_default) + } + + if (OB_SUCC(ret)) { + ObObj gmt_modified_default; + ObObj gmt_modified_default_null; + + gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_modified_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + true, //is_on_update_for_timestamp + gmt_modified_default_null, + gmt_modified_default) + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("task_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj src_tablespace_default; + src_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablespace_default, + src_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_tablegroup_default; + src_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablegroup_default, + src_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_database_default; + src_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_database_default, + src_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_table_default; + src_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_table_default, + src_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_partition_default; + src_partition_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_partition", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_PARTITION_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_partition_default, + src_partition_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablespace_default; + target_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablespace_default, + target_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablegroup_default; + target_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablegroup_default, + target_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_database_default; + target_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_database_default, + target_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_table_default; + target_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_table_default, + target_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("table_column", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("completion_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("cumulative_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_TASK_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_task_history_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_TABLE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_TASK_HISTORY_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ObObj gmt_create_default; + ObObj gmt_create_default_null; + + gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_create_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + false, //is_on_update_for_timestamp + gmt_create_default_null, + gmt_create_default) + } + + if (OB_SUCC(ret)) { + ObObj gmt_modified_default; + ObObj gmt_modified_default_null; + + gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG); + gmt_modified_default_null.set_null(); + ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObTimestampType, //column_type + CS_TYPE_BINARY,//collation_type + 0, //column length + -1, //column_precision + 6, //column_scale + true,//is nullable + false, //is_autoincrement + true, //is_on_update_for_timestamp + gmt_modified_default_null, + gmt_modified_default) + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("tenant_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("task_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_INNER_TABLE_DEFAULT_VALUE_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("job_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("src_tenant_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj src_tablespace_default; + src_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablespace_default, + src_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_tablegroup_default; + src_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_tablegroup_default, + src_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_database_default; + src_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_database_default, + src_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_table_default; + src_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_table_default, + src_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj src_partition_default; + src_partition_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("src_partition", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_PARTITION_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + src_partition_default, + src_partition_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablespace_default; + target_tablespace_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablespace", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLESPACE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablespace_default, + target_tablespace_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_tablegroup_default; + target_tablegroup_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_tablegroup", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_TABLEGROUP_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_tablegroup_default, + target_tablegroup_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_database_default; + target_database_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_database", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_DATABASE_NAME_LENGTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_database_default, + target_database_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj target_table_default; + target_table_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("target_table", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + target_table_default, + target_table_default); //default_value + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("table_column", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("status", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("start_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("completion_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("cumulative_ts", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_bytes", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_rows", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_index_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_ref_constraint_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("total_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("imported_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("failed_trigger_count", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObIntType, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(int64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ObObj result_default; + result_default.set_varchar(ObString::make_string("")); + ADD_COLUMN_SCHEMA_T("result", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_INVALID, //column_collation_type + OB_DEFAULT_STATUS_LENTH, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + result_default, + result_default); //default_value + } + + if (OB_SUCC(ret)) { + ObObj comment_default; + comment_default.set_lob_value(ObLongTextType, "", static_cast(strlen(""))); + ADD_COLUMN_SCHEMA_T("comment", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObLongTextType, //column_type + CS_TYPE_INVALID, //column_collation_type + 0, //column_length + -1, //column_precision + -1, //column_scale + true, //is_nullable + false, //is_autoincrement + comment_default, + comment_default); //default_value + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID); + table_schema.set_aux_lob_meta_tid(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID); + table_schema.set_aux_lob_piece_tid(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.50451_50500.cpp b/src/share/inner_table/ob_inner_table_schema.50451_50500.cpp index 70aeb469e5..3720f62e6e 100644 --- a/src/share/inner_table/ob_inner_table_schema.50451_50500.cpp +++ b/src/share/inner_table/ob_inner_table_schema.50451_50500.cpp @@ -1645,6 +1645,816 @@ int ObInnerTableSchema::all_tenant_scheduler_job_class_aux_lob_meta_schema(ObTab return ret; } +int ObInnerTableSchema::all_recover_table_job_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_RECOVER_TABLE_JOB_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_recover_table_job_history_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_job_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_JOB_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_job_history_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_task_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_TASK_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_task_history_aux_lob_meta_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(2); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_META); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 16, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("seq_id", //column_name + ++column_id, //column_id + 2, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 8192, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("binary_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("char_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 262144, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.60451_60500.cpp b/src/share/inner_table/ob_inner_table_schema.60451_60500.cpp index cd17d395ef..4bf26e43b9 100644 --- a/src/share/inner_table/ob_inner_table_schema.60451_60500.cpp +++ b/src/share/inner_table/ob_inner_table_schema.60451_60500.cpp @@ -1105,6 +1105,546 @@ int ObInnerTableSchema::all_tenant_scheduler_job_class_aux_lob_piece_schema(ObTa return ret; } +int ObInnerTableSchema::all_recover_table_job_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_RECOVER_TABLE_JOB_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_recover_table_job_history_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_job_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_JOB_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_job_history_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_task_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_TASK_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::all_import_table_task_history_aux_lob_piece_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(1); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(AUX_LOB_PIECE); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("piece_id", //column_name + ++column_id, //column_id + 1, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt64Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint64_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("data_len", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObUInt32Type, //column_type + CS_TYPE_INVALID, //column_collation_type + sizeof(uint32_t), //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + + if (OB_SUCC(ret)) { + ADD_COLUMN_SCHEMA("lob_data", //column_name + ++column_id, //column_id + 0, //rowkey_id + 0, //index_id + 0, //part_key_pos + ObVarcharType, //column_type + CS_TYPE_BINARY, //column_collation_type + 32, //column_length + -1, //column_precision + -1, //column_scale + false, //is_nullable + false); //is_autoincrement + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID); + table_schema.set_data_table_id(OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.h b/src/share/inner_table/ob_inner_table_schema.h index 4ba8c3e8a1..4b721d1c18 100644 --- a/src/share/inner_table/ob_inner_table_schema.h +++ b/src/share/inner_table/ob_inner_table_schema.h @@ -540,6 +540,12 @@ public: static int wr_control_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_event_history_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_scheduler_job_class_schema(share::schema::ObTableSchema &table_schema); + static int all_recover_table_job_schema(share::schema::ObTableSchema &table_schema); + static int all_recover_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_job_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_task_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_task_history_schema(share::schema::ObTableSchema &table_schema); static int tenant_virtual_all_table_schema(share::schema::ObTableSchema &table_schema); static int tenant_virtual_table_column_schema(share::schema::ObTableSchema &table_schema); static int tenant_virtual_table_index_schema(share::schema::ObTableSchema &table_schema); @@ -614,6 +620,7 @@ public: static int all_virtual_audit_action_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_dag_warning_history_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_tablet_encrypt_info_schema(share::schema::ObTableSchema &table_schema); + static int tenant_virtual_show_restore_preview_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_master_key_version_info_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_dag_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_dag_scheduler_schema(share::schema::ObTableSchema &table_schema); @@ -931,6 +938,12 @@ public: static int all_virtual_balance_group_ls_stat_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_flt_config_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_tenant_scheduler_job_class_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_recover_table_job_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_recover_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_job_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_task_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_task_history_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_sql_audit_ora_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_plan_stat_ora_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_plan_cache_plan_explain_ora_schema(share::schema::ObTableSchema &table_schema); @@ -1172,6 +1185,12 @@ public: static int all_virtual_flt_config_ora_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_tenant_scheduler_job_run_detail_real_agent_ora_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_tenant_scheduler_job_class_real_agent_ora_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_recover_table_job_ora_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_recover_table_job_history_ora_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_job_ora_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_job_history_ora_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_task_ora_schema(share::schema::ObTableSchema &table_schema); + static int all_virtual_import_table_task_history_ora_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_ls_info_ora_schema(share::schema::ObTableSchema &table_schema); static int gv_ob_plan_cache_stat_schema(share::schema::ObTableSchema &table_schema); static int gv_ob_plan_cache_plan_stat_schema(share::schema::ObTableSchema &table_schema); @@ -1522,6 +1541,18 @@ public: static int dba_ob_tenant_event_history_schema(share::schema::ObTableSchema &table_schema); static int cdb_ob_tenant_event_history_schema(share::schema::ObTableSchema &table_schema); static int gv_ob_flt_trace_config_schema(share::schema::ObTableSchema &table_schema); + static int cdb_ob_recover_table_jobs_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_recover_table_jobs_schema(share::schema::ObTableSchema &table_schema); + static int cdb_ob_recover_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_recover_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int cdb_ob_import_table_jobs_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_jobs_schema(share::schema::ObTableSchema &table_schema); + static int cdb_ob_import_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_job_history_schema(share::schema::ObTableSchema &table_schema); + static int cdb_ob_import_table_tasks_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_tasks_schema(share::schema::ObTableSchema &table_schema); + static int cdb_ob_import_table_task_history_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_task_history_schema(share::schema::ObTableSchema &table_schema); static int gv_ob_tenant_runtime_info_schema(share::schema::ObTableSchema &table_schema); static int v_ob_tenant_runtime_info_schema(share::schema::ObTableSchema &table_schema); static int dba_synonyms_schema(share::schema::ObTableSchema &table_schema); @@ -1772,6 +1803,12 @@ public: static int dba_ob_tenant_event_history_ora_schema(share::schema::ObTableSchema &table_schema); static int dba_scheduler_job_run_details_ora_schema(share::schema::ObTableSchema &table_schema); static int dba_scheduler_job_classes_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_recover_table_jobs_ora_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_recover_table_job_history_ora_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_jobs_ora_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_job_history_ora_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_tasks_ora_schema(share::schema::ObTableSchema &table_schema); + static int dba_ob_import_table_task_history_ora_schema(share::schema::ObTableSchema &table_schema); static int gv_ob_sql_audit_ora_schema(share::schema::ObTableSchema &table_schema); static int v_ob_sql_audit_ora_schema(share::schema::ObTableSchema &table_schema); static int gv_instance_schema(share::schema::ObTableSchema &table_schema); @@ -2193,6 +2230,12 @@ public: static int wr_control_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_event_history_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_scheduler_job_class_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_recover_table_job_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_recover_table_job_history_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_job_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_job_history_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_task_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_task_history_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema); static int all_table_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_column_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_ddl_operation_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); @@ -2447,6 +2490,12 @@ public: static int wr_control_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_event_history_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_tenant_scheduler_job_class_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_recover_table_job_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_recover_table_job_history_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_job_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_job_history_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_task_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); + static int all_import_table_task_history_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_ash_all_virtual_ash_i1_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_sql_plan_monitor_all_virtual_sql_plan_monitor_i1_schema(share::schema::ObTableSchema &table_schema); static int all_virtual_sql_audit_all_virtual_sql_audit_i1_schema(share::schema::ObTableSchema &table_schema); @@ -2905,6 +2954,12 @@ const schema_create_func sys_table_schema_creators [] = { ObInnerTableSchema::wr_control_schema, ObInnerTableSchema::all_tenant_event_history_schema, ObInnerTableSchema::all_tenant_scheduler_job_class_schema, + ObInnerTableSchema::all_recover_table_job_schema, + ObInnerTableSchema::all_recover_table_job_history_schema, + ObInnerTableSchema::all_import_table_job_schema, + ObInnerTableSchema::all_import_table_job_history_schema, + ObInnerTableSchema::all_import_table_task_schema, + ObInnerTableSchema::all_import_table_task_history_schema, NULL,}; const schema_create_func virtual_table_schema_creators [] = { @@ -2982,6 +3037,7 @@ const schema_create_func virtual_table_schema_creators [] = { ObInnerTableSchema::all_virtual_audit_action_schema, ObInnerTableSchema::all_virtual_dag_warning_history_schema, ObInnerTableSchema::all_virtual_tablet_encrypt_info_schema, + ObInnerTableSchema::tenant_virtual_show_restore_preview_schema, ObInnerTableSchema::all_virtual_master_key_version_info_schema, ObInnerTableSchema::all_virtual_dag_schema, ObInnerTableSchema::all_virtual_dag_scheduler_schema, @@ -3299,6 +3355,12 @@ const schema_create_func virtual_table_schema_creators [] = { ObInnerTableSchema::all_virtual_balance_group_ls_stat_schema, ObInnerTableSchema::all_virtual_flt_config_schema, ObInnerTableSchema::all_virtual_tenant_scheduler_job_class_schema, + ObInnerTableSchema::all_virtual_recover_table_job_schema, + ObInnerTableSchema::all_virtual_recover_table_job_history_schema, + ObInnerTableSchema::all_virtual_import_table_job_schema, + ObInnerTableSchema::all_virtual_import_table_job_history_schema, + ObInnerTableSchema::all_virtual_import_table_task_schema, + ObInnerTableSchema::all_virtual_import_table_task_history_schema, ObInnerTableSchema::all_virtual_ash_all_virtual_ash_i1_schema, ObInnerTableSchema::all_virtual_sql_plan_monitor_all_virtual_sql_plan_monitor_i1_schema, ObInnerTableSchema::all_virtual_sql_audit_all_virtual_sql_audit_i1_schema, @@ -3550,6 +3612,12 @@ const schema_create_func virtual_table_schema_creators [] = { ObInnerTableSchema::all_virtual_flt_config_ora_schema, ObInnerTableSchema::all_virtual_tenant_scheduler_job_run_detail_real_agent_ora_schema, ObInnerTableSchema::all_virtual_tenant_scheduler_job_class_real_agent_ora_schema, + ObInnerTableSchema::all_virtual_recover_table_job_ora_schema, + ObInnerTableSchema::all_virtual_recover_table_job_history_ora_schema, + ObInnerTableSchema::all_virtual_import_table_job_ora_schema, + ObInnerTableSchema::all_virtual_import_table_job_history_ora_schema, + ObInnerTableSchema::all_virtual_import_table_task_ora_schema, + ObInnerTableSchema::all_virtual_import_table_task_history_ora_schema, ObInnerTableSchema::all_virtual_ls_info_ora_schema, ObInnerTableSchema::all_virtual_table_real_agent_ora_idx_data_table_id_real_agent_schema, ObInnerTableSchema::all_virtual_table_real_agent_ora_idx_db_tb_name_real_agent_schema, @@ -3984,6 +4052,18 @@ const schema_create_func sys_view_schema_creators [] = { ObInnerTableSchema::dba_ob_tenant_event_history_schema, ObInnerTableSchema::cdb_ob_tenant_event_history_schema, ObInnerTableSchema::gv_ob_flt_trace_config_schema, + ObInnerTableSchema::cdb_ob_recover_table_jobs_schema, + ObInnerTableSchema::dba_ob_recover_table_jobs_schema, + ObInnerTableSchema::cdb_ob_recover_table_job_history_schema, + ObInnerTableSchema::dba_ob_recover_table_job_history_schema, + ObInnerTableSchema::cdb_ob_import_table_jobs_schema, + ObInnerTableSchema::dba_ob_import_table_jobs_schema, + ObInnerTableSchema::cdb_ob_import_table_job_history_schema, + ObInnerTableSchema::dba_ob_import_table_job_history_schema, + ObInnerTableSchema::cdb_ob_import_table_tasks_schema, + ObInnerTableSchema::dba_ob_import_table_tasks_schema, + ObInnerTableSchema::cdb_ob_import_table_task_history_schema, + ObInnerTableSchema::dba_ob_import_table_task_history_schema, ObInnerTableSchema::gv_ob_tenant_runtime_info_schema, ObInnerTableSchema::v_ob_tenant_runtime_info_schema, ObInnerTableSchema::dba_synonyms_schema, @@ -4234,6 +4314,12 @@ const schema_create_func sys_view_schema_creators [] = { ObInnerTableSchema::dba_ob_tenant_event_history_ora_schema, ObInnerTableSchema::dba_scheduler_job_run_details_ora_schema, ObInnerTableSchema::dba_scheduler_job_classes_schema, + ObInnerTableSchema::dba_ob_recover_table_jobs_ora_schema, + ObInnerTableSchema::dba_ob_recover_table_job_history_ora_schema, + ObInnerTableSchema::dba_ob_import_table_jobs_ora_schema, + ObInnerTableSchema::dba_ob_import_table_job_history_ora_schema, + ObInnerTableSchema::dba_ob_import_table_tasks_ora_schema, + ObInnerTableSchema::dba_ob_import_table_task_history_ora_schema, ObInnerTableSchema::gv_ob_sql_audit_ora_schema, ObInnerTableSchema::v_ob_sql_audit_ora_schema, ObInnerTableSchema::gv_instance_schema, @@ -4748,6 +4834,12 @@ const uint64_t tenant_space_tables [] = { OB_WR_CONTROL_TID, OB_ALL_TENANT_EVENT_HISTORY_TID, OB_ALL_TENANT_SCHEDULER_JOB_CLASS_TID, + OB_ALL_RECOVER_TABLE_JOB_TID, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID, + OB_ALL_IMPORT_TABLE_JOB_TID, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID, + OB_ALL_IMPORT_TABLE_TASK_TID, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID, OB_TENANT_VIRTUAL_ALL_TABLE_TID, OB_TENANT_VIRTUAL_TABLE_COLUMN_TID, OB_TENANT_VIRTUAL_TABLE_INDEX_TID, @@ -4938,6 +5030,12 @@ const uint64_t tenant_space_tables [] = { OB_ALL_VIRTUAL_WR_CONTROL_TID, OB_ALL_VIRTUAL_TENANT_EVENT_HISTORY_TID, OB_ALL_VIRTUAL_FLT_CONFIG_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID, OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID, OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TID, OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID, @@ -5188,6 +5286,12 @@ const uint64_t tenant_space_tables [] = { OB_ALL_VIRTUAL_FLT_CONFIG_ORA_TID, OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_RUN_DETAIL_REAL_AGENT_ORA_TID, OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_REAL_AGENT_ORA_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID, OB_ALL_VIRTUAL_LS_INFO_ORA_TID, OB_GV_OB_PLAN_CACHE_STAT_TID, OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID, @@ -5442,6 +5546,12 @@ const uint64_t tenant_space_tables [] = { OB_DBA_OB_LS_HISTORY_TID, OB_DBA_OB_TENANT_EVENT_HISTORY_TID, OB_GV_OB_FLT_TRACE_CONFIG_TID, + OB_DBA_OB_RECOVER_TABLE_JOBS_TID, + OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_TID, + OB_DBA_OB_IMPORT_TABLE_JOBS_TID, + OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_TID, + OB_DBA_OB_IMPORT_TABLE_TASKS_TID, + OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TID, OB_DBA_SYNONYMS_TID, OB_DBA_OBJECTS_ORA_TID, OB_ALL_OBJECTS_TID, @@ -5690,6 +5800,12 @@ const uint64_t tenant_space_tables [] = { OB_DBA_OB_TENANT_EVENT_HISTORY_ORA_TID, OB_DBA_SCHEDULER_JOB_RUN_DETAILS_ORA_TID, OB_DBA_SCHEDULER_JOB_CLASSES_TID, + OB_DBA_OB_RECOVER_TABLE_JOBS_ORA_TID, + OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_ORA_TID, + OB_DBA_OB_IMPORT_TABLE_JOBS_ORA_TID, + OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_ORA_TID, + OB_DBA_OB_IMPORT_TABLE_TASKS_ORA_TID, + OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_ORA_TID, OB_GV_OB_SQL_AUDIT_ORA_TID, OB_V_OB_SQL_AUDIT_ORA_TID, OB_GV_INSTANCE_TID, @@ -6254,6 +6370,12 @@ const uint64_t tenant_space_tables [] = { OB_WR_CONTROL_AUX_LOB_META_TID, OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_META_TID, OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_META_TID, + OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TID, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID, OB_ALL_TABLE_AUX_LOB_PIECE_TID, OB_ALL_COLUMN_AUX_LOB_PIECE_TID, OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TID, @@ -6486,7 +6608,13 @@ const uint64_t tenant_space_tables [] = { OB_ALL_DBMS_LOCK_ALLOCATED_AUX_LOB_PIECE_TID, OB_WR_CONTROL_AUX_LOB_PIECE_TID, OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_PIECE_TID, - OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_PIECE_TID, }; + OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_PIECE_TID, + OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TID, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID, + OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TID, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID, + OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID, }; const uint64_t all_ora_mapping_virtual_table_org_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_TID, @@ -6619,6 +6747,12 @@ const uint64_t all_ora_mapping_virtual_table_org_tables [] = { OB_ALL_VIRTUAL_WR_CONTROL_TID, OB_ALL_VIRTUAL_TENANT_EVENT_HISTORY_TID, OB_ALL_VIRTUAL_FLT_CONFIG_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID, OB_ALL_VIRTUAL_LS_INFO_TID, }; const uint64_t all_ora_mapping_virtual_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID @@ -6751,6 +6885,12 @@ const uint64_t all_ora_mapping_virtual_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_O , OB_ALL_VIRTUAL_WR_CONTROL_ORA_TID , OB_ALL_VIRTUAL_TENANT_EVENT_HISTORY_ORA_TID , OB_ALL_VIRTUAL_FLT_CONFIG_ORA_TID +, OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TID +, OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TID +, OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TID +, OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TID +, OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TID +, OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID , OB_ALL_VIRTUAL_LS_INFO_ORA_TID , }; @@ -7000,6 +7140,12 @@ const char* const tenant_space_table_names [] = { OB_WR_CONTROL_TNAME, OB_ALL_TENANT_EVENT_HISTORY_TNAME, OB_ALL_TENANT_SCHEDULER_JOB_CLASS_TNAME, + OB_ALL_RECOVER_TABLE_JOB_TNAME, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_TNAME, + OB_ALL_IMPORT_TABLE_JOB_TNAME, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_TNAME, + OB_ALL_IMPORT_TABLE_TASK_TNAME, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_TNAME, OB_TENANT_VIRTUAL_ALL_TABLE_TNAME, OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME, OB_TENANT_VIRTUAL_TABLE_INDEX_TNAME, @@ -7190,6 +7336,12 @@ const char* const tenant_space_table_names [] = { OB_ALL_VIRTUAL_WR_CONTROL_TNAME, OB_ALL_VIRTUAL_TENANT_EVENT_HISTORY_TNAME, OB_ALL_VIRTUAL_FLT_CONFIG_TNAME, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TNAME, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TNAME, OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TNAME, OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TNAME, OB_ALL_VIRTUAL_PLAN_STAT_ORA_TNAME, @@ -7440,6 +7592,12 @@ const char* const tenant_space_table_names [] = { OB_ALL_VIRTUAL_FLT_CONFIG_ORA_TNAME, OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_RUN_DETAIL_REAL_AGENT_ORA_TNAME, OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_REAL_AGENT_ORA_TNAME, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TNAME, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TNAME, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME, OB_ALL_VIRTUAL_LS_INFO_ORA_TNAME, OB_GV_OB_PLAN_CACHE_STAT_TNAME, OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME, @@ -7694,6 +7852,12 @@ const char* const tenant_space_table_names [] = { OB_DBA_OB_LS_HISTORY_TNAME, OB_DBA_OB_TENANT_EVENT_HISTORY_TNAME, OB_GV_OB_FLT_TRACE_CONFIG_TNAME, + OB_DBA_OB_RECOVER_TABLE_JOBS_TNAME, + OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_TNAME, + OB_DBA_OB_IMPORT_TABLE_JOBS_TNAME, + OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_TNAME, + OB_DBA_OB_IMPORT_TABLE_TASKS_TNAME, + OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TNAME, OB_DBA_SYNONYMS_TNAME, OB_DBA_OBJECTS_ORA_TNAME, OB_ALL_OBJECTS_TNAME, @@ -7942,6 +8106,12 @@ const char* const tenant_space_table_names [] = { OB_DBA_OB_TENANT_EVENT_HISTORY_ORA_TNAME, OB_DBA_SCHEDULER_JOB_RUN_DETAILS_ORA_TNAME, OB_DBA_SCHEDULER_JOB_CLASSES_TNAME, + OB_DBA_OB_RECOVER_TABLE_JOBS_ORA_TNAME, + OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_ORA_TNAME, + OB_DBA_OB_IMPORT_TABLE_JOBS_ORA_TNAME, + OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_ORA_TNAME, + OB_DBA_OB_IMPORT_TABLE_TASKS_ORA_TNAME, + OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME, OB_GV_OB_SQL_AUDIT_ORA_TNAME, OB_V_OB_SQL_AUDIT_ORA_TNAME, OB_GV_INSTANCE_TNAME, @@ -8506,6 +8676,12 @@ const char* const tenant_space_table_names [] = { OB_WR_CONTROL_AUX_LOB_META_TNAME, OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_META_TNAME, OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_META_TNAME, + OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TNAME, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TNAME, + OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TNAME, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TNAME, + OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TNAME, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TNAME, OB_ALL_TABLE_AUX_LOB_PIECE_TNAME, OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME, OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TNAME, @@ -8738,7 +8914,13 @@ const char* const tenant_space_table_names [] = { OB_ALL_DBMS_LOCK_ALLOCATED_AUX_LOB_PIECE_TNAME, OB_WR_CONTROL_AUX_LOB_PIECE_TNAME, OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_PIECE_TNAME, - OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_PIECE_TNAME, }; + OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_PIECE_TNAME, + OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TNAME, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME, + OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TNAME, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME, + OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TNAME, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TNAME, }; const uint64_t only_rs_vtables [] = { OB_ALL_VIRTUAL_CORE_META_TABLE_TID, @@ -9059,7 +9241,13 @@ const uint64_t restrict_access_virtual_tables[] = { OB_ALL_VIRTUAL_TENANT_PARAMETER_ORA_TID, OB_ALL_VIRTUAL_DBMS_LOCK_ALLOCATED_REAL_AGENT_ORA_TID, OB_ALL_VIRTUAL_WR_CONTROL_ORA_TID, - OB_ALL_VIRTUAL_FLT_CONFIG_ORA_TID }; + OB_ALL_VIRTUAL_FLT_CONFIG_ORA_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TID, + OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TID, + OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID }; static inline bool is_restrict_access_virtual_table(const uint64_t tid) @@ -11238,6 +11426,54 @@ LOBMapping const lob_aux_table_mappings [] = { ObInnerTableSchema::all_tenant_scheduler_job_class_aux_lob_piece_schema }, + { + OB_ALL_RECOVER_TABLE_JOB_TID, + OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TID, + OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_recover_table_job_aux_lob_meta_schema, + ObInnerTableSchema::all_recover_table_job_aux_lob_piece_schema + }, + + { + OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TID, + OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_recover_table_job_history_aux_lob_meta_schema, + ObInnerTableSchema::all_recover_table_job_history_aux_lob_piece_schema + }, + + { + OB_ALL_IMPORT_TABLE_JOB_TID, + OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_import_table_job_aux_lob_meta_schema, + ObInnerTableSchema::all_import_table_job_aux_lob_piece_schema + }, + + { + OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_import_table_job_history_aux_lob_meta_schema, + ObInnerTableSchema::all_import_table_job_history_aux_lob_piece_schema + }, + + { + OB_ALL_IMPORT_TABLE_TASK_TID, + OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_import_table_task_aux_lob_meta_schema, + ObInnerTableSchema::all_import_table_task_aux_lob_piece_schema + }, + + { + OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID, + OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID, + ObInnerTableSchema::all_import_table_task_history_aux_lob_meta_schema, + ObInnerTableSchema::all_import_table_task_history_aux_lob_piece_schema + }, + }; static inline bool get_sys_table_lob_aux_table_id(const uint64_t tid, uint64_t& meta_tid, uint64_t& piece_tid) @@ -11275,12 +11511,12 @@ static inline int get_sys_table_lob_aux_schema(const uint64_t tid, } const int64_t OB_CORE_TABLE_COUNT = 4; -const int64_t OB_SYS_TABLE_COUNT = 251; -const int64_t OB_VIRTUAL_TABLE_COUNT = 724; -const int64_t OB_SYS_VIEW_COUNT = 766; -const int64_t OB_SYS_TENANT_TABLE_COUNT = 1746; +const int64_t OB_SYS_TABLE_COUNT = 257; +const int64_t OB_VIRTUAL_TABLE_COUNT = 737; +const int64_t OB_SYS_VIEW_COUNT = 784; +const int64_t OB_SYS_TENANT_TABLE_COUNT = 1783; const int64_t OB_CORE_SCHEMA_VERSION = 1; -const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1749; +const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1786; } // end namespace share } // end namespace oceanbase diff --git a/src/share/inner_table/ob_inner_table_schema.lob.cpp b/src/share/inner_table/ob_inner_table_schema.lob.cpp index 6fa98de143..30564b9d9a 100644 --- a/src/share/inner_table/ob_inner_table_schema.lob.cpp +++ b/src/share/inner_table/ob_inner_table_schema.lob.cpp @@ -21,7 +21,7 @@ inner_lob_map_t inner_lob_map; bool lob_mapping_init() { int ret = OB_SUCCESS; - if (OB_FAIL(inner_lob_map.create(254, ObModIds::OB_INNER_LOB_HASH_SET))) { + if (OB_FAIL(inner_lob_map.create(260, ObModIds::OB_INNER_LOB_HASH_SET))) { SERVER_LOG(WARN, "fail to create inner lob map", K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < ARRAYSIZEOF(lob_aux_table_mappings); ++i) { diff --git a/src/share/inner_table/ob_inner_table_schema_constants.h b/src/share/inner_table/ob_inner_table_schema_constants.h index 6fa68be352..3d223b9f83 100644 --- a/src/share/inner_table/ob_inner_table_schema_constants.h +++ b/src/share/inner_table/ob_inner_table_schema_constants.h @@ -282,6 +282,12 @@ const uint64_t OB_ALL_DBMS_LOCK_ALLOCATED_TID = 471; // "__all_dbms_lock_allocat const uint64_t OB_WR_CONTROL_TID = 472; // "__wr_control" const uint64_t OB_ALL_TENANT_EVENT_HISTORY_TID = 473; // "__all_tenant_event_history" const uint64_t OB_ALL_TENANT_SCHEDULER_JOB_CLASS_TID = 474; // "__all_tenant_scheduler_job_class" +const uint64_t OB_ALL_RECOVER_TABLE_JOB_TID = 475; // "__all_recover_table_job" +const uint64_t OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID = 476; // "__all_recover_table_job_history" +const uint64_t OB_ALL_IMPORT_TABLE_JOB_TID = 477; // "__all_import_table_job" +const uint64_t OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID = 478; // "__all_import_table_job_history" +const uint64_t OB_ALL_IMPORT_TABLE_TASK_TID = 479; // "__all_import_table_task" +const uint64_t OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID = 480; // "__all_import_table_task_history" const uint64_t OB_TENANT_VIRTUAL_ALL_TABLE_TID = 10001; // "__tenant_virtual_all_table" const uint64_t OB_TENANT_VIRTUAL_TABLE_COLUMN_TID = 10002; // "__tenant_virtual_table_column" const uint64_t OB_TENANT_VIRTUAL_TABLE_INDEX_TID = 10003; // "__tenant_virtual_table_index" @@ -356,6 +362,7 @@ const uint64_t OB_ALL_VIRTUAL_AUDIT_OPERATION_TID = 11097; // "__all_virtual_aud const uint64_t OB_ALL_VIRTUAL_AUDIT_ACTION_TID = 11098; // "__all_virtual_audit_action" const uint64_t OB_ALL_VIRTUAL_DAG_WARNING_HISTORY_TID = 11099; // "__all_virtual_dag_warning_history" const uint64_t OB_ALL_VIRTUAL_TABLET_ENCRYPT_INFO_TID = 11100; // "__all_virtual_tablet_encrypt_info" +const uint64_t OB_TENANT_VIRTUAL_SHOW_RESTORE_PREVIEW_TID = 11102; // "__tenant_virtual_show_restore_preview" const uint64_t OB_ALL_VIRTUAL_MASTER_KEY_VERSION_INFO_TID = 11104; // "__all_virtual_master_key_version_info" const uint64_t OB_ALL_VIRTUAL_DAG_TID = 11105; // "__all_virtual_dag" const uint64_t OB_ALL_VIRTUAL_DAG_SCHEDULER_TID = 11106; // "__all_virtual_dag_scheduler" @@ -673,6 +680,12 @@ const uint64_t OB_ALL_VIRTUAL_BALANCE_TASK_HELPER_TID = 12416; // "__all_virtual const uint64_t OB_ALL_VIRTUAL_BALANCE_GROUP_LS_STAT_TID = 12417; // "__all_virtual_balance_group_ls_stat" const uint64_t OB_ALL_VIRTUAL_FLT_CONFIG_TID = 12420; // "__all_virtual_flt_config" const uint64_t OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_TID = 12421; // "__all_virtual_tenant_scheduler_job_class" +const uint64_t OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TID = 12422; // "__all_virtual_recover_table_job" +const uint64_t OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TID = 12423; // "__all_virtual_recover_table_job_history" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TID = 12424; // "__all_virtual_import_table_job" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID = 12425; // "__all_virtual_import_table_job_history" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID = 12426; // "__all_virtual_import_table_task" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID = 12427; // "__all_virtual_import_table_task_history" const uint64_t OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID = 15009; // "ALL_VIRTUAL_SQL_AUDIT_ORA" const uint64_t OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID = 15010; // "ALL_VIRTUAL_PLAN_STAT_ORA" const uint64_t OB_ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA_TID = 15012; // "ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA" @@ -914,6 +927,12 @@ const uint64_t OB_ALL_VIRTUAL_LS_REAL_AGENT_ORA_TID = 15402; // "ALL_VIRTUAL_LS_ const uint64_t OB_ALL_VIRTUAL_FLT_CONFIG_ORA_TID = 15403; // "ALL_VIRTUAL_FLT_CONFIG_ORA" const uint64_t OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_RUN_DETAIL_REAL_AGENT_ORA_TID = 15404; // "ALL_VIRTUAL_TENANT_SCHEDULER_JOB_RUN_DETAIL_REAL_AGENT_ORA" const uint64_t OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_REAL_AGENT_ORA_TID = 15406; // "ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_REAL_AGENT_ORA" +const uint64_t OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TID = 15407; // "ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA" +const uint64_t OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TID = 15408; // "ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TID = 15409; // "ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TID = 15410; // "ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TID = 15411; // "ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA" +const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID = 15412; // "ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA" const uint64_t OB_ALL_VIRTUAL_LS_INFO_ORA_TID = 15414; // "ALL_VIRTUAL_LS_INFO_ORA" const uint64_t OB_GV_OB_PLAN_CACHE_STAT_TID = 20001; // "GV$OB_PLAN_CACHE_STAT" const uint64_t OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID = 20002; // "GV$OB_PLAN_CACHE_PLAN_STAT" @@ -1264,6 +1283,18 @@ const uint64_t OB_CDB_OB_LS_HISTORY_TID = 21446; // "CDB_OB_LS_HISTORY" const uint64_t OB_DBA_OB_TENANT_EVENT_HISTORY_TID = 21447; // "DBA_OB_TENANT_EVENT_HISTORY" const uint64_t OB_CDB_OB_TENANT_EVENT_HISTORY_TID = 21448; // "CDB_OB_TENANT_EVENT_HISTORY" const uint64_t OB_GV_OB_FLT_TRACE_CONFIG_TID = 21449; // "GV$OB_FLT_TRACE_CONFIG" +const uint64_t OB_CDB_OB_RECOVER_TABLE_JOBS_TID = 21463; // "CDB_OB_RECOVER_TABLE_JOBS" +const uint64_t OB_DBA_OB_RECOVER_TABLE_JOBS_TID = 21464; // "DBA_OB_RECOVER_TABLE_JOBS" +const uint64_t OB_CDB_OB_RECOVER_TABLE_JOB_HISTORY_TID = 21465; // "CDB_OB_RECOVER_TABLE_JOB_HISTORY" +const uint64_t OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_TID = 21466; // "DBA_OB_RECOVER_TABLE_JOB_HISTORY" +const uint64_t OB_CDB_OB_IMPORT_TABLE_JOBS_TID = 21467; // "CDB_OB_IMPORT_TABLE_JOBS" +const uint64_t OB_DBA_OB_IMPORT_TABLE_JOBS_TID = 21468; // "DBA_OB_IMPORT_TABLE_JOBS" +const uint64_t OB_CDB_OB_IMPORT_TABLE_JOB_HISTORY_TID = 21469; // "CDB_OB_IMPORT_TABLE_JOB_HISTORY" +const uint64_t OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_TID = 21470; // "DBA_OB_IMPORT_TABLE_JOB_HISTORY" +const uint64_t OB_CDB_OB_IMPORT_TABLE_TASKS_TID = 21471; // "CDB_OB_IMPORT_TABLE_TASKS" +const uint64_t OB_DBA_OB_IMPORT_TABLE_TASKS_TID = 21472; // "DBA_OB_IMPORT_TABLE_TASKS" +const uint64_t OB_CDB_OB_IMPORT_TABLE_TASK_HISTORY_TID = 21473; // "CDB_OB_IMPORT_TABLE_TASK_HISTORY" +const uint64_t OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TID = 21474; // "DBA_OB_IMPORT_TABLE_TASK_HISTORY" const uint64_t OB_GV_OB_TENANT_RUNTIME_INFO_TID = 21477; // "GV$OB_TENANT_RUNTIME_INFO" const uint64_t OB_V_OB_TENANT_RUNTIME_INFO_TID = 21478; // "V$OB_TENANT_RUNTIME_INFO" const uint64_t OB_DBA_SYNONYMS_TID = 25001; // "DBA_SYNONYMS" @@ -1514,6 +1545,12 @@ const uint64_t OB_DBA_OB_LS_HISTORY_ORA_TID = 25258; // "DBA_OB_LS_HISTORY_ORA" const uint64_t OB_DBA_OB_TENANT_EVENT_HISTORY_ORA_TID = 25259; // "DBA_OB_TENANT_EVENT_HISTORY_ORA" const uint64_t OB_DBA_SCHEDULER_JOB_RUN_DETAILS_ORA_TID = 25260; // "DBA_SCHEDULER_JOB_RUN_DETAILS_ORA" const uint64_t OB_DBA_SCHEDULER_JOB_CLASSES_TID = 25261; // "DBA_SCHEDULER_JOB_CLASSES" +const uint64_t OB_DBA_OB_RECOVER_TABLE_JOBS_ORA_TID = 25262; // "DBA_OB_RECOVER_TABLE_JOBS_ORA" +const uint64_t OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_ORA_TID = 25263; // "DBA_OB_RECOVER_TABLE_JOB_HISTORY_ORA" +const uint64_t OB_DBA_OB_IMPORT_TABLE_JOBS_ORA_TID = 25264; // "DBA_OB_IMPORT_TABLE_JOBS_ORA" +const uint64_t OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_ORA_TID = 25265; // "DBA_OB_IMPORT_TABLE_JOB_HISTORY_ORA" +const uint64_t OB_DBA_OB_IMPORT_TABLE_TASKS_ORA_TID = 25266; // "DBA_OB_IMPORT_TABLE_TASKS_ORA" +const uint64_t OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_ORA_TID = 25267; // "DBA_OB_IMPORT_TABLE_TASK_HISTORY_ORA" const uint64_t OB_GV_OB_SQL_AUDIT_ORA_TID = 28002; // "GV$OB_SQL_AUDIT_ORA" const uint64_t OB_V_OB_SQL_AUDIT_ORA_TID = 28003; // "V$OB_SQL_AUDIT_ORA" const uint64_t OB_GV_INSTANCE_TID = 28004; // "GV$INSTANCE" @@ -1935,6 +1972,12 @@ const uint64_t OB_ALL_DBMS_LOCK_ALLOCATED_AUX_LOB_META_TID = 50471; // "__all_db const uint64_t OB_WR_CONTROL_AUX_LOB_META_TID = 50472; // "__wr_control_aux_lob_meta" const uint64_t OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_META_TID = 50473; // "__all_tenant_event_history_aux_lob_meta" const uint64_t OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_META_TID = 50474; // "__all_tenant_scheduler_job_class_aux_lob_meta" +const uint64_t OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TID = 50475; // "__all_recover_table_job_aux_lob_meta" +const uint64_t OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TID = 50476; // "__all_recover_table_job_history_aux_lob_meta" +const uint64_t OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TID = 50477; // "__all_import_table_job_aux_lob_meta" +const uint64_t OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID = 50478; // "__all_import_table_job_history_aux_lob_meta" +const uint64_t OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID = 50479; // "__all_import_table_task_aux_lob_meta" +const uint64_t OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID = 50480; // "__all_import_table_task_history_aux_lob_meta" const uint64_t OB_ALL_TABLE_AUX_LOB_PIECE_TID = 60003; // "__all_table_aux_lob_piece" const uint64_t OB_ALL_COLUMN_AUX_LOB_PIECE_TID = 60004; // "__all_column_aux_lob_piece" const uint64_t OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TID = 60005; // "__all_ddl_operation_aux_lob_piece" @@ -2189,6 +2232,12 @@ const uint64_t OB_ALL_DBMS_LOCK_ALLOCATED_AUX_LOB_PIECE_TID = 60471; // "__all_d const uint64_t OB_WR_CONTROL_AUX_LOB_PIECE_TID = 60472; // "__wr_control_aux_lob_piece" const uint64_t OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_PIECE_TID = 60473; // "__all_tenant_event_history_aux_lob_piece" const uint64_t OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_PIECE_TID = 60474; // "__all_tenant_scheduler_job_class_aux_lob_piece" +const uint64_t OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TID = 60475; // "__all_recover_table_job_aux_lob_piece" +const uint64_t OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID = 60476; // "__all_recover_table_job_history_aux_lob_piece" +const uint64_t OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TID = 60477; // "__all_import_table_job_aux_lob_piece" +const uint64_t OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID = 60478; // "__all_import_table_job_history_aux_lob_piece" +const uint64_t OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID = 60479; // "__all_import_table_task_aux_lob_piece" +const uint64_t OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID = 60480; // "__all_import_table_task_history_aux_lob_piece" const uint64_t OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TID = 14999; // "__all_virtual_plan_cache_stat" const uint64_t OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TID = 14998; // "__all_virtual_session_event" const uint64_t OB_ALL_VIRTUAL_SESSION_WAIT_ALL_VIRTUAL_SESSION_WAIT_I1_TID = 14997; // "__all_virtual_session_wait" @@ -2634,6 +2683,12 @@ const char *const OB_ALL_DBMS_LOCK_ALLOCATED_TNAME = "__all_dbms_lock_allocated" const char *const OB_WR_CONTROL_TNAME = "__wr_control"; const char *const OB_ALL_TENANT_EVENT_HISTORY_TNAME = "__all_tenant_event_history"; const char *const OB_ALL_TENANT_SCHEDULER_JOB_CLASS_TNAME = "__all_tenant_scheduler_job_class"; +const char *const OB_ALL_RECOVER_TABLE_JOB_TNAME = "__all_recover_table_job"; +const char *const OB_ALL_RECOVER_TABLE_JOB_HISTORY_TNAME = "__all_recover_table_job_history"; +const char *const OB_ALL_IMPORT_TABLE_JOB_TNAME = "__all_import_table_job"; +const char *const OB_ALL_IMPORT_TABLE_JOB_HISTORY_TNAME = "__all_import_table_job_history"; +const char *const OB_ALL_IMPORT_TABLE_TASK_TNAME = "__all_import_table_task"; +const char *const OB_ALL_IMPORT_TABLE_TASK_HISTORY_TNAME = "__all_import_table_task_history"; const char *const OB_TENANT_VIRTUAL_ALL_TABLE_TNAME = "__tenant_virtual_all_table"; const char *const OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME = "__tenant_virtual_table_column"; const char *const OB_TENANT_VIRTUAL_TABLE_INDEX_TNAME = "__tenant_virtual_table_index"; @@ -2708,6 +2763,7 @@ const char *const OB_ALL_VIRTUAL_AUDIT_OPERATION_TNAME = "__all_virtual_audit_op const char *const OB_ALL_VIRTUAL_AUDIT_ACTION_TNAME = "__all_virtual_audit_action"; const char *const OB_ALL_VIRTUAL_DAG_WARNING_HISTORY_TNAME = "__all_virtual_dag_warning_history"; const char *const OB_ALL_VIRTUAL_TABLET_ENCRYPT_INFO_TNAME = "__all_virtual_tablet_encrypt_info"; +const char *const OB_TENANT_VIRTUAL_SHOW_RESTORE_PREVIEW_TNAME = "__tenant_virtual_show_restore_preview"; const char *const OB_ALL_VIRTUAL_MASTER_KEY_VERSION_INFO_TNAME = "__all_virtual_master_key_version_info"; const char *const OB_ALL_VIRTUAL_DAG_TNAME = "__all_virtual_dag"; const char *const OB_ALL_VIRTUAL_DAG_SCHEDULER_TNAME = "__all_virtual_dag_scheduler"; @@ -3025,6 +3081,12 @@ const char *const OB_ALL_VIRTUAL_BALANCE_TASK_HELPER_TNAME = "__all_virtual_bala const char *const OB_ALL_VIRTUAL_BALANCE_GROUP_LS_STAT_TNAME = "__all_virtual_balance_group_ls_stat"; const char *const OB_ALL_VIRTUAL_FLT_CONFIG_TNAME = "__all_virtual_flt_config"; const char *const OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_TNAME = "__all_virtual_tenant_scheduler_job_class"; +const char *const OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TNAME = "__all_virtual_recover_table_job"; +const char *const OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TNAME = "__all_virtual_recover_table_job_history"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TNAME = "__all_virtual_import_table_job"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TNAME = "__all_virtual_import_table_job_history"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TNAME = "__all_virtual_import_table_task"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TNAME = "__all_virtual_import_table_task_history"; const char *const OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TNAME = "ALL_VIRTUAL_SQL_AUDIT"; const char *const OB_ALL_VIRTUAL_PLAN_STAT_ORA_TNAME = "ALL_VIRTUAL_PLAN_STAT"; const char *const OB_ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA_TNAME = "ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN"; @@ -3266,6 +3328,12 @@ const char *const OB_ALL_VIRTUAL_LS_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL_LS_REAL_ const char *const OB_ALL_VIRTUAL_FLT_CONFIG_ORA_TNAME = "ALL_VIRTUAL_FLT_CONFIG"; const char *const OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_RUN_DETAIL_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL_TENANT_SCHEDULER_JOB_RUN_DETAIL_REAL_AGENT"; const char *const OB_ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL_TENANT_SCHEDULER_JOB_CLASS_REAL_AGENT"; +const char *const OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_ORA_TNAME = "ALL_VIRTUAL_RECOVER_TABLE_JOB"; +const char *const OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_ORA_TNAME = "ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TNAME = "ALL_VIRTUAL_IMPORT_TABLE_JOB"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TNAME = "ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TNAME = "ALL_VIRTUAL_IMPORT_TABLE_TASK"; +const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME = "ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY"; const char *const OB_ALL_VIRTUAL_LS_INFO_ORA_TNAME = "ALL_VIRTUAL_LS_INFO"; const char *const OB_GV_OB_PLAN_CACHE_STAT_TNAME = "GV$OB_PLAN_CACHE_STAT"; const char *const OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME = "GV$OB_PLAN_CACHE_PLAN_STAT"; @@ -3616,6 +3684,18 @@ const char *const OB_CDB_OB_LS_HISTORY_TNAME = "CDB_OB_LS_HISTORY"; const char *const OB_DBA_OB_TENANT_EVENT_HISTORY_TNAME = "DBA_OB_TENANT_EVENT_HISTORY"; const char *const OB_CDB_OB_TENANT_EVENT_HISTORY_TNAME = "CDB_OB_TENANT_EVENT_HISTORY"; const char *const OB_GV_OB_FLT_TRACE_CONFIG_TNAME = "GV$OB_FLT_TRACE_CONFIG"; +const char *const OB_CDB_OB_RECOVER_TABLE_JOBS_TNAME = "CDB_OB_RECOVER_TABLE_JOBS"; +const char *const OB_DBA_OB_RECOVER_TABLE_JOBS_TNAME = "DBA_OB_RECOVER_TABLE_JOBS"; +const char *const OB_CDB_OB_RECOVER_TABLE_JOB_HISTORY_TNAME = "CDB_OB_RECOVER_TABLE_JOB_HISTORY"; +const char *const OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_TNAME = "DBA_OB_RECOVER_TABLE_JOB_HISTORY"; +const char *const OB_CDB_OB_IMPORT_TABLE_JOBS_TNAME = "CDB_OB_IMPORT_TABLE_JOBS"; +const char *const OB_DBA_OB_IMPORT_TABLE_JOBS_TNAME = "DBA_OB_IMPORT_TABLE_JOBS"; +const char *const OB_CDB_OB_IMPORT_TABLE_JOB_HISTORY_TNAME = "CDB_OB_IMPORT_TABLE_JOB_HISTORY"; +const char *const OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_TNAME = "DBA_OB_IMPORT_TABLE_JOB_HISTORY"; +const char *const OB_CDB_OB_IMPORT_TABLE_TASKS_TNAME = "CDB_OB_IMPORT_TABLE_TASKS"; +const char *const OB_DBA_OB_IMPORT_TABLE_TASKS_TNAME = "DBA_OB_IMPORT_TABLE_TASKS"; +const char *const OB_CDB_OB_IMPORT_TABLE_TASK_HISTORY_TNAME = "CDB_OB_IMPORT_TABLE_TASK_HISTORY"; +const char *const OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TNAME = "DBA_OB_IMPORT_TABLE_TASK_HISTORY"; const char *const OB_GV_OB_TENANT_RUNTIME_INFO_TNAME = "GV$OB_TENANT_RUNTIME_INFO"; const char *const OB_V_OB_TENANT_RUNTIME_INFO_TNAME = "V$OB_TENANT_RUNTIME_INFO"; const char *const OB_DBA_SYNONYMS_TNAME = "DBA_SYNONYMS"; @@ -3866,6 +3946,12 @@ const char *const OB_DBA_OB_LS_HISTORY_ORA_TNAME = "DBA_OB_LS_HISTORY"; const char *const OB_DBA_OB_TENANT_EVENT_HISTORY_ORA_TNAME = "DBA_OB_TENANT_EVENT_HISTORY"; const char *const OB_DBA_SCHEDULER_JOB_RUN_DETAILS_ORA_TNAME = "DBA_SCHEDULER_JOB_RUN_DETAILS"; const char *const OB_DBA_SCHEDULER_JOB_CLASSES_TNAME = "DBA_SCHEDULER_JOB_CLASSES"; +const char *const OB_DBA_OB_RECOVER_TABLE_JOBS_ORA_TNAME = "DBA_OB_RECOVER_TABLE_JOBS"; +const char *const OB_DBA_OB_RECOVER_TABLE_JOB_HISTORY_ORA_TNAME = "DBA_OB_RECOVER_TABLE_JOB_HISTORY"; +const char *const OB_DBA_OB_IMPORT_TABLE_JOBS_ORA_TNAME = "DBA_OB_IMPORT_TABLE_JOBS"; +const char *const OB_DBA_OB_IMPORT_TABLE_JOB_HISTORY_ORA_TNAME = "DBA_OB_IMPORT_TABLE_JOB_HISTORY"; +const char *const OB_DBA_OB_IMPORT_TABLE_TASKS_ORA_TNAME = "DBA_OB_IMPORT_TABLE_TASKS"; +const char *const OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME = "DBA_OB_IMPORT_TABLE_TASK_HISTORY"; const char *const OB_GV_OB_SQL_AUDIT_ORA_TNAME = "GV$OB_SQL_AUDIT"; const char *const OB_V_OB_SQL_AUDIT_ORA_TNAME = "V$OB_SQL_AUDIT"; const char *const OB_GV_INSTANCE_TNAME = "GV$INSTANCE"; @@ -4287,6 +4373,12 @@ const char *const OB_ALL_DBMS_LOCK_ALLOCATED_AUX_LOB_META_TNAME = "__all_dbms_lo const char *const OB_WR_CONTROL_AUX_LOB_META_TNAME = "__wr_control_aux_lob_meta"; const char *const OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_META_TNAME = "__all_tenant_event_history_aux_lob_meta"; const char *const OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_META_TNAME = "__all_tenant_scheduler_job_class_aux_lob_meta"; +const char *const OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TNAME = "__all_recover_table_job_aux_lob_meta"; +const char *const OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TNAME = "__all_recover_table_job_history_aux_lob_meta"; +const char *const OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TNAME = "__all_import_table_job_aux_lob_meta"; +const char *const OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TNAME = "__all_import_table_job_history_aux_lob_meta"; +const char *const OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TNAME = "__all_import_table_task_aux_lob_meta"; +const char *const OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TNAME = "__all_import_table_task_history_aux_lob_meta"; const char *const OB_ALL_TABLE_AUX_LOB_PIECE_TNAME = "__all_table_aux_lob_piece"; const char *const OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME = "__all_column_aux_lob_piece"; const char *const OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TNAME = "__all_ddl_operation_aux_lob_piece"; @@ -4541,6 +4633,12 @@ const char *const OB_ALL_DBMS_LOCK_ALLOCATED_AUX_LOB_PIECE_TNAME = "__all_dbms_l const char *const OB_WR_CONTROL_AUX_LOB_PIECE_TNAME = "__wr_control_aux_lob_piece"; const char *const OB_ALL_TENANT_EVENT_HISTORY_AUX_LOB_PIECE_TNAME = "__all_tenant_event_history_aux_lob_piece"; const char *const OB_ALL_TENANT_SCHEDULER_JOB_CLASS_AUX_LOB_PIECE_TNAME = "__all_tenant_scheduler_job_class_aux_lob_piece"; +const char *const OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TNAME = "__all_recover_table_job_aux_lob_piece"; +const char *const OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME = "__all_recover_table_job_history_aux_lob_piece"; +const char *const OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TNAME = "__all_import_table_job_aux_lob_piece"; +const char *const OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME = "__all_import_table_job_history_aux_lob_piece"; +const char *const OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TNAME = "__all_import_table_task_aux_lob_piece"; +const char *const OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TNAME = "__all_import_table_task_history_aux_lob_piece"; const char *const OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TNAME = "__idx_11003_all_virtual_plan_cache_stat_i1"; const char *const OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TNAME = "__idx_11013_all_virtual_session_event_i1"; const char *const OB_ALL_VIRTUAL_SESSION_WAIT_ALL_VIRTUAL_SESSION_WAIT_I1_TNAME = "__idx_11014_all_virtual_session_wait_i1"; diff --git a/src/share/inner_table/ob_inner_table_schema_def.py b/src/share/inner_table/ob_inner_table_schema_def.py index f774c0f205..c8c08b9da2 100644 --- a/src/share/inner_table/ob_inner_table_schema_def.py +++ b/src/share/inner_table/ob_inner_table_schema_def.py @@ -5930,7 +5930,6 @@ def_table_schema( ('tenant_id', 'int', 'false', 'OB_INVALID_TENANT_ID'), ('gmt_create', 'timestamp:6', 'false') ], - in_tenant_space = True, is_cluster_private = True, meta_record_in_sys = False, @@ -5980,18 +5979,322 @@ def_table_schema( ('comments', 'varchar:240', 'true'), ], ) -# 475 : __all_recover_table_job -# 476 : __all_recover_table_job_history -# 477 : __all_import_table_job -# 478 : __all_import_table_job_history -# 479 : __all_import_table_task -# 480 : __all_import_table_task_history + +def_table_schema( + owner = 'chongrong.th', + table_name = '__all_recover_table_job', + table_id = '475', + table_type = 'SYSTEM_TABLE', + gm_columns = ['gmt_create', 'gmt_modified'], + rowkey_columns = [ + ('tenant_id', 'int'), + ('job_id', 'int'), + ], + in_tenant_space = True, + is_cluster_private = True, + meta_record_in_sys = False, + + normal_columns = [ + ('initiator_tenant_id', 'int'), + ('initiator_job_id', 'int'), + ('start_ts', 'int'), + ('end_ts', 'int'), + ('status', 'varchar:OB_DEFAULT_STATUS_LENTH'), + ('aux_tenant_name', 'varchar:OB_MAX_TENANT_NAME_LENGTH'), + ('target_tenant_name', 'varchar:OB_MAX_TENANT_NAME_LENGTH'), + ('target_tenant_id', 'int'), + ('import_all', 'bool'), + ('db_list', 'longtext', 'true', ''), + ('hex_db_list', 'longtext', 'true', ''), + ('table_list', 'longtext', 'true', ''), + ('hex_table_list', 'longtext', 'true', ''), + ('partition_list', 'longtext', 'true', ''), + ('hex_partition_list', 'longtext', 'true', ''), + ('restore_scn', 'uint'), + ('restore_option', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH'), + ('backup_dest', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH'), + ('backup_set_list', 'longtext'), + ('backup_piece_list', 'longtext'), + ('backup_passwd', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH', 'true', ''), + ('external_kms_info', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH', 'true', ''), + ('remap_db_list', 'longtext', 'true', ''), + ('hex_remap_db_list', 'longtext', 'true', ''), + ('remap_table_list', 'longtext', 'true', ''), + ('hex_remap_table_list', 'longtext', 'true', ''), + ('remap_partition_list', 'longtext', 'true', ''), + ('hex_remap_partition_list', 'longtext', 'true', ''), + ('remap_tablegroup_list', 'longtext', 'true', ''), + ('hex_remap_tablegroup_list', 'longtext', 'true', ''), + ('remap_tablespace_list', 'longtext', 'true', ''), + ('hex_remap_tablespace_list', 'longtext', 'true', ''), + ('result', 'varchar:OB_DEFAULT_STATUS_LENTH', 'true', ''), + ('comment', 'longtext', 'true', ''), + ('description', 'longtext', 'true', ''), + ], +) + +def_table_schema( + owner = 'chongrong.th', + table_name = '__all_recover_table_job_history', + table_id = '476', + table_type = 'SYSTEM_TABLE', + gm_columns = ['gmt_create', 'gmt_modified'], + rowkey_columns = [ + ('tenant_id', 'int'), + ('job_id', 'int'), + ], + in_tenant_space = True, + is_cluster_private = True, + meta_record_in_sys = False, + normal_columns = [ + ('initiator_tenant_id', 'int'), + ('initiator_job_id', 'int'), + ('start_ts', 'int'), + ('end_ts', 'int'), + ('status', 'varchar:OB_DEFAULT_STATUS_LENTH'), + ('aux_tenant_name', 'varchar:OB_MAX_TENANT_NAME_LENGTH'), + ('target_tenant_name', 'varchar:OB_MAX_TENANT_NAME_LENGTH'), + ('target_tenant_id', 'int'), + ('import_all', 'bool'), + ('db_list', 'longtext', 'true', ''), + ('hex_db_list', 'longtext', 'true', ''), + ('table_list', 'longtext', 'true', ''), + ('hex_table_list', 'longtext', 'true', ''), + ('partition_list', 'longtext', 'true', ''), + ('hex_partition_list', 'longtext', 'true', ''), + ('restore_scn', 'uint'), + ('restore_option', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH'), + ('backup_dest', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH'), + ('backup_set_list', 'longtext'), + ('backup_piece_list', 'longtext'), + ('backup_passwd', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH', 'true', ''), + ('external_kms_info', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH', 'true', ''), + ('remap_db_list', 'longtext', 'true', ''), + ('hex_remap_db_list', 'longtext', 'true', ''), + ('remap_table_list', 'longtext', 'true', ''), + ('hex_remap_table_list', 'longtext', 'true', ''), + ('remap_partition_list', 'longtext', 'true', ''), + ('hex_remap_partition_list', 'longtext', 'true', ''), + ('remap_tablegroup_list', 'longtext', 'true', ''), + ('hex_remap_tablegroup_list', 'longtext', 'true', ''), + ('remap_tablespace_list', 'longtext', 'true', ''), + ('hex_remap_tablespace_list', 'longtext', 'true', ''), + ('result', 'varchar:OB_DEFAULT_STATUS_LENTH', 'true', ''), + ('comment', 'longtext', 'true', ''), + ('description', 'longtext', 'true', ''), + ], +) + +def_table_schema( + owner = 'chongrong.th', + table_name = '__all_import_table_job', + table_id = '477', + table_type = 'SYSTEM_TABLE', + gm_columns = ['gmt_create', 'gmt_modified'], + rowkey_columns = [ + ('tenant_id', 'int'), + ('job_id', 'int'), + ], + in_tenant_space = True, + is_cluster_private = True, + meta_record_in_sys = False, + normal_columns = [ + ('initiator_tenant_id', 'int'), + ('initiator_job_id', 'int'), + ('start_ts', 'int'), + ('end_ts', 'int'), + ('src_tenant_name', 'varchar:OB_MAX_TENANT_NAME_LENGTH'), + ('src_tenant_id', 'int'), + ('status', 'varchar:OB_DEFAULT_STATUS_LENTH'), + ('import_all', 'bool'), + ('db_list', 'longtext', 'true', ''), + ('hex_db_list', 'longtext', 'true', ''), + ('table_list', 'longtext', 'true', ''), + ('hex_table_list', 'longtext', 'true', ''), + ('partition_list', 'longtext', 'true', ''), + ('hex_partition_list', 'longtext', 'true', ''), + ('remap_db_list', 'longtext', 'true', ''), + ('hex_remap_db_list', 'longtext', 'true', ''), + ('remap_table_list', 'longtext', 'true', ''), + ('hex_remap_table_list', 'longtext', 'true', ''), + ('remap_partition_list', 'longtext', 'true', ''), + ('hex_remap_partition_list', 'longtext', 'true', ''), + ('remap_tablegroup_list', 'longtext', 'true', ''), + ('hex_remap_tablegroup_list', 'longtext', 'true', ''), + ('remap_tablespace_list', 'longtext', 'true', ''), + ('hex_remap_tablespace_list', 'longtext', 'true', ''), + ('total_table_count', 'int'), + ('finished_table_count', 'int'), + ('failed_table_count', 'int'), + ('total_bytes', 'int'), + ('finished_bytes', 'int'), + ('failed_bytes', 'int'), + ('result', 'varchar:OB_DEFAULT_STATUS_LENTH', 'true', ''), + ('comment', 'longtext', 'true', ''), + ('description', 'longtext', 'true', ''), + ], +) + +def_table_schema( + owner = 'chongrong.th', + table_name = '__all_import_table_job_history', + table_id = '478', + table_type = 'SYSTEM_TABLE', + gm_columns = ['gmt_create', 'gmt_modified'], + rowkey_columns = [ + ('tenant_id', 'int'), + ('job_id', 'int'), + ], + in_tenant_space = True, + is_cluster_private = True, + meta_record_in_sys = False, + + normal_columns = [ + ('initiator_tenant_id', 'int'), + ('initiator_job_id', 'int'), + ('start_ts', 'int'), + ('end_ts', 'int'), + ('src_tenant_name', 'varchar:OB_MAX_TENANT_NAME_LENGTH'), + ('src_tenant_id', 'int'), + ('status', 'varchar:OB_DEFAULT_STATUS_LENTH'), + ('import_all', 'bool'), + ('db_list', 'longtext', 'true', ''), + ('hex_db_list', 'longtext', 'true', ''), + ('table_list', 'longtext', 'true', ''), + ('hex_table_list', 'longtext', 'true', ''), + ('partition_list', 'longtext', 'true', ''), + ('hex_partition_list', 'longtext', 'true', ''), + ('remap_db_list', 'longtext', 'true', ''), + ('hex_remap_db_list', 'longtext', 'true', ''), + ('remap_table_list', 'longtext', 'true', ''), + ('hex_remap_table_list', 'longtext', 'true', ''), + ('remap_partition_list', 'longtext', 'true', ''), + ('hex_remap_partition_list', 'longtext', 'true', ''), + ('remap_tablegroup_list', 'longtext', 'true', ''), + ('hex_remap_tablegroup_list', 'longtext', 'true', ''), + ('remap_tablespace_list', 'longtext', 'true', ''), + ('hex_remap_tablespace_list', 'longtext', 'true', ''), + ('total_table_count', 'int'), + ('finished_table_count', 'int'), + ('failed_table_count', 'int'), + ('total_bytes', 'int'), + ('finished_bytes', 'int'), + ('failed_bytes', 'int'), + ('result', 'varchar:OB_DEFAULT_STATUS_LENTH', 'true', ''), + ('comment', 'longtext', 'true', ''), + ('description', 'longtext', 'true', ''), + ], +) + +def_table_schema( + owner = 'chongrong.th', + table_name = '__all_import_table_task', + table_id = '479', + table_type = 'SYSTEM_TABLE', + gm_columns = ['gmt_create', 'gmt_modified'], + rowkey_columns = [ + ('tenant_id', 'int'), + ('task_id', 'int'), + ], + in_tenant_space = True, + is_cluster_private = True, + meta_record_in_sys = False, + + normal_columns = [ + ('job_id', 'int'), + ('src_tenant_id', 'int'), + ('src_tablespace', 'varchar:OB_MAX_TABLESPACE_NAME_LENGTH', 'true', ''), + ('src_tablegroup', 'varchar:OB_MAX_TABLEGROUP_NAME_LENGTH', 'true', ''), + ('src_database', 'varchar:OB_MAX_DATABASE_NAME_LENGTH', 'true', ''), + ('src_table', 'varchar:OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE', 'true', ''), + ('src_partition', 'varchar:OB_MAX_PARTITION_NAME_LENGTH', 'true', ''), + ('target_tablespace', 'varchar:OB_MAX_TABLESPACE_NAME_LENGTH', 'true', ''), + ('target_tablegroup', 'varchar:OB_MAX_TABLEGROUP_NAME_LENGTH', 'true', ''), + ('target_database', 'varchar:OB_MAX_DATABASE_NAME_LENGTH', 'true', ''), + ('target_table', 'varchar:OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE', 'true', ''), + ('table_column', 'int'), + ('status', 'varchar:OB_DEFAULT_STATUS_LENTH'), + ('start_ts', 'int'), + ('completion_ts', 'int'), + ('cumulative_ts', 'int'), + ('total_bytes', 'int'), + ('total_rows', 'int'), + ('imported_bytes', 'int'), + ('imported_rows', 'int'), + ('total_index_count', 'int'), + ('imported_index_count', 'int'), + ('failed_index_count', 'int'), + ('total_constraint_count', 'int'), + ('imported_constraint_count', 'int'), + ('failed_constraint_count', 'int'), + ('total_ref_constraint_count', 'int'), + ('imported_ref_constraint_count', 'int'), + ('failed_ref_constraint_count', 'int'), + ('total_trigger_count', 'int'), + ('imported_trigger_count', 'int'), + ('failed_trigger_count', 'int'), + ('result', 'varchar:OB_DEFAULT_STATUS_LENTH', 'true', ''), + ('comment', 'longtext', 'true', ''), + ], +) + +def_table_schema( + owner = 'chongrong.th', + table_name = '__all_import_table_task_history', + table_id = '480', + table_type = 'SYSTEM_TABLE', + gm_columns = ['gmt_create', 'gmt_modified'], + rowkey_columns = [ + ('tenant_id', 'int'), + ('task_id', 'varchar:OB_INNER_TABLE_DEFAULT_VALUE_LENTH'), + ], + in_tenant_space = True, + is_cluster_private = True, + meta_record_in_sys = False, + + normal_columns = [ + ('job_id', 'int'), + ('src_tenant_id', 'int'), + ('src_tablespace', 'varchar:OB_MAX_TABLESPACE_NAME_LENGTH', 'true', ''), + ('src_tablegroup', 'varchar:OB_MAX_TABLEGROUP_NAME_LENGTH', 'true', ''), + ('src_database', 'varchar:OB_MAX_DATABASE_NAME_LENGTH', 'true', ''), + ('src_table', 'varchar:OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE', 'true', ''), + ('src_partition', 'varchar:OB_MAX_PARTITION_NAME_LENGTH', 'true', ''), + ('target_tablespace', 'varchar:OB_MAX_TABLESPACE_NAME_LENGTH', 'true', ''), + ('target_tablegroup', 'varchar:OB_MAX_TABLEGROUP_NAME_LENGTH', 'true', ''), + ('target_database', 'varchar:OB_MAX_DATABASE_NAME_LENGTH', 'true', ''), + ('target_table', 'varchar:OB_MAX_USER_TABLE_NAME_LENGTH_ORACLE', 'true', ''), + ('table_column', 'int'), + ('status', 'varchar:OB_DEFAULT_STATUS_LENTH'), + ('start_ts', 'int'), + ('completion_ts', 'int'), + ('cumulative_ts', 'int'), + ('total_bytes', 'int'), + ('total_rows', 'int'), + ('imported_bytes', 'int'), + ('imported_rows', 'int'), + ('total_index_count', 'int'), + ('imported_index_count', 'int'), + ('failed_index_count', 'int'), + ('total_constraint_count', 'int'), + ('imported_constraint_count', 'int'), + ('failed_constraint_count', 'int'), + ('total_ref_constraint_count', 'int'), + ('imported_ref_constraint_count', 'int'), + ('failed_ref_constraint_count', 'int'), + ('total_trigger_count', 'int'), + ('imported_trigger_count', 'int'), + ('failed_trigger_count', 'int'), + ('result', 'varchar:OB_DEFAULT_STATUS_LENTH', 'true', ''), + ('comment', 'longtext', 'true', ''), + ], +) + # 481 : __all_import_stmt_exec_history # 482 : __all_tablet_reorganize_history # # 余留位置 - ################################################################################ @@ -8270,7 +8573,22 @@ def_table_schema( vtable_route_policy = 'distributed', ) -#11102 __virtual_show_restore_preview abandoned on 4.0 +def_table_schema( + owner = 'chongrong.th', + table_name = '__tenant_virtual_show_restore_preview', + table_id = '11102', + table_type = 'VIRTUAL_TABLE', + gm_columns = [], + rowkey_columns = [ + ], + + normal_columns = [ + ('backup_type', 'varchar:20'), + ('backup_id', 'int'), + ('backup_dest', 'varchar:OB_MAX_BACKUP_DEST_LENGTH', 'true'), + ('description', 'varchar:OB_MAX_BACKUP_DEST_LENGTH', 'true'), + ], +) def_table_schema( owner = 'sean.yyj', @@ -12604,12 +12922,40 @@ def_table_schema(**gen_iterate_virtual_table_def( table_name = '__all_virtual_tenant_scheduler_job_class', keywords = all_def_keywords['__all_tenant_scheduler_job_class'])) -# 12422: __all_virtual_recover_table_job -# 12423: __all_virtual_recover_table_job_history -# 12424: __all_virtual_import_table_job -# 12425: __all_virtual_import_table_job_history -# 12426: __all_virtual_import_table_task -# 12427: __all_virtual_import_table_task_history +def_table_schema(**gen_iterate_private_virtual_table_def( + table_id = '12422', + table_name = '__all_virtual_recover_table_job', + keywords = all_def_keywords['__all_recover_table_job'], + in_tenant_space = True)) + +def_table_schema(**gen_iterate_private_virtual_table_def( + table_id = '12423', + table_name = '__all_virtual_recover_table_job_history', + keywords = all_def_keywords['__all_recover_table_job_history'], + in_tenant_space = True)) +def_table_schema(**gen_iterate_private_virtual_table_def( + table_id = '12424', + table_name = '__all_virtual_import_table_job', + keywords = all_def_keywords['__all_import_table_job'], + in_tenant_space = True)) + +def_table_schema(**gen_iterate_private_virtual_table_def( + table_id = '12425', + table_name = '__all_virtual_import_table_job_history', + keywords = all_def_keywords['__all_import_table_job_history'], + in_tenant_space = True)) + +def_table_schema(**gen_iterate_private_virtual_table_def( + table_id = '12426', + table_name = '__all_virtual_import_table_task', + keywords = all_def_keywords['__all_import_table_task'], + in_tenant_space = True)) + +def_table_schema(**gen_iterate_private_virtual_table_def( + table_id = '12427', + table_name = '__all_virtual_import_table_task_history', + keywords = all_def_keywords['__all_import_table_task_history'], + in_tenant_space = True)) # 12428: __all_virtual_import_stmt_exec_history # 12429: __all_virtual_data_activity_metrics @@ -13004,13 +13350,12 @@ def_table_schema(**gen_oracle_mapping_real_virtual_table_def('15404', all_def_ke # 15405: __all_virtual_session_info def_table_schema(**gen_oracle_mapping_real_virtual_table_def('15406', all_def_keywords['__all_tenant_scheduler_job_class'])) - -# 15407: __all_virtual_recover_table_job -# 15408: __all_virtual_recover_table_job_history -# 15409: __all_virtual_import_table_job -# 15410: __all_virtual_import_table_job_history -# 15411: __all_virtual_import_table_task -# 15412: __all_virtual_import_table_task_history +def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15407', all_def_keywords['__all_virtual_recover_table_job']))) +def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15408', all_def_keywords['__all_virtual_recover_table_job_history']))) +def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15409', all_def_keywords['__all_virtual_import_table_job']))) +def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15410', all_def_keywords['__all_virtual_import_table_job_history']))) +def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15411', all_def_keywords['__all_virtual_import_table_task']))) +def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15412', all_def_keywords['__all_virtual_import_table_task_history']))) # 15413: __all_virtual_import_stmt_exec_history def_table_schema(**gen_oracle_mapping_virtual_table_def('15414', all_def_keywords['__all_virtual_ls_info'])) # 15415: idx_dbms_lock_allocated_lockhandle_real_agent @@ -28914,18 +29259,532 @@ def_table_schema( # 21461: GV$OB_PL_CACHE_OBJECT # 21462: V$OB_PL_CACHE_OBJECT -# 21463: CDB_OB_RECOVER_TABLE_JOBS -# 21464: DBA_OB_RECOVER_TABLE_JOBS -# 21465: CDB_OB_RECOVER_TABLE_JOB_HISTORY -# 21466: DBA_OB_RECOVER_TABLE_JOB_HISTORY -# 21467: CDB_OB_IMPORT_TABLE_JOBS -# 21468: DBA_OB_IMPORT_TABLE_JOBS -# 21469: CDB_OB_IMPORT_TABLE_JOB_HISTORY -# 21470: DBA_OB_IMPORT_TABLE_JOB_HISTORY -# 21471: CDB_OB_IMPORT_TABLE_TASKS -# 21472: DBA_OB_IMPORT_TABLE_TASKS -# 21473: CDB_OB_IMPORT_TABLE_TASK_HISTORY -# 21474: DBA_OB_IMPORT_TABLE_TASK_HISTORY +def_table_schema( + owner = 'chongrong.th', + table_name = 'CDB_OB_RECOVER_TABLE_JOBS', + table_id = '21463', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + view_definition = """ + SELECT + TENANT_ID, + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + CASE + WHEN END_TS = 0 + THEN NULL + ELSE + USEC_TO_TIME(END_TS) + END AS FINISH_TIMESTAMP, + STATUS, + AUX_TENANT_NAME, + TARGET_TENANT_NAME, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + RESTORE_SCN, + CASE + WHEN RESTORE_SCN = 0 + THEN NULL + ELSE + SCN_TO_TIMESTAMP(RESTORE_SCN) + END AS RESTORE_SCN_DISPLAY, + RESTORE_OPTION, + BACKUP_DEST, + BACKUP_SET_LIST, + BACKUP_PIECE_LIST, + BACKUP_PASSWD, + EXTERNAL_KMS_INFO, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB; +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_RECOVER_TABLE_JOBS', + table_id = '21464', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + CASE + WHEN END_TS = 0 + THEN NULL + ELSE + USEC_TO_TIME(END_TS) + END AS FINISH_TIMESTAMP, + STATUS, + AUX_TENANT_NAME, + TARGET_TENANT_NAME, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + RESTORE_SCN, + CASE + WHEN RESTORE_SCN = 0 + THEN NULL + ELSE + SCN_TO_TIMESTAMP(RESTORE_SCN) + END AS RESTORE_SCN_DISPLAY, + RESTORE_OPTION, + BACKUP_DEST, + BACKUP_SET_LIST, + BACKUP_PIECE_LIST, + BACKUP_PASSWD, + EXTERNAL_KMS_INFO, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'CDB_OB_RECOVER_TABLE_JOB_HISTORY', + table_id = '21465', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + view_definition = """ + SELECT + TENANT_ID, + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, + STATUS, + AUX_TENANT_NAME, + TARGET_TENANT_NAME, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + RESTORE_SCN, + CASE + WHEN RESTORE_SCN = 0 + THEN NULL + ELSE + SCN_TO_TIMESTAMP(RESTORE_SCN) + END AS RESTORE_SCN_DISPLAY, + RESTORE_OPTION, + BACKUP_DEST, + BACKUP_SET_LIST, + BACKUP_PIECE_LIST, + BACKUP_PASSWD, + EXTERNAL_KMS_INFO, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY; +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_RECOVER_TABLE_JOB_HISTORY', + table_id = '21466', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, + STATUS, + AUX_TENANT_NAME, + TARGET_TENANT_NAME, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + RESTORE_SCN, + CASE + WHEN RESTORE_SCN = 0 + THEN NULL + ELSE + SCN_TO_TIMESTAMP(RESTORE_SCN) + END AS RESTORE_SCN_DISPLAY, + RESTORE_OPTION, + BACKUP_DEST, + BACKUP_SET_LIST, + BACKUP_PIECE_LIST, + BACKUP_PASSWD, + EXTERNAL_KMS_INFO, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'CDB_OB_IMPORT_TABLE_JOBS', + table_id = '21467', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + view_definition = """ + SELECT + TENANT_ID, + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + CASE + WHEN END_TS = 0 + THEN NULL + ELSE + USEC_TO_TIME(END_TS) + END AS FINISH_TIMESTAMP, + SRC_TENANT_NAME, + SRC_TENANT_ID, + STATUS, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + TOTAL_TABLE_COUNT, + FINISHED_TABLE_COUNT, + FAILED_TABLE_COUNT, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB; +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_JOBS', + table_id = '21468', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + CASE + WHEN END_TS = 0 + THEN NULL + ELSE + USEC_TO_TIME(END_TS) + END AS FINISH_TIMESTAMP, + SRC_TENANT_NAME, + SRC_TENANT_ID, + STATUS, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + TOTAL_TABLE_COUNT, + FINISHED_TABLE_COUNT, + FAILED_TABLE_COUNT, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'CDB_OB_IMPORT_TABLE_JOB_HISTORY', + table_id = '21469', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + view_definition = """ + SELECT + TENANT_ID, + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, + SRC_TENANT_NAME, + SRC_TENANT_ID, + STATUS, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + TOTAL_TABLE_COUNT, + FINISHED_TABLE_COUNT, + FAILED_TABLE_COUNT, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY; +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_JOB_HISTORY', + table_id = '21470', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(END_TS) AS FINISH_TIMESTAMP, + SRC_TENANT_NAME, + SRC_TENANT_ID, + STATUS, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + TOTAL_TABLE_COUNT, + FINISHED_TABLE_COUNT, + FAILED_TABLE_COUNT, + RESULT, + COMMENT, + DESCRIPTION + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'CDB_OB_IMPORT_TABLE_TASKS', + table_id = '21471', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + view_definition = """ + SELECT + TENANT_ID, + TASK_ID, + JOB_ID, + SRC_TENANT_ID, + SRC_TABLESPACE, + SRC_TABLEGROUP, + SRC_DATABASE, + SRC_TABLE, + SRC_PARTITION, + TARGET_TABLESPACE, + TARGET_TABLEGROUP, + TARGET_DATABASE, + TARGET_TABLE, + TABLE_COLUMN, + STATUS, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, + CUMULATIVE_TS, + TOTAL_INDEX_COUNT, + IMPORTED_INDEX_COUNT, + FAILED_INDEX_COUNT, + TOTAL_CONSTRAINT_COUNT, + IMPORTED_CONSTRAINT_COUNT, + FAILED_CONSTRAINT_COUNT, + TOTAL_REF_CONSTRAINT_COUNT, + IMPORTED_REF_CONSTRAINT_COUNT, + FAILED_REF_CONSTRAINT_COUNT, + RESULT, + COMMENT + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK; +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_TASKS', + table_id = '21472', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + TASK_ID, + JOB_ID, + SRC_TENANT_ID, + SRC_TABLESPACE, + SRC_TABLEGROUP, + SRC_DATABASE, + SRC_TABLE, + SRC_PARTITION, + TARGET_TABLESPACE, + TARGET_TABLEGROUP, + TARGET_DATABASE, + TARGET_TABLE, + TABLE_COLUMN, + STATUS, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, + CUMULATIVE_TS, + TOTAL_INDEX_COUNT, + IMPORTED_INDEX_COUNT, + FAILED_INDEX_COUNT, + TOTAL_CONSTRAINT_COUNT, + IMPORTED_CONSTRAINT_COUNT, + FAILED_CONSTRAINT_COUNT, + TOTAL_REF_CONSTRAINT_COUNT, + IMPORTED_REF_CONSTRAINT_COUNT, + FAILED_REF_CONSTRAINT_COUNT, + RESULT, + COMMENT + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'CDB_OB_IMPORT_TABLE_TASK_HISTORY', + table_id = '21473', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + view_definition = """ + SELECT + TENANT_ID, + TASK_ID, + JOB_ID, + SRC_TENANT_ID, + SRC_TABLESPACE, + SRC_TABLEGROUP, + SRC_DATABASE, + SRC_TABLE, + SRC_PARTITION, + TARGET_TABLESPACE, + TARGET_TABLEGROUP, + TARGET_DATABASE, + TARGET_TABLE, + TABLE_COLUMN, + STATUS, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, + CUMULATIVE_TS, + TOTAL_INDEX_COUNT, + IMPORTED_INDEX_COUNT, + FAILED_INDEX_COUNT, + TOTAL_CONSTRAINT_COUNT, + IMPORTED_CONSTRAINT_COUNT, + FAILED_CONSTRAINT_COUNT, + TOTAL_REF_CONSTRAINT_COUNT, + IMPORTED_REF_CONSTRAINT_COUNT, + FAILED_REF_CONSTRAINT_COUNT, + RESULT, + COMMENT + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY; +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_TASK_HISTORY', + table_id = '21474', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + TASK_ID, + JOB_ID, + SRC_TENANT_ID, + SRC_TABLESPACE, + SRC_TABLEGROUP, + SRC_DATABASE, + SRC_TABLE, + SRC_PARTITION, + TARGET_TABLESPACE, + TARGET_TABLEGROUP, + TARGET_DATABASE, + TARGET_TABLE, + TABLE_COLUMN, + STATUS, + USEC_TO_TIME(START_TS) AS START_TIMESTAMP, + USEC_TO_TIME(COMPLETION_TS) AS COMPLETION_TIMESTAMP, + CUMULATIVE_TS, + TOTAL_INDEX_COUNT, + IMPORTED_INDEX_COUNT, + FAILED_INDEX_COUNT, + TOTAL_CONSTRAINT_COUNT, + IMPORTED_CONSTRAINT_COUNT, + FAILED_CONSTRAINT_COUNT, + TOTAL_REF_CONSTRAINT_COUNT, + IMPORTED_REF_CONSTRAINT_COUNT, + FAILED_REF_CONSTRAINT_COUNT, + RESULT, + COMMENT + FROM OCEANBASE.__ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + # 21475: CDB_OB_IMPORT_STMT_EXEC_HISTORY # 21476: DBA_OB_IMPORT_STMT_EXEC_HISTORY @@ -47151,12 +48010,283 @@ def_table_schema( """.replace("\n", " ") ) -# 25262: DBA_OB_RECOVER_TABLE_JOBS -# 25263: DBA_OB_RECOVER_TABLE_JOB_HISTORY -# 25264: DBA_OB_IMPORT_TABLE_JOBS -# 25265: DBA_OB_IMPORT_TABLE_JOB_HISTORY -# 25266: DBA_OB_IMPORT_TABLE_TASKS -# 25267: DBA_OB_IMPORT_TABLE_TASK_HISTORY +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_RECOVER_TABLE_JOBS', + name_postfix = '_ORA', + database_id = 'OB_ORA_SYS_DATABASE_ID', + table_id = '25262', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, + CASE + WHEN END_TS = 0 + THEN NULL + ELSE + TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') + END AS FINISH_TIMESTAMP, + STATUS, + AUX_TENANT_NAME, + TARGET_TENANT_NAME, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + RESTORE_SCN, + CASE + WHEN RESTORE_SCN = 0 + THEN NULL + ELSE + SCN_TO_TIMESTAMP(RESTORE_SCN) + END AS RESTORE_SCN_DISPLAY, + RESTORE_OPTION, + BACKUP_DEST, + BACKUP_SET_LIST, + BACKUP_PIECE_LIST, + BACKUP_PASSWD, + EXTERNAL_KMS_INFO, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + RESULT, + "COMMENT", + DESCRIPTION + FROM SYS.ALL_VIRTUAL_RECOVER_TABLE_JOB + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_RECOVER_TABLE_JOB_HISTORY', + name_postfix = '_ORA', + database_id = 'OB_ORA_SYS_DATABASE_ID', + table_id = '25263', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, + TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS FINISH_TIMESTAMP, + STATUS, + AUX_TENANT_NAME, + TARGET_TENANT_NAME, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + RESTORE_SCN, + CASE + WHEN RESTORE_SCN = 0 + THEN NULL + ELSE + SCN_TO_TIMESTAMP(RESTORE_SCN) + END AS RESTORE_SCN_DISPLAY, + RESTORE_OPTION, + BACKUP_DEST, + BACKUP_SET_LIST, + BACKUP_PIECE_LIST, + BACKUP_PASSWD, + EXTERNAL_KMS_INFO, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + RESULT, + "COMMENT", + DESCRIPTION + FROM SYS.ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_JOBS', + name_postfix = '_ORA', + database_id = 'OB_ORA_SYS_DATABASE_ID', + table_id = '25264', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, + CASE + WHEN END_TS = 0 + THEN NULL + ELSE + TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') + END AS FINISH_TIMESTAMP, + SRC_TENANT_NAME, + SRC_TENANT_ID, + STATUS, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + TOTAL_TABLE_COUNT, + FINISHED_TABLE_COUNT, + FAILED_TABLE_COUNT, + RESULT, + "COMMENT", + DESCRIPTION + FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_JOB + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_JOB_HISTORY', + name_postfix = '_ORA', + database_id = 'OB_ORA_SYS_DATABASE_ID', + table_id = '25265', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + JOB_ID, + INITIATOR_TENANT_ID, + INITIATOR_JOB_ID, + TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, + TO_CHAR(END_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS FINISH_TIMESTAMP, + SRC_TENANT_NAME, + SRC_TENANT_ID, + STATUS, + IMPORT_ALL, + DB_LIST, + TABLE_LIST, + REMAP_DB_LIST, + REMAP_TABLE_LIST, + REMAP_TABLEGROUP_LIST, + REMAP_TABLESPACE_LIST, + TOTAL_TABLE_COUNT, + FINISHED_TABLE_COUNT, + FAILED_TABLE_COUNT, + RESULT, + "COMMENT", + DESCRIPTION + FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_TASKS', + name_postfix = '_ORA', + database_id = 'OB_ORA_SYS_DATABASE_ID', + table_id = '25266', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + TASK_ID, + JOB_ID, + SRC_TENANT_ID, + SRC_TABLESPACE, + SRC_TABLEGROUP, + SRC_DATABASE, + SRC_TABLE, + SRC_PARTITION, + TARGET_TABLESPACE, + TARGET_TABLEGROUP, + TARGET_DATABASE, + TARGET_TABLE, + TABLE_COLUMN, + STATUS, + TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, + TO_CHAR(COMPLETION_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS COMPLETION_TIMESTAMP, + CUMULATIVE_TS, + TOTAL_INDEX_COUNT, + IMPORTED_INDEX_COUNT, + FAILED_INDEX_COUNT, + TOTAL_CONSTRAINT_COUNT, + IMPORTED_CONSTRAINT_COUNT, + FAILED_CONSTRAINT_COUNT, + TOTAL_REF_CONSTRAINT_COUNT, + IMPORTED_REF_CONSTRAINT_COUNT, + FAILED_REF_CONSTRAINT_COUNT, + RESULT, + "COMMENT" + FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_TASK + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) + +def_table_schema( + owner = 'chongrong.th', + table_name = 'DBA_OB_IMPORT_TABLE_TASK_HISTORY', + name_postfix = '_ORA', + database_id = 'OB_ORA_SYS_DATABASE_ID', + table_id = '25267', + table_type = 'SYSTEM_VIEW', + rowkey_columns = [], + normal_columns = [], + gm_columns = [], + in_tenant_space = True, + view_definition = """ + SELECT + TASK_ID, + JOB_ID, + SRC_TENANT_ID, + SRC_TABLESPACE, + SRC_TABLEGROUP, + SRC_DATABASE, + SRC_TABLE, + SRC_PARTITION, + TARGET_TABLESPACE, + TARGET_TABLEGROUP, + TARGET_DATABASE, + TARGET_TABLE, + TABLE_COLUMN, + STATUS, + TO_CHAR(START_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS START_TIMESTAMP, + TO_CHAR(COMPLETION_TS / (1000 * 60 * 60 * 24 * 1000) + TO_DATE('1970-01-01 08:00:00', 'yyyy-mm-dd hh:mi:ss'), 'yyyy-mm-dd hh24:mi:ss') AS COMPLETION_TIMESTAMP, + CUMULATIVE_TS, + TOTAL_INDEX_COUNT, + IMPORTED_INDEX_COUNT, + FAILED_INDEX_COUNT, + TOTAL_CONSTRAINT_COUNT, + IMPORTED_CONSTRAINT_COUNT, + FAILED_CONSTRAINT_COUNT, + TOTAL_REF_CONSTRAINT_COUNT, + IMPORTED_REF_CONSTRAINT_COUNT, + FAILED_REF_CONSTRAINT_COUNT, + RESULT, + "COMMENT" + FROM SYS.ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY + WHERE TENANT_ID = EFFECTIVE_TENANT_ID() +""".replace("\n", " ") +) # 25268: DBA_OB_IMPORT_STMT_EXEC_HISTORY # 余留位置 diff --git a/src/share/inner_table/ob_inner_table_schema_misc.ipp b/src/share/inner_table/ob_inner_table_schema_misc.ipp index 0858d0ed13..9fc3fef378 100644 --- a/src/share/inner_table/ob_inner_table_schema_misc.ipp +++ b/src/share/inner_table/ob_inner_table_schema_misc.ipp @@ -455,6 +455,10 @@ case OB_ALL_VIRTUAL_COLUMN_CHECKSUM_ERROR_INFO_TID: case OB_ALL_VIRTUAL_DEADLOCK_EVENT_HISTORY_TID: case OB_ALL_VIRTUAL_GLOBAL_CONTEXT_VALUE_TID: case OB_ALL_VIRTUAL_GLOBAL_TRANSACTION_TID: +case OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TID: +case OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID: +case OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID: +case OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID: case OB_ALL_VIRTUAL_KV_TTL_TASK_TID: case OB_ALL_VIRTUAL_KV_TTL_TASK_HISTORY_TID: case OB_ALL_VIRTUAL_LOG_ARCHIVE_DEST_PARAMETER_TID: @@ -474,6 +478,8 @@ case OB_ALL_VIRTUAL_LS_RESTORE_PROGRESS_TID: case OB_ALL_VIRTUAL_LS_STATUS_TID: case OB_ALL_VIRTUAL_LS_TRANSFER_MEMBER_LIST_LOCK_INFO_TID: case OB_ALL_VIRTUAL_MERGE_INFO_TID: +case OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TID: +case OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TID: case OB_ALL_VIRTUAL_RESTORE_JOB_TID: case OB_ALL_VIRTUAL_RESTORE_JOB_HISTORY_TID: case OB_ALL_VIRTUAL_RESTORE_PROGRESS_TID: @@ -948,6 +954,70 @@ case OB_ALL_VIRTUAL_ZONE_MERGE_INFO_TID: break; } + case OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_TID: { + ObIteratePrivateVirtualTable *iter = NULL; + const bool meta_record_in_sys = false; + if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIteratePrivateVirtualTable, iter))) { + SERVER_LOG(WARN, "create iterate private virtual table iterator failed", KR(ret)); + } else if (OB_FAIL(iter->init(OB_ALL_IMPORT_TABLE_JOB_TID, meta_record_in_sys, index_schema, params))) { + SERVER_LOG(WARN, "iterate private virtual table iter init failed", KR(ret)); + iter->~ObIteratePrivateVirtualTable(); + allocator.free(iter); + iter = NULL; + } else { + vt_iter = iter; + } + break; + } + + case OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID: { + ObIteratePrivateVirtualTable *iter = NULL; + const bool meta_record_in_sys = false; + if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIteratePrivateVirtualTable, iter))) { + SERVER_LOG(WARN, "create iterate private virtual table iterator failed", KR(ret)); + } else if (OB_FAIL(iter->init(OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID, meta_record_in_sys, index_schema, params))) { + SERVER_LOG(WARN, "iterate private virtual table iter init failed", KR(ret)); + iter->~ObIteratePrivateVirtualTable(); + allocator.free(iter); + iter = NULL; + } else { + vt_iter = iter; + } + break; + } + + case OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID: { + ObIteratePrivateVirtualTable *iter = NULL; + const bool meta_record_in_sys = false; + if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIteratePrivateVirtualTable, iter))) { + SERVER_LOG(WARN, "create iterate private virtual table iterator failed", KR(ret)); + } else if (OB_FAIL(iter->init(OB_ALL_IMPORT_TABLE_TASK_TID, meta_record_in_sys, index_schema, params))) { + SERVER_LOG(WARN, "iterate private virtual table iter init failed", KR(ret)); + iter->~ObIteratePrivateVirtualTable(); + allocator.free(iter); + iter = NULL; + } else { + vt_iter = iter; + } + break; + } + + case OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID: { + ObIteratePrivateVirtualTable *iter = NULL; + const bool meta_record_in_sys = false; + if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIteratePrivateVirtualTable, iter))) { + SERVER_LOG(WARN, "create iterate private virtual table iterator failed", KR(ret)); + } else if (OB_FAIL(iter->init(OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID, meta_record_in_sys, index_schema, params))) { + SERVER_LOG(WARN, "iterate private virtual table iter init failed", KR(ret)); + iter->~ObIteratePrivateVirtualTable(); + allocator.free(iter); + iter = NULL; + } else { + vt_iter = iter; + } + break; + } + case OB_ALL_VIRTUAL_KV_TTL_TASK_TID: { ObIteratePrivateVirtualTable *iter = NULL; const bool meta_record_in_sys = false; @@ -1075,7 +1145,9 @@ case OB_ALL_VIRTUAL_ZONE_MERGE_INFO_TID: } break; } + END_CREATE_VT_ITER_SWITCH_LAMBDA + BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA case OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_TID: { ObIteratePrivateVirtualTable *iter = NULL; const bool meta_record_in_sys = false; @@ -1139,9 +1211,7 @@ case OB_ALL_VIRTUAL_ZONE_MERGE_INFO_TID: } break; } - END_CREATE_VT_ITER_SWITCH_LAMBDA - BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA case OB_ALL_VIRTUAL_LS_RECOVERY_STAT_TID: { ObIteratePrivateVirtualTable *iter = NULL; const bool meta_record_in_sys = true; @@ -1254,6 +1324,38 @@ case OB_ALL_VIRTUAL_ZONE_MERGE_INFO_TID: break; } + case OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_TID: { + ObIteratePrivateVirtualTable *iter = NULL; + const bool meta_record_in_sys = false; + if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIteratePrivateVirtualTable, iter))) { + SERVER_LOG(WARN, "create iterate private virtual table iterator failed", KR(ret)); + } else if (OB_FAIL(iter->init(OB_ALL_RECOVER_TABLE_JOB_TID, meta_record_in_sys, index_schema, params))) { + SERVER_LOG(WARN, "iterate private virtual table iter init failed", KR(ret)); + iter->~ObIteratePrivateVirtualTable(); + allocator.free(iter); + iter = NULL; + } else { + vt_iter = iter; + } + break; + } + + case OB_ALL_VIRTUAL_RECOVER_TABLE_JOB_HISTORY_TID: { + ObIteratePrivateVirtualTable *iter = NULL; + const bool meta_record_in_sys = false; + if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIteratePrivateVirtualTable, iter))) { + SERVER_LOG(WARN, "create iterate private virtual table iterator failed", KR(ret)); + } else if (OB_FAIL(iter->init(OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID, meta_record_in_sys, index_schema, params))) { + SERVER_LOG(WARN, "iterate private virtual table iter init failed", KR(ret)); + iter->~ObIteratePrivateVirtualTable(); + allocator.free(iter); + iter = NULL; + } else { + vt_iter = iter; + } + break; + } + case OB_ALL_VIRTUAL_RESTORE_JOB_TID: { ObIteratePrivateVirtualTable *iter = NULL; const bool meta_record_in_sys = false; @@ -1365,7 +1467,9 @@ case OB_ALL_VIRTUAL_ZONE_MERGE_INFO_TID: } break; } + END_CREATE_VT_ITER_SWITCH_LAMBDA + BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA case OB_ALL_VIRTUAL_TENANT_EVENT_HISTORY_TID: { ObIteratePrivateVirtualTable *iter = NULL; const bool meta_record_in_sys = false; @@ -1461,9 +1565,7 @@ case OB_ALL_VIRTUAL_ZONE_MERGE_INFO_TID: } break; } - END_CREATE_VT_ITER_SWITCH_LAMBDA - BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA case OB_ALL_VIRTUAL_WR_SNAPSHOT_TID: { ObIteratePrivateVirtualTable *iter = NULL; const bool meta_record_in_sys = false; @@ -4043,6 +4145,18 @@ case OB_ALL_DEADLOCK_EVENT_HISTORY_AUX_LOB_PIECE_TID: case OB_ALL_GLOBAL_CONTEXT_VALUE_TID: case OB_ALL_GLOBAL_CONTEXT_VALUE_AUX_LOB_META_TID: case OB_ALL_GLOBAL_CONTEXT_VALUE_AUX_LOB_PIECE_TID: +case OB_ALL_IMPORT_TABLE_JOB_TID: +case OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_META_TID: +case OB_ALL_IMPORT_TABLE_JOB_AUX_LOB_PIECE_TID: +case OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID: +case OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID: +case OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID: +case OB_ALL_IMPORT_TABLE_TASK_TID: +case OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID: +case OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID: +case OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID: +case OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID: +case OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID: case OB_ALL_KV_TTL_TASK_TID: case OB_ALL_KV_TTL_TASK_IDX_KV_TTL_TASK_TABLE_ID_TID: case OB_ALL_KV_TTL_TASK_AUX_LOB_META_TID: @@ -4103,6 +4217,12 @@ case OB_ALL_LS_TRANSFER_MEMBER_LIST_LOCK_INFO_AUX_LOB_PIECE_TID: case OB_ALL_MERGE_INFO_TID: case OB_ALL_MERGE_INFO_AUX_LOB_META_TID: case OB_ALL_MERGE_INFO_AUX_LOB_PIECE_TID: +case OB_ALL_RECOVER_TABLE_JOB_TID: +case OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_META_TID: +case OB_ALL_RECOVER_TABLE_JOB_AUX_LOB_PIECE_TID: +case OB_ALL_RECOVER_TABLE_JOB_HISTORY_TID: +case OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_META_TID: +case OB_ALL_RECOVER_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID: case OB_ALL_RESERVED_SNAPSHOT_TID: case OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_META_TID: case OB_ALL_RESERVED_SNAPSHOT_AUX_LOB_PIECE_TID: diff --git a/src/share/ob_common_rpc_proxy.h b/src/share/ob_common_rpc_proxy.h index babc52783e..fb7ff86958 100644 --- a/src/share/ob_common_rpc_proxy.h +++ b/src/share/ob_common_rpc_proxy.h @@ -64,6 +64,7 @@ public: RPC_S(PRD create_database, obrpc::OB_CREATE_DATABASE, (ObCreateDatabaseArg), UInt64); RPC_S(PRD create_tablegroup, obrpc::OB_CREATE_TABLEGROUP, (ObCreateTablegroupArg), UInt64); RPC_S(PRD create_table, obrpc::OB_CREATE_TABLE, (ObCreateTableArg), ObCreateTableRes); + RPC_S(PRD recover_restore_table_ddl, obrpc::OB_RECOVER_RESTORE_TABLE_DDL, (ObRecoverRestoreTableDDLArg)); RPC_S(PRD parallel_create_table, obrpc::OB_PARALLEL_CREATE_TABLE, (ObCreateTableArg), ObCreateTableRes); RPC_S(PRD alter_table, obrpc::OB_ALTER_TABLE, (ObAlterTableArg), ObAlterTableRes); RPC_S(PRD create_hidden_table, obrpc::OB_CREATE_HIDDEN_TABLE, (obrpc::ObCreateHiddenTableArg), ObCreateHiddenTableRes); @@ -270,6 +271,7 @@ public: RPC_S(PR5 backup_manage, obrpc::OB_BACKUP_MANAGE, (ObBackupManageArg)); RPC_S(PR5 backup_delete, obrpc::OB_BACKUP_CLEAN, (obrpc::ObBackupCleanArg)); RPC_S(PR5 delete_policy, obrpc::OB_DELETE_POLICY, (obrpc::ObDeletePolicyArg)); + RPC_S(PR5 recover_table, obrpc::OB_RECOVER_TABLE, (obrpc::ObRecoverTableArg)); //RPC_S(PRD standby_upgrade_virtual_schema, obrpc::OB_UPGRADE_STANDBY_SCHEMA, // (ObDDLNopOpreatorArg)); // use ddl thread RPC_S(PR5 check_backup_scheduler_working, obrpc::OB_CHECK_BACKUP_SCHEDULER_WORKING, Bool); diff --git a/src/share/ob_ddl_common.cpp b/src/share/ob_ddl_common.cpp index db0a8799fc..01375419ba 100644 --- a/src/share/ob_ddl_common.cpp +++ b/src/share/ob_ddl_common.cpp @@ -407,7 +407,12 @@ int ObDDLUtil::generate_column_name_str( } // append original column name if (OB_SUCC(ret) && with_origin_name) { - if (OB_FAIL(sql_string.append_fmt("%s%.*s%s", split_char, column_name_info.column_name_.length(), column_name_info.column_name_.ptr(), split_char))) { + if (column_name_info.is_enum_set_need_cast_) { + // Enum and set in Recover restore table ddl operation will be cast to unsigned, and then append into macro block. + if (OB_FAIL(sql_string.append_fmt("cast(%s%.*s%s as unsigned)", split_char, column_name_info.column_name_.length(), column_name_info.column_name_.ptr(), split_char))) { + LOG_WARN("append origin column name failed", K(ret)); + } + } else if (OB_FAIL(sql_string.append_fmt("%s%.*s%s", split_char, column_name_info.column_name_.length(), column_name_info.column_name_.ptr(), split_char))) { LOG_WARN("append origin column name failed", K(ret)); } } @@ -529,14 +534,12 @@ int ObDDLUtil::generate_build_replica_sql( LOG_WARN("fail to check formal guard", K(ret)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, source_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(data_table_id)); - } else if (OB_ISNULL(source_table_schema)) { - ret = OB_TABLE_NOT_EXIST; - LOG_WARN("fail to get table schema", K(ret), K(data_table_id)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, dest_table_id, dest_table_schema))) { LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(dest_table_id)); - } else if (OB_ISNULL(dest_table_schema)) { + } else if (OB_ISNULL(source_table_schema) || OB_ISNULL(dest_table_schema)) { ret = OB_TABLE_NOT_EXIST; - LOG_WARN("fail to get table schema", K(ret), K(dest_table_id)); + LOG_WARN("fail to get table schema", K(ret), KP(source_table_schema), KP(dest_table_schema), + K(tenant_id), K(data_table_id), K(dest_table_id)); } else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_table_id(tenant_id, data_table_id, oracle_mode))) { LOG_WARN("check if oracle mode failed", K(ret), K(data_table_id)); } else { @@ -1163,12 +1166,19 @@ static inline void try_replace_user_tenant_id(const uint64_t user_tenant_id, uin check_tenant_id = !is_user_tenant(check_tenant_id) ? check_tenant_id : user_tenant_id; } -int ObDDLUtil::replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObAlterTableArg &alter_table_arg) +int ObDDLUtil::replace_user_tenant_id( + const ObDDLType &ddl_type, + const uint64_t tenant_id, + obrpc::ObAlterTableArg &alter_table_arg) { int ret = OB_SUCCESS; - if (!is_user_tenant(tenant_id)) { + if (OB_UNLIKELY(is_invalid_ddl_type(ddl_type))) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(ddl_type)); + } else if (!is_user_tenant(tenant_id)) { LOG_TRACE("not user tenant, no need to replace", K(tenant_id)); } else { + const bool need_replace_schema_info = DDL_TABLE_RESTORE != ddl_type; try_replace_user_tenant_id(tenant_id, alter_table_arg.exec_tenant_id_); for (int64_t i = 0; OB_SUCC(ret) && i < alter_table_arg.index_arg_list_.count(); ++i) { obrpc::ObIndexArg *index_arg = alter_table_arg.index_arg_list_.at(i); @@ -1180,7 +1190,7 @@ int ObDDLUtil::replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObAlterTa try_replace_user_tenant_id(tenant_id, fk_arg.exec_tenant_id_); try_replace_user_tenant_id(tenant_id, fk_arg.tenant_id_); } - if (is_user_tenant(alter_table_arg.alter_table_schema_.get_tenant_id())) { + if (need_replace_schema_info && is_user_tenant(alter_table_arg.alter_table_schema_.get_tenant_id())) { alter_table_arg.alter_table_schema_.set_tenant_id(tenant_id); } try_replace_user_tenant_id(tenant_id, alter_table_arg.sequence_ddl_arg_.exec_tenant_id_); @@ -1226,6 +1236,83 @@ REPLACE_DDL_ARG_FUNC(obrpc::ObTruncateTableArg) #undef REPLACE_DDL_ARG_FUNC +int ObDDLUtil::reshape_ddl_column_obj( + common::ObDatum &datum, + const ObObjMeta &obj_meta) +{ + int ret = OB_SUCCESS; + if (datum.is_null()) { + // do not need to reshape + } else if (obj_meta.is_lob_storage()) { + ObLobLocatorV2 lob(datum.get_string(), obj_meta.has_lob_header()); + ObString disk_loc; + if (!lob.is_valid()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid lob locator", K(ret)); + } else if (!lob.is_lob_disk_locator() && !lob.is_persist_lob()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid lob locator, should be persist lob", K(ret), K(lob)); + } else if (OB_FAIL(lob.get_disk_locator(disk_loc))) { + LOG_WARN("get disk locator failed", K(ret), K(lob)); + } + if (OB_SUCC(ret)) { + datum.set_string(disk_loc); + } + } else if (OB_UNLIKELY(!obj_meta.is_fixed_len_char_type())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("no need to reshape non-char", K(ret)); + } else { + const char *ptr = datum.ptr_; + int32_t len = datum.len_; + int32_t trunc_len_byte = static_cast(ObCharset::strlen_byte_no_sp( + obj_meta.get_collation_type(), ptr, len)); + datum.set_string(ObString(trunc_len_byte, ptr)); + } + return ret; +} + +int ObDDLUtil::get_tenant_schema_guard( + 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; + rootserver::ObRootService *root_service = GCTX.root_service_; + if (OB_UNLIKELY(common::OB_INVALID_ID == src_tenant_id || common::OB_INVALID_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_ISNULL(root_service)) { + ret = OB_ERR_SYS; + LOG_WARN("error sys, root service must not be nullptr", K(ret)); + } else { + share::schema::ObMultiVersionSchemaService &schema_service = root_service->get_schema_service(); + if (OB_FAIL(schema_service.get_tenant_schema_guard(dst_tenant_id, hold_buf_dst_tenant_schema_guard))) { + LOG_WARN("get tanant schema guard failed", K(ret), K(dst_tenant_id)); + } else if (src_tenant_id != dst_tenant_id) { + if (OB_FAIL(schema_service.get_tenant_schema_guard(src_tenant_id, hold_buf_src_tenant_schema_guard))) { + LOG_WARN("get tanant schema guard failed", K(ret), 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; + } + } else { + src_tenant_schema_guard = &hold_buf_dst_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; +} /****************** ObCheckTabletDataComplementOp *************/ diff --git a/src/share/ob_ddl_common.h b/src/share/ob_ddl_common.h index 3bcd01aef3..9c94ef06d0 100644 --- a/src/share/ob_ddl_common.h +++ b/src/share/ob_ddl_common.h @@ -80,7 +80,7 @@ enum ObDDLType DDL_TABLE_REDEFINITION = 1010, DDL_DIRECT_LOAD = 1011, // load data DDL_DIRECT_LOAD_INSERT = 1012, // insert into select - + DDL_TABLE_RESTORE = 1013, // table restore // @note new normal ddl type to be defined here !!! DDL_NORMAL_TYPE = 10001, @@ -106,6 +106,7 @@ enum ObDDLTaskType UPDATE_AUTOINC_SCHEMA = 9, CANCEL_DDL_TASK = 10, MODIFY_NOT_NULL_COLUMN_STATE_TASK = 11, + MAKE_RECOVER_RESTORE_TABLE_TASK_TAKE_EFFECT = 12, }; enum ObDDLTaskStatus { @@ -236,16 +237,17 @@ struct ObColumnNameInfo final { public: ObColumnNameInfo() - : column_name_(), is_shadow_column_(false) + : column_name_(), is_shadow_column_(false), is_enum_set_need_cast_(false) {} - ObColumnNameInfo(const ObString &column_name, const bool is_shadow_column) - : column_name_(column_name), is_shadow_column_(is_shadow_column) + ObColumnNameInfo(const ObString &column_name, const bool is_shadow_column, const bool is_enum_set_need_cast = false) + : column_name_(column_name), is_shadow_column_(is_shadow_column), is_enum_set_need_cast_(is_enum_set_need_cast) {} ~ObColumnNameInfo() = default; - TO_STRING_KV(K_(column_name), K_(is_shadow_column)); + TO_STRING_KV(K_(column_name), K_(is_shadow_column), K_(is_enum_set_need_cast)); public: ObString column_name_; bool is_shadow_column_; + bool is_enum_set_need_cast_; }; class ObColumnNameMap final { @@ -412,14 +414,16 @@ public: const uint64_t task_id, int64_t &data_format_version); - static int replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObAlterTableArg &alter_table_arg); + static int replace_user_tenant_id( + const ObDDLType &ddl_type, + const uint64_t tenant_id, + obrpc::ObAlterTableArg &alter_table_arg); static int replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObDropDatabaseArg &drop_db_arg); static int replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObDropTableArg &drop_table_arg); static int replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObDropIndexArg &drop_index_arg); static int replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObTruncateTableArg &trucnate_table_arg); static int replace_user_tenant_id(const uint64_t tenant_id, obrpc::ObCreateIndexArg &create_index_arg); -private: static int generate_column_name_str( const common::ObIArray &column_names, const bool is_oracle_mode, @@ -434,6 +438,40 @@ private: const bool with_alias_name, const bool with_comma, ObSqlString &sql_string); + static int reshape_ddl_column_obj( + common::ObDatum &datum, + const ObObjMeta &obj_meta); + + /** + * NOTICE: The interface is designed for Offline DDL operation only. + * The caller can not obtain the schema via the hold_buf_src_tenant_schema_guard whose + * validity is limited by whether src_tenant_id and dst_tenant_id are the same. + * + * 1. This interface will provide the same tenant schema guard when src_tenant_id = dst_tenant_id, + * to avoid using two different versions of the guard caused by the parallel ddl under the tenant. + * 2. This interface will provide corresponding tenant schema guard when src_tenant_id != dst_tenant_id. + * + * @param [in] src_tenant_id + * @param [in] dst_tenant_id + * @param [in] hold_buf_src_tenant_schema_guard: hold buf, invalid when src_tenant_id = dst_tenant_id. + * @param [in] hold_buf_dst_tenant_schema_guard: hold buf. + * @param [out] src_tenant_schema_guard: + * pointer to the hold_buf_dst_tenant_schema_guard if src_tenant_id = dst_tenant_id, + * pointer to the hold_buf_src_tenant_schema_guard if src_tenant_id != dst_tenant_id, + * is always not nullptr if the interface return OB_SUCC. + * @param [out] dst_tenant_schema_guard: + * pointer to the hold_buf_dst_tenant_schema_guard, + * is always not nullptr if the interface return OB_SUCC. + */ + static int get_tenant_schema_guard( + 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); + +private: static int generate_order_by_str( const ObIArray &select_column_ids, const ObIArray &order_column_ids, diff --git a/src/share/ob_debug_sync_point.h b/src/share/ob_debug_sync_point.h index 573627e356..e4fe8dbff5 100755 --- a/src/share/ob_debug_sync_point.h +++ b/src/share/ob_debug_sync_point.h @@ -381,6 +381,10 @@ class ObString; ACT(BEFORE_REPLAY_DDL_PREPRARE,)\ ACT(BEFORE_REPLAY_DDL_COMMIT,)\ ACT(BEFORE_BACKUP_UESR_META,)\ + ACT(BEFORE_INSERT_UERR_RECOVER_TABLE_JOB,)\ + ACT(BEFORE_GENERATE_IMPORT_TABLE_TASK,)\ + ACT(BEFORE_RECOVER_UESR_RECOVER_TABLE_JOB,)\ + ACT(BEFORE_RESTORE_AUX_TENANT,)\ ACT(BEFORE_BACKUP_SYS_TABLETS,)\ ACT(BEFORE_WRITE_DDL_PREPARE_LOG,)\ ACT(AFTER_REMOTE_WRITE_DDL_PREPARE_LOG,)\ diff --git a/src/share/ob_inner_table_operator.cpp b/src/share/ob_inner_table_operator.cpp index 6bd08cb769..5e241dec0b 100644 --- a/src/share/ob_inner_table_operator.cpp +++ b/src/share/ob_inner_table_operator.cpp @@ -541,7 +541,7 @@ int ObInnerTableOperator::compare_and_swap( LOG_WARN("invalid new_value", K(ret), K(key), K(column_name), K(old_value), K(new_value)); } else if (OB_FAIL(sql.assign_fmt("%s='%s'", column_name, new_value))) { LOG_WARN("fail to assign sql", K(ret), K(key), K(column_name), K(old_value), K(new_value)); - } else if (OB_FAIL(predicates.assign_fmt("%s=%s", column_name, old_value))) { + } else if (OB_FAIL(predicates.assign_fmt("%s='%s'", column_name, old_value))) { LOG_WARN("fail to assign sql", K(ret), K(key), K(column_name), K(old_value), K(new_value)); } else if (OB_FAIL(do_compare_and_swap_(proxy, key, sql.ptr(), predicates.ptr(), affected_rows))) { LOG_WARN("fail to update column", K(ret), K(key), K(column_name), K(old_value), K(new_value)); diff --git a/src/share/ob_rpc_struct.cpp b/src/share/ob_rpc_struct.cpp index 51296f10dc..31a10b4879 100755 --- a/src/share/ob_rpc_struct.cpp +++ b/src/share/ob_rpc_struct.cpp @@ -514,6 +514,158 @@ int ObUpdateDDLTaskActiveTimeArg::assign(const ObUpdateDDLTaskActiveTimeArg &arg return ret; } +ObRecoverRestoreTableDDLArg::~ObRecoverRestoreTableDDLArg() +{ + reset(); +} + +void ObRecoverRestoreTableDDLArg::reset() +{ + target_schema_.reset(); + src_tenant_id_ = common::OB_INVALID_ID; + src_table_id_ = common::OB_INVALID_ID; + ddl_task_id_ = common::OB_INVALID_ID; + allocator_.reset(); + ObDDLArg::reset(); +} + +bool ObRecoverRestoreTableDDLArg::is_valid() const +{ + return OB_INVALID_ID != target_schema_.get_tenant_id() + && OB_INVALID_ID != target_schema_.get_database_id() + && !target_schema_.get_table_name_str().empty() + && OB_INVALID_ID != src_tenant_id_ + && OB_INVALID_ID != src_table_id_ + && ddl_task_id_ > 0; +} + +int ObRecoverRestoreTableDDLArg::assign(const ObRecoverRestoreTableDDLArg &other) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!other.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(other)); + } else if (OB_FAIL(ObDDLArg::assign(other))) { + LOG_WARN("assign failed", K(ret)); + } else if (OB_FAIL(target_schema_.assign(other.target_schema_))) { + LOG_WARN("assign failed", K(ret)); + } else if (OB_FAIL(tz_info_.assign(other.tz_info_))) { + LOG_WARN("assign failed", K(ret)); + } else if (OB_FAIL(tz_info_wrap_.deep_copy(other.tz_info_wrap_))) { + LOG_WARN("assign failed", K(ret)); + } else { + src_tenant_id_ = other.src_tenant_id_; + src_table_id_ = other.src_table_id_; + ddl_task_id_ = ddl_task_id_; + char *tmp_ptr[ObNLSFormatEnum::NLS_MAX] = {}; + for (int64_t i = 0; OB_SUCC(ret) && i < ObNLSFormatEnum::NLS_MAX; ++i) { + const ObString &cur_str = other.nls_formats_[i]; + if (OB_ISNULL(tmp_ptr[i] = (char *)allocator_.alloc(cur_str.length()))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("alloc memory failed", K(ret), "size", cur_str.length()); + } else { + MEMCPY(tmp_ptr[i], cur_str.ptr(), cur_str.length()); + nls_formats_[i].assign_ptr(tmp_ptr[i], cur_str.length()); + } + } + if (OB_FAIL(ret)) { + for (int64_t i = 0; i < ObNLSFormatEnum::NLS_MAX; ++i) { + allocator_.free(tmp_ptr[i]); + } + } + } + return ret; +} + +OB_DEF_SERIALIZE(ObRecoverRestoreTableDDLArg) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), KPC(this)); + } else if (OB_FAIL(ObDDLArg::serialize(buf, buf_len, pos))) { + LOG_WARN("fail to serialize DDLArg", K(ret), K(buf_len), K(pos)); + } else { + LST_DO_CODE(OB_UNIS_ENCODE, + target_schema_, + src_tenant_id_, + src_table_id_, + ddl_task_id_, + tz_info_, + tz_info_wrap_); + for (int64_t i = 0; OB_SUCC(ret) && i < ObNLSFormatEnum::NLS_MAX; i++) { + if (OB_FAIL(nls_formats_[i].serialize(buf, buf_len, pos))) { + LOG_WARN("fail to serialize nls_formats_[i]", K(ret), K(nls_formats_[i])); + } + } + } + return ret; +} + +OB_DEF_DESERIALIZE(ObRecoverRestoreTableDDLArg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObDDLArg::deserialize(buf, data_len, pos))) { + LOG_WARN("fail to deserialize DDLArg", K(ret), K(data_len), K(pos)); + } else { + LST_DO_CODE(OB_UNIS_DECODE, + target_schema_, + src_tenant_id_, + src_table_id_, + ddl_task_id_, + tz_info_, + tz_info_wrap_); + ObString tmp_string; + char *tmp_ptr[ObNLSFormatEnum::NLS_MAX] = {}; + for (int64_t i = 0; OB_SUCC(ret) && i < ObNLSFormatEnum::NLS_MAX; i++) { + if (OB_FAIL(tmp_string.deserialize(buf, data_len, pos))) { + LOG_WARN("fail to deserialize nls_formats_", K(ret), K(i)); + } else if (OB_ISNULL(tmp_ptr[i] = (char *)allocator_.alloc(tmp_string.length()))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc memory!", K(ret)); + } else { + MEMCPY(tmp_ptr[i], tmp_string.ptr(), tmp_string.length()); + nls_formats_[i].assign_ptr(tmp_ptr[i], tmp_string.length()); + tmp_string.reset(); + } + } + if (OB_FAIL(ret)) { + for (int64_t i = 0; i < ObNLSFormatEnum::NLS_MAX; i++) { + allocator_.free(tmp_ptr[i]); + } + } + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObRecoverRestoreTableDDLArg) +{ + int ret = OB_SUCCESS; + int64_t len = 0; + if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret)); + } else { + len += ObDDLArg::get_serialize_size(); + LST_DO_CODE(OB_UNIS_ADD_LEN, + target_schema_, + src_tenant_id_, + src_table_id_, + ddl_task_id_, + tz_info_, + tz_info_wrap_); + if (OB_SUCC(ret)) { + for (int64_t i = 0; i < ObNLSFormatEnum::NLS_MAX; i++) { + len += nls_formats_[i].get_serialize_size(); + } + } + } + if (OB_FAIL(ret)) { + len = -1; + } + return len; +} + bool ObCreateHiddenTableArg::is_valid() const { return (OB_INVALID_ID != tenant_id_ @@ -522,7 +674,6 @@ bool ObCreateHiddenTableArg::is_valid() const && OB_INVALID_ID != dest_tenant_id_ && share::DDL_INVALID != ddl_type_); } - int ObCreateHiddenTableArg::assign(const ObCreateHiddenTableArg &arg) { int ret = OB_SUCCESS; @@ -547,7 +698,6 @@ int ObCreateHiddenTableArg::assign(const ObCreateHiddenTableArg &arg) } return ret; } - OB_DEF_SERIALIZE(ObCreateHiddenTableArg) { int ret = OB_SUCCESS; @@ -578,7 +728,6 @@ OB_DEF_SERIALIZE(ObCreateHiddenTableArg) } return ret; } - OB_DEF_DESERIALIZE(ObCreateHiddenTableArg) { int ret = OB_SUCCESS; @@ -618,7 +767,6 @@ OB_DEF_DESERIALIZE(ObCreateHiddenTableArg) } return ret; } - OB_DEF_SERIALIZE_SIZE(ObCreateHiddenTableArg) { int64_t len = 0; @@ -663,6 +811,7 @@ int ObCreateHiddenTableRes::assign(const ObCreateHiddenTableRes &res) schema_version_ = res.schema_version_; return ret; } + OB_SERIALIZE_MEMBER(ObCreateHiddenTableRes, tenant_id_, table_id_, @@ -671,6 +820,7 @@ OB_SERIALIZE_MEMBER(ObCreateHiddenTableRes, trace_id_, task_id_, schema_version_); + OB_SERIALIZE_MEMBER(ObStartRedefTableRes, task_id_, tenant_id_, @@ -3164,7 +3314,8 @@ bool ObCalcColumnChecksumRequestArg::is_valid() const { bool bret = OB_INVALID_ID != tenant_id_ && OB_INVALID_ID != target_table_id_ && OB_INVALID_VERSION != schema_version_ && execution_id_ >= 0 - && OB_INVALID_VERSION != snapshot_version_ && OB_INVALID_ID != source_table_id_ && task_id_ > 0; + && OB_INVALID_VERSION != snapshot_version_ && OB_INVALID_ID != source_table_id_ + && task_id_ > 0; for (int64_t i = 0; bret && i < calc_items_.count(); ++i) { bret = calc_items_.at(i).is_valid(); } @@ -4390,7 +4541,9 @@ OB_SERIALIZE_MEMBER((ObPhysicalRestoreTenantArg, ObCmdArg), encrypt_key_, kms_uri_, kms_encrypt_key_, - restore_timestamp_); + restore_timestamp_, + initiator_job_id_, + initiator_tenant_id_); ObPhysicalRestoreTenantArg::ObPhysicalRestoreTenantArg() : ObCmdArg(), @@ -4407,7 +4560,9 @@ ObPhysicalRestoreTenantArg::ObPhysicalRestoreTenantArg() encrypt_key_(), kms_uri_(), kms_encrypt_key_(), - restore_timestamp_() + restore_timestamp_(), + initiator_job_id_(0), + initiator_tenant_id_(OB_INVALID_TENANT_ID) { } @@ -4431,6 +4586,8 @@ int ObPhysicalRestoreTenantArg::assign(const ObPhysicalRestoreTenantArg &other) kms_uri_ = other.kms_uri_; kms_encrypt_key_ = other.kms_encrypt_key_; restore_timestamp_ = other.restore_timestamp_; + initiator_job_id_ = other.initiator_job_id_; + initiator_tenant_id_ = other.initiator_tenant_id_; } return ret; } @@ -6981,19 +7138,22 @@ OB_SERIALIZE_MEMBER(ObLogReqLoadProxyProgressRequest, agency_addr_seq_, principa OB_SERIALIZE_MEMBER(ObLogReqLoadProxyProgressResponse, err_, progress_); OB_SERIALIZE_MEMBER(ObDDLBuildSingleReplicaRequestArg, tenant_id_, ls_id_, source_tablet_id_, dest_tablet_id_, - source_table_id_, dest_schema_id_, schema_version_, snapshot_version_, ddl_type_, task_id_, execution_id_, - parallelism_, tablet_task_id_, data_format_version_, consumer_group_id_); + source_table_id_, dest_schema_id_, schema_version_, snapshot_version_, ddl_type_, task_id_, + execution_id_, parallelism_, tablet_task_id_, data_format_version_, consumer_group_id_, dest_tenant_id_, dest_ls_id_, dest_schema_version_); int ObDDLBuildSingleReplicaRequestArg::assign(const ObDDLBuildSingleReplicaRequestArg &other) { int ret = OB_SUCCESS; tenant_id_ = other.tenant_id_; ls_id_ = other.ls_id_; + dest_tenant_id_ = other.dest_tenant_id_; + dest_ls_id_ = other.dest_ls_id_; source_tablet_id_ = other.source_tablet_id_; dest_tablet_id_ = other.dest_tablet_id_; source_table_id_ = other.source_table_id_; dest_schema_id_ = other.dest_schema_id_; schema_version_ = other.schema_version_; + dest_schema_version_ = other.dest_schema_version_; snapshot_version_ = other.snapshot_version_; ddl_type_ = other.ddl_type_; task_id_ = other.task_id_; @@ -7016,23 +7176,30 @@ int ObDDLBuildSingleReplicaRequestResult::assign(const ObDDLBuildSingleReplicaRe return ret; } -OB_SERIALIZE_MEMBER(ObDDLBuildSingleReplicaResponseArg, tenant_id_, ls_id_, tablet_id_, source_table_id_, dest_schema_id_, ret_code_, snapshot_version_, schema_version_, task_id_, execution_id_, row_scanned_, row_inserted_); +OB_SERIALIZE_MEMBER(ObDDLBuildSingleReplicaResponseArg, tenant_id_, ls_id_, tablet_id_, + source_table_id_, dest_schema_id_, ret_code_, snapshot_version_, schema_version_, + task_id_, execution_id_, row_scanned_, row_inserted_, dest_tenant_id_, dest_ls_id_, dest_schema_version_, + server_addr_); int ObDDLBuildSingleReplicaResponseArg::assign(const ObDDLBuildSingleReplicaResponseArg &other) { int ret = OB_SUCCESS; tenant_id_ = other.tenant_id_; + dest_tenant_id_ = other.dest_tenant_id_; ls_id_ = other.ls_id_; + dest_ls_id_ = other.dest_ls_id_; tablet_id_ = other.tablet_id_; source_table_id_ = other.source_table_id_; dest_schema_id_ = other.dest_schema_id_; ret_code_ = other.ret_code_; snapshot_version_ = other.snapshot_version_; schema_version_ = other.schema_version_; + dest_schema_version_ = other.dest_schema_version_; task_id_ = other.task_id_; execution_id_ = other.execution_id_; row_scanned_ = other.row_scanned_; row_inserted_ = other.row_inserted_; + server_addr_ = other.server_addr_; return ret; } @@ -9267,6 +9434,36 @@ int ObTTLRequestArg::assign(const ObTTLRequestArg &other) return ret; } +OB_SERIALIZE_MEMBER(ObRecoverTableArg, tenant_id_, tenant_name_, import_arg_, restore_tenant_arg_, action_); + +ObRecoverTableArg::ObRecoverTableArg() + : tenant_id_(OB_INVALID_TENANT_ID), tenant_name_(), import_arg_(), restore_tenant_arg_(), action_() {} + +bool ObRecoverTableArg::is_valid() const +{ + bool ret = OB_INVALID_TENANT_ID != tenant_id_ + && (Action::CANCEL == action_ || !restore_tenant_arg_.restore_option_.empty()); + return ret; +} + +int ObRecoverTableArg::assign(const ObRecoverTableArg &that) +{ + int ret = OB_SUCCESS; + if (!that.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid recover table arg", K(ret), K(that)); + } else if (OB_FAIL(import_arg_.assign(that.import_arg_))) { + LOG_WARN("fail to assign", K(that.import_arg_)); + } else if (OB_FAIL(restore_tenant_arg_.assign(that.restore_tenant_arg_))) { + LOG_WARN("failed to assign restore tenant arg", K(ret)); + } else { + tenant_id_ = that.tenant_id_; + tenant_name_ = that.tenant_name_; + action_ = that.action_; + } + return ret; +} + OB_SERIALIZE_MEMBER(ObBroadcastConsensusVersionRes, ret_); int ObLoadBaselineRes::assign(const ObLoadBaselineRes &other) diff --git a/src/share/ob_rpc_struct.h b/src/share/ob_rpc_struct.h index c3f78a0bc5..1bd3ead7f0 100755 --- a/src/share/ob_rpc_struct.h +++ b/src/share/ob_rpc_struct.h @@ -72,6 +72,7 @@ #include "logservice/palf/log_meta_info.h"//LogConfigVersion #include "share/scn.h"//SCN #include "share/ob_server_table_operator.h" +#include "share/restore/ob_import_arg.h" namespace oceanbase { @@ -1233,6 +1234,7 @@ public: table_name_.reset(); database_name_.reset(); index_action_type_ = INVALID_ACTION; + ObDDLArg::reset(); } bool is_valid() const; virtual bool is_allow_when_upgrade() const; @@ -1557,7 +1559,6 @@ public: uint64_t tenant_id_; common::ObSArray rename_table_items_; }; - struct ObStartRedefTableArg final { OB_UNIS_VERSION(1); @@ -1755,6 +1756,38 @@ public: uint64_t tenant_id_; }; +struct ObRecoverRestoreTableDDLArg : public ObDDLArg +{ + OB_UNIS_VERSION(1); +public: + ObRecoverRestoreTableDDLArg() : + ObDDLArg(), + target_schema_(), + src_tenant_id_(common::OB_INVALID_ID), + src_table_id_(common::OB_INVALID_ID), + ddl_task_id_(common::OB_INVALID_ID), + tz_info_wrap_(), + nls_formats_{}, + allocator_("RestorTableDDL") + { + } + virtual ~ObRecoverRestoreTableDDLArg(); + int assign(const ObRecoverRestoreTableDDLArg &other); + bool is_valid() const; + void reset(); + TO_STRING_KV(K_(target_schema), K_(src_tenant_id), K_(src_table_id), K_(tz_info_wrap), + "nls_formats", common::ObArrayWrap(nls_formats_, common::ObNLSFormatEnum::NLS_MAX)); +public: + share::schema::ObTableSchema target_schema_; // with dest tenant_id, database_id, and table name. + uint64_t src_tenant_id_; + int64_t src_table_id_; + int64_t ddl_task_id_; + common::ObTimeZoneInfo tz_info_; + common::ObTimeZoneInfoWrap tz_info_wrap_; + common::ObString nls_formats_[common::ObNLSFormatEnum::NLS_MAX]; + common::ObArenaAllocator allocator_; +}; + struct ObCreateHiddenTableArg : public ObDDLArg { OB_UNIS_VERSION(1); @@ -1856,7 +1889,6 @@ public: share::ObTaskId trace_id_; }; - struct ObAlterTableArg : public ObDDLArg { OB_UNIS_VERSION(1); @@ -2404,6 +2436,7 @@ public: common::ObString nls_timestamp_tz_format_; ObSQLMode sql_mode_; common::ObAddr inner_sql_exec_addr_; + }; typedef ObCreateIndexArg ObAlterPrimaryArg; @@ -2701,10 +2734,10 @@ public: virtual bool is_allow_when_upgrade() const { return true; } void reset() { - ObDDLArg::reset(); tenant_id_ = OB_INVALID_ID; table_id_ = common::OB_INVALID_ID; table_name_.reset(); + ObDDLArg::reset(); } uint64_t tenant_id_; uint64_t database_id_; @@ -5158,7 +5191,9 @@ public: K_(encrypt_key), K_(kms_uri), K_(kms_encrypt_key), - K_(restore_timestamp)); + K_(restore_timestamp), + K_(initiator_job_id), + K_(initiator_tenant_id)); common::ObString tenant_name_; common::ObString uri_; @@ -5174,6 +5209,8 @@ public: common::ObString kms_uri_; common::ObString kms_encrypt_key_; common::ObString restore_timestamp_; + uint64_t initiator_job_id_; + uint64_t initiator_tenant_id_; }; struct ObServerZoneArg @@ -8594,18 +8631,19 @@ struct ObDDLBuildSingleReplicaRequestArg final public: ObDDLBuildSingleReplicaRequestArg() : tenant_id_(OB_INVALID_ID), ls_id_(), source_tablet_id_(), dest_tablet_id_(), source_table_id_(OB_INVALID_ID), dest_schema_id_(OB_INVALID_ID), - schema_version_(0), snapshot_version_(0), ddl_type_(0), task_id_(0), - parallelism_(0), execution_id_(-1), tablet_task_id_(0), data_format_version_(0), - consumer_group_id_(0) {} + schema_version_(0), snapshot_version_(0), ddl_type_(0), task_id_(0), parallelism_(0), execution_id_(-1), tablet_task_id_(0), + data_format_version_(0), consumer_group_id_(0), dest_tenant_id_(OB_INVALID_ID), dest_ls_id_(), dest_schema_version_(0) + {} bool is_valid() const { return OB_INVALID_ID != tenant_id_ && ls_id_.is_valid() && source_tablet_id_.is_valid() && dest_tablet_id_.is_valid() - && OB_INVALID_ID != source_table_id_ && OB_INVALID_ID != dest_schema_id_ && schema_version_ > 0 && snapshot_version_ > 0 - && task_id_ > 0 && parallelism_ > 0 && tablet_task_id_ > 0 && data_format_version_ > 0 && consumer_group_id_ >= 0; + && OB_INVALID_ID != source_table_id_ && OB_INVALID_ID != dest_schema_id_ && schema_version_ > 0 + && snapshot_version_ > 0 && dest_schema_version_ > 0 && task_id_ > 0 && parallelism_ > 0 && tablet_task_id_ > 0 + && data_format_version_ > 0 && consumer_group_id_ >= 0; } int assign(const ObDDLBuildSingleReplicaRequestArg &other); - TO_STRING_KV(K_(tenant_id), K_(ls_id), K_(source_tablet_id), K_(dest_tablet_id), - K_(source_table_id), K_(dest_schema_id), K_(schema_version), K_(snapshot_version), - K_(task_id), K_(parallelism), K_(execution_id), K_(tablet_task_id), K_(data_format_version), + TO_STRING_KV(K_(tenant_id), K_(dest_tenant_id), K_(ls_id), K_(dest_ls_id), K_(source_tablet_id), + K_(dest_tablet_id), K_(source_table_id), K_(dest_schema_id), K_(schema_version), K_(dest_schema_version), + K_(snapshot_version), K_(task_id), K_(parallelism), K_(execution_id), K_(tablet_task_id), K_(data_format_version), K_(consumer_group_id)); public: uint64_t tenant_id_; @@ -8623,6 +8661,9 @@ public: int64_t tablet_task_id_; int64_t data_format_version_; int64_t consumer_group_id_; + uint64_t dest_tenant_id_; + share::ObLSID dest_ls_id_; + int64_t dest_schema_version_; }; struct ObDDLBuildSingleReplicaRequestResult final @@ -8646,17 +8687,21 @@ struct ObDDLBuildSingleReplicaResponseArg final OB_UNIS_VERSION(1); public: ObDDLBuildSingleReplicaResponseArg() - : tenant_id_(OB_INVALID_ID), ls_id_(), tablet_id_(), source_table_id_(), dest_schema_id_(OB_INVALID_ID), - ret_code_(OB_SUCCESS), snapshot_version_(0), schema_version_(0), task_id_(0), execution_id_(-1), - row_scanned_(0), row_inserted_(0) + : tenant_id_(OB_INVALID_ID), ls_id_(), tablet_id_(), source_table_id_(OB_INVALID_ID), dest_schema_id_(OB_INVALID_ID), + ret_code_(OB_SUCCESS), snapshot_version_(0), schema_version_(0), task_id_(0), execution_id_(-1), row_scanned_(0), + row_inserted_(0), dest_tenant_id_(OB_INVALID_ID), dest_ls_id_(), dest_schema_version_(0), server_addr_() {} ~ObDDLBuildSingleReplicaResponseArg() = default; - bool is_valid() const { return OB_INVALID_ID != tenant_id_ && ls_id_.is_valid() && tablet_id_.is_valid() + bool is_valid() const { return OB_INVALID_ID != tenant_id_ && OB_INVALID_ID != dest_tenant_id_ + && ls_id_.is_valid() && dest_ls_id_.is_valid() && tablet_id_.is_valid() && OB_INVALID_ID != source_table_id_ && OB_INVALID_ID != dest_schema_id_ - && snapshot_version_ > 0 && schema_version_ > 0 && task_id_ > 0 && execution_id_ >= 0; } + && snapshot_version_ > 0 && schema_version_ > 0 && dest_schema_version_ > 0 + && task_id_ > 0 && execution_id_ >= 0; } int assign(const ObDDLBuildSingleReplicaResponseArg &other); - TO_STRING_KV(K_(tenant_id), K_(ls_id), K_(tablet_id), K_(source_table_id), K_(dest_schema_id), K_(ret_code), - K_(snapshot_version), K_(schema_version), K_(task_id), K_(execution_id), K_(row_scanned), K_(row_inserted)); + TO_STRING_KV(K_(tenant_id), K_(dest_tenant_id), K_(ls_id), K_(dest_ls_id), + K_(tablet_id), K_(source_table_id), K_(dest_schema_id), K_(ret_code), + K_(snapshot_version), K_(schema_version), K_(dest_schema_version), K_(task_id), + K_(execution_id), K_(row_scanned), K_(row_inserted), K_(server_addr)); public: uint64_t tenant_id_; share::ObLSID ls_id_; @@ -8670,6 +8715,10 @@ public: int64_t execution_id_; int64_t row_scanned_; int64_t row_inserted_; + uint64_t dest_tenant_id_; + share::ObLSID dest_ls_id_; + int64_t dest_schema_version_; + common::ObAddr server_addr_; }; struct ObLogReqLoadProxyRequest @@ -9954,6 +10003,30 @@ private: int ret_; }; +struct ObRecoverTableArg +{ +public: + OB_UNIS_VERSION(1); +public: + enum Action + { + INITIATE = 0, + CANCEL, + MAX + }; + ObRecoverTableArg(); + ~ObRecoverTableArg() {} + bool is_valid() const; + int assign(const ObRecoverTableArg &that); + TO_STRING_KV(K_(tenant_id), K_(tenant_name), K_(import_arg), K_(restore_tenant_arg), K_(action)); +public: + uint64_t tenant_id_; // tenant which is the table recover to. + common::ObString tenant_name_; + share::ObImportArg import_arg_; + ObPhysicalRestoreTenantArg restore_tenant_arg_; + Action action_; +}; + struct ObLoadBaselineRes { OB_UNIS_VERSION(1); diff --git a/src/share/parameter/ob_parameter_seed.ipp b/src/share/parameter/ob_parameter_seed.ipp index 157dd1e4a9..b0a50e87de 100755 --- a/src/share/parameter/ob_parameter_seed.ipp +++ b/src/share/parameter/ob_parameter_seed.ipp @@ -248,6 +248,10 @@ DEF_BOOL(_sql_insert_multi_values_split_opt, OB_CLUSTER_PARAMETER, "True", "True means that the split + batch optimization for inserting multiple rows of the insert values ​​statement can be done", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); +DEF_BOOL(_auto_drop_recovering_auxiliary_tenant, OB_CLUSTER_PARAMETER, "True", + "control whether to delete auxiliary tenant after recovering tables failed", + ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); + DEF_INT(_min_malloc_sample_interval, OB_CLUSTER_PARAMETER, "16", "[1, 10000]", "the min malloc times between two samples, " "which is not more than _max_malloc_sample_interval. " diff --git a/src/share/restore/ob_import_arg.cpp b/src/share/restore/ob_import_arg.cpp new file mode 100644 index 0000000000..9b44b45515 --- /dev/null +++ b/src/share/restore/ob_import_arg.cpp @@ -0,0 +1,153 @@ +/** + * 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 SHARE +#include "share/restore/ob_import_arg.h" + +namespace oceanbase +{ +namespace share +{ + +// ObImportArg +OB_SERIALIZE_MEMBER(ObImportArg, import_table_arg_, remap_table_arg_); + +const ObImportDatabaseArray &ObImportArg::get_import_database_array() const +{ + return import_table_arg_.get_import_database_array(); +} + +const ObImportTableArray &ObImportArg::get_import_table_array() const +{ + return import_table_arg_.get_import_table_array(); +} + +const ObImportPartitionArray &ObImportArg::get_import_partition_array() const +{ + return import_table_arg_.get_import_partition_array(); +} + +const ObRemapDatabaseArray &ObImportArg::get_remap_database_array() const +{ + return remap_table_arg_.get_remap_database_array(); +} + +const ObRemapTableArray &ObImportArg::get_remap_table_array() const +{ + return remap_table_arg_.get_remap_table_array(); +} + +const ObRemapPartitionArray &ObImportArg::get_remap_partition_array() const +{ + return remap_table_arg_.get_remap_partition_array(); +} + +const ObRemapTablegroupArray &ObImportArg::get_remap_tablegroup_array() const +{ + return remap_table_arg_.get_remap_tablegroup_array(); +} + +const ObRemapTablespaceArray &ObImportArg::get_remap_tablespace_array() const +{ + return remap_table_arg_.get_remap_tablespace_array(); +} + +void ObImportArg::reset() +{ + import_table_arg_.reset(); + remap_table_arg_.reset(); +} + +int ObImportArg::assign(const ObImportArg &other) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(import_table_arg_.assign(other.get_import_table_arg()))) { + LOG_WARN("failed to assign import table arg", K(ret), "import table arg", other.get_import_table_arg()); + } else if (OB_FAIL(remap_table_arg_.assign(other.get_remap_table_arg()))) { + LOG_WARN("failed to assign remap table arg", K(ret), "remap table arg", other.get_remap_table_arg()); + } + return ret; +} + +int ObImportArg::add_import_database(const ObImportDatabaseItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(import_table_arg_.add_database(item))) { + LOG_WARN("failed to add import database", K(ret), K(item)); + } + return ret; +} + +int ObImportArg::add_import_table(const ObImportTableItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(import_table_arg_.add_table(item))) { + LOG_WARN("failed to add import table", K(ret), K(item)); + } + return ret; +} + +int ObImportArg::add_import_parition(const ObImportPartitionItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(import_table_arg_.add_partition(item))) { + LOG_WARN("failed to add import partition", K(ret), K(item)); + } + return ret; +} + +int ObImportArg::add_remap_database(const ObRemapDatabaseItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_table_arg_.add_remap_database(item))) { + LOG_WARN("failed to add remap database", K(ret), K(item)); + } + return ret; +} + +int ObImportArg::add_remap_table(const ObRemapTableItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_table_arg_.add_remap_table(item))) { + LOG_WARN("failed to add remap table", K(ret), K(item)); + } + return ret; +} + +int ObImportArg::add_remap_parition(const ObRemapPartitionItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_table_arg_.add_remap_parition(item))) { + LOG_WARN("failed to add remap partition", K(ret), K(item)); + } + return ret; +} + +int ObImportArg::add_remap_tablegroup(const ObRemapTablegroupItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_table_arg_.add_remap_tablegroup(item))) { + LOG_WARN("failed to add remap tablegroup", K(ret), K(item)); + } + return ret; +} + +int ObImportArg::add_remap_tablespace(const ObRemapTablespaceItem &item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_table_arg_.add_remap_tablespace(item))) { + LOG_WARN("failed to add remap tablespace", K(ret), K(item)); + } + return ret; +} + +} +} \ No newline at end of file diff --git a/src/share/restore/ob_import_arg.h b/src/share/restore/ob_import_arg.h new file mode 100644 index 0000000000..58e7df362c --- /dev/null +++ b/src/share/restore/ob_import_arg.h @@ -0,0 +1,69 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_ARG_H +#define OCEANBASE_SHARE_IMPORT_ARG_H + +#include "share/restore/ob_import_table_arg.h" +#include "share/restore/ob_import_remap_arg.h" + +namespace oceanbase +{ +namespace share +{ + +class ObImportArg final +{ + OB_UNIS_VERSION(1); +public: + ObImportArg() : import_table_arg_(), remap_table_arg_() + {} + + const ObImportTableArg &get_import_table_arg() const { return import_table_arg_; } + const ObImportRemapArg &get_remap_table_arg() const { return remap_table_arg_; } + ObImportTableArg &get_import_table_arg() { return import_table_arg_; } + ObImportRemapArg &get_remap_table_arg() { return remap_table_arg_; } + + const ObImportDatabaseArray &get_import_database_array() const; + const ObImportTableArray &get_import_table_array() const; + const ObImportPartitionArray &get_import_partition_array() const; + + const ObRemapDatabaseArray &get_remap_database_array() const; + const ObRemapTableArray &get_remap_table_array() const; + const ObRemapPartitionArray &get_remap_partition_array() const; + const ObRemapTablegroupArray &get_remap_tablegroup_array() const; + const ObRemapTablespaceArray &get_remap_tablespace_array() const; + + void reset(); + int assign(const ObImportArg &other); + int add_import_database(const ObImportDatabaseItem &item); + int add_import_table(const ObImportTableItem &item); + int add_import_parition(const ObImportPartitionItem &item); + int add_remap_database(const ObRemapDatabaseItem &item); + int add_remap_table(const ObRemapTableItem &item); + int add_remap_parition(const ObRemapPartitionItem &item); + int add_remap_tablegroup(const ObRemapTablegroupItem &item); + int add_remap_tablespace(const ObRemapTablespaceItem &item); + + TO_STRING_KV(K_(import_table_arg), K_(remap_table_arg)); + +private: + ObImportTableArg import_table_arg_; + ObImportRemapArg remap_table_arg_; + DISALLOW_COPY_AND_ASSIGN(ObImportArg); +}; + + +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_item_format_provider.cpp b/src/share/restore/ob_import_item_format_provider.cpp new file mode 100644 index 0000000000..38474ace86 --- /dev/null +++ b/src/share/restore/ob_import_item_format_provider.cpp @@ -0,0 +1,96 @@ +/** + * 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 SHARE +#include "share/restore/ob_import_item_format_provider.h" +#include "lib/allocator/page_arena.h" + +namespace oceanbase +{ +namespace share +{ + +// ObIImportItemFormatProvider +int ObIImportItemFormatProvider::format_serialize(common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + char *serialize_buf = NULL; + int64_t serialize_pos = 0; + int64_t serialize_size = get_format_serialize_size(); + if (OB_ISNULL(serialize_buf = static_cast(allocator.alloc(serialize_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret), K(serialize_size)); + } else if (OB_FAIL(format_serialize(serialize_buf, serialize_size, serialize_pos))) { + LOG_WARN("fail to format serialize", K(ret)); + } else { + str.assign_ptr(serialize_buf, serialize_size); + } + return ret; +} + +// ObImportItemHexFormatImpl +int64_t ObImportItemHexFormatImpl::get_hex_format_serialize_size() const +{ + return 2 * get_serialize_size(); +} + +int ObImportItemHexFormatImpl::hex_format_serialize( + char *buf, const int64_t buf_len, int64_t &pos) const +{ + int ret = OB_SUCCESS; + common::ObArenaAllocator tmp_allocator; + char *serialize_buf = NULL; + int64_t serialize_pos = 0; + int64_t serialize_size = get_serialize_size(); + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (OB_ISNULL(serialize_buf = static_cast(tmp_allocator.alloc(serialize_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret), K(serialize_size)); + } else if (OB_FAIL(serialize(serialize_buf, serialize_size, serialize_pos))) { + LOG_WARN("fail to serialize", K(ret)); + } else if (OB_FAIL(hex_print(serialize_buf, serialize_pos, buf, buf_len, pos))) { + LOG_WARN("fail to print hex", K(ret), K(serialize_pos), K(buf_len), K(pos)); + } + + return ret; +} + + +int ObImportItemHexFormatImpl::hex_format_deserialize( + const char *buf, const int64_t data_len, int64_t &pos) +{ + int ret = OB_SUCCESS; + common::ObArenaAllocator tmp_allocator; + int64_t hex_size = data_len - pos; + char *deserialize_buf = NULL; + int64_t deserialize_size = hex_size / 2 + 1; + int64_t deserialize_pos = 0; + if ((NULL == buf) || (data_len < 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments.", K(ret), KP(buf), K(data_len)); + } else if (hex_size <= 0) { + // skip + } else if (OB_ISNULL(deserialize_buf = static_cast(tmp_allocator.alloc(deserialize_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret), K(deserialize_size)); + } else if (OB_FAIL(hex_to_cstr(buf, hex_size, deserialize_buf, deserialize_size))) { + LOG_WARN("fail to get cstr from hex", K(ret), K(hex_size), K(deserialize_size)); + } else if (OB_FAIL(deserialize(deserialize_buf, deserialize_size, deserialize_pos))) { + LOG_WARN("fail to deserialize", K(ret)); + } + + return ret; +} + +} +} \ No newline at end of file diff --git a/src/share/restore/ob_import_item_format_provider.h b/src/share/restore/ob_import_item_format_provider.h new file mode 100644 index 0000000000..a526730710 --- /dev/null +++ b/src/share/restore/ob_import_item_format_provider.h @@ -0,0 +1,63 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_ITEM_FORMAT_PROVIEDER_H_ +#define OCEANBASE_SHARE_IMPORT_ITEM_FORMAT_PROVIEDER_H_ + +#include +#include "lib/ob_define.h" +#include "lib/utility/ob_print_utils.h" +#include "lib/utility/ob_unify_serialize.h" +#include "lib/oblog/ob_log_module.h" + +namespace oceanbase +{ +namespace share +{ +// Format string used for show to user +class ObIImportItemFormatProvider +{ +public: + virtual int64_t get_format_serialize_size() const = 0; + + virtual int format_serialize(char *buf, const int64_t buf_len, int64_t &pos) const = 0; + + int format_serialize(common::ObIAllocator &allocator, common::ObString &str) const; +}; + + +// Format string used to persist to table +class ObIImportItemHexFormatProvider +{ +public: + virtual int64_t get_hex_format_serialize_size() const = 0; + + virtual int hex_format_serialize(char *buf, const int64_t buf_len, int64_t &pos) const = 0; + + virtual int hex_format_deserialize(const char *buf, const int64_t data_len, int64_t &pos) = 0; +}; + + +class ObImportItemHexFormatImpl : public ObIImportItemHexFormatProvider +{ + OB_UNIS_VERSION_PV(); // pure virtual +public: + virtual int64_t get_hex_format_serialize_size() const override; + + virtual int hex_format_serialize(char *buf, const int64_t buf_len, int64_t &pos) const override; + + virtual int hex_format_deserialize(const char *buf, const int64_t data_len, int64_t &pos) override; +}; + +} +} +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_partition_item.cpp b/src/share/restore/ob_import_partition_item.cpp new file mode 100644 index 0000000000..23a6c75266 --- /dev/null +++ b/src/share/restore/ob_import_partition_item.cpp @@ -0,0 +1,144 @@ +/** + * 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 SHARE +#include "share/restore/ob_import_partition_item.h" + +namespace oceanbase +{ +namespace share +{ + +// ObImportPartitionItem +OB_SERIALIZE_MEMBER(ObImportPartitionItem, mode_, database_name_, table_name_, partition_name_); + +void ObImportPartitionItem::reset() +{ + mode_ = common::OB_NAME_CASE_INVALID; + database_name_.reset(); + table_name_.reset(); + partition_name_.reset(); +} + +bool ObImportPartitionItem::is_valid() const +{ + return common::OB_NAME_CASE_INVALID != mode_ + && !database_name_.empty() + && !table_name_.empty() + && !partition_name_.empty(); +} + +bool ObImportPartitionItem::case_mode_equal(const ObIImportItem &other) const +{ + bool is_equal = false; + if (get_item_type() == other.get_item_type()) { + const ObImportPartitionItem &the_other = static_cast(other); + is_equal = ObCharset::case_mode_equal(mode_, database_name_, the_other.database_name_) + && ObCharset::case_mode_equal(mode_, table_name_, the_other.table_name_) + && ObCharset::case_mode_equal(mode_, partition_name_, the_other.partition_name_); + } + return is_equal; +} + +int64_t ObImportPartitionItem::get_format_serialize_size() const +{ + // For example, database name is SH, table name is SALES, and partition name is SALES_1998. + // The format string is `SH`.`SALES`:`SALES_1998`. + // Pre allocate twice the size to handle escape character. + int64_t size = 0; + size += database_name_.length() * 2; + size += table_name_.length() * 2; + size += partition_name_.length() * 2; + size += 8; + return size; +} + +int ObImportPartitionItem::format_serialize(char *buf, const int64_t buf_len, int64_t &pos) const +{ + // For example, database name is SH, table name is SALES, and partition name is SALES_1998. + // The format string is `SH`.`SALES`:`SALES_1998`. + int ret = OB_SUCCESS; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret)); + } else { + const bool skip_escape = false; + const bool do_oracle_mode_escape = false; + int64_t string_size = 0; + common::ObHexEscapeSqlStr hex_escape_db_name(database_name_, skip_escape, do_oracle_mode_escape); + common::ObHexEscapeSqlStr hex_escape_table_name(table_name_, skip_escape, do_oracle_mode_escape); + common::ObHexEscapeSqlStr hex_escape_part_name(partition_name_, skip_escape, do_oracle_mode_escape); + if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`"))) { + LOG_WARN("fail to format str", K(ret), K(pos), K(buf_len)); + } else if (OB_FALSE_IT(string_size = hex_escape_db_name.to_string(buf + pos, buf_len - pos))) { + } else if (OB_FALSE_IT(pos += string_size)) { + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`.`"))) { + LOG_WARN("fail to format str", K(ret), K(pos), K(buf_len)); + } else if (OB_FALSE_IT(string_size = hex_escape_table_name.to_string(buf + pos, buf_len - pos))) { + } else if (OB_FALSE_IT(pos += string_size)) { + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`:`"))) { + LOG_WARN("fail to format str", K(ret), K(pos), K(buf_len)); + } else if (OB_FALSE_IT(string_size = hex_escape_part_name.to_string(buf + pos, buf_len - pos))) { + } else if (OB_FALSE_IT(pos += string_size)) { + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`"))) { + LOG_WARN("fail to format str", K(ret), K(pos), K(buf_len)); + } + } + + return ret; +} + +int ObImportPartitionItem::deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src) +{ + int ret = OB_SUCCESS; + if (!src.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(src)); + } else if (get_item_type() != src.get_item_type()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("item type not match", K(ret), "src", src.get_item_type(), "dest", get_item_type()); + } else { + const ObImportPartitionItem &other = static_cast(src); + if (OB_FAIL(ob_write_string(allocator, other.database_name_, database_name_))) { + LOG_WARN("failed to copy item", K(ret), K(other)); + } else if (OB_FAIL(ob_write_string(allocator, other.table_name_, table_name_))) { + LOG_WARN("failed to copy item", K(ret), K(other)); + } else if (OB_FAIL(ob_write_string(allocator, other.partition_name_, partition_name_))) { + LOG_WARN("failed to copy item", K(ret), K(other)); + } else { + mode_ = other.mode_; + } + } + + return ret; +} + +int ObImportPartitionItem::assign(const ObImportPartitionItem &other) +{ + int ret = OB_SUCCESS; + if (!other.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(other)); + } else { + mode_ = other.mode_; + database_name_ = other.database_name_; + table_name_ = other.table_name_; + partition_name_ = other.partition_name_; + } + + return ret; +} + +} +} \ No newline at end of file diff --git a/src/share/restore/ob_import_partition_item.h b/src/share/restore/ob_import_partition_item.h new file mode 100644 index 0000000000..b7f8457e05 --- /dev/null +++ b/src/share/restore/ob_import_partition_item.h @@ -0,0 +1,70 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_PARTITION_ITEM_H +#define OCEANBASE_SHARE_IMPORT_PARTITION_ITEM_H + +#include "share/restore/ob_import_schema_item.h" + +namespace oceanbase +{ +namespace share +{ +struct ObImportPartitionItem final : public ObIImportSchemaItem +{ + OB_UNIS_VERSION(1); +public: + ObImportPartitionItem() : + ObIImportSchemaItem(ItemType::PARTITION), + database_name_(), + table_name_(), + partition_name_() + {} + + ObImportPartitionItem(common::ObNameCaseMode mode, const char *db_name, const int64_t db_len, + const char *table_name, const int64_t table_len, const char *part_name, const int64_t part_len) : + ObIImportSchemaItem(ItemType::PARTITION, mode), + database_name_(db_len, db_name), + table_name_(table_len, table_name), + partition_name_(part_len, part_name) + {} + + virtual void reset() override; + virtual bool is_valid() const override; + // ignore case + virtual bool case_mode_equal(const ObIImportItem &other) const override; + virtual int64_t get_format_serialize_size() const override; + virtual int format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const override; + + virtual int deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src) override; + int assign(const ObImportPartitionItem &other); + + TO_STRING_KV(K_(mode), K_(database_name), K_(table_name), K_(partition_name)); + +public: + // The following 3 names are all c_style string, and '\0' is not included + // into the length. + common::ObString database_name_; + common::ObString table_name_; + common::ObString partition_name_; +}; + + +using ObImportPartitionArray = ObImportSchemaItemArray; + + +} +} +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_remap_arg.cpp b/src/share/restore/ob_import_remap_arg.cpp new file mode 100644 index 0000000000..377c78b026 --- /dev/null +++ b/src/share/restore/ob_import_remap_arg.cpp @@ -0,0 +1,548 @@ +/** + * 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 SHARE +#include "share/restore/ob_import_remap_arg.h" +#include "share/schema/ob_multi_version_schema_service.h" +//#include "share/schema/ob_column_schema.h" + +namespace oceanbase +{ +namespace share +{ + +// ObImportRemapArg +OB_SERIALIZE_MEMBER( + ObImportRemapArg, + remap_database_array_, + remap_table_array_, + remap_partition_array_, + remap_tablegroup_array_, + remap_tablespace_array_); + +ObImportRemapArg::ObImportRemapArg() + : remap_database_array_(), + remap_table_array_(), + remap_partition_array_(), + remap_tablegroup_array_(), + remap_tablespace_array_() +{} + + +void ObImportRemapArg::reset() +{ + remap_database_array_.reset(); + remap_table_array_.reset(); + remap_partition_array_.reset(); + remap_tablegroup_array_.reset(); + remap_tablespace_array_.reset(); +} + +int ObImportRemapArg::assign(const ObImportRemapArg &other) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_database_array_.assign(other.get_remap_database_array()))) { + LOG_WARN("failed to assign database array", K(ret), "database array", other.get_remap_database_array()); + } else if (OB_FAIL(remap_table_array_.assign(other.get_remap_table_array()))) { + LOG_WARN("failed to assign table array", K(ret), "table array", other.get_remap_table_array()); + } else if (OB_FAIL(remap_partition_array_.assign(other.get_remap_partition_array()))) { + LOG_WARN("failed to assign partition array", K(ret), "partition array", other.get_remap_partition_array()); + } else if (OB_FAIL(remap_tablegroup_array_.assign(other.get_remap_tablegroup_array()))) { + LOG_WARN("failed to assign partition array", K(ret), "tablegroup array", other.get_remap_tablegroup_array()); + } else if (OB_FAIL(remap_tablespace_array_.assign(other.get_remap_tablespace_array()))) { + LOG_WARN("failed to assign partition array", K(ret), "tablespace array", other.get_remap_tablespace_array()); + } + return ret; +} + +int ObImportRemapArg::add_remap_database(const ObRemapDatabaseItem &item) +{ + int ret = OB_SUCCESS; + const ObImportDatabaseItem *dup_db_item = NULL; + common::ObArenaAllocator allocator; + ObString source_err_str; + ObString target_err_str; + + if (remap_database_array_.is_src_exist(item.src_, dup_db_item) + || remap_database_array_.is_target_exist(item.target_, dup_db_item)) { + if (OB_FAIL(dup_db_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_db_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input remap database is conflict", K(ret), K(item), KPC(dup_db_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (OB_FAIL(remap_database_array_.add_item(item))) { + LOG_WARN("failed to add remap database item", K(ret), K(item)); + } else { + LOG_INFO("add one remap database", K(item)); + } + return ret; +} + +int ObImportRemapArg::add_remap_table(const ObRemapTableItem &item) +{ + int ret = OB_SUCCESS; + const ObImportTableItem *dup_table_item = NULL; + common::ObArenaAllocator allocator; + ObString source_err_str; + ObString target_err_str; + + if (remap_table_array_.is_src_exist(item.src_, dup_table_item) + || remap_table_array_.is_target_exist(item.target_, dup_table_item)) { + if (OB_FAIL(dup_table_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_table_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input remap table is conflict", K(ret), K(item), KPC(dup_table_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (OB_FAIL(remap_table_array_.add_item(item))) { + LOG_WARN("failed to add remap table item", K(ret), K(item)); + } else { + LOG_INFO("add one remap table", K(item)); + } + return ret; +} + +int ObImportRemapArg::add_remap_parition(const ObRemapPartitionItem &item) +{ + int ret = OB_SUCCESS; + bool is_dup = false; + ObSqlString dup_item; + if (OB_FAIL(check_remap_partition_dup(item, is_dup, dup_item))) { + LOG_WARN("failed to check remap tablespace dup", K(ret), K(item)); + } else if (is_dup) { + ret = OB_ENTRY_EXIST; + LOG_WARN("duplicate partition", K(ret), K(item)); + LOG_USER_ERROR(OB_ENTRY_EXIST, dup_item.ptr()); + } else if (OB_FAIL(remap_partition_array_.add_item(item))) { + LOG_WARN("failed to add remap partition item", K(ret), K(item)); + } + return ret; +} + +int ObImportRemapArg::add_remap_tablegroup(const ObRemapTablegroupItem &item) +{ + int ret = OB_SUCCESS; + const ObImportTablegroupItem *dup_tg_item = NULL; + common::ObArenaAllocator allocator; + ObString source_err_str; + ObString target_err_str; + + if (remap_tablegroup_array_.is_src_exist(item.src_, dup_tg_item) + || remap_tablegroup_array_.is_target_exist(item.target_, dup_tg_item)) { + if (OB_FAIL(dup_tg_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_tg_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input remap tablegroup is conflict", K(ret), K(item), KPC(dup_tg_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (OB_FAIL(remap_tablegroup_array_.add_item(item))) { + LOG_WARN("failed to add remap tablegroup item", K(ret), K(item)); + } else { + LOG_INFO("add one remap tablegroup", K(item)); + } + return ret; +} + +int ObImportRemapArg::add_remap_tablespace(const ObRemapTablespaceItem &item) +{ + int ret = OB_SUCCESS; + const ObImportTablespaceItem *dup_ts_item = NULL; + common::ObArenaAllocator allocator; + ObString source_err_str; + ObString target_err_str; + + if (remap_tablespace_array_.is_src_exist(item.src_, dup_ts_item) + || remap_tablespace_array_.is_target_exist(item.target_, dup_ts_item)) { + if (OB_FAIL(dup_ts_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_ts_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input remap tablespace is conflict", K(ret), K(item), KPC(dup_ts_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (OB_FAIL(remap_tablespace_array_.add_item(item))) { + LOG_WARN("failed to add remap tablespace item", K(ret), K(item)); + } else { + LOG_INFO("add one remap tablespace", K(item)); + } + return ret; +} + +int ObImportRemapArg::check_remap_database_dup( + const ObRemapDatabaseItem &item, bool &is_dup, ObSqlString &dup_item) +{ + int ret = OB_SUCCESS; + const ObImportDatabaseItem *dup_item_ptr = nullptr; + is_dup = false; + if (remap_database_array_.is_src_exist(item.src_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate src database, %.*s", item.src_.name_.length(), item.src_.name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } else if (remap_database_array_.is_target_exist(item.target_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate target database, %.*s", item.target_.name_.length(), item.target_.name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} +int ObImportRemapArg::check_remap_table_dup( + const ObRemapTableItem &item, bool &is_dup, ObSqlString &dup_item) +{ + int ret = OB_SUCCESS; + const ObImportTableItem *dup_item_ptr = nullptr; + is_dup = false; + if (remap_table_array_.is_src_exist(item.src_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate src table, %.*s.%.*s", + item.src_.database_name_.length(), item.src_.database_name_.ptr(), + item.src_.table_name_.length(), item.src_.table_name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } else if (remap_table_array_.is_target_exist(item.target_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate target table, %.*s.%.*s", + item.target_.database_name_.length(), item.target_.database_name_.ptr(), + item.target_.table_name_.length(), item.target_.table_name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} +int ObImportRemapArg::check_remap_partition_dup( + const ObRemapPartitionItem &item, bool &is_dup, ObSqlString &dup_item) +{ + int ret = OB_SUCCESS; + const ObImportPartitionItem *src_dup_item_ptr = nullptr; + const ObImportTableItem *target_dup_item_ptr = nullptr; + is_dup = false; + if (remap_partition_array_.is_src_exist(item.src_, src_dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(src_dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate src partition, %.*s.%.*s:%.*s", + item.src_.database_name_.length(), item.src_.database_name_.ptr(), + item.src_.table_name_.length(), item.src_.table_name_.ptr(), + item.src_.partition_name_.length(), item.src_.partition_name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } else if (remap_table_array_.is_target_exist(item.target_, target_dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(target_dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate target table, %.*s.%.*s", + item.target_.database_name_.length(), item.target_.database_name_.ptr(), + item.target_.table_name_.length(), item.target_.table_name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} +int ObImportRemapArg::check_remap_tablegroup_dup( + const ObRemapTablegroupItem &item, bool &is_dup, ObSqlString &dup_item) +{ + int ret = OB_SUCCESS; + const ObImportTablegroupItem *dup_item_ptr = nullptr; + is_dup = false; + if (remap_tablegroup_array_.is_src_exist(item.src_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate src tablegroup, %.*s", item.src_.name_.length(), item.src_.name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } else if (remap_tablegroup_array_.is_target_exist(item.target_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate target tablegroup, %.*s", item.target_.name_.length(), item.target_.name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} +int ObImportRemapArg::check_remap_tablespace_dup( + const ObRemapTablespaceItem &item, bool &is_dup, ObSqlString &dup_item) +{ + int ret = OB_SUCCESS; + const ObImportTablespaceItem *dup_item_ptr = nullptr; + is_dup = false; + if (remap_tablespace_array_.is_src_exist(item.src_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate src tablegroup, %.*s", item.src_.name_.length(), item.src_.name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } else if (remap_tablespace_array_.is_target_exist(item.target_, dup_item_ptr)) { + is_dup = true; + if (OB_ISNULL(dup_item_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("dup item ptr must not be nullptr", K(ret)); + } else if (OB_FAIL(dup_item.assign_fmt("remap duplicate target tablegroup, %.*s", item.target_.name_.length(), item.target_.name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} + +int ObImportRemapArg::get_remap_database(const ObImportDatabaseItem &src, ObImportDatabaseItem &target) const +{ + int ret = OB_SUCCESS; + bool is_exist = false; + const ObImportDatabaseItem *p = NULL; + if (!remap_database_array_.is_remap_target_exist(src, p)) { + ret = OB_ENTRY_NOT_EXIST; + } else { + target = *p; + } + return ret; +} + +int ObImportRemapArg::get_remap_table(const ObImportTableItem &src, ObImportTableItem &target) const +{ + int ret = OB_SUCCESS; + bool is_exist = false; + const ObImportTableItem *p = NULL; + if (!remap_table_array_.is_remap_target_exist(src, p)) { + ret = OB_ENTRY_NOT_EXIST; + } else { + target = *p; + } + return ret; +} + +int ObImportRemapArg::get_remap_partition(const ObImportPartitionItem &src, ObImportTableItem &target) const +{ + int ret = OB_SUCCESS; + bool is_exist = false; + const ObImportTableItem *p = NULL; + if (!remap_partition_array_.is_remap_target_exist(src, p)) { + ret = OB_ENTRY_NOT_EXIST; + } else { + target = *p; + } + return ret; +} + +int ObImportRemapArg::get_remap_tablegroup(const ObImportTablegroupItem &src, ObImportTablegroupItem &target) const +{ + int ret = OB_SUCCESS; + bool is_exist = false; + const ObImportTablegroupItem *p = NULL; + if (!remap_tablegroup_array_.is_remap_target_exist(src, p)) { + ret = OB_ENTRY_NOT_EXIST; + } else { + target = *p; + } + return ret; +} + +int ObImportRemapArg::get_remap_tablespace(const ObImportTablespaceItem &src, ObImportTablespaceItem &target) const +{ + int ret = OB_SUCCESS; + bool is_exist = false; + const ObImportTablespaceItem *p = NULL; + if (!remap_tablespace_array_.is_remap_target_exist(src, p)) { + ret = OB_ENTRY_NOT_EXIST; + } else { + target = *p; + } + return ret; +} + +int ObImportRemapArg::get_remap_db_list_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_format_str_(remap_database_array_, allocator, str))) { + LOG_WARN("failed to get remap db list format str", K(ret), K_(remap_database_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_table_list_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_format_str_(remap_table_array_, allocator, str))) { + LOG_WARN("failed to get remap table list format str", K(ret), K_(remap_table_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_partition_list_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_format_str_(remap_partition_array_, allocator, str))) { + LOG_WARN("failed to get remap partition list format str", K(ret), K_(remap_partition_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_tablegroup_list_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_format_str_(remap_tablegroup_array_, allocator, str))) { + LOG_WARN("failed to get remap tablegroup list format str", K(ret), K_(remap_tablegroup_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_tablespace_list_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_format_str_(remap_tablespace_array_, allocator, str))) { + LOG_WARN("failed to get remap tablespace list format str", K(ret), K_(remap_tablespace_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_db_list_hex_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_hex_format_str_(remap_database_array_, allocator, str))) { + LOG_WARN("failed to get remap db list hex_format str", K(ret), K_(remap_database_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_table_list_hex_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_hex_format_str_(remap_table_array_, allocator, str))) { + LOG_WARN("failed to get remap table list hex_format str", K(ret), K_(remap_table_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_partition_list_hex_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_hex_format_str_(remap_partition_array_, allocator, str))) { + LOG_WARN("failed to get remap partition list hex format str", K(ret), K_(remap_partition_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_tablegroup_list_hex_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_hex_format_str_(remap_tablegroup_array_, allocator, str))) { + LOG_WARN("failed to get remap tablegroup list hex format str", K(ret), K_(remap_tablegroup_array)); + } + return ret; +} + +int ObImportRemapArg::get_remap_tablespace_list_hex_format_str( + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_remap_item_array_hex_format_str_(remap_tablespace_array_, allocator, str))) { + LOG_WARN("failed to get remap tablespace list hex format str", K(ret), K_(remap_tablespace_array)); + } + return ret; +} + +int ObImportRemapArg::remap_db_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_item_array_deserialize_hex_format_str_(remap_database_array_, str))) { + LOG_WARN("fail to deserialize hex format remap database array", K(ret), K(str)); + } + return ret; +} + +int ObImportRemapArg::remap_table_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_item_array_deserialize_hex_format_str_(remap_table_array_, str))) { + LOG_WARN("fail to deserialize hex format remap table array", K(ret), K(str)); + } + return ret; +} + +int ObImportRemapArg::remap_partition_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_item_array_deserialize_hex_format_str_(remap_partition_array_, str))) { + LOG_WARN("fail to deserialize hex format remap partition array", K(ret), K(str)); + } + return ret; +} + +int ObImportRemapArg::remap_tablegroup_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_item_array_deserialize_hex_format_str_(remap_tablegroup_array_, str))) { + LOG_WARN("fail to deserialize hex format remap tablegroup array", K(ret), K(str)); + } + return ret; +} + +int ObImportRemapArg::remap_tablespace_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(remap_item_array_deserialize_hex_format_str_(remap_tablespace_array_, str))) { + LOG_WARN("fail to deserialize hex format remap tablespace array", K(ret), K(str)); + } + return ret; +} + +} +} \ No newline at end of file diff --git a/src/share/restore/ob_import_remap_arg.h b/src/share/restore/ob_import_remap_arg.h new file mode 100644 index 0000000000..c394f96fd0 --- /dev/null +++ b/src/share/restore/ob_import_remap_arg.h @@ -0,0 +1,173 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_REMAP_ARG_H +#define OCEANBASE_SHARE_IMPORT_REMAP_ARG_H + +#include "share/restore/ob_remap_schema_item.h" + +namespace oceanbase +{ +namespace share +{ + +class ObImportRemapArg final +{ + OB_UNIS_VERSION(1); +public: + ObImportRemapArg(); + void reset(); + int assign(const ObImportRemapArg &other); + const ObRemapDatabaseArray &get_remap_database_array() const { return remap_database_array_; } + const ObRemapTableArray &get_remap_table_array() const { return remap_table_array_; } + const ObRemapPartitionArray &get_remap_partition_array() const { return remap_partition_array_; } + const ObRemapTablegroupArray &get_remap_tablegroup_array() const { return remap_tablegroup_array_; } + const ObRemapTablespaceArray &get_remap_tablespace_array() const { return remap_tablespace_array_; } + ObRemapDatabaseArray &get_remap_database_array() { return remap_database_array_; } + ObRemapTableArray &get_remap_table_array() { return remap_table_array_; } + ObRemapPartitionArray &get_remap_partition_array() { return remap_partition_array_; } + ObRemapTablegroupArray &get_remap_tablegroup_array() { return remap_tablegroup_array_; } + ObRemapTablespaceArray &get_remap_tablespace_array() { return remap_tablespace_array_; } + int add_remap_database(const ObRemapDatabaseItem &item); + int add_remap_table(const ObRemapTableItem &item); + int add_remap_parition(const ObRemapPartitionItem &item); + int add_remap_tablegroup(const ObRemapTablegroupItem &item); + int add_remap_tablespace(const ObRemapTablespaceItem &item); + int check_remap_database_dup(const ObRemapDatabaseItem &item, bool &is_dup, ObSqlString &dup_item_str); + int check_remap_table_dup(const ObRemapTableItem &item, bool &is_dup, ObSqlString &dup_item_str); + int check_remap_partition_dup(const ObRemapPartitionItem &item, bool &is_dup, ObSqlString &dup_item_str); + int check_remap_tablegroup_dup(const ObRemapTablegroupItem &item, bool &is_dup, ObSqlString &dup_item_str); + int check_remap_tablespace_dup(const ObRemapTablespaceItem &item, bool &is_dup, ObSqlString &dup_item_str); + int get_remap_database(const ObImportDatabaseItem &src, ObImportDatabaseItem &target) const; + int get_remap_table(const ObImportTableItem &src, ObImportTableItem &target) const; + int get_remap_partition(const ObImportPartitionItem &src, ObImportTableItem &target) const; + int get_remap_tablegroup(const ObImportTablegroupItem &src, ObImportTablegroupItem &target) const; + int get_remap_tablespace(const ObImportTablespaceItem &src, ObImportTablespaceItem &target) const; + + int get_remap_db_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_table_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_partition_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_tablegroup_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_tablespace_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_db_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_table_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_partition_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_tablegroup_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_remap_tablespace_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + + int remap_db_list_deserialize_hex_format_str(const common::ObString &str); + int remap_table_list_deserialize_hex_format_str(const common::ObString &str); + int remap_partition_list_deserialize_hex_format_str(const common::ObString &str); + int remap_tablegroup_list_deserialize_hex_format_str(const common::ObString &str); + int remap_tablespace_list_deserialize_hex_format_str(const common::ObString &str); + + TO_STRING_KV(K_(remap_database_array), K_(remap_table_array), K_(remap_partition_array), + K_(remap_tablegroup_array), K_(remap_tablespace_array)); + +private: + template + int get_remap_item_array_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const; + + template + int get_remap_item_array_hex_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const; + + template + int remap_item_array_deserialize_hex_format_str_( + T& array, + const common::ObString &str); + +private: + ObRemapDatabaseArray remap_database_array_; + ObRemapTableArray remap_table_array_; + ObRemapPartitionArray remap_partition_array_; + ObRemapTablegroupArray remap_tablegroup_array_; + ObRemapTablespaceArray remap_tablespace_array_; + DISALLOW_COPY_AND_ASSIGN(ObImportRemapArg); +}; + + +template +int ObImportRemapArg::get_remap_item_array_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + int64_t str_buf_len = array.get_format_serialize_size(); + char *str_buf = NULL; + + if (str_buf_len >= OB_MAX_LONGTEXT_LENGTH) { + ret = OB_SIZE_OVERFLOW; + SHARE_LOG(WARN, "format str is too long", K(ret), K(str_buf_len)); + } else if (OB_ISNULL(str_buf = static_cast(allocator.alloc(str_buf_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SHARE_LOG(WARN, "fail to alloc buf", K(ret), K(str_buf_len)); + } else if (OB_FAIL(array.format_serialize(str_buf, str_buf_len, pos))) { + SHARE_LOG(WARN, "fail to format remap item array", K(ret), K(pos), K(str_buf_len)); + } else { + str.assign_ptr(str_buf, str_buf_len); + SHARE_LOG(INFO, "get format remap item array str", K(str)); + } + + return ret; +} + +template +int ObImportRemapArg::get_remap_item_array_hex_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + int64_t str_buf_len = array.get_hex_format_serialize_size(); + char *str_buf = NULL; + + if (str_buf_len >= OB_MAX_LONGTEXT_LENGTH) { + ret = OB_SIZE_OVERFLOW; + SHARE_LOG(WARN, "hex format str is too long", K(ret), K(str_buf_len)); + } else if (OB_ISNULL(str_buf = static_cast(allocator.alloc(str_buf_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SHARE_LOG(WARN, "fail to alloc buf", K(ret), K(str_buf_len)); + } else if (OB_FAIL(array.hex_format_serialize(str_buf, str_buf_len, pos))) { + SHARE_LOG(WARN, "fail to hex format remap item array", K(ret), K(pos), K(str_buf_len)); + } else { + str.assign_ptr(str_buf, str_buf_len); + } + + return ret; +} + +template +int ObImportRemapArg::remap_item_array_deserialize_hex_format_str_( + T& array, + const common::ObString &str) +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + if (OB_FAIL(array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + SHARE_LOG(WARN, "fail to deserialize hex format remap item array", K(ret), K(str)); + } + return ret; +} + +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_schema_item.h b/src/share/restore/ob_import_schema_item.h new file mode 100644 index 0000000000..1717d3243d --- /dev/null +++ b/src/share/restore/ob_import_schema_item.h @@ -0,0 +1,179 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_SCHEMA_ITEM_H +#define OCEANBASE_SHARE_IMPORT_SCHEMA_ITEM_H + +#include "lib/ob_define.h" +#include "lib/utility/ob_print_utils.h" +#include "lib/utility/ob_unify_serialize.h" +#include "lib/oblog/ob_log_module.h" +#include "lib/container/ob_array_serialization.h" +#include "common/object/ob_object.h" +#include "share/restore/ob_import_item_format_provider.h" + +namespace oceanbase +{ +namespace share +{ +class ObIImportItem : public ObIImportItemFormatProvider +{ + OB_UNIS_VERSION_PV(); // pure virtual +public: + enum ItemType + { + DATABASE = 0, + TABLE, + PARTITION, + TABLEGROUP, + TABLESPACE, + REMAP + }; + +public: + explicit ObIImportItem(ItemType item_type) + : item_type_(item_type) + {} + virtual ~ObIImportItem() {} + virtual void reset() = 0; + virtual bool is_valid() const = 0; + virtual int deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src) = 0; + virtual bool operator==(const ObIImportItem &other) const = 0; + + ItemType get_item_type() const { return item_type_; } + + DECLARE_PURE_VIRTUAL_TO_STRING; + +private: + ItemType item_type_; +}; + + +struct ObIImportSchemaItem : public ObIImportItem +{ + ObIImportSchemaItem(ItemType item_type) : + ObIImportItem(item_type), + mode_(common::OB_NAME_CASE_INVALID) + {} + + ObIImportSchemaItem(ItemType item_type, common::ObNameCaseMode mode) : + ObIImportItem(item_type), + mode_(mode) + {} + + virtual bool case_mode_equal(const ObIImportItem &other) const = 0; + virtual bool operator==(const ObIImportItem &other) const override + { + return case_mode_equal(other); + } + +public: + common::ObNameCaseMode mode_; // for compare +}; + + +// Used for these schema which can be named with only one string, such as database, tablespace, tablegroup. +template +struct ObImportSimpleSchemaItem final : public ObIImportSchemaItem +{ + OB_UNIS_VERSION(1); +public: + ObImportSimpleSchemaItem() : + ObIImportSchemaItem(ITEM_TYPE), + name_() + {} + + ObImportSimpleSchemaItem(common::ObNameCaseMode mode, const char *name, const int64_t len) : + ObIImportSchemaItem(ITEM_TYPE, mode), + name_(len, name) + {} + + virtual void reset() override; + virtual bool is_valid() const override; + virtual bool case_mode_equal(const ObIImportItem &other) const override; + virtual int64_t get_format_serialize_size() const override; + virtual int format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const override; + + virtual int deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src); + + using ObIImportSchemaItem::format_serialize; + int assign(const ObImportSimpleSchemaItem &other); + + TO_STRING_KV(K_(mode), K_(name)); + +public: + // "name_" is c_style string, and '\0' is not included + // into the length. + common::ObString name_; +}; + + +template +class ObImportSchemaItemArray : public ObIImportItemFormatProvider, public ObImportItemHexFormatImpl +{ + OB_UNIS_VERSION(1); +public: + ObImportSchemaItemArray() {} + virtual int64_t get_format_serialize_size() const override; + virtual int format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const override; + + virtual void reset(); + + int assign(const ObImportSchemaItemArray &other); + int add_item(const T& item); + bool is_empty() const { return items_.empty(); } + const common::ObSArray &get_items() const { return items_; } + bool is_exist(const T& item) const; + bool is_exist(const T& item, const T *&out) const; + + template + int foreach(Operator &op) const + { + int ret = common::OB_SUCCESS; + ARRAY_FOREACH(items_, idx) { + const T &item = items_.at(idx); + if (OB_FAIL(op(item))) { + SHARE_LOG(WARN, "fail to do operator", K(ret), K(item)); + } + } + return ret; + } + + TO_STRING_KV(K_(items)); + +protected: + common::ObArenaAllocator allocator_; + common::ObSArray items_; +}; + + +using ObImportDatabaseItem = ObImportSimpleSchemaItem; +using ObImportTablegroupItem = ObImportSimpleSchemaItem; +using ObImportTablespaceItem = ObImportSimpleSchemaItem; +using ObImportDatabaseArray = ObImportSchemaItemArray; + +} +} + + +#ifndef INCLUDE_OB_IMPORT_SCHEMA_ITEM_HPP +#define INCLUDE_OB_IMPORT_SCHEMA_ITEM_HPP +#include "ob_import_schema_item.hpp" +#endif + +#endif diff --git a/src/share/restore/ob_import_schema_item.hpp b/src/share/restore/ob_import_schema_item.hpp new file mode 100644 index 0000000000..b06cbfa880 --- /dev/null +++ b/src/share/restore/ob_import_schema_item.hpp @@ -0,0 +1,325 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef INCLUDE_OB_IMPORT_SCHEMA_ITEM_HPP +#define INCLUDE_OB_IMPORT_SCHEMA_ITEM_HPP +#include "share/restore/ob_import_schema_item.h" +#include "lib/charset/ob_charset.h" +#endif + + +namespace oceanbase +{ +namespace share +{ +// ObImportSimpleSchemaItem +template +int64_t ObImportSimpleSchemaItem::get_serialize_size() const +{ + int64_t len = 0; + len += serialization::encoded_length_vi32(mode_); + len += name_.get_serialize_size(); + return len; +} + +template +int ObImportSimpleSchemaItem::serialize(SERIAL_PARAMS) const +{ + int ret = OB_SUCCESS; + int64_t new_pos = pos; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (OB_FAIL(serialization::encode_vi32(buf, buf_len, new_pos, mode_))) { + SHARE_LOG(WARN, "fail to serialize name case mode", K(ret)); + } else if (OB_FAIL(name_.serialize(buf, buf_len, new_pos))) { + SHARE_LOG(WARN, "serialize name failed", K(ret)); + } else { + pos = new_pos; + } + return ret; +} + +template +int ObImportSimpleSchemaItem::deserialize(DESERIAL_PARAMS) +{ + int ret = OB_SUCCESS; + int64_t new_pos = pos; + if ((NULL == buf) || (data_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(data_len)); + } else if (OB_FAIL(serialization::decode_vi32(buf, data_len, new_pos, ((int32_t *)(&mode_))))) { + SHARE_LOG(WARN, "fail to deserialize name case mode, ", K(ret)); + } else if (OB_FAIL(name_.deserialize(buf, data_len, new_pos))) { + SHARE_LOG(WARN, "deserialize name failed", K(ret)); + } else { + pos = new_pos; + } + return ret; +} + +template +void ObImportSimpleSchemaItem::reset() +{ + mode_ = common::OB_NAME_CASE_INVALID; + name_.reset(); +} + +template +bool ObImportSimpleSchemaItem::is_valid() const +{ + return common::OB_NAME_CASE_INVALID != mode_ + && !name_.empty() + && 0 < name_.length() + && MAX_SIZE >= name_.length(); +} + +template +bool ObImportSimpleSchemaItem::case_mode_equal(const ObIImportItem &other) const +{ + bool is_equal = false; + if (get_item_type() == other.get_item_type()) { + const ObImportSimpleSchemaItem &the_other = static_cast &>(other); + is_equal = ObCharset::case_mode_equal(mode_, name_, the_other.name_); + } + return is_equal; +} + +template +int64_t ObImportSimpleSchemaItem::get_format_serialize_size() const +{ + // For example, db1 is formated as `db1` + // Pre allocate twice the size to handle escape character. + return name_.length() * 2 + 2; +} + +template +int ObImportSimpleSchemaItem::format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const +{ + int ret = OB_SUCCESS; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret)); + } else { + const bool skip_escape = false; + const bool do_oracle_mode_escape = false; + int64_t string_size = 0; + common::ObHexEscapeSqlStr hex_escape_name(name_, skip_escape, do_oracle_mode_escape); + if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`"))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(pos), K(buf_len)); + } else if (OB_FALSE_IT(string_size = hex_escape_name.to_string(buf + pos, buf_len - pos))) { + } else if (OB_FALSE_IT(pos += string_size)) { + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`"))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(pos), K(buf_len)); + } + } + + return ret; +} + +template +int ObImportSimpleSchemaItem::deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src) +{ + int ret = OB_SUCCESS; + if (!src.is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), K(src)); + } else if (get_item_type() != src.get_item_type()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "item type not match", K(ret), "src", src.get_item_type(), "dest", get_item_type()); + } else { + const ObImportSimpleSchemaItem &other = static_cast &>(src); + if (OB_FAIL(ob_write_string(allocator, other.name_, name_))) { + SHARE_LOG(WARN, "failed to copy item", K(ret), K(other)); + } else { + mode_ = other.mode_; + } + } + + return ret; +} + +template +int ObImportSimpleSchemaItem::assign(const ObImportSimpleSchemaItem &other) +{ + int ret = OB_SUCCESS; + if (!other.is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), K(other)); + } else { + mode_ = other.mode_; + name_ = other.name_; + } + + return ret; +} + + +// ObImportSchemaItemArray +OB_DEF_SERIALIZE_SIZE(ObImportSchemaItemArray, template ) +{ + return items_.get_serialize_size(); +} + +OB_DEF_SERIALIZE(ObImportSchemaItemArray, template ) +{ + int ret = OB_SUCCESS; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), KP(buf), K(buf_len), K(pos)); + } else if (OB_FAIL(items_.serialize(buf, buf_len, pos))) { + SHARE_LOG(WARN, "failed to serialize", K(ret), KP(buf), K(buf_len), K(pos)); + } + return ret; +} + +OB_DEF_DESERIALIZE(ObImportSchemaItemArray, template ) +{ + int ret = OB_SUCCESS; + common::ObSArray tmp_items; + if ((NULL == buf) || (data_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), KP(buf), K(data_len), K(pos)); + } else if (OB_FALSE_IT(reset())) { + } else if (OB_FAIL(tmp_items.deserialize(buf, data_len, pos))) { + SHARE_LOG(WARN, "failed to deserialize", K(ret), KP(buf), K(data_len), K(pos)); + } + ARRAY_FOREACH(tmp_items, idx) { + const T &item = tmp_items.at(idx); + if (OB_FAIL(add_item(item))) { + SHARE_LOG(WARN, "failed to add item", K(ret), K(idx), K(item)); + } + } + return ret; +} + +template +int64_t ObImportSchemaItemArray::get_format_serialize_size() const +{ + int64_t size = 1; // include '\0' + ARRAY_FOREACH_NORET(items_, idx) { + const T &item = items_.at(idx); + // item is concatenated with ',' + size += 0 == idx ? 0 : 1; + size += item.get_format_serialize_size(); + } + + return size; +} + +template +int ObImportSchemaItemArray::format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const +{ + int ret = OB_SUCCESS; + int64_t new_pos = pos; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (items_.empty()) { + buf[new_pos++] = 0; + } + + ARRAY_FOREACH(items_, idx) { + const T &item = items_.at(idx); + // concatenated items with ',' + if (0 < idx && OB_FAIL(databuff_printf(buf, buf_len, new_pos, "%s", ","))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(new_pos), K(buf_len)); + } else if (OB_FAIL(item.format_serialize(buf, buf_len, new_pos))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(new_pos), K(buf_len)); + } + } + + if (OB_SUCC(ret)) { + pos = new_pos; + } + + return ret; +} + +template +void ObImportSchemaItemArray::reset() +{ + items_.reset(); + allocator_.reset(); +} + +template +int ObImportSchemaItemArray::assign(const ObImportSchemaItemArray &other) +{ + int ret = OB_SUCCESS; + const common::ObSArray &items = other.get_items(); + + reset(); + ARRAY_FOREACH(items, idx) { + const T &item = items.at(idx); + if (OB_FAIL(add_item(item))) { + SHARE_LOG(WARN, "failed to add item", K(ret), K(idx), K(item)); + } + } + + return ret; +} + +template +int ObImportSchemaItemArray::add_item(const T& item) +{ + int ret = OB_SUCCESS; + T tmp_item; + if (!item.is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), K(item)); + } else if (OB_FAIL(tmp_item.deep_copy(allocator_, item))) { + SHARE_LOG(WARN, "failed to copy item", K(ret), K(item)); + } else if (OB_FAIL(items_.push_back(tmp_item))) { + SHARE_LOG(WARN, "failed to push back item", K(ret), K(item)); + } else { + SHARE_LOG(INFO, "add one item", K(item)); + } + return ret; +} + + +template +bool ObImportSchemaItemArray::is_exist(const T& item) const +{ + bool ret = false; + const T *out = NULL; + ret = is_exist(item, out); + return ret; +} + +template +bool ObImportSchemaItemArray::is_exist(const T& item, const T *&out) const +{ + bool ret = false; + ARRAY_FOREACH_NORET(items_, idx) { + const T &tmp = items_.at(idx); + if (tmp == item) { + ret = true; + out = &tmp; + break; + } + } + return ret; +} + +} +} \ No newline at end of file diff --git a/src/share/restore/ob_import_table_arg.cpp b/src/share/restore/ob_import_table_arg.cpp new file mode 100644 index 0000000000..947be840ea --- /dev/null +++ b/src/share/restore/ob_import_table_arg.cpp @@ -0,0 +1,288 @@ +/** + * 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 SHARE +#include "share/restore/ob_import_table_arg.h" +#include "lib/string/ob_sql_string.h" +#include "lib/oblog/ob_log_module.h" + +namespace oceanbase +{ +namespace share +{ +// ObImportTableArg +OB_SERIALIZE_MEMBER( + ObImportTableArg, + is_import_all_, + database_array_, + table_array_, + partition_array_); + +ObImportTableArg::ObImportTableArg() + : is_import_all_(false), + database_array_(), + table_array_(), + partition_array_() +{} + +int ObImportTableArg::set_import_all() +{ + int ret = OB_SUCCESS; + is_import_all_ = true; + return ret; +} + +void ObImportTableArg::reset() +{ + is_import_all_ = false; + database_array_.reset(); + table_array_.reset(); + partition_array_.reset(); +} + +int ObImportTableArg::assign(const ObImportTableArg &other) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(database_array_.assign(other.get_import_database_array()))) { + LOG_WARN("failed to assign database array", K(ret), "database array", other.get_import_database_array()); + } else if (OB_FAIL(table_array_.assign(other.get_import_table_array()))) { + LOG_WARN("failed to assign database array", K(ret), "table array", other.get_import_table_array()); + } else if (OB_FAIL(partition_array_.assign(other.get_import_partition_array()))) { + LOG_WARN("failed to assign database array", K(ret), "partition array", other.get_import_partition_array()); + } else { + is_import_all_ = other.is_import_all(); + } + return ret; +} + +int ObImportTableArg::add_database(const ObImportDatabaseItem &item) +{ + int ret = OB_SUCCESS; + const ObImportDatabaseItem *dup_db_item = NULL; + const ObImportTableItem *dup_table_item = NULL; + common::ObArenaAllocator allocator; + ObString source_err_str; + ObString target_err_str; + + if (database_array_.is_exist(item, dup_db_item)) { + if (OB_FAIL(dup_db_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_db_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input database is conflict", K(ret), K(item), KPC(dup_db_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (table_array_.is_exist_table_in_database(item, dup_table_item)) { + if (OB_FAIL(dup_table_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_table_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input database is conflict", K(ret), K(item), KPC(dup_db_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (OB_FAIL(database_array_.add_item(item))) { + LOG_WARN("failed to add table", K(ret), K(item)); + } else { + LOG_INFO("add one database", K(item)); + } + return ret; +} + +int ObImportTableArg::add_table(const ObImportTableItem &item) +{ + int ret = OB_SUCCESS; + const ObImportDatabaseItem *dup_db_item = NULL; + const ObImportTableItem *dup_table_item = NULL; + common::ObArenaAllocator allocator; + ObString source_err_str; + ObString target_err_str; + const ObImportDatabaseItem db_item = item.get_database(); + if (database_array_.is_exist(db_item, dup_db_item)) { + if (OB_FAIL(dup_db_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_db_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input table is conflict", K(ret), K(item), KPC(dup_db_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (table_array_.is_exist(item, dup_table_item)) { + if (OB_FAIL(dup_table_item->format_serialize(allocator, source_err_str))) { + LOG_WARN("failed to format serialize", K(ret), KPC(dup_table_item)); + } else if (OB_FAIL(item.format_serialize(allocator, target_err_str))) { + LOG_WARN("failed to format serialize", K(ret), K(item)); + } else { + ret = OB_BACKUP_CONFLICT_VALUE; + LOG_WARN("input table is conflict", K(ret), K(item), KPC(dup_db_item)); + LOG_USER_ERROR(OB_BACKUP_CONFLICT_VALUE, target_err_str.length(), target_err_str.ptr(), source_err_str.length(), source_err_str.ptr()); + } + } else if (OB_FAIL(table_array_.add_item(item))) { + LOG_WARN("failed to add table", K(ret), K(item)); + } else { + LOG_INFO("add one table", K(item)); + } + return ret; +} + +int ObImportTableArg::add_partition(const ObImportPartitionItem &item) +{ + int ret = OB_SUCCESS; + bool is_dup = false; + ObSqlString dup_item; + if (OB_FAIL(check_partion_dup(item, is_dup, dup_item))) { + LOG_WARN("failed to check able dup", K(ret), K(item)); + } else if (is_dup) { + ret = OB_ENTRY_EXIST; + LOG_WARN("duplicate partition", K(ret), K(item)); + LOG_USER_ERROR(OB_ENTRY_EXIST, dup_item.ptr()); + } else if (OB_FAIL(partition_array_.add_item(item))) { + LOG_WARN("failed to add table", K(ret), K(item)); + } + return ret; +} + +int ObImportTableArg::check_database_dup(const ObImportDatabaseItem &item, bool &is_dup, ObSqlString &dup_item) const +{ + int ret = OB_SUCCESS; + is_dup = false; + if (database_array_.is_exist(item)) { + is_dup = true; + if (OB_FAIL(dup_item.assign_fmt("duplicate recover database, %.*s", item.name_.length(), item.name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} +int ObImportTableArg::check_table_dup(const ObImportTableItem &item, bool &is_dup, ObSqlString &dup_item) const +{ + int ret = OB_SUCCESS; + is_dup = false; + if (table_array_.is_exist(item)) { + is_dup = true; + if (OB_FAIL(dup_item.assign_fmt("duplicate recover table, %.*s.%.*s", + item.database_name_.length(), item.database_name_.ptr(), + item.table_name_.length(), item.table_name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} +int ObImportTableArg::check_partion_dup(const ObImportPartitionItem &item, bool &is_dup, ObSqlString &dup_item) const +{ + int ret = OB_SUCCESS; + is_dup = false; + if (partition_array_.is_exist(item)) { + is_dup = true; + if (OB_FAIL(dup_item.assign_fmt("duplicate recover partition, %.*s.%.*s:%.*s", + item.database_name_.length(), item.database_name_.ptr(), + item.table_name_.length(), item.table_name_.ptr(), + item.partition_name_.length(), item.partition_name_.ptr()))) { + LOG_WARN("failed to assign fmt", K(ret)); + } + } + return ret; +} + +int ObImportTableArg::get_db_list_format_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_item_array_format_str_(database_array_, allocator, str))) { + LOG_WARN("failed to get import database list format str", K(ret), K_(database_array)); + } + return ret; +} + +int ObImportTableArg::get_table_list_format_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_item_array_format_str_(table_array_, allocator, str))) { + LOG_WARN("failed to get import table list format str", K(ret), K_(table_array)); + } + return ret; +} + +int ObImportTableArg::get_partition_list_format_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_item_array_format_str_(partition_array_, allocator, str))) { + LOG_WARN("failed to get import partition list format str", K(ret), K_(partition_array)); + } + return ret; +} + + +int ObImportTableArg::get_db_list_hex_format_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_item_array_hex_format_str_(database_array_, allocator, str))) { + LOG_WARN("failed to get import database list hex format str", K(ret), K_(database_array)); + } + return ret; +} + +int ObImportTableArg::get_table_list_hex_format_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_item_array_hex_format_str_(table_array_, allocator, str))) { + LOG_WARN("failed to get import table list hex format str", K(ret), K_(table_array)); + } + return ret; +} + +int ObImportTableArg::get_partition_list_hex_format_str( + common::ObIAllocator &allocator, common::ObString &str) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_item_array_hex_format_str_(partition_array_, allocator, str))) { + LOG_WARN("failed to get import partition list hex format str", K(ret), K_(partition_array)); + } + return ret; +} + +int ObImportTableArg::db_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(item_array_deserialize_hex_format_str_(database_array_, str))) { + LOG_WARN("fail to deserialize hex format import database array", K(ret), K(str)); + } + return ret; +} + +int ObImportTableArg::table_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(item_array_deserialize_hex_format_str_(table_array_, str))) { + LOG_WARN("fail to deserialize hex format import table array", K(ret), K(str)); + } + return ret; +} + +int ObImportTableArg::partition_list_deserialize_hex_format_str(const common::ObString &str) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(item_array_deserialize_hex_format_str_(partition_array_, str))) { + LOG_WARN("fail to deserialize hex format import partition array", K(ret), K(str)); + } + return ret; +} + +} +} \ No newline at end of file diff --git a/src/share/restore/ob_import_table_arg.h b/src/share/restore/ob_import_table_arg.h new file mode 100644 index 0000000000..0e32f8b93d --- /dev/null +++ b/src/share/restore/ob_import_table_arg.h @@ -0,0 +1,156 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_TABLE_ARG_H +#define OCEANBASE_SHARE_IMPORT_TABLE_ARG_H + +#include "lib/ob_define.h" +#include "share/restore/ob_import_table_item.h" +#include "share/restore/ob_import_partition_item.h" + +namespace oceanbase +{ +namespace share +{ + +class ObImportTableArg final +{ + OB_UNIS_VERSION(1); +public: + ObImportTableArg(); + void reset(); + int assign(const ObImportTableArg &other); + int set_import_all(); + bool is_import_all() const { return is_import_all_; } + const ObImportDatabaseArray &get_import_database_array() const { return database_array_; } + const ObImportTableArray &get_import_table_array() const { return table_array_; } + const ObImportPartitionArray &get_import_partition_array() const { return partition_array_; } + ObImportDatabaseArray &get_import_database_array() { return database_array_; } + ObImportTableArray &get_import_table_array() { return table_array_; } + ObImportPartitionArray &get_import_partition_array() { return partition_array_; } + int add_database(const ObImportDatabaseItem &item); + int add_table(const ObImportTableItem &item); + int add_partition(const ObImportPartitionItem &item); + int check_database_dup(const ObImportDatabaseItem &item, bool &is_dup, ObSqlString &dup_item) const; + int check_table_dup(const ObImportTableItem &item, bool &is_dup, ObSqlString &dup_item) const; + int check_partion_dup(const ObImportPartitionItem &item, bool &is_dup, ObSqlString &dup_item) const; + int get_db_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_table_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_partition_list_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_db_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_table_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + int get_partition_list_hex_format_str(common::ObIAllocator &allocator, common::ObString &str) const; + + int db_list_deserialize_hex_format_str(const common::ObString &str); + int table_list_deserialize_hex_format_str(const common::ObString &str); + int partition_list_deserialize_hex_format_str(const common::ObString &str); + + TO_STRING_KV(K_(is_import_all), K_(database_array), K_(table_array), K_(partition_array)); + +private: + template + int get_item_array_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const; + + template + int get_item_array_hex_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const; + + template + int item_array_deserialize_hex_format_str_( + T& array, + const common::ObString &str); + +private: + bool is_import_all_; + ObImportDatabaseArray database_array_; + ObImportTableArray table_array_; + ObImportPartitionArray partition_array_; + DISALLOW_COPY_AND_ASSIGN(ObImportTableArg); +}; + + + +template +int ObImportTableArg::get_item_array_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + int64_t str_buf_len = array.get_format_serialize_size(); + char *str_buf = NULL; + + if (str_buf_len >= OB_MAX_LONGTEXT_LENGTH) { + ret = OB_SIZE_OVERFLOW; + SHARE_LOG(WARN, "format str is too long", K(ret), K(str_buf_len)); + } else if (OB_ISNULL(str_buf = static_cast(allocator.alloc(str_buf_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SHARE_LOG(WARN, "fail to alloc buf", K(ret), K(str_buf_len)); + } else if (OB_FAIL(array.format_serialize(str_buf, str_buf_len, pos))) { + SHARE_LOG(WARN, "fail to format import item array", K(ret), K(pos), K(str_buf_len)); + } else { + str.assign_ptr(str_buf, str_buf_len); + SHARE_LOG(INFO, "get format import item array str", K(str)); + } + + return ret; +} + +template +int ObImportTableArg::get_item_array_hex_format_str_( + const T &array, + common::ObIAllocator &allocator, + common::ObString &str) const +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + int64_t str_buf_len = array.get_hex_format_serialize_size(); + char *str_buf = NULL; + + if (str_buf_len >= OB_MAX_LONGTEXT_LENGTH) { + ret = OB_SIZE_OVERFLOW; + SHARE_LOG(WARN, "hex format str is too long", K(ret), K(str_buf_len)); + } else if (OB_ISNULL(str_buf = static_cast(allocator.alloc(str_buf_len)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SHARE_LOG(WARN, "fail to alloc buf", K(ret), K(str_buf_len)); + } else if (OB_FAIL(array.hex_format_serialize(str_buf, str_buf_len, pos))) { + SHARE_LOG(WARN, "fail to hex format import item array", K(ret), K(pos), K(str_buf_len)); + } else { + str.assign_ptr(str_buf, str_buf_len); + } + + return ret; +} + +template +int ObImportTableArg::item_array_deserialize_hex_format_str_( + T& array, + const common::ObString &str) +{ + int ret = OB_SUCCESS; + int64_t pos = 0; + if (OB_FAIL(array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + SHARE_LOG(WARN, "fail to deserialize hex format import item array", K(ret), K(str)); + } + return ret; +} + +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_table_item.cpp b/src/share/restore/ob_import_table_item.cpp new file mode 100644 index 0000000000..e6ee00991d --- /dev/null +++ b/src/share/restore/ob_import_table_item.cpp @@ -0,0 +1,164 @@ +/** + * 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 SHARE +#include "share/restore/ob_import_table_item.h" + +namespace oceanbase +{ +namespace share +{ + +// ObImportTableItem +OB_SERIALIZE_MEMBER(ObImportTableItem, mode_, database_name_, table_name_); + +void ObImportTableItem::reset() +{ + mode_ = common::OB_NAME_CASE_INVALID; + database_name_.reset(); + table_name_.reset(); +} + +bool ObImportTableItem::is_valid() const +{ + return common::OB_NAME_CASE_INVALID != mode_ + && !database_name_.empty() + && !table_name_.empty(); +} + +bool ObImportTableItem::case_mode_equal(const ObIImportItem &other) const +{ + bool is_equal = false; + if (get_item_type() == other.get_item_type()) { + const ObImportTableItem &dest = static_cast(other); + is_equal = ObCharset::case_mode_equal(mode_, database_name_, dest.database_name_) + && ObCharset::case_mode_equal(mode_, table_name_, dest.table_name_); + } + return is_equal; +} + +int64_t ObImportTableItem::get_format_serialize_size() const +{ + // For example, database name is SH, and table name is SALES. + // The format string is `SH`.`SALES`. + // Pre allocate twice the size to handle escape character. + int64_t size = 0; + size += database_name_.length() * 2; + size += table_name_.length() * 2; + size += 5; + return size; +} + +int ObImportTableItem::format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const +{ + // For example, database name is SH, and table name is SALES. + // The format string is `SH`.`SALES`. + int ret = OB_SUCCESS; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret)); + } else { + const bool skip_escape = false; + const bool do_oracle_mode_escape = false; + int64_t string_size = 0; + common::ObHexEscapeSqlStr hex_escape_db_name(database_name_, skip_escape, do_oracle_mode_escape); + common::ObHexEscapeSqlStr hex_escape_table_name(table_name_, skip_escape, do_oracle_mode_escape); + if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`"))) { + LOG_WARN("fail to format str", K(ret), K(pos), K(buf_len)); + } else if (OB_FALSE_IT(string_size = hex_escape_db_name.to_string(buf + pos, buf_len - pos))) { + } else if (OB_FALSE_IT(pos += string_size)) { + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`.`"))) { + LOG_WARN("fail to format str", K(ret), K(pos), K(buf_len)); + } else if (OB_FALSE_IT(string_size = hex_escape_table_name.to_string(buf + pos, buf_len - pos))) { + } else if (OB_FALSE_IT(pos += string_size)) { + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", "`"))) { + LOG_WARN("fail to format str", K(ret), K(pos), K(buf_len)); + } + } + + return ret; +} + +int ObImportTableItem::deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src) +{ + int ret = OB_SUCCESS; + if (!src.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(src)); + } else if (get_item_type() != src.get_item_type()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("item type not match", K(ret), "src", src.get_item_type(), "dest", get_item_type()); + } else { + const ObImportTableItem &other = static_cast(src); + if (OB_FAIL(ob_write_string(allocator, other.database_name_, database_name_))) { + LOG_WARN("failed to copy item", K(ret), K(other)); + } else if (OB_FAIL(ob_write_string(allocator, other.table_name_, table_name_))) { + LOG_WARN("failed to copy item", K(ret), K(other)); + } else { + mode_ = other.mode_; + } + } + + return ret; +} + +int ObImportTableItem::assign(const ObImportTableItem &other) +{ + int ret = OB_SUCCESS; + if (!other.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(other)); + } else { + mode_ = other.mode_; + database_name_ = other.database_name_; + table_name_ = other.table_name_; + } + + return ret; +} + +bool ObImportTableItem::is_database_equal(const ObImportDatabaseItem &other) const +{ + const ObImportDatabaseItem db_item = get_database(); + return db_item == other; +} + +const ObImportDatabaseItem ObImportTableItem::get_database() const +{ + ObImportDatabaseItem db_item; + db_item.name_ = database_name_; + db_item.mode_ = mode_; + return db_item; +} + +// ObImportTableArray +bool ObImportTableArray::is_exist_table_in_database(const ObImportDatabaseItem &db_item, const ObImportTableItem *&out) const +{ + bool ret = false; + ARRAY_FOREACH_NORET(items_, idx) { + const ObImportTableItem &table_item = items_.at(idx); + if (table_item.is_database_equal(db_item)) { + ret = true; + out = &table_item; + break; + } + } + return ret; +} + +} +} \ No newline at end of file diff --git a/src/share/restore/ob_import_table_item.h b/src/share/restore/ob_import_table_item.h new file mode 100644 index 0000000000..cb67be2d2a --- /dev/null +++ b/src/share/restore/ob_import_table_item.h @@ -0,0 +1,76 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_TABLE_ITEM_H +#define OCEANBASE_SHARE_IMPORT_TABLE_ITEM_H + +#include "share/restore/ob_import_schema_item.h" + +namespace oceanbase +{ +namespace share +{ + +struct ObImportTableItem final : public ObIImportSchemaItem +{ + OB_UNIS_VERSION(1); +public: + ObImportTableItem() : + ObIImportSchemaItem(ItemType::TABLE), + database_name_(), + table_name_() + {} + + ObImportTableItem(common::ObNameCaseMode mode, const char *db_name, const int64_t db_len, + const char *table_name, const int64_t table_len) : + ObIImportSchemaItem(ItemType::TABLE, mode), + database_name_(db_len, db_name), + table_name_(table_len, table_name) + {} + + virtual void reset() override; + virtual bool is_valid() const override; + virtual bool case_mode_equal(const ObIImportItem &other) const override; + // ignore case + virtual int64_t get_format_serialize_size() const override; + virtual int format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const override; + + virtual int deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src) override; + + using ObIImportSchemaItem::format_serialize; + int assign(const ObImportTableItem &other); + bool is_database_equal(const ObImportDatabaseItem &other) const; + const ObImportDatabaseItem get_database() const; + + TO_STRING_KV(K_(mode), K_(database_name), K_(table_name)); + +public: + // The following 2 names both are c_style string, and '\0' is not included + // into the length. + common::ObString database_name_; + common::ObString table_name_; +}; + + +class ObImportTableArray final : public ObImportSchemaItemArray +{ +public: + bool is_exist_table_in_database(const ObImportDatabaseItem &db_item, const ObImportTableItem *&out) const; +}; + + +} +} +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_table_persist_helper.cpp b/src/share/restore/ob_import_table_persist_helper.cpp new file mode 100644 index 0000000000..b03aad15b9 --- /dev/null +++ b/src/share/restore/ob_import_table_persist_helper.cpp @@ -0,0 +1,488 @@ +/** + * 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 SHARE + +#include "ob_import_table_persist_helper.h" +#include "ob_import_table_struct.h" +#include "share/inner_table/ob_inner_table_schema_constants.h" + +using namespace oceanbase; +using namespace share; +ObImportTableJobPersistHelper::ObImportTableJobPersistHelper() + : is_inited_(false), tenant_id_(), table_op_() +{ +} + +int ObImportTableJobPersistHelper::init(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + if(!is_sys_tenant(tenant_id) && !is_user_tenant(tenant_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid tenant id", K(ret), K(tenant_id)); + } else if (OB_FAIL(table_op_.init(OB_ALL_IMPORT_TABLE_JOB_TNAME, *this))) { + LOG_WARN("failed to init table op", K(ret)); + } else { + tenant_id_ = tenant_id; + is_inited_ = true; + } + return ret; +} + + +int ObImportTableJobPersistHelper::insert_import_table_job( + common::ObISQLClient &proxy, const ObImportTableJob &job) const +{ + int ret = OB_SUCCESS; + int64_t affect_rows = 0; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.insert_or_update_row(proxy, job, affect_rows))) { + LOG_WARN("failed to insert or update row", K(ret), K(job)); + } + return ret; +} + +int ObImportTableJobPersistHelper::get_import_table_job( + common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id, ObImportTableJob &job) const +{ + int ret = OB_SUCCESS; + ObImportTableJob::Key key; + key.tenant_id_ = tenant_id; + key.job_id_ = job_id; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableJobPersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.get_row(proxy, false, key, job))) { + LOG_WARN("failed to get row", KR(ret), K(key)); + } + return ret; +} + +int ObImportTableJobPersistHelper::get_all_import_table_jobs( + common::ObISQLClient &proxy, common::ObIArray &jobs) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(); + jobs.reset(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_IMPORT_TABLE_JOB_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else { + while (OB_SUCC(ret)) { + ObImportTableJob job; + job.reset(); + if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + break; + } else { + LOG_WARN("failed to get next row", K(ret)); + } + } else if (OB_FAIL(job.parse_from(*result))) { + LOG_WARN("failed to parse job result", K(ret)); + } else if (OB_FAIL(jobs.push_back(job))) { + LOG_WARN("failed to push back job", K(ret), K(job)); + } + } + } + } + } + LOG_INFO("get targets", K(ret), K(jobs), K(exec_tenant_id), K(sql)); + return ret; +} + +int ObImportTableJobPersistHelper::advance_status( + common::ObISQLClient &proxy, const ObImportTableJob &job, const ObImportTableJobStatus &next_status) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = -1; + ObDMLSqlSplicer dml; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableJobPersistHelper not init", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_JOB_ID, job.get_job_id()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, job.get_tenant_id()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_RESULT, job.get_result().get_result_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_COMMENT, job.get_result().get_comment()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_END_TS, job.get_end_ts()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, next_status.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.splice_update_sql(OB_ALL_IMPORT_TABLE_JOB_TNAME, sql))) { + LOG_WARN("failed to splice update sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" and %s='%s'", OB_STR_STATUS, job.get_status().get_str()))) { + LOG_WARN("failed to append fmt", K(ret)); + } else if (OB_FAIL(proxy.write(get_exec_tenant_id(), sql.ptr(), affected_rows))) { + LOG_WARN("failed to write", K(ret), K(sql)); + } + return ret; +} + +int ObImportTableJobPersistHelper::force_cancel_import_job(common::ObISQLClient &proxy) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = -1; + ObDMLSqlSplicer dml; + ObRecoverTableStatus status(ObRecoverTableStatus::CANCELING); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableJobPersistHelper not init", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, status.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.splice_update_sql(OB_ALL_IMPORT_TABLE_JOB_TNAME, sql))) { + LOG_WARN("failed to splice update sql", K(ret)); + } else if (OB_FAIL(proxy.write(get_exec_tenant_id(), sql.ptr(), affected_rows))) { + LOG_WARN("failed to exec sql", K(ret), K(sql)); + } else { + LOG_INFO("success cancel import job", K(tenant_id_)); + } + return ret; +} + +int ObImportTableJobPersistHelper::move_import_job_to_history( + common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = -1; + if (tenant_id == OB_INVALID_TENANT_ID || job_id <= 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(tenant_id), K(job_id)); + } else if (OB_FAIL(sql.assign_fmt( + "insert into %s select * from %s where %s=%lu", + OB_ALL_IMPORT_TABLE_JOB_HISTORY_TNAME, OB_ALL_IMPORT_TABLE_JOB_TNAME, + OB_STR_JOB_ID, job_id))) { + LOG_WARN("failed to init sql", K(ret)); + } else if (OB_FAIL(proxy.write(gen_meta_tenant_id(tenant_id), sql.ptr(), affected_rows))) { + LOG_WARN("failed to exec sql", K(ret), K(sql)); + } else if (OB_FALSE_IT(sql.reset())) { + } else if (OB_FAIL(sql.assign_fmt("delete from %s where %s=%lu", + OB_ALL_IMPORT_TABLE_JOB_TNAME, OB_STR_JOB_ID, job_id))) { + LOG_WARN("failed to init sql", K(ret)); + } else if (OB_FAIL(proxy.write(gen_meta_tenant_id(tenant_id), sql.ptr(), affected_rows))) { + LOG_WARN("failed to exec sql", K(ret), K(sql)); + } else { + LOG_INFO("succeed move import job to history table", K(tenant_id), K(job_id)); + } + return ret; +} + +int ObImportTableJobPersistHelper::get_import_table_job_history_by_initiator(common::ObISQLClient &proxy, + const uint64_t initiator_tenant_id, const uint64_t initiator_job_id, ObImportTableJob &job) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_IMPORT_TABLE_JOB_HISTORY_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" where initiator_tenant_id =%lu and initiator_job_id=%ld", initiator_tenant_id, initiator_job_id))) { + LOG_WARN("failed to append sql", K(ret), K(sql), K(initiator_tenant_id), K(initiator_job_id)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + ret = OB_ENTRY_NOT_EXIST; + } else { + LOG_WARN("failed to get next", K(ret)); + } + } else if (OB_FAIL(job.parse_from(*result))) { + LOG_WARN("failed to parse row", K(ret)); + } else if (OB_ITER_END != result->next()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("multi value exist", K(ret), K(sql), K(exec_tenant_id)); + } else { + ret = OB_SUCCESS; + } + } + } + LOG_INFO("get import table job history", K(ret), K(job), K(sql)); + return ret; +} + +int ObImportTableJobPersistHelper::get_import_table_job_by_initiator(common::ObISQLClient &proxy, + const uint64_t initiator_tenant_id, const uint64_t initiator_job_id, ObImportTableJob &job) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_IMPORT_TABLE_JOB_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" where initiator_tenant_id =%lu and initiator_job_id=%ld", initiator_tenant_id, initiator_job_id))) { + LOG_WARN("failed to append sql", K(ret), K(sql), K(initiator_tenant_id), K(initiator_job_id)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + ret = OB_ENTRY_NOT_EXIST; + } else { + LOG_WARN("failed to get next", K(ret)); + } + } else if (OB_FAIL(job.parse_from(*result))) { + LOG_WARN("failed to parse row", K(ret)); + } else if (OB_ITER_END != result->next()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("multi value exist", K(ret), K(sql), K(exec_tenant_id)); + } else { + ret = OB_SUCCESS; + } + } + } + LOG_INFO("get import table job", K(ret), K(job), K(sql)); + return ret; +} + +int ObImportTableJobPersistHelper::report_import_job_statistics( + common::ObISQLClient &proxy, const ObImportTableJob &job) const +{ + int ret = OB_SUCCESS; + int64_t affected_rows = 0; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableJobPersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.update_row(proxy, job, affected_rows))) { + LOG_WARN("failed to compare and swap status", K(ret), K(job)); + } + return ret; +} + +ObImportTableTaskPersistHelper::ObImportTableTaskPersistHelper() + : is_inited_(false), tenant_id_(), table_op_() +{ +} + +int ObImportTableTaskPersistHelper::init(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + if(!is_sys_tenant(tenant_id) && !is_user_tenant(tenant_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid tenant id", K(ret), K(tenant_id)); + } else if (OB_FAIL(table_op_.init(OB_ALL_IMPORT_TABLE_TASK_TNAME, *this))) { + LOG_WARN("failed to init table op", K(ret)); + } else { + tenant_id_ = tenant_id; + is_inited_ = true; + } + return ret; +} + + +int ObImportTableTaskPersistHelper::insert_import_table_task( + common::ObISQLClient &proxy, const ObImportTableTask &task) const +{ + int ret = OB_SUCCESS; + int64_t affect_rows = 0; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.insert_or_update_row(proxy, task, affect_rows))) { + LOG_WARN("failed to insert or update row", K(ret), K(task)); + } + return ret; +} + +int ObImportTableTaskPersistHelper::get_recover_table_task( + common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t task_id, ObImportTableTask &task) const +{ + int ret = OB_SUCCESS; + ObImportTableTask::Key key; + key.tenant_id_ = tenant_id; + key.task_id_ = task_id; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableTaskPersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.get_row(proxy, false, key, task))) { + LOG_WARN("failed to get row", KR(ret), K(key)); + } + return ret; +} + +int ObImportTableTaskPersistHelper::advance_status( + common::ObISQLClient &proxy, const ObImportTableTask &task, const ObImportTableTaskStatus &next_status) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = -1; + ObDMLSqlSplicer dml; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableTaskPersistHelper not init", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TASK_ID, task.get_task_id()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, task.get_tenant_id()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_RESULT, task.get_result().get_result_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_COMMENT, task.get_result().get_comment()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, next_status.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.splice_update_sql(OB_ALL_IMPORT_TABLE_TASK_TNAME, sql))) { + LOG_WARN("failed to splice update sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" and %s='%s'", OB_STR_STATUS, task.get_status().get_str()))) { + LOG_WARN("failed to append fmt", K(ret)); + } else if (OB_FAIL(proxy.write(get_exec_tenant_id(), sql.ptr(), affected_rows))) { + LOG_WARN("failed to write", K(ret), K(sql)); + } + return ret; +} + +int ObImportTableTaskPersistHelper::get_all_import_table_tasks_by_initiator(common::ObISQLClient &proxy, + const ObImportTableJob &job, common::ObIArray &tasks) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(); + tasks.reset(); + const int64_t job_id = job.get_job_id(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_IMPORT_TABLE_TASK_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" where job_id=%ld", job_id))) { + LOG_WARN("failed to append sql", K(ret), K(sql), K(job_id)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else { + while (OB_SUCC(ret)) { + ObImportTableTask task; + if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + break; + } else { + LOG_WARN("failed to get next row", K(ret)); + } + } else if (OB_FAIL(task.parse_from(*result))) { + LOG_WARN("failed to parse job result", K(ret)); + } else if (OB_FAIL(tasks.push_back(task))) { + LOG_WARN("failed to push back job", K(ret), K(task)); + } + } + } + } + } + LOG_INFO("get import table tasks", K(ret), K(tasks)); + return ret; +} + +int ObImportTableTaskPersistHelper::get_one_not_finish_task_by_initiator( + common::ObISQLClient &proxy, const ObImportTableJob &job, bool &all_finish, ObImportTableTask &task) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(); + task.reset(); + all_finish = false; + const int64_t job_id = job.get_job_id(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_IMPORT_TABLE_TASK_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" where job_id=%ld and status != 'FINISH' order by task_id limit 1", job_id))) { + LOG_WARN("failed to append sql", K(ret), K(sql), K(job_id)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + all_finish = true; + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to get next", K(ret)); + } + } else if (OB_FAIL(task.parse_from(*result))) { + LOG_WARN("failed to parse from result", K(ret)); + } + } + } + LOG_INFO("get import table tasks", K(ret), K(task)); + return ret; +} + +int ObImportTableTaskPersistHelper::move_import_task_to_history( + common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = -1; + if (tenant_id == OB_INVALID_TENANT_ID || job_id <= 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(tenant_id), K(job_id)); + } else if (OB_FAIL(sql.assign_fmt( + "insert into %s select * from %s where %s=%lu", + OB_ALL_IMPORT_TABLE_TASK_HISTORY_TNAME, OB_ALL_IMPORT_TABLE_TASK_TNAME, + OB_STR_JOB_ID, job_id))) { + LOG_WARN("failed to init sql", K(ret)); + } else if (OB_FAIL(proxy.write(gen_meta_tenant_id(tenant_id), sql.ptr(), affected_rows))) { + LOG_WARN("failed to exec sql", K(ret), K(sql)); + } else if (OB_FALSE_IT(sql.reset())) { + } else if (OB_FAIL(sql.assign_fmt("delete from %s where %s=%lu", + OB_ALL_IMPORT_TABLE_TASK_TNAME, OB_STR_JOB_ID, job_id))) { + LOG_WARN("failed to init sql", K(ret)); + } else if (OB_FAIL(proxy.write(gen_meta_tenant_id(tenant_id), sql.ptr(), affected_rows))) { + LOG_WARN("failed to exec sql", K(ret), K(sql)); + } else { + LOG_INFO("succeed move import task to history table", K(tenant_id), K(job_id)); + } + return ret; +} + +int ObImportTableTaskPersistHelper::report_import_task_statistics( + common::ObISQLClient &proxy, const ObImportTableTask &task) const +{ + int ret = OB_SUCCESS; + int64_t affected_rows = 0; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObImportTableTaskPersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.update_row(proxy, task, affected_rows))) { + LOG_WARN("failed to compare and swap status", K(ret), K(task)); + } + return ret; +} \ No newline at end of file diff --git a/src/share/restore/ob_import_table_persist_helper.h b/src/share/restore/ob_import_table_persist_helper.h new file mode 100644 index 0000000000..bb396ee55c --- /dev/null +++ b/src/share/restore/ob_import_table_persist_helper.h @@ -0,0 +1,89 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_TABLE_PERSIST_HELPER_H +#define OCEANBASE_SHARE_IMPORT_TABLE_PERSIST_HELPER_H + +#include "lib/ob_define.h" +#include "share/ob_inner_table_operator.h" +#include "share/restore/ob_import_table_struct.h" + +namespace oceanbase +{ +namespace share +{ + +class ObImportTableJobPersistHelper final : public ObIExecTenantIdProvider +{ +public: + ObImportTableJobPersistHelper(); + virtual ~ObImportTableJobPersistHelper() {} + int init(const uint64_t tenant_id); + void reset() { is_inited_ = false; } + uint64_t get_exec_tenant_id() const override { return gen_meta_tenant_id(tenant_id_); } + int insert_import_table_job(common::ObISQLClient &proxy, const ObImportTableJob &job) const; + + int get_import_table_job(common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id, + ObImportTableJob &job) const; + int get_all_import_table_jobs(common::ObISQLClient &proxy, common::ObIArray &jobs) const; + int advance_status(common::ObISQLClient &proxy, + const ObImportTableJob &job, const ObImportTableJobStatus &next_status) const; + + int report_import_job_statistics(common::ObISQLClient &proxy, const ObImportTableJob &job) const; + int move_import_job_to_history(common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id) const; + int get_import_table_job_history_by_initiator(common::ObISQLClient &proxy, + const uint64_t initiator_tenant_id, const uint64_t initiator_job_id, ObImportTableJob &job) const; + int get_import_table_job_by_initiator(common::ObISQLClient &proxy, + const uint64_t initiator_tenant_id, const uint64_t initiator_job_id, ObImportTableJob &job) const; + int force_cancel_import_job(common::ObISQLClient &proxy) const; + TO_STRING_KV(K_(is_inited), K_(tenant_id)); +private: + DISALLOW_COPY_AND_ASSIGN(ObImportTableJobPersistHelper); + bool is_inited_; + uint64_t tenant_id_; // sys or user tenant id + ObInnerTableOperator table_op_; +}; + +class ObImportTableTaskPersistHelper final : public ObIExecTenantIdProvider +{ +public: + ObImportTableTaskPersistHelper(); + virtual ~ObImportTableTaskPersistHelper() {} + int init(const uint64_t tenant_id); + void reset() { is_inited_ = false; } + uint64_t get_exec_tenant_id() const override { return gen_meta_tenant_id(tenant_id_); } + int insert_import_table_task(common::ObISQLClient &proxy, const ObImportTableTask &task) const; + + int get_all_import_table_tasks_by_initiator(common::ObISQLClient &proxy, + const ObImportTableJob &job, common::ObIArray &tasks) const; + int get_one_not_finish_task_by_initiator(common::ObISQLClient &proxy, + const ObImportTableJob &job, bool &all_finish, ObImportTableTask &task) const; + int get_recover_table_task(common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t task_id, + ObImportTableTask &task) const; + + int advance_status(common::ObISQLClient &proxy, + const ObImportTableTask &task, const ObImportTableTaskStatus &next_status) const; + + int move_import_task_to_history(common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id) const; + int report_import_task_statistics(common::ObISQLClient &proxy, const ObImportTableTask &task) const; + TO_STRING_KV(K_(is_inited), K_(tenant_id)); +private: + DISALLOW_COPY_AND_ASSIGN(ObImportTableTaskPersistHelper); + bool is_inited_; + uint64_t tenant_id_; // sys or user tenant id + ObInnerTableOperator table_op_; +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_table_struct.cpp b/src/share/restore/ob_import_table_struct.cpp new file mode 100644 index 0000000000..1461c2fe29 --- /dev/null +++ b/src/share/restore/ob_import_table_struct.cpp @@ -0,0 +1,1390 @@ +/** + * 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 SHARE +#include "ob_import_table_struct.h" +#include "share/schema/ob_multi_version_schema_service.h" +#include "share/restore/ob_import_util.h" +//#include "share/schema/ob_column_schema.h" + +using namespace oceanbase; +using namespace share; + +void ObImportResult::set_result(const bool is_succeed, const Comment &comment) +{ + is_succeed_ = is_succeed; + comment_ = comment; +} + +int ObImportResult::set_result(const bool is_succeed, const char *buf) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(databuff_printf(comment_.ptr(), comment_.capacity(), "%.*s", static_cast(comment_.capacity()), buf))) { + LOG_WARN("failed to databuff printf", K(ret)); + } else { + is_succeed_ = is_succeed; + } + return ret; +} + +int ObImportResult::set_result(const int err_code, const share::ObTaskId &trace_id, const ObAddr &addr, const ObString &extra_info) +{ + int ret = OB_SUCCESS; + char trace_id_buf[OB_MAX_TRACE_ID_BUFFER_SIZE] = ""; + char addr_buf[OB_MAX_SERVER_ADDR_SIZE] = ""; + if (OB_SUCCESS == err_code) { + is_succeed_ = true; + } else if (OB_FALSE_IT(trace_id.to_string(trace_id_buf, OB_MAX_TRACE_ID_BUFFER_SIZE))) { + } else if (OB_FAIL(addr.ip_port_to_string(addr_buf, OB_MAX_SERVER_ADDR_SIZE))) { + LOG_WARN("failed to convert addr to string", K(ret), K(addr)); + } else if (OB_FAIL(databuff_printf(comment_.ptr(), comment_.capacity(), "result:%d(%s), addr:%s, trace_id:%s", + err_code, extra_info.empty() ? "" : extra_info.ptr(), + addr_buf, trace_id_buf))) { + LOG_WARN("failed to databuff printf", K(ret), K(err_code), K(trace_id), K(addr)); + } else { + is_succeed_ = false; + } + return ret; +} +void ObImportResult::reset() +{ + is_succeed_ = true; + comment_.reset(); +} + +ObImportResult &ObImportResult::operator=(const ObImportResult &result) +{ + is_succeed_ = result.is_succeed_; + comment_ = result.comment_; + return *this; +} +const char* ObImportTableTaskStatus::get_str() const +{ + const char *str = "UNKNOWN"; + const char *status_strs[] = { + "INIT", + "DOING", + "FINISH", + }; + + STATIC_ASSERT(MAX == ARRAYSIZEOF(status_strs), "status count mismatch"); + if (status_ < INIT || status_ >= MAX) { + LOG_ERROR_RET(OB_ERR_UNEXPECTED, "invalid status", K(status_)); + } else { + str = status_strs[status_]; + } + return str; +} + +int ObImportTableTaskStatus::set_status(const char *str) +{ + int ret = OB_SUCCESS; + ObString s(str); + const char *status_strs[] = { + "INIT", + "DOING", + "FINISH", + }; + const int64_t count = ARRAYSIZEOF(status_strs); + if (s.empty()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("status can't empty", K(ret)); + } else { + for (int64_t i = 0; i < count; ++i) { + if (0 == s.case_compare(status_strs[i])) { + status_ = static_cast(i); + } + } + } + return ret; +} + +ObImportTableTaskStatus ObImportTableTaskStatus::get_next_status(const int err_code) +{ + ObImportTableTaskStatus status; + if (!is_valid()) { + LOG_ERROR_RET(OB_ERR_UNEXPECTED, "unexpected import table task status", K(status_)); + } else if (OB_SUCCESS != err_code || Status::DOING == status_) { + status = Status::FINISH; + } else { + status = Status::DOING; + } + return status; +} + + + +#define PARSE_INT_VALUE(COLUMN_NAME) \ + if (OB_SUCC(ret)) { \ + int64_t value = 0; \ + EXTRACT_INT_FIELD_MYSQL(result, #COLUMN_NAME, value, int64_t); \ + if (OB_SUCC(ret)) { \ + set_##COLUMN_NAME(value); \ + } \ + } + +#define PARSE_UINT_VALUE(COLUMN_NAME) \ + if (OB_SUCC(ret)) { \ + uint64_t value = 0; \ + EXTRACT_UINT_FIELD_MYSQL(result, #COLUMN_NAME, value, uint64_t); \ + if (OB_SUCC(ret)) { \ + set_##COLUMN_NAME(value); \ + } \ + } + +#define PARSE_STR_VALUE(COLUMN_NAME) \ + if (OB_SUCC(ret)) { \ + ObString value; \ + EXTRACT_VARCHAR_FIELD_MYSQL_WITH_DEFAULT_VALUE(result, #COLUMN_NAME, value, true, false, value); \ + if (OB_FAIL(set_##COLUMN_NAME(value))) { \ + LOG_WARN("failed to set column value", KR(ret), K(value)); \ + } \ + } + +#define FILL_INT_COLUMN(COLUMN_NAME) \ + if (OB_SUCC(ret)) { \ + if (OB_FAIL(dml.add_column(#COLUMN_NAME, (COLUMN_NAME##_)))) { \ + LOG_WARN("failed to add column", K(ret)); \ + } \ + } + +#define FILL_UINT_COLUMN(COLUMN_NAME) \ + if (OB_SUCC(ret)) { \ + if (OB_FAIL(dml.add_uint64_column(#COLUMN_NAME, (COLUMN_NAME##_)))) { \ + LOG_WARN("failed to add column", K(ret)); \ + } \ + } + +#define FILL_STR_COLUMN(COLUMN_NAME) \ + if (OB_SUCC(ret)) { \ + if (OB_FAIL((dml.add_column(#COLUMN_NAME, (COLUMN_NAME##_))))) { \ + LOG_WARN("failed to add column", K(ret)); \ + } \ + } + + +void ObImportTableTask::reset() +{ + tenant_id_ = OB_INVALID_TENANT_ID; + task_id_ = 0; + job_id_ = 0; + src_tenant_id_ = OB_INVALID_TENANT_ID; + src_tablespace_.reset(); + src_tablegroup_.reset(); + src_database_.reset(); + src_table_.reset(); + src_partition_.reset(); + target_tablespace_.reset(); + target_tablegroup_.reset(); + target_database_.reset(); + target_table_.reset(); + table_column_ = 0; + status_ = ObImportTableTaskStatus::MAX; + start_ts_ = OB_INVALID_TIMESTAMP; + completion_ts_ = OB_INVALID_TIMESTAMP; + cumulative_ts_ = OB_INVALID_TIMESTAMP; + total_bytes_ = 0; + total_rows_ = 0; + imported_bytes_ = 0; + imported_rows_ = 0; + total_index_count_ = 0; + imported_index_count_ = 0; + failed_index_count_ = 0; + total_constraint_count_ = 0; + imported_constraint_count_ = 0; + failed_constraint_count_ = 0; + total_ref_constraint_count_ = 0; + imported_ref_constraint_count_ = 0; + failed_ref_constraint_count_ = 0; + total_trigger_count_ = 0; + imported_trigger_count_ = 0; + failed_trigger_count_ = 0; + result_.reset(); +} + +int ObImportTableTask::assign(const ObImportTableTask &that) +{ + int ret = OB_SUCCESS; + if (!that.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(that)); + } else if (OB_FAIL(set_src_tablespace(that.get_src_tablespace()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_src_tablegroup(that.get_src_tablegroup()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_src_database(that.get_src_database()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_src_table(that.get_src_table()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_src_partition(that.get_src_partition()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_target_tablespace(that.get_target_tablespace()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_target_tablegroup(that.get_target_tablegroup()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_target_database(that.get_target_database()))) { + LOG_WARN("failed to set", K(ret)); + } else if (OB_FAIL(set_target_table(that.get_target_table()))) { + LOG_WARN("failed to set", K(ret)); + } else { + tenant_id_ = that.tenant_id_; + task_id_ = that.task_id_; + job_id_ = that.job_id_; + src_tenant_id_ = that.src_tenant_id_; + table_column_ = that.table_column_; + status_ = that.status_; + start_ts_ = that.start_ts_; + completion_ts_ = that.completion_ts_; + cumulative_ts_ = that.cumulative_ts_; + total_bytes_ = that.total_bytes_; + total_rows_ = that.total_rows_; + imported_bytes_ = that.imported_bytes_; + imported_rows_ = that.imported_rows_; + total_index_count_ = that.total_index_count_; + imported_index_count_ = that.imported_index_count_; + failed_index_count_ = that.failed_index_count_; + total_constraint_count_ = that.total_constraint_count_; + imported_constraint_count_ = that.imported_constraint_count_; + failed_constraint_count_ = that.failed_constraint_count_; + total_ref_constraint_count_ = that.total_ref_constraint_count_; + imported_ref_constraint_count_ = that.imported_ref_constraint_count_; + failed_ref_constraint_count_ = that.failed_ref_constraint_count_; + total_trigger_count_ = that.total_trigger_count_; + imported_trigger_count_ = that.imported_trigger_count_; + failed_trigger_count_ = that.failed_trigger_count_; + result_ = that.result_; + } + return ret; +} + +int ObImportTableTask::Key::fill_pkey_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + if (!is_pkey_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid key", K(ret), K(*this)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TASK_ID, task_id_))) { + LOG_WARN("failed to add column", K(ret)); + } + return ret; +} + +int ObImportTableTask::get_pkey(Key &key) const +{ + int ret = OB_SUCCESS; + if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid pkey", K(ret), KPC(this)); + } else { + key.tenant_id_ = get_tenant_id(); + key.task_id_ = get_task_id(); + } + return ret; +} + +int ObImportTableTask::fill_pkey_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + if (!is_pkey_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid key", K(ret), K(*this)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TASK_ID, task_id_))) { + LOG_WARN("failed to add column", K(ret)); + } + return ret; +} + +int ObImportTableTask::parse_from(common::sqlclient::ObMySQLResult &result) +{ + int ret = OB_SUCCESS; + const int64_t OB_MAX_RESULT_BUF_LEN = 12; + char result_buf[OB_MAX_RESULT_BUF_LEN] = ""; + ObImportResult::Comment comment; + EXTRACT_INT_FIELD_MYSQL(result, "tenant_id", tenant_id_, uint64_t); + EXTRACT_INT_FIELD_MYSQL(result, "src_tenant_id", src_tenant_id_, uint64_t); + PARSE_INT_VALUE(task_id) + PARSE_INT_VALUE(job_id) + PARSE_STR_VALUE(src_tablespace) + PARSE_STR_VALUE(src_tablegroup) + PARSE_STR_VALUE(src_database) + PARSE_STR_VALUE(src_table) + PARSE_STR_VALUE(src_partition) + PARSE_STR_VALUE(target_tablespace) + PARSE_STR_VALUE(target_tablegroup) + PARSE_STR_VALUE(target_database) + PARSE_STR_VALUE(target_table) + PARSE_INT_VALUE(table_column) + if (OB_SUCC(ret)) { + int64_t real_length = 0; + char status_str[OB_DEFAULT_STATUS_LENTH] = ""; + EXTRACT_STRBUF_FIELD_MYSQL(result, OB_STR_STATUS, status_str, OB_DEFAULT_STATUS_LENTH, real_length); + if (OB_SUCC(ret)) { + if (OB_FAIL(status_.set_status(status_str))) { + LOG_WARN("failed to set status", K(ret), K(status_str)); + } + } + } + PARSE_INT_VALUE(start_ts) + PARSE_INT_VALUE(completion_ts) + PARSE_INT_VALUE(cumulative_ts) + PARSE_INT_VALUE(total_bytes) + PARSE_INT_VALUE(total_rows) + PARSE_INT_VALUE(imported_bytes) + PARSE_INT_VALUE(imported_rows) + PARSE_INT_VALUE(total_index_count) + PARSE_INT_VALUE(imported_index_count) + PARSE_INT_VALUE(failed_index_count) + PARSE_INT_VALUE(total_constraint_count) + PARSE_INT_VALUE(imported_constraint_count) + PARSE_INT_VALUE(failed_constraint_count) + PARSE_INT_VALUE(total_ref_constraint_count) + PARSE_INT_VALUE(imported_ref_constraint_count) + PARSE_INT_VALUE(failed_ref_constraint_count) + PARSE_INT_VALUE(total_trigger_count) + PARSE_INT_VALUE(imported_trigger_count) + PARSE_INT_VALUE(failed_trigger_count) + + int64_t real_length = 0; + EXTRACT_STRBUF_FIELD_MYSQL(result, OB_STR_RESULT, result_buf, OB_MAX_RESULT_BUF_LEN, real_length); + EXTRACT_STRBUF_FIELD_MYSQL_SKIP_RET(result, OB_STR_COMMENT, comment.ptr(), comment.capacity(), real_length); + if (OB_SUCC(ret)) { + bool is_succeed = true; + if (0 == STRCMP("FAILED", result_buf)) { + is_succeed = false; + } else if (0 == STRCMP("SUCCEESS", result_buf)) { + is_succeed = true; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid result str", K(ret)); + } + result_.set_result(is_succeed, comment); + } + + if (OB_SUCC(ret)) { + LOG_INFO("succeed to parse ObImportTableTask", KPC(this)); + } + return ret; +} + +int ObImportTableTask::fill_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + if (OB_FAIL(fill_pkey_dml(dml))) { + LOG_WARN("failed to fill pkey dml", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, status_.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } + FILL_INT_COLUMN(job_id) + FILL_INT_COLUMN(src_tenant_id) + FILL_STR_COLUMN(src_tablespace) + FILL_STR_COLUMN(src_tablegroup) + FILL_STR_COLUMN(src_database) + FILL_STR_COLUMN(src_table) + FILL_STR_COLUMN(src_partition) + FILL_STR_COLUMN(target_tablespace) + FILL_STR_COLUMN(target_tablegroup) + FILL_STR_COLUMN(target_database) + FILL_STR_COLUMN(target_table) + FILL_INT_COLUMN(table_column) + FILL_INT_COLUMN(start_ts) + FILL_INT_COLUMN(completion_ts) + FILL_INT_COLUMN(cumulative_ts) + FILL_INT_COLUMN(total_bytes) + FILL_INT_COLUMN(total_rows) + FILL_INT_COLUMN(imported_bytes) + FILL_INT_COLUMN(imported_rows) + FILL_INT_COLUMN(total_index_count) + FILL_INT_COLUMN(imported_index_count) + FILL_INT_COLUMN(failed_index_count) + FILL_INT_COLUMN(total_constraint_count) + FILL_INT_COLUMN(imported_constraint_count) + FILL_INT_COLUMN(failed_constraint_count) + FILL_INT_COLUMN(total_ref_constraint_count) + FILL_INT_COLUMN(imported_ref_constraint_count) + FILL_INT_COLUMN(failed_ref_constraint_count) + FILL_INT_COLUMN(total_trigger_count) + FILL_INT_COLUMN(imported_trigger_count) + FILL_INT_COLUMN(failed_trigger_count) + if (OB_SUCC(ret) && status_.is_finish()) { + if (FAILEDx(dml.add_column(OB_STR_RESULT, result_.get_result_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_COMMENT, result_.get_comment()))) { + LOG_WARN("failed to add column", K(ret)); + } + } + + return ret; +} + + +const char* ObImportTableJobStatus::get_str() const +{ + const char *str = "UNKNOWN"; + const char *status_strs[] = { + "INIT", + "IMPORT_TABLE", + "RECONSTRUCT_REF_CONSTRAINT", + "CANCELING", + "IMPORT_FINISH", + }; + + STATIC_ASSERT(MAX_STATUS == ARRAYSIZEOF(status_strs), "status count mismatch"); + if (status_ < INIT || status_ >= MAX_STATUS) { + LOG_ERROR_RET(OB_ERR_UNEXPECTED, "invalid status", K(status_)); + } else { + str = status_strs[status_]; + } + return str; +} + +int ObImportTableJobStatus::set_status(const char *str) +{ + int ret = OB_SUCCESS; + ObString s(str); + const char *status_strs[] = { + "INIT", + "IMPORT_TABLE", + "RECONSTRUCT_REF_CONSTRAINT", + "CANCELING", + "IMPORT_FINISH", + }; + const int64_t count = ARRAYSIZEOF(status_strs); + if (s.empty()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("status can't empty", K(ret)); + } else { + for (int64_t i = 0; i < count; ++i) { + if (0 == s.case_compare(status_strs[i])) { + status_ = static_cast(i); + } + } + } + return ret; +} + +ObImportTableJobStatus ObImportTableJobStatus::get_next_status(const ObImportTableJobStatus &cur_status) +{ + ObImportTableJobStatus ret; + switch(cur_status) { + case ObImportTableJobStatus::Status::INIT: { + ret = ObImportTableJobStatus::Status::IMPORT_TABLE; + break; + } + case ObImportTableJobStatus::Status::IMPORT_TABLE: { + ret = ObImportTableJobStatus::Status::RECONSTRUCT_REF_CONSTRAINT; + break; + } + case ObImportTableJobStatus::Status::RECONSTRUCT_REF_CONSTRAINT: { + ret = ObImportTableJobStatus::Status::IMPORT_FINISH; + break; + } + case ObImportTableJobStatus::Status::IMPORT_FINISH: { + ret = ObImportTableJobStatus::Status::IMPORT_FINISH; + break; + } + default: { + break; + } + } + return ret; +} + + + +void ObImportTableJob::reset() +{ + tenant_id_ = OB_INVALID_TENANT_ID; + job_id_ = 0; + initiator_tenant_id_ = OB_INVALID_TENANT_ID; + initiator_job_id_ = 0; + start_ts_ = 0; + end_ts_ = 0; + src_tenant_name_.reset(); + src_tenant_id_ = OB_INVALID_TENANT_ID; + status_ = ObImportTableJobStatus::Status::MAX_STATUS; + total_table_count_ = 0; + finished_table_count_ = 0; + failed_table_count_ = 0; + total_bytes_ = 0; + finished_bytes_ = 0; + failed_bytes_ = 0; + result_.reset(); + import_arg_.reset(); + description_.reset(); +} + +int ObImportTableJob::Key::fill_pkey_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + if (!is_pkey_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid key", K(ret), K(*this)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_JOB_ID, job_id_))) { + LOG_WARN("failed to add column", K(ret)); + } + return ret; +} + +int ObImportTableJob::get_pkey(Key &key) const +{ + int ret = OB_SUCCESS; + if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid pkey", K(ret), KPC(this)); + } else { + key.tenant_id_ = get_tenant_id(); + key.job_id_ = get_job_id(); + } + return ret; +} + +int ObImportTableJob::fill_pkey_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + if (!is_pkey_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid key", K(ret), K(*this)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_JOB_ID, job_id_))) { + LOG_WARN("failed to add column", K(ret)); + } + return ret; +} + +int ObImportTableJob::parse_from(common::sqlclient::ObMySQLResult &result) +{ + int ret = OB_SUCCESS; + ObNameCaseMode name_case_mode; + bool import_all = false; + const int64_t OB_MAX_RESULT_BUF_LEN = 12; + char result_buf[OB_MAX_RESULT_BUF_LEN] = ""; + ObImportResult::Comment comment; + EXTRACT_INT_FIELD_MYSQL(result, "tenant_id", tenant_id_, uint64_t); + if (FAILEDx(ObImportTableUtil::get_tenant_name_case_mode(tenant_id_, name_case_mode))) { + LOG_WARN("failed to get tenant name case mode", K(ret), K(tenant_id_)); + } + EXTRACT_INT_FIELD_MYSQL(result, "initiator_tenant_id", initiator_tenant_id_, uint64_t); + EXTRACT_INT_FIELD_MYSQL(result, "src_tenant_id", src_tenant_id_, uint64_t); + EXTRACT_BOOL_FIELD_MYSQL(result, "import_all", import_all); + if (OB_SUCC(ret) && import_all) { + import_arg_.get_import_table_arg().set_import_all(); + } + + PARSE_INT_VALUE(job_id) + PARSE_INT_VALUE(initiator_job_id) + PARSE_INT_VALUE(start_ts) + PARSE_INT_VALUE(end_ts) + PARSE_STR_VALUE(src_tenant_name) + PARSE_STR_VALUE(description) + if (OB_SUCC(ret)) { + int64_t real_length = 0; + char status_str[OB_DEFAULT_STATUS_LENTH] = ""; + EXTRACT_STRBUF_FIELD_MYSQL(result, OB_STR_STATUS, status_str, OB_DEFAULT_STATUS_LENTH, real_length); + if (OB_SUCC(ret)) { + if (OB_FAIL(status_.set_status(status_str))) { + LOG_WARN("failed to set status", K(ret), K(status_str)); + } + } + } + PARSE_INT_VALUE(total_table_count) + PARSE_INT_VALUE(finished_table_count) + PARSE_INT_VALUE(failed_table_count) + PARSE_INT_VALUE(total_bytes) + PARSE_INT_VALUE(finished_bytes) + PARSE_INT_VALUE(failed_bytes) + + int64_t real_length = 0; + EXTRACT_STRBUF_FIELD_MYSQL(result, OB_STR_RESULT, result_buf, OB_MAX_RESULT_BUF_LEN, real_length); + EXTRACT_STRBUF_FIELD_MYSQL_SKIP_RET(result, OB_STR_COMMENT, comment.ptr(), comment.capacity(), real_length); + if (OB_SUCC(ret)) { + bool is_succeed = true; + if (0 == STRCMP("FAILED", result_buf)) { + is_succeed = false; + } + result_.set_result(is_succeed, comment); + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObImportDatabaseArray &import_database_array = import_arg_.get_import_table_arg().get_import_database_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_db_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(import_database_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize database array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObImportTableArray &import_table_array = import_arg_.get_import_table_arg().get_import_table_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_table_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(import_table_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize table array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObImportPartitionArray &import_part_array = import_arg_.get_import_table_arg().get_import_partition_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_partition_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(import_part_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize partition array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapDatabaseArray &remap_database_array = import_arg_.get_remap_table_arg().get_remap_database_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_db_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_database_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap database array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapTableArray &remap_table_array = import_arg_.get_remap_table_arg().get_remap_table_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_table_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_table_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap table array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapPartitionArray &remap_partition_array = import_arg_.get_remap_table_arg().get_remap_partition_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_partition_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_partition_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap partition array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapTablegroupArray &remap_tablegroup_array = import_arg_.get_remap_table_arg().get_remap_tablegroup_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_tablegroup_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_tablegroup_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap tablegroup array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapTablespaceArray &remap_tablespace_array = import_arg_.get_remap_table_arg().get_remap_tablespace_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_tablespace_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_tablespace_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap tablespace array", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + LOG_INFO("succeed parse import table job", KPC(this)); + } + + return ret; +} + +int ObImportTableJob::fill_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + const bool import_all = import_arg_.get_import_table_arg().is_import_all(); + if (OB_FAIL(fill_pkey_dml(dml))) { + LOG_WARN("failed to fill key", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, status_.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column("import_all", import_all))) { + LOG_WARN("failed to add column", K(ret)); + } + FILL_INT_COLUMN(initiator_tenant_id) + FILL_INT_COLUMN(initiator_job_id) + FILL_INT_COLUMN(start_ts) + FILL_INT_COLUMN(end_ts) + FILL_STR_COLUMN(src_tenant_name) + FILL_INT_COLUMN(src_tenant_id) + FILL_INT_COLUMN(total_table_count) + FILL_INT_COLUMN(finished_table_count) + FILL_INT_COLUMN(failed_table_count) + FILL_INT_COLUMN(total_bytes) + FILL_INT_COLUMN(finished_bytes) + FILL_INT_COLUMN(failed_bytes) + FILL_STR_COLUMN(description) + + if (OB_SUCC(ret) && status_.is_finish()) { + if (OB_FAIL(dml.add_column(OB_STR_RESULT, result_.get_result_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_COMMENT, result_.get_comment()))) { + LOG_WARN("failed to add column", K(ret)); + } + } + + if (OB_SUCC(ret)) { + ObArenaAllocator allocator; + + ObString db_list_; + ObString table_list_; + ObString partition_list_; + ObString remap_db_list_; + ObString remap_table_list_; + ObString remap_partition_list_; + ObString remap_tablegroup_list_; + ObString remap_tablespace_list_; + + ObString hex_db_list_; + ObString hex_table_list_; + ObString hex_partition_list_; + ObString hex_remap_db_list_; + ObString hex_remap_table_list_; + ObString hex_remap_partition_list_; + ObString hex_remap_tablegroup_list_; + ObString hex_remap_tablespace_list_; + + const share::ObImportTableArg &import_table_arg = import_arg_.get_import_table_arg(); + const share::ObImportRemapArg &remap_table_arg = import_arg_.get_remap_table_arg(); + + if (OB_FAIL(import_table_arg.get_db_list_format_str(allocator, db_list_))) { + LOG_WARN("fail to get db list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_table_list_format_str(allocator, table_list_))) { + LOG_WARN("fail to get table list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_partition_list_format_str(allocator, partition_list_))) { + LOG_WARN("fail to get partition list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_db_list_format_str(allocator, remap_db_list_))) { + LOG_WARN("fail to get remap db list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_table_list_format_str(allocator, remap_table_list_))) { + LOG_WARN("fail to get remap table list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_partition_list_format_str(allocator, remap_partition_list_))) { + LOG_WARN("fail to get remap partition list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablegroup_list_format_str(allocator, remap_tablegroup_list_))) { + LOG_WARN("fail to get remap tablegroup list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablespace_list_format_str(allocator, remap_tablespace_list_))) { + LOG_WARN("fail to get remap tablespace list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(import_table_arg.get_db_list_hex_format_str(allocator, hex_db_list_))) { + LOG_WARN("fail to get hex db list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_table_list_hex_format_str(allocator, hex_table_list_))) { + LOG_WARN("fail to get hex table list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_partition_list_hex_format_str(allocator, hex_partition_list_))) { + LOG_WARN("fail to get hex partition list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_db_list_hex_format_str(allocator, hex_remap_db_list_))) { + LOG_WARN("fail to get remap hex db list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_table_list_hex_format_str(allocator, hex_remap_table_list_))) { + LOG_WARN("fail to get remap hex table list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_partition_list_hex_format_str(allocator, hex_remap_partition_list_))) { + LOG_WARN("fail to get remap hex partition list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablegroup_list_hex_format_str(allocator, hex_remap_tablegroup_list_))) { + LOG_WARN("fail to get remap hex tablegroup list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablespace_list_hex_format_str(allocator, hex_remap_tablespace_list_))) { + LOG_WARN("fail to get remap hex tablespace list format str", KR(ret), K(remap_table_arg)); + } else { + FILL_STR_COLUMN(db_list) + FILL_STR_COLUMN(table_list) + FILL_STR_COLUMN(partition_list) + FILL_STR_COLUMN(remap_db_list) + FILL_STR_COLUMN(remap_table_list) + FILL_STR_COLUMN(remap_partition_list) + FILL_STR_COLUMN(remap_tablegroup_list) + FILL_STR_COLUMN(remap_tablespace_list) + + FILL_STR_COLUMN(hex_db_list) + FILL_STR_COLUMN(hex_table_list) + FILL_STR_COLUMN(hex_partition_list) + FILL_STR_COLUMN(hex_remap_db_list) + FILL_STR_COLUMN(hex_remap_table_list) + FILL_STR_COLUMN(hex_remap_partition_list) + FILL_STR_COLUMN(hex_remap_tablegroup_list) + FILL_STR_COLUMN(hex_remap_tablespace_list) + } + } + return ret; +} + +int ObImportTableJob::assign(const ObImportTableJob &that) +{ + int ret = OB_SUCCESS; + if (!that.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret)); + } else if (OB_FAIL(import_arg_.assign(that.import_arg_))) { + LOG_WARN("failed to assign import arg", K(ret)); + } else if (OB_FAIL(set_src_tenant_name(that.get_src_tenant_name()))) { + LOG_WARN("failed to set src tenant name", K(ret)); + } else if (OB_FAIL(set_description(that.get_description()))) { + LOG_WARN("failed to set description", K(ret)); + } else { + set_tenant_id(that.get_tenant_id()); + set_job_id(that.get_job_id()); + set_initiator_job_id(that.get_initiator_job_id()); + set_initiator_tenant_id(that.get_initiator_tenant_id()); + set_start_ts(that.get_start_ts()); + set_end_ts(that.get_end_ts()); + set_src_tenant_id(that.get_src_tenant_id()); + set_status(that.get_status()); + set_total_table_count(that.get_total_table_count()); + set_finished_table_count(that.get_finished_table_count()); + set_failed_table_count(that.get_failed_table_count()); + set_total_bytes(that.get_total_bytes()); + set_finished_bytes(that.get_finished_bytes()); + set_failed_bytes(that.get_failed_bytes()); + } + return ret; +} + +const char* ObRecoverTableStatus::get_str() const +{ + const char *str = "UNKNOWN"; + const char *status_strs[] = { + "PREPARE", + "RECOVERING", + "RESTORE_AUX_TENANT", + "PRECHECK_IMPORT", + "GEN_IMPORT_JOB", + "IMPORTING", + "CANCELING", + "COMPLETED", + "FAILED", + }; + + STATIC_ASSERT(MAX_STATUS == ARRAYSIZEOF(status_strs), "status count mismatch"); + if (status_ < PREPARE || status_ >= MAX_STATUS) { + LOG_ERROR_RET(OB_ERR_UNEXPECTED, "invalid status", K(status_)); + } else { + str = status_strs[status_]; + } + return str; +} + +int ObRecoverTableStatus::set_status(const char *str) +{ + int ret = OB_SUCCESS; + ObString s(str); + const char *status_strs[] = { + "PREPARE", + "RECOVERING", + "RESTORE_AUX_TENANT", + "PRECHECK_IMPORT", + "GEN_IMPORT_JOB", + "IMPORTING", + "CANCELING", + "COMPLETED", + "FAILED", + }; + const int64_t count = ARRAYSIZEOF(status_strs); + if (s.empty()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("status can't empty", K(ret)); + } else { + for (int64_t i = 0; i < count; ++i) { + if (0 == s.case_compare(status_strs[i])) { + status_ = static_cast(i); + } + } + } + return ret; +} + +ObRecoverTableStatus ObRecoverTableStatus::get_sys_next_status(const ObRecoverTableStatus &cur_status) +{ + ObRecoverTableStatus ret; + switch(cur_status) { + case ObRecoverTableStatus::Status::PREPARE: { + ret = ObRecoverTableStatus::Status::RECOVERING; + break; + } + case ObRecoverTableStatus::Status::RECOVERING: { + ret = ObRecoverTableStatus::Status::COMPLETED; + break; + } + case ObRecoverTableStatus::Status::COMPLETED: { + ret = ObRecoverTableStatus::Status::COMPLETED; + break; + } + default: { + break; + } + } + return ret; +} + +ObRecoverTableStatus ObRecoverTableStatus::get_user_next_status(const ObRecoverTableStatus &cur_status) +{ + ObRecoverTableStatus ret; + switch(cur_status) { + case ObRecoverTableStatus::Status::PREPARE: { + ret = ObRecoverTableStatus::Status::RESTORE_AUX_TENANT; + break; + } + case ObRecoverTableStatus::Status::RESTORE_AUX_TENANT: { + ret = ObRecoverTableStatus::Status::GEN_IMPORT_JOB; + break; + } + case ObRecoverTableStatus::Status::GEN_IMPORT_JOB: { + ret = ObRecoverTableStatus::Status::IMPORTING; + break; + } + case ObRecoverTableStatus::Status::IMPORTING: { + ret = ObRecoverTableStatus::Status::COMPLETED; + break; + } + case ObRecoverTableStatus::Status::COMPLETED: { + ret = ObRecoverTableStatus::Status::COMPLETED; + break; + } + default: { + break; + } + } + return ret; +} + + +int ObRecoverTableJob::assign(const ObRecoverTableJob &that) +{ + int ret = OB_SUCCESS; + if (!that.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(that)); + } else if (OB_FAIL(set_aux_tenant_name(that.get_aux_tenant_name()))) { + LOG_WARN("failed to set aux tenant name", K(ret)); + } else if (OB_FAIL(set_target_tenant_name(that.get_target_tenant_name()))) { + LOG_WARN("failed to set target tenant name", K(ret)); + } else if (OB_FAIL(set_restore_option(that.get_restore_option()))) { + LOG_WARN("failed to set restore option", K(ret)); + } else if (OB_FAIL(set_backup_dest(that.get_backup_dest()))) { + LOG_WARN("failed to set backup dest", K(ret)); + } else if (OB_FAIL(set_backup_passwd(that.get_backup_passwd()))) { + LOG_WARN("failed to set backup passwd", K(ret)); + } else if (OB_FAIL(set_external_kms_info(that.get_external_kms_info()))) { + LOG_WARN("failed to set external kms", K(ret)); + } else if (OB_FAIL(multi_restore_path_list_.assign(that.get_multi_restore_path_list()))) { + LOG_WARN("failed to assign multi restore path", K(ret)); + } else if (OB_FAIL(import_arg_.assign(that.get_import_arg()))) { + LOG_WARN("failed to assign import arg", K(ret)); + } else if (OB_FAIL(set_description(that.get_description()))) { + LOG_WARN("failed to set description", K(ret)); + } else { + set_tenant_id(that.get_tenant_id()); + set_job_id(that.get_job_id()); + set_initiator_job_id(that.get_initiator_job_id()); + set_initiator_tenant_id(that.get_initiator_tenant_id()); + set_start_ts(that.get_start_ts()); + set_end_ts(that.get_end_ts()); + set_status(that.get_status()); + set_target_tenant_id(that.get_target_tenant_id()); + set_restore_scn(that.get_restore_scn()); + set_result(that.get_result()); + } + return ret; +} + +int ObRecoverTableJob::Key::fill_pkey_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + if (!is_pkey_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid key", K(ret), K(*this)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_JOB_ID, job_id_))) { + LOG_WARN("failed to add column", K(ret)); + } + return ret; +} + +void ObRecoverTableJob::reset() +{ + tenant_id_ = OB_INVALID_TENANT_ID; + job_id_ = 0; + initiator_tenant_id_ = OB_INVALID_TENANT_ID; + initiator_job_id_ = 0; + start_ts_ = 0; + end_ts_ = 0; + status_ = ObRecoverTableStatus::Status::MAX_STATUS; + aux_tenant_name_.reset(); + target_tenant_name_.reset(); + target_tenant_id_ = OB_INVALID_TENANT_ID; + restore_scn_.reset(); + restore_option_.reset(); + backup_dest_.reset(); + backup_passwd_.reset(); + external_kms_info_.reset(); + result_.reset(); + import_arg_.reset(); + multi_restore_path_list_.reset(); + description_.reset(); +} + +int ObRecoverTableJob::fill_pkey_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + if (!is_pkey_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid key", K(ret), K(*this)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_JOB_ID, job_id_))) { + LOG_WARN("failed to add column", K(ret)); + } + return ret; +} + +int ObRecoverTableJob::get_pkey(Key &key) const +{ + int ret = OB_SUCCESS; + if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid pkey", K(ret), KPC(this)); + } else { + key.tenant_id_ = get_tenant_id(); + key.job_id_ = get_job_id(); + } + return ret; +} + + +int ObRecoverTableJob::parse_from(common::sqlclient::ObMySQLResult &result) +{ + int ret = OB_SUCCESS; + ObNameCaseMode name_case_mode; + bool import_all = false; + const int64_t OB_MAX_RESULT_BUF_LEN = 12; + char result_buf[OB_MAX_RESULT_BUF_LEN] = ""; + ObImportResult::Comment comment; + EXTRACT_INT_FIELD_MYSQL(result, "tenant_id", tenant_id_, uint64_t); + if (FAILEDx(ObImportTableUtil::get_tenant_name_case_mode(tenant_id_, name_case_mode))) { + LOG_WARN("failed to get tenant name case mode", K(ret), K(tenant_id_)); + } + EXTRACT_INT_FIELD_MYSQL(result, "initiator_tenant_id", initiator_tenant_id_, uint64_t); + EXTRACT_INT_FIELD_MYSQL(result, "target_tenant_id", target_tenant_id_, uint64_t); + PARSE_INT_VALUE(job_id) + PARSE_INT_VALUE(initiator_job_id) + PARSE_INT_VALUE(start_ts) + PARSE_INT_VALUE(end_ts) + EXTRACT_BOOL_FIELD_MYSQL(result, "import_all", import_all); + if (OB_SUCC(ret) && import_all) { + import_arg_.get_import_table_arg().set_import_all(); + } + if (OB_SUCC(ret)) { + int64_t real_length = 0; + char status_str[OB_DEFAULT_STATUS_LENTH] = ""; + EXTRACT_STRBUF_FIELD_MYSQL(result, OB_STR_STATUS, status_str, OB_DEFAULT_STATUS_LENTH, real_length); + if (OB_SUCC(ret)) { + if (OB_FAIL(status_.set_status(status_str))) { + LOG_WARN("failed to set status", K(ret), K(status_str)); + } + } + } + + PARSE_STR_VALUE(aux_tenant_name) + PARSE_STR_VALUE(target_tenant_name) + + if (OB_SUCC(ret)) { + uint64_t restore_scn = 0; + EXTRACT_UINT_FIELD_MYSQL(result, OB_STR_RESTORE_SCN, restore_scn, uint64_t); + if (FAILEDx(restore_scn_.convert_for_inner_table_field(restore_scn))) { + LOG_WARN("failed to conver for inner table", K(ret), K(restore_scn)); + } + } + + PARSE_STR_VALUE(restore_option) + PARSE_STR_VALUE(backup_dest) + PARSE_STR_VALUE(backup_passwd) + PARSE_STR_VALUE(external_kms_info) + PARSE_STR_VALUE(description) + + int64_t real_length = 0; + EXTRACT_STRBUF_FIELD_MYSQL(result, OB_STR_RESULT, result_buf, OB_MAX_RESULT_BUF_LEN, real_length); + EXTRACT_STRBUF_FIELD_MYSQL_SKIP_RET(result, OB_STR_COMMENT, comment.ptr(), comment.capacity(), real_length); + if (OB_SUCC(ret)) { + bool is_succeed = true; + if (0 == STRCMP("FAILED", result_buf)) { + is_succeed = false; + } + result_.set_result(is_succeed, comment); + } + + if (OB_SUCC(ret)) { + ObString str; + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, OB_STR_BACKUP_SET_LIST, str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(multi_restore_path_list_.backup_set_list_assign_with_format_str(str))) { + LOG_WARN("fail to assign backup set list", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, OB_STR_BACKUP_PIECE_LIST, str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(multi_restore_path_list_.backup_piece_list_assign_with_format_str(str))) { + LOG_WARN("fail to assign backup set list", KR(ret), K(str)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObImportDatabaseArray &import_database_array = import_arg_.get_import_table_arg().get_import_database_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_db_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(import_database_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize database array", KR(ret), K(str)); + } else { + LOG_INFO("import database array hex format deserialize", K(str), K(import_database_array)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObImportTableArray &import_table_array = import_arg_.get_import_table_arg().get_import_table_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_table_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(import_table_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize table array", KR(ret), K(str)); + } else { + LOG_INFO("import table array hex format deserialize", K(str), K(import_table_array)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObImportPartitionArray &import_part_array = import_arg_.get_import_table_arg().get_import_partition_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_partition_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(import_part_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize partition array", KR(ret), K(str)); + } else { + LOG_INFO("import partition array hex format deserialize", K(str), K(import_part_array)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapDatabaseArray &remap_database_array = import_arg_.get_remap_table_arg().get_remap_database_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_db_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_database_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap database array", KR(ret), K(str)); + } else { + LOG_INFO("remap database array hex format deserialize", K(str), K(remap_database_array)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapTableArray &remap_table_array = import_arg_.get_remap_table_arg().get_remap_table_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_table_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_table_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap table array", KR(ret), K(str)); + } else { + LOG_INFO("remap table array hex format deserialize", K(str), K(remap_table_array)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapPartitionArray &remap_part_array = import_arg_.get_remap_table_arg().get_remap_partition_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_partition_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_part_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap partition array", KR(ret), K(str)); + } else { + LOG_INFO("remap partition array hex format deserialize", K(str), K(remap_part_array)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapTablegroupArray &remap_tablegroup_array = import_arg_.get_remap_table_arg().get_remap_tablegroup_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_tablegroup_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_tablegroup_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap tablegroup array", KR(ret), K(str)); + } else { + LOG_INFO("remap tablegroup array hex format deserialize", K(str), K(remap_tablegroup_array)); + } + } + + if (OB_SUCC(ret)) { + ObString str; + int64_t pos = 0; + share::ObRemapTablespaceArray &remap_tablespace_array = import_arg_.get_remap_table_arg().get_remap_tablespace_array(); + EXTRACT_VARCHAR_FIELD_MYSQL_SKIP_RET(result, "hex_remap_tablespace_list", str); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(remap_tablespace_array.hex_format_deserialize(str.ptr(), str.length(), pos))) { + LOG_WARN("failed to deserialize remap tablespace array", KR(ret), K(str)); + } else { + LOG_INFO("remap tablespace array hex format deserialize", K(str), K(remap_tablespace_array)); + } + } + + if (OB_SUCC(ret)) { + LOG_INFO("parse recover table job succeed", KPC(this)); + } + + return ret; +} + +int ObRecoverTableJob::fill_dml(share::ObDMLSqlSplicer &dml) const +{ + int ret = OB_SUCCESS; + const bool import_all = import_arg_.get_import_table_arg().is_import_all(); + if (OB_FAIL(fill_pkey_dml(dml))) { + LOG_WARN("failed to fill key", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, status_.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_uint64_column(OB_STR_RESTORE_SCN, restore_scn_.get_val_for_inner_table_field()))) { + LOG_WARN("failed to add restore scn", K(ret)); + } else if (OB_FAIL(dml.add_column("import_all", import_all))) { + LOG_WARN("failed to add column", K(ret)); + } + + FILL_INT_COLUMN(initiator_tenant_id) + FILL_INT_COLUMN(initiator_job_id) + FILL_INT_COLUMN(start_ts) + FILL_INT_COLUMN(end_ts) + FILL_STR_COLUMN(aux_tenant_name) + FILL_STR_COLUMN(target_tenant_name) + FILL_INT_COLUMN(target_tenant_id) + FILL_STR_COLUMN(restore_option) + FILL_STR_COLUMN(backup_dest) + FILL_STR_COLUMN(backup_passwd) + FILL_STR_COLUMN(external_kms_info) + FILL_STR_COLUMN(description) + + if (OB_SUCC(ret) && status_.is_finish()) { + if (OB_FAIL(dml.add_column(OB_STR_RESULT, result_.get_result_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_COMMENT, result_.get_comment()))) { + LOG_WARN("failed to add column", K(ret)); + } + } + + if (OB_SUCC(ret)) { + ObArenaAllocator allocator; + ObString backup_set_list_; + ObString backup_piece_list_; + + ObString db_list_; + ObString table_list_; + ObString partition_list_; + ObString remap_db_list_; + ObString remap_table_list_; + ObString remap_partition_list_; + ObString remap_tablegroup_list_; + ObString remap_tablespace_list_; + + ObString hex_db_list_; + ObString hex_table_list_; + ObString hex_partition_list_; + ObString hex_remap_db_list_; + ObString hex_remap_table_list_; + ObString hex_remap_partition_list_; + ObString hex_remap_tablegroup_list_; + ObString hex_remap_tablespace_list_; + + const share::ObImportTableArg &import_table_arg = import_arg_.get_import_table_arg(); + const share::ObImportRemapArg &remap_table_arg = import_arg_.get_remap_table_arg(); + + if (OB_FAIL(multi_restore_path_list_.get_backup_set_list_format_str(allocator, backup_set_list_))) { + LOG_WARN("fail to get format str", KR(ret), K(multi_restore_path_list_)); + } else if (OB_FAIL(multi_restore_path_list_.get_backup_piece_list_format_str(allocator, backup_piece_list_))) { + LOG_WARN("fail to get format str", KR(ret), K(multi_restore_path_list_)); + } else if (OB_FAIL(import_table_arg.get_db_list_format_str(allocator, db_list_))) { + LOG_WARN("fail to get db list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_table_list_format_str(allocator, table_list_))) { + LOG_WARN("fail to get table list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_partition_list_format_str(allocator, partition_list_))) { + LOG_WARN("fail to get partition list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_db_list_format_str(allocator, remap_db_list_))) { + LOG_WARN("fail to get remap db list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_table_list_format_str(allocator, remap_table_list_))) { + LOG_WARN("fail to get remap table list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_partition_list_format_str(allocator, remap_partition_list_))) { + LOG_WARN("fail to get remap partition list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablegroup_list_format_str(allocator, remap_tablegroup_list_))) { + LOG_WARN("fail to get remap tablegroup list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablespace_list_format_str(allocator, remap_tablespace_list_))) { + LOG_WARN("fail to get remap tablespace list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(import_table_arg.get_db_list_hex_format_str(allocator, hex_db_list_))) { + LOG_WARN("fail to get hex db list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_table_list_hex_format_str(allocator, hex_table_list_))) { + LOG_WARN("fail to get hex table list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(import_table_arg.get_partition_list_hex_format_str(allocator, hex_partition_list_))) { + LOG_WARN("fail to get hex partition list format str", KR(ret), K(import_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_db_list_hex_format_str(allocator, hex_remap_db_list_))) { + LOG_WARN("fail to get remap hex db list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_table_list_hex_format_str(allocator, hex_remap_table_list_))) { + LOG_WARN("fail to get remap hex table list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_partition_list_hex_format_str(allocator, hex_remap_partition_list_))) { + LOG_WARN("fail to get remap hex partition list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablegroup_list_hex_format_str(allocator, hex_remap_tablegroup_list_))) { + LOG_WARN("fail to get remap hex tablegroup list format str", KR(ret), K(remap_table_arg)); + } else if (OB_FAIL(remap_table_arg.get_remap_tablespace_list_hex_format_str(allocator, hex_remap_tablespace_list_))) { + LOG_WARN("fail to get remap hex tablespace list format str", KR(ret), K(remap_table_arg)); + } else { + FILL_STR_COLUMN(backup_set_list) + FILL_STR_COLUMN(backup_piece_list) + FILL_STR_COLUMN(db_list) + FILL_STR_COLUMN(table_list) + FILL_STR_COLUMN(partition_list) + FILL_STR_COLUMN(remap_db_list) + FILL_STR_COLUMN(remap_table_list) + FILL_STR_COLUMN(remap_partition_list) + FILL_STR_COLUMN(remap_tablegroup_list) + FILL_STR_COLUMN(remap_tablespace_list) + + FILL_STR_COLUMN(hex_db_list) + FILL_STR_COLUMN(hex_table_list) + FILL_STR_COLUMN(hex_partition_list) + FILL_STR_COLUMN(hex_remap_db_list) + FILL_STR_COLUMN(hex_remap_table_list) + FILL_STR_COLUMN(hex_remap_partition_list) + FILL_STR_COLUMN(hex_remap_tablegroup_list) + FILL_STR_COLUMN(hex_remap_tablespace_list) + } + } + + return ret; +} + +#undef PARSE_INT_VALUE +#undef PARSE_UINT_VALUE +#undef PARSE_STR_VALUE +#undef FILL_INT_COLUMN +#undef FILL_UINT_COLUMN +#undef FILL_STR_COLUMN diff --git a/src/share/restore/ob_import_table_struct.h b/src/share/restore/ob_import_table_struct.h new file mode 100644 index 0000000000..0f6f413912 --- /dev/null +++ b/src/share/restore/ob_import_table_struct.h @@ -0,0 +1,441 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_TABLE_STRUCT_H +#define OCEANBASE_SHARE_IMPORT_TABLE_STRUCT_H + +#include "lib/ob_define.h" +#include "share/backup/ob_backup_struct.h" +#include "share/ob_inner_table_operator.h" +#include "share/restore/ob_import_arg.h" + +namespace oceanbase +{ +namespace share +{ +class ObImportResult final +{ +public: + using Comment = common::ObFixedLengthString; + ObImportResult(): is_succeed_(true), comment_() {} + ~ObImportResult() {} + void set_result(const bool is_succeed, const Comment &comment = ""); + int set_result(const bool is_succeed, const char *buf); + int set_result(const int err_code, const share::ObTaskId &trace_id, const ObAddr &addr, const ObString &extra_info = ObString()); + void reset(); + const char *get_result_str() const { return is_succeed_ ? "SUCCEESS" : "FAILED"; } + const char *get_comment() const { return comment_.ptr(); } + bool is_succeed() const { return is_succeed_; } + bool is_comment_setted() const { return !comment_.is_empty(); } + ObImportResult &operator=(const ObImportResult &result); + TO_STRING_KV(K_(is_succeed), K_(comment)); +private: + bool is_succeed_; + Comment comment_; +}; + +#ifdef Property_declare_int +#undef Property_declare_int +#endif + +#define Property_declare_int(variable_type, variable_name)\ +private:\ + variable_type variable_name##_;\ +public:\ + variable_type get_##variable_name() const\ + { return variable_name##_;}\ + void set_##variable_name(variable_type other)\ + { variable_name##_ = other;} + +#ifdef Property_declare_ObString +#undef Property_declare_ObString +#endif +#define Property_declare_ObString(variable_name)\ +private:\ + ObString variable_name##_;\ +public:\ + const ObString &get_##variable_name() const\ + { return variable_name##_;}\ + int set_##variable_name(const ObString &str)\ + { return deep_copy_ob_string(allocator_, str, variable_name##_);} + +#ifdef Property_declare_struct +#undef Property_declare_struct +#endif + +#define Property_declare_struct(variable_type, variable_name)\ +private:\ + variable_type variable_name##_;\ +public:\ + const variable_type &get_##variable_name() const\ + { return variable_name##_;}\ + variable_type &get_##variable_name() \ + { return variable_name##_;}\ + void set_##variable_name(variable_type other)\ + { variable_name##_ = other;} + +class ObImportTableTaskStatus final +{ +public: + enum Status + { + INIT = 0, + DOING = 1, + FINISH = 2, + MAX + }; + ObImportTableTaskStatus(): status_(MAX) {} + ~ObImportTableTaskStatus() {} + ObImportTableTaskStatus(const Status &status): status_(status) {} + ObImportTableTaskStatus(const ObImportTableTaskStatus &status): status_(status.status_) {} + ObImportTableTaskStatus &operator=(const ObImportTableTaskStatus &status) { + if (this != &status) { + status_ = status.status_; + } + return *this; + } + ObImportTableTaskStatus &operator=(const Status &status) { status_ = status; return *this; } + bool operator ==(const ObImportTableTaskStatus &other) const { return status_ == other.status_; } + bool operator !=(const ObImportTableTaskStatus &other) const { return status_ != other.status_; } + operator Status() const { return status_; } + bool is_valid() const { return status_ >= INIT && status_ < MAX; } + bool is_finish() const { return FINISH == status_; } + + Status get_status() const { return status_; } + ObImportTableTaskStatus get_next_status(const int err_code); + const char* get_str() const; + int set_status(const char *str); + TO_STRING_KV(K_(status)) +private: + Status status_; +}; + +struct ObImportTableTask final : public ObIInnerTableRow +{ +public: + struct Key final : public ObIInnerTableKey + { + Key() : tenant_id_(OB_INVALID_TENANT_ID), task_id_() {} + ~Key() {} + void reset() { tenant_id_ = OB_INVALID_TENANT_ID; task_id_ = 0; } + bool is_pkey_valid() const override { return (is_user_tenant(tenant_id_) || is_sys_tenant(tenant_id_)) && task_id_ != 0; } + bool operator==(const Key &other) const + { + return task_id_ == other.task_id_ && tenant_id_ == other.tenant_id_; + } + bool operator!=(const Key &other) const + { + return task_id_ != other.task_id_ || tenant_id_ != other.tenant_id_; + } + int fill_pkey_dml(share::ObDMLSqlSplicer &dml) const override; + TO_STRING_KV(K_(tenant_id), K_(task_id)); + uint64_t tenant_id_; + int64_t task_id_; + }; +public: + ObImportTableTask() { reset(); } + virtual ~ObImportTableTask() {} + bool is_valid() const { return is_pkey_valid(); } + void reset(); + int assign(const ObImportTableTask &that); + int get_pkey(Key &key) const; + bool is_pkey_valid() const override { return (is_user_tenant(tenant_id_) || is_sys_tenant(tenant_id_)) && task_id_ != 0; } + int fill_pkey_dml(share::ObDMLSqlSplicer &dml) const override; + int parse_from(common::sqlclient::ObMySQLResult &result) override; + int fill_dml(share::ObDMLSqlSplicer &dml) const override; + + TO_STRING_KV(K_(tenant_id), K_(task_id), K_(job_id), K_(src_tenant_id), K_(src_tablespace), K_(src_tablegroup), + K_(src_database), K_(src_table), K_(src_partition), K_(target_tablespace), K_(target_tablegroup), K_(target_database), + K_(target_table), K_(table_column), K_(status), K_(start_ts), K_(completion_ts), K_(cumulative_ts), K_(total_bytes), + K_(total_rows), K_(imported_bytes), K_(imported_rows), K_(total_index_count), K_(imported_index_count), + K_(failed_index_count), K_(total_constraint_count), K_(imported_constraint_count), K_(failed_constraint_count), + K_(total_ref_constraint_count), K_(imported_ref_constraint_count), K_(failed_ref_constraint_count), + K_(total_trigger_count), K_(imported_trigger_count), K_(failed_trigger_count), K_(result)); + + Property_declare_int(uint64_t, tenant_id) + Property_declare_int(int64_t, task_id) + Property_declare_int(int64_t, job_id) + Property_declare_int(uint64_t, src_tenant_id) + Property_declare_ObString(src_tablespace) + Property_declare_ObString(src_tablegroup) + Property_declare_ObString(src_database) + Property_declare_ObString(src_table) + Property_declare_ObString(src_partition) + Property_declare_ObString(target_tablespace) + Property_declare_ObString(target_tablegroup) + Property_declare_ObString(target_database) + Property_declare_ObString(target_table) + Property_declare_int(int64_t, table_column) + Property_declare_int(ObImportTableTaskStatus, status) + Property_declare_int(int64_t, start_ts) + Property_declare_int(int64_t, completion_ts) + Property_declare_int(int64_t, cumulative_ts) + Property_declare_int(int64_t, total_bytes) + Property_declare_int(int64_t, total_rows) + Property_declare_int(int64_t, imported_bytes) + Property_declare_int(int64_t, imported_rows) + Property_declare_int(int64_t, total_index_count) + Property_declare_int(int64_t, imported_index_count) + Property_declare_int(int64_t, failed_index_count) + Property_declare_int(int64_t, total_constraint_count) + Property_declare_int(int64_t, imported_constraint_count) + Property_declare_int(int64_t, failed_constraint_count) + Property_declare_int(int64_t, total_ref_constraint_count) + Property_declare_int(int64_t, imported_ref_constraint_count) + Property_declare_int(int64_t, failed_ref_constraint_count) + Property_declare_int(int64_t, total_trigger_count) + Property_declare_int(int64_t, imported_trigger_count) + Property_declare_int(int64_t, failed_trigger_count) + Property_declare_struct(ObImportResult, result) + +private: + common::ObArenaAllocator allocator_; +}; + +class ObImportTableJobStatus final +{ +public: + enum Status + { + INIT = 0, + IMPORT_TABLE = 1, + RECONSTRUCT_REF_CONSTRAINT = 2, + CANCELING = 3, + IMPORT_FINISH = 4, + MAX_STATUS + }; + ObImportTableJobStatus(): status_(MAX_STATUS) {} + ~ObImportTableJobStatus() {} + ObImportTableJobStatus(const Status &status): status_(status) {} + ObImportTableJobStatus(const ObImportTableJobStatus &status): status_(status.status_) {} + ObImportTableJobStatus &operator=(const ObImportTableJobStatus &status) { + if (this != &status) { + status_ = status.status_; + } + return *this; + } + ObImportTableJobStatus &operator=(const Status &status) { status_ = status; return *this; } + bool operator ==(const ObImportTableJobStatus &other) const { return status_ == other.status_; } + bool operator ==(const Status &other) const { return status_ == other; } + bool operator !=(const ObImportTableJobStatus &other) const { return status_ != other.status_; } + bool operator !=(const Status &other) const { return status_ != other; } + operator Status() const { return status_; } + bool is_valid() const { return status_ >= INIT && status_ < MAX_STATUS; } + bool is_finish() const { return IMPORT_FINISH == status_; } + + Status get_status() const { return status_; } + static ObImportTableJobStatus get_next_status(const ObImportTableJobStatus &cur_status); + + const char* get_str() const; + int set_status(const char *str); + + TO_STRING_KV(K_(status)) +private: + Status status_; +}; + + +struct ObImportTableJob final : public ObIInnerTableRow +{ +public: + + struct Key final : public ObIInnerTableKey + { + Key() : tenant_id_(OB_INVALID_TENANT_ID), job_id_(-1) {} + ~Key() {} + void reset() { tenant_id_ = OB_INVALID_TENANT_ID; job_id_ = -1; } + bool is_pkey_valid() const override { return (is_user_tenant(tenant_id_) || is_sys_tenant(tenant_id_)) && job_id_ >= 0; } + bool operator==(const Key &other) const + { + return job_id_ == other.job_id_ && tenant_id_ == other.tenant_id_; + } + bool operator!=(const Key &other) const + { + return job_id_ != other.job_id_ || tenant_id_ != other.tenant_id_; + } + int fill_pkey_dml(share::ObDMLSqlSplicer &dml) const override; + TO_STRING_KV(K_(tenant_id), K_(job_id)); + uint64_t tenant_id_; + int64_t job_id_; + }; + +public: + ObImportTableJob() { reset(); } + virtual ~ObImportTableJob() {} + bool is_valid() const override { return is_pkey_valid(); } + void reset(); + int get_pkey(Key &key) const; + bool is_pkey_valid() const override { return (is_user_tenant(tenant_id_) || is_sys_tenant(tenant_id_)) && job_id_ >= 0; } + int fill_pkey_dml(share::ObDMLSqlSplicer &dml) const override; + int parse_from(common::sqlclient::ObMySQLResult &result) override; + int fill_dml(share::ObDMLSqlSplicer &dml) const override; + int assign(const ObImportTableJob &that); + + TO_STRING_KV(K_(tenant_id), K_(job_id), K_(initiator_tenant_id), K_(initiator_job_id), K_(start_ts), K_(end_ts), + K_(src_tenant_name), K_(src_tenant_id), K_(status), K_(total_table_count), K_(finished_table_count), + K_(failed_table_count), K_(total_bytes), K_(finished_bytes), K_(failed_bytes), K_(result), K_(import_arg)); + + Property_declare_int(uint64_t, tenant_id) + Property_declare_int(int64_t, job_id) + Property_declare_int(uint64_t, initiator_tenant_id) + Property_declare_int(int64_t, initiator_job_id) + Property_declare_int(int64_t, start_ts) + Property_declare_int(int64_t, end_ts) + Property_declare_ObString(src_tenant_name) + Property_declare_int(uint64_t, src_tenant_id) + Property_declare_int(ObImportTableJobStatus, status) + Property_declare_int(int64_t, total_table_count) + Property_declare_int(int64_t, finished_table_count) + Property_declare_int(int64_t, failed_table_count) + Property_declare_int(int64_t, total_bytes) + Property_declare_int(int64_t, finished_bytes) + Property_declare_int(int64_t, failed_bytes) + Property_declare_struct(ObImportResult, result) + Property_declare_ObString(description) + + const share::ObImportArg &get_import_arg() const { return import_arg_; } + share::ObImportArg &get_import_arg() { return import_arg_; } + +private: + share::ObImportArg import_arg_; + common::ObArenaAllocator allocator_; + DISALLOW_COPY_AND_ASSIGN(ObImportTableJob); +}; + +class ObRecoverTableStatus final +{ +public: + enum Status + { + PREPARE = 0, + RECOVERING = 1, + RESTORE_AUX_TENANT = 2, + PRECHECK_IMPORT = 3, + GEN_IMPORT_JOB = 4, + IMPORTING = 5, + CANCELING = 6, + COMPLETED = 7, + FAILED = 8, + MAX_STATUS + }; +public: + ObRecoverTableStatus(): status_(MAX_STATUS) {} + ~ObRecoverTableStatus() {} + ObRecoverTableStatus(const Status &status): status_(status) {} + ObRecoverTableStatus(const ObRecoverTableStatus &status): status_(status.status_) {} + ObRecoverTableStatus &operator=(const ObRecoverTableStatus &status) { + if (this != &status) { + status_ = status.status_; + } + return *this; + } + + ObRecoverTableStatus &operator=(const Status &status) { status_ = status; return *this; } + bool operator ==(const ObRecoverTableStatus &other) const { return status_ == other.status_; } + bool operator ==(const ObRecoverTableStatus::Status &other) const { return status_ == other; } + bool operator !=(const ObRecoverTableStatus::Status &other) const { return status_ != other; } + bool operator !=(const ObRecoverTableStatus &other) const { return status_ != other.status_; } + operator Status() const { return status_; } + bool is_valid() const { return status_ >= PREPARE && status_ < MAX_STATUS; } + bool is_finish() const { return COMPLETED == status_ || FAILED == status_; } + bool is_completed() const { return COMPLETED == status_; } + + Status get_status() const { return status_; } + static ObRecoverTableStatus get_sys_next_status(const ObRecoverTableStatus &cur_status); // used by sys job + static ObRecoverTableStatus get_user_next_status(const ObRecoverTableStatus &cur_status); // used by user job + + const char* get_str() const; + int set_status(const char *str); + + TO_STRING_KV(K_(status)) + +private: + Status status_; +}; + +struct ObRecoverTableJob final : public ObIInnerTableRow +{ +public: + struct Key final : public ObIInnerTableKey + { + Key() : tenant_id_(OB_INVALID_TENANT_ID), job_id_(-1) {} + ~Key() {} + void reset() { tenant_id_ = OB_INVALID_TENANT_ID; job_id_ = -1; } + bool is_pkey_valid() const override { + return (is_user_tenant(tenant_id_) || is_sys_tenant(tenant_id_)) && job_id_ >= 0; } + + bool operator==(const Key &other) const + { + return job_id_ == other.job_id_ && tenant_id_ == other.tenant_id_; + } + bool operator!=(const Key &other) const + { + return job_id_ != other.job_id_ || tenant_id_ != other.tenant_id_; + } + int fill_pkey_dml(share::ObDMLSqlSplicer &dml) const override; + TO_STRING_KV(K_(tenant_id), K_(job_id)); + uint64_t tenant_id_; + int64_t job_id_; + }; + +public: + ObRecoverTableJob() { reset(); } + virtual ~ObRecoverTableJob() {} + bool is_valid() const override { return is_pkey_valid(); } + void reset(); + int assign(const ObRecoverTableJob &that); + bool is_pkey_valid() const override { return (is_user_tenant(tenant_id_) || is_sys_tenant(tenant_id_)) && job_id_ >= 0; } + + int get_pkey(Key &key) const; + int fill_pkey_dml(share::ObDMLSqlSplicer &dml) const override; + int parse_from(common::sqlclient::ObMySQLResult &result) override; + int fill_dml(share::ObDMLSqlSplicer &dml) const override; + + TO_STRING_KV(K_(tenant_id), K_(job_id), K_(initiator_tenant_id), K_(initiator_job_id), K_(start_ts), K_(end_ts), K_(status), + K_(aux_tenant_name), K_(target_tenant_name), K_(target_tenant_id), K_(restore_scn), K_(restore_option), + K_(backup_dest), K_(backup_dest), K_(backup_passwd), K_(external_kms_info), K_(result), + K_(import_arg), K_(multi_restore_path_list)); + + Property_declare_int(uint64_t, tenant_id) + Property_declare_int(int64_t, job_id) + Property_declare_int(uint64_t, initiator_tenant_id) + Property_declare_int(int64_t, initiator_job_id) + Property_declare_int(int64_t, start_ts) + Property_declare_int(int64_t, end_ts) + Property_declare_int(ObRecoverTableStatus, status) + Property_declare_ObString(aux_tenant_name) + Property_declare_ObString(target_tenant_name) + Property_declare_int(uint64_t, target_tenant_id) + Property_declare_int(share::SCN, restore_scn) + Property_declare_ObString(restore_option) + Property_declare_ObString(backup_dest) + Property_declare_ObString(backup_passwd) + Property_declare_ObString(external_kms_info) + Property_declare_struct(ObImportResult, result) + Property_declare_ObString(description) + ObPhysicalRestoreBackupDestList& get_multi_restore_path_list() { return multi_restore_path_list_; } + const ObPhysicalRestoreBackupDestList& get_multi_restore_path_list() const { return multi_restore_path_list_; } + share::ObImportArg &get_import_arg() { return import_arg_; } + const share::ObImportArg &get_import_arg() const { return import_arg_; } +private: + share::ObImportArg import_arg_; + share::ObPhysicalRestoreBackupDestList multi_restore_path_list_; + common::ObArenaAllocator allocator_; +}; + +#undef Property_declare_int +#undef Property_declare_ObString +#undef Property_declare_struct +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_import_util.cpp b/src/share/restore/ob_import_util.cpp new file mode 100644 index 0000000000..2ae8f77f73 --- /dev/null +++ b/src/share/restore/ob_import_util.cpp @@ -0,0 +1,155 @@ +/** + * 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 SHARE +#include "ob_import_util.h" + +using namespace oceanbase; +using namespace share; + +bool ObImportTableUtil::can_retrieable_err(const int err_code) +{ + bool bret = true; + switch(err_code) { + case OB_NOT_INIT : + case OB_INIT_TWICE: + case OB_INVALID_ARGUMENT : + case OB_ERR_UNEXPECTED : + case OB_ERR_SYS : + case OB_CANCELED : + case OB_NOT_SUPPORTED : + case OB_TENANT_HAS_BEEN_DROPPED : + case OB_TENANT_NOT_EXIST: + case OB_ERR_INVALID_TENANT_NAME: + case OB_HASH_EXIST: + case OB_HASH_NOT_EXIST: + case OB_ENTRY_EXIST: + case OB_ENTRY_NOT_EXIST: + case OB_VERSION_NOT_MATCH: + case OB_STATE_NOT_MATCH: + case OB_TABLE_NOT_EXIST: + case OB_ERR_BAD_DATABASE: + case OB_LS_RESTORE_FAILED: + case OB_OP_NOT_ALLOW: + case OB_TABLEGROUP_NOT_EXIST: + case OB_TABLESPACE_NOT_EXIST: + case OB_ERR_TABLE_EXIST: + bret = false; + break; + default: + break; + } + return bret; +} + +int ObImportTableUtil::get_tenant_schema_guard(share::schema::ObMultiVersionSchemaService &schema_service, uint64_t tenant_id, + share::schema::ObSchemaGetterGuard &guard) +{ + int ret = OB_SUCCESS; + schema::ObTenantStatus status; + if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id, guard))) { + LOG_WARN("faield to get tenant schema guard", K(ret)); + } else if (OB_FAIL(guard.get_tenant_status(tenant_id, status))) { + LOG_WARN("failed to get tenant status", K(ret)); + } else if (schema::ObTenantStatus::TENANT_STATUS_NORMAL != status) { + ret = OB_STATE_NOT_MATCH; + LOG_WARN("tenant status is not normal", K(ret), K(tenant_id)); + } + return ret; +} + +int ObImportTableUtil::check_database_schema_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &db_name, bool &is_exist) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard guard; + if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id, guard))) { + LOG_WARN("faield to get tenant schema guard", K(ret)); + } else if (OB_FAIL(guard.check_database_exist(tenant_id, db_name, is_exist))) { + LOG_WARN("failed to check database exist", K(ret), K(tenant_id), K(db_name)); + } + return ret; +} + +int ObImportTableUtil::check_table_schema_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &db_name, const ObString &table_name, bool &is_exist) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard guard; + uint64_t database_id = OB_INVALID_ID; + is_exist = false; + if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id, guard))) { + LOG_WARN("faield to get tenant schema guard", K(ret)); + } else if (OB_FAIL(guard.get_database_id(tenant_id, db_name, database_id))) { + LOG_WARN("failed to get database id", K(ret), K(tenant_id)); + } else if (OB_INVALID_ID == database_id) { + LOG_WARN("database not exist", K(tenant_id), K(db_name)); + } else if (OB_FAIL(guard.check_table_exist(tenant_id, database_id, table_name, false, + schema::ObSchemaGetterGuard::CheckTableType::ALL_NON_HIDDEN_TYPES, is_exist))) { + LOG_WARN("failed to check table exist", K(ret)); + } + return ret; +} + +int ObImportTableUtil::check_tablegroup_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &tablegroup, bool &is_exist) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard guard; + uint64_t table_group_id = OB_INVALID_ID; + is_exist = false; + if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id, guard))) { + LOG_WARN("faield to get tenant schema guard", K(ret)); + } else if (OB_FAIL(guard.get_tablegroup_id(tenant_id, tablegroup, table_group_id))) { + LOG_WARN("failed to get tablegroup id", K(ret), K(tenant_id)); + } else if (OB_INVALID_ID == table_group_id) { + LOG_INFO("tablegroup not exist", K(tenant_id), K(tablegroup)); + } else { + is_exist = true; + } + return ret; +} + +int ObImportTableUtil::check_tablespace_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &tablespace, bool &is_exist) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard guard; + const schema::ObTablespaceSchema *schema = nullptr; + is_exist = false; + if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id, guard))) { + LOG_WARN("faield to get tenant schema guard", K(ret)); + } else if (OB_FAIL(guard.get_tablespace_schema_with_name(tenant_id, tablespace, schema))) { + LOG_WARN("failed to get tablespace id", K(ret), K(tenant_id)); + } else if (OB_ISNULL(schema)) { + LOG_INFO("tablespace not exist", K(tenant_id), K(tablespace)); + } else { + is_exist = true; + } + return ret; +} + +int ObImportTableUtil::get_tenant_name_case_mode(const uint64_t tenant_id, ObNameCaseMode &name_case_mode) +{ + int ret = OB_SUCCESS; + schema::ObSchemaGetterGuard guard; + share::schema::ObMultiVersionSchemaService *schema_service = nullptr; + if (!is_valid_tenant_id(tenant_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid tenant id", K(ret), K(tenant_id)); + } else if (OB_ISNULL(schema_service = GCTX.schema_service_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("schema service must not be null", K(ret)); + } else if (OB_FAIL(schema_service->get_tenant_name_case_mode(tenant_id, name_case_mode))) { + LOG_WARN("faield to get tenant schema guard", K(ret), K(tenant_id)); + } + return ret; +} \ No newline at end of file diff --git a/src/share/restore/ob_import_util.h b/src/share/restore/ob_import_util.h new file mode 100644 index 0000000000..78aa0c5f4d --- /dev/null +++ b/src/share/restore/ob_import_util.h @@ -0,0 +1,41 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_IMPORT_TABLE_UTIL_H +#define OCEANBASE_SHARE_IMPORT_TABLE_UTIL_H +#include "lib/ob_define.h" +#include "share/schema/ob_multi_version_schema_service.h" +namespace oceanbase +{ +namespace share +{ +class ObImportTableUtil final +{ +public: +static bool can_retrieable_err(const int err_code); +static int get_tenant_schema_guard(share::schema::ObMultiVersionSchemaService &schema_service, uint64_t tenant_id, + share::schema::ObSchemaGetterGuard &guard); +static int check_database_schema_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &db_name, bool &is_exist); +static int check_table_schema_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &db_name, const ObString &table_name, bool &is_exist); +static int check_tablegroup_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &tablegroup, bool &is_exist); +static int check_tablespace_exist(share::schema::ObMultiVersionSchemaService &schema_service, + uint64_t tenant_id, const ObString &tablespace, bool &is_exist); +static int get_tenant_name_case_mode(const uint64_t tenant_id, ObNameCaseMode &name_case_mode); +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_physical_restore_table_operator.cpp b/src/share/restore/ob_physical_restore_table_operator.cpp index fa52801008..6e83804518 100644 --- a/src/share/restore/ob_physical_restore_table_operator.cpp +++ b/src/share/restore/ob_physical_restore_table_operator.cpp @@ -518,6 +518,16 @@ int ObPhysicalRestoreTableOperator::retrieve_restore_option( RETRIEVE_STR_VALUE(kms_encrypt_key, job); RETRIEVE_INT_VALUE(concurrency, job); + if (OB_SUCC(ret)) { + if (name == "backup_dest") { + ObString value; + EXTRACT_VARCHAR_FIELD_MYSQL_WITH_DEFAULT_VALUE(result, "value", value, true, false, value); + if (FAILEDx((job).set_backup_dest(value))) { + LOG_WARN("failed to set column value", KR(ret), K(value)); + } + } + } + if (OB_SUCC(ret)) { if (name == "kms_encrypt") { int64_t kms_encrypt = 0; @@ -955,6 +965,31 @@ int ObPhysicalRestoreTableOperator::get_job_by_tenant_name( } return ret; } + +int ObPhysicalRestoreTableOperator::get_job_by_restore_tenant_name( + const ObString &tenant_name, + ObPhysicalRestoreJob &job_info) +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(tenant_id_); + if (!inited_) { + ret = OB_NOT_INIT; + LOG_WARN("sql proxy is null", K(ret)); + } else if (OB_UNLIKELY(tenant_name.empty())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid exec_tenant_id", K(ret), K(tenant_name)); + } else if (OB_FAIL(sql.assign_fmt("SELECT * FROM %s WHERE job_id in (SELECT job_id " + "FROM %s WHERE name = 'restore_tenant_name' AND value = '%.*s')", + OB_ALL_RESTORE_JOB_TNAME, + OB_ALL_RESTORE_JOB_TNAME, + tenant_name.length(), tenant_name.ptr()))) { + LOG_WARN("failed to assign sql", K(ret), K(tenant_name)); + } else if (OB_FAIL(get_restore_job_by_sql_(exec_tenant_id, sql, job_info))) { + LOG_WARN("failed to get restore job by sql", KR(ret), K(sql), K(exec_tenant_id)); + } + return ret; +} int ObPhysicalRestoreTableOperator::get_restore_job_by_sql_( const uint64_t exec_tenant_id, const ObSqlString &sql, ObPhysicalRestoreJob &job_info) diff --git a/src/share/restore/ob_physical_restore_table_operator.h b/src/share/restore/ob_physical_restore_table_operator.h index 5e6de3107a..2c8f8cc083 100644 --- a/src/share/restore/ob_physical_restore_table_operator.h +++ b/src/share/restore/ob_physical_restore_table_operator.h @@ -115,6 +115,8 @@ public: * */ int get_job_by_tenant_name(const ObString &tenant_name, ObPhysicalRestoreJob &job_info); + int get_job_by_restore_tenant_name(const ObString &tenant_name, + ObPhysicalRestoreJob &job_info); /* * description: check all ls has restored to consistent_scn diff --git a/src/share/restore/ob_recover_table_persist_helper.cpp b/src/share/restore/ob_recover_table_persist_helper.cpp new file mode 100644 index 0000000000..40ba6cbb63 --- /dev/null +++ b/src/share/restore/ob_recover_table_persist_helper.cpp @@ -0,0 +1,313 @@ +/** + * 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 SHARE + +#include "ob_recover_table_persist_helper.h" +#include "ob_import_table_struct.h" +#include "share/inner_table/ob_inner_table_schema_constants.h" + +using namespace oceanbase; +using namespace share; +ObRecoverTablePersistHelper::ObRecoverTablePersistHelper() + : is_inited_(false), tenant_id_() +{ +} + +int ObRecoverTablePersistHelper::init(const uint64_t tenant_id) +{ + int ret = OB_SUCCESS; + if(!is_sys_tenant(tenant_id) && !is_user_tenant(tenant_id)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid tenant id", K(ret), K(tenant_id)); + } else if (OB_FAIL(table_op_.init(OB_ALL_RECOVER_TABLE_JOB_TNAME, *this))) { + LOG_WARN("failed to init table op", K(ret)); + } else { + tenant_id_ = tenant_id; + is_inited_ = true; + } + return ret; +} + +int ObRecoverTablePersistHelper::insert_recover_table_job( + common::ObISQLClient &proxy, const ObRecoverTableJob &job) const +{ + int ret = OB_SUCCESS; + int64_t affect_rows = 0; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.insert_or_update_row(proxy, job, affect_rows))) { + LOG_WARN("failed to insert or update row", K(ret), K(job)); + } + return ret; +} + +int ObRecoverTablePersistHelper::get_recover_table_job( + common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id, + ObRecoverTableJob &job) const +{ + int ret = OB_SUCCESS; + ObRecoverTableJob::Key key; + key.tenant_id_ = tenant_id; + key.job_id_ = job_id; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op_.get_row(proxy, false, key, job))) { + LOG_WARN("failed to get row", KR(ret), K(key)); + } + return ret; +} + +int ObRecoverTablePersistHelper::is_recover_table_job_exist( + common::ObISQLClient &proxy, + const uint64_t target_tenant_id, + bool &is_exist) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(); + if (OB_FAIL(sql.assign_fmt("select count(*) cnt from %s where target_tenant_id=%lu", OB_ALL_RECOVER_TABLE_JOB_TNAME, target_tenant_id))) { + LOG_WARN("fail to assign sql", K(ret)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_FAIL(result->next())) { + LOG_WARN("failed to get next row", K(ret)); + } else { + int64_t cnt = 0; + EXTRACT_INT_FIELD_MYSQL(*result, "cnt", cnt, int64_t); + if (OB_FAIL(ret)) { + } else { + is_exist = 0 != cnt; + } + } + } + } + + return ret; +} + +int ObRecoverTablePersistHelper::advance_status( + common::ObISQLClient &proxy, const ObRecoverTableJob &job, const ObRecoverTableStatus &next_status) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = -1; + ObDMLSqlSplicer dml; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_JOB_ID, job.get_job_id()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, job.get_tenant_id()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_RESULT, job.get_result().get_result_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_COMMENT, job.get_result().get_comment()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (next_status.is_finish() && OB_FAIL(dml.add_column(OB_STR_END_TS, job.get_end_ts()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, next_status.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.splice_update_sql(OB_ALL_RECOVER_TABLE_JOB_TNAME, sql))) { + LOG_WARN("failed to splice update sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" and %s='%s'", OB_STR_STATUS, job.get_status().get_str()))) { + LOG_WARN("failed to append fmt", K(ret)); + } else if (OB_FAIL(proxy.write(get_exec_tenant_id(), sql.ptr(), affected_rows))) { + LOG_WARN("failed to write", K(ret), K(sql)); + } + return ret; +} + +int ObRecoverTablePersistHelper::force_cancel_recover_job(common::ObISQLClient &proxy) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + int64_t affected_rows = -1; + ObDMLSqlSplicer dml; + ObRecoverTableStatus status(ObRecoverTableStatus::CANCELING); + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(dml.add_pk_column(OB_STR_TENANT_ID, tenant_id_))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.add_column(OB_STR_STATUS, status.get_str()))) { + LOG_WARN("failed to add column", K(ret)); + } else if (OB_FAIL(dml.splice_update_sql(OB_ALL_RECOVER_TABLE_JOB_TNAME, sql))) { + LOG_WARN("failed to splice update sql", K(ret)); + } else if (OB_FAIL(proxy.write(get_exec_tenant_id(), sql.ptr(), affected_rows))) { + LOG_WARN("failed to exec sql", K(ret), K(sql)); + } else { + LOG_INFO("success cancel recover job", K(tenant_id_)); + } + return ret; +} + +int ObRecoverTablePersistHelper::get_all_recover_table_job( + common::ObISQLClient &proxy, common::ObIArray &jobs) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + const uint64_t exec_tenant_id = get_exec_tenant_id(); + jobs.reset(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_RECOVER_TABLE_JOB_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else { + while (OB_SUCC(ret)) { + ObRecoverTableJob job; + job.reset(); + if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + break; + } else { + LOG_WARN("failed to get next row", K(ret)); + } + } else if (OB_FAIL(job.parse_from(*result))) { + LOG_WARN("failed to parse job result", K(ret)); + } else if (OB_FAIL(jobs.push_back(job))) { + LOG_WARN("failed to push back job", K(ret), K(job)); + } + } + } + } + } + LOG_INFO("get targets", K(ret), K(jobs), K(exec_tenant_id), K(sql)); + return ret; +} + +int ObRecoverTablePersistHelper::get_recover_table_job_by_initiator(common::ObISQLClient &proxy, + const ObRecoverTableJob &initiator_job, ObRecoverTableJob &target_job) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + uint64_t initiator_tenant_id = initiator_job.get_tenant_id(); + int64_t initiator_job_id = initiator_job.get_job_id(); + const uint64_t exec_tenant_id = get_exec_tenant_id(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_RECOVER_TABLE_JOB_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" where initiator_tenant_id =%lu and initiator_job_id=%ld", initiator_tenant_id, initiator_job_id))) { + LOG_WARN("failed to append sql", K(ret), K(sql), K(initiator_tenant_id), K(initiator_job_id)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + ret = OB_ENTRY_NOT_EXIST; + } + LOG_WARN("no row exist", K(ret)); + } else if (OB_FAIL(target_job.parse_from(*result))) { + LOG_WARN("failed to parse row", K(ret)); + } else if (OB_ITER_END != result->next()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("multi value exist", K(ret), K(sql), K(exec_tenant_id)); + } else { + ret = OB_SUCCESS; + } + } + } + LOG_INFO("get recover table job", K(ret), K(target_job), K(sql)); + return ret; +} + +int ObRecoverTablePersistHelper::delete_recover_table_job( + common::ObISQLClient &proxy, const ObRecoverTableJob &job) const +{ + int ret = OB_SUCCESS; + int64_t affect_rows = 0; + ObRecoverTableJob::Key key; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(job.get_pkey(key))) { + LOG_WARN("failed to get pkey", K(ret)); + } else if (OB_FAIL(table_op_.delete_row(proxy, key, affect_rows))) { + LOG_WARN("failed to delete row", K(ret), K(job)); + } + return ret; +} + +int ObRecoverTablePersistHelper::insert_recover_table_job_history( + common::ObISQLClient &proxy, const ObRecoverTableJob &job) const +{ + int ret = OB_SUCCESS; + int64_t affect_rows = 0; + ObInnerTableOperator table_op; + if (IS_NOT_INIT) { + ret = OB_NOT_INIT; + LOG_WARN("ObRecoverTablePersistHelper not init", K(ret)); + } else if (OB_FAIL(table_op.init(OB_ALL_RECOVER_TABLE_JOB_HISTORY_TNAME, *this))) { + LOG_WARN("failed to init table op", K(ret)); + } else if (OB_FAIL(table_op.insert_or_update_row(proxy, job, affect_rows))) { + LOG_WARN("failed to insert or update row", K(ret), K(job)); + } + return ret; +} + +int ObRecoverTablePersistHelper::get_recover_table_job_history_by_initiator(common::ObISQLClient &proxy, + const ObRecoverTableJob &initiator_job, ObRecoverTableJob &target_job) const +{ + int ret = OB_SUCCESS; + ObSqlString sql; + uint64_t initiator_tenant_id = initiator_job.get_tenant_id(); + int64_t initiator_job_id = initiator_job.get_job_id(); + const uint64_t exec_tenant_id = get_exec_tenant_id(); + if (OB_FAIL(sql.assign_fmt("select * from %s", OB_ALL_RECOVER_TABLE_JOB_HISTORY_TNAME))) { + LOG_WARN("fail to assign sql", K(ret)); + } else if (OB_FAIL(sql.append_fmt(" where initiator_tenant_id =%lu and initiator_job_id=%ld", initiator_tenant_id, initiator_job_id))) { + LOG_WARN("failed to append sql", K(ret), K(sql), K(initiator_tenant_id), K(initiator_job_id)); + } else { + HEAP_VAR(ObMySQLProxy::ReadResult, res) { + ObMySQLResult *result = NULL; + if (OB_FAIL(proxy.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to exec sql", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is null", K(ret), K(sql), K(exec_tenant_id)); + } else if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + ret = OB_ENTRY_NOT_EXIST; + } + LOG_WARN("no row exist", K(ret)); + } else if (OB_FAIL(target_job.parse_from(*result))) { + LOG_WARN("failed to parse row", K(ret)); + } else if (OB_ITER_END != result->next()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("multi value exist", K(ret), K(sql), K(exec_tenant_id)); + } else { + ret = OB_SUCCESS; + } + } + } + LOG_INFO("get recover table job history", K(ret), K(target_job), K(sql)); + return ret; +} \ No newline at end of file diff --git a/src/share/restore/ob_recover_table_persist_helper.h b/src/share/restore/ob_recover_table_persist_helper.h new file mode 100644 index 0000000000..1ed1763228 --- /dev/null +++ b/src/share/restore/ob_recover_table_persist_helper.h @@ -0,0 +1,64 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_RECOVER_TABLE_PERSIST_HELPER_H +#define OCEANBASE_SHARE_RECOVER_TABLE_PERSIST_HELPER_H + +#include "lib/ob_define.h" +#include "share/ob_inner_table_operator.h" +#include "share/restore/ob_import_table_struct.h" + +namespace oceanbase +{ +namespace share +{ +class ObRecoverTablePersistHelper final : public ObIExecTenantIdProvider +{ +public: + ObRecoverTablePersistHelper(); + virtual ~ObRecoverTablePersistHelper() {} + int init(const uint64_t tenant_id); + void reset() { is_inited_ = false; } + uint64_t get_exec_tenant_id() const override { return gen_meta_tenant_id(tenant_id_); } + int insert_recover_table_job(common::ObISQLClient &proxy, const ObRecoverTableJob &job) const; + + int get_all_recover_table_job(common::ObISQLClient &proxy, common::ObIArray &jobs) const; + + int get_recover_table_job(common::ObISQLClient &proxy, const uint64_t tenant_id, const int64_t job_id, + ObRecoverTableJob &job) const; + + int is_recover_table_job_exist(common::ObISQLClient &proxy, const uint64_t target_tenant_id, bool &is_exist) const; + + int advance_status(common::ObISQLClient &proxy, + const ObRecoverTableJob &job, const ObRecoverTableStatus &next_status) const; + int force_cancel_recover_job(common::ObISQLClient &proxy) const; + + int get_recover_table_job_by_initiator(common::ObISQLClient &proxy, + const ObRecoverTableJob &initiator_job, ObRecoverTableJob &target_job) const; + + int delete_recover_table_job(common::ObISQLClient &proxy, const ObRecoverTableJob &job) const; + int insert_recover_table_job_history(common::ObISQLClient &proxy, const ObRecoverTableJob &job) const; + + int get_recover_table_job_history_by_initiator(common::ObISQLClient &proxy, + const ObRecoverTableJob &initiator_job, ObRecoverTableJob &target_job) const; + TO_STRING_KV(K_(is_inited), K_(tenant_id)); +private: + DISALLOW_COPY_AND_ASSIGN(ObRecoverTablePersistHelper); + bool is_inited_; + uint64_t tenant_id_; // sys or user tenant id + ObInnerTableOperator table_op_; +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_recover_table_util.cpp b/src/share/restore/ob_recover_table_util.cpp new file mode 100644 index 0000000000..e992e48c48 --- /dev/null +++ b/src/share/restore/ob_recover_table_util.cpp @@ -0,0 +1,49 @@ +/** + * 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 SHARE +#include "share/restore/ob_recover_table_util.h" + +using namespace oceanbase; +using namespace share; + +int ObRecoverTableUtil::check_compatible(const uint64_t target_tenant_id) +{ + int ret = OB_SUCCESS; + uint64_t data_version = 0; + // 1. check sys tenant data version + if (OB_FAIL(GET_MIN_DATA_VERSION(OB_SYS_TENANT_ID, data_version))) { + LOG_WARN("fail to get sys data version", K(ret)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("Tenant COMPATIBLE is below 4.2.1.0, recover table is not supported", K(ret), K(data_version)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "Tenant COMPATIBLE is below 4.2.1.0, recover table is"); + } + + // 2. check target meta tenant data version + if (FAILEDx(GET_MIN_DATA_VERSION(gen_meta_tenant_id(target_tenant_id), data_version))) { + LOG_WARN("fail to get meta tenant data version", K(ret), K(target_tenant_id)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("Tenant COMPATIBLE is below 4.2.1.0, recover table is not supported", K(ret), K(data_version)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "Tenant COMPATIBLE is below 4.2.1.0, recover table is"); + } + + // 3. check target user tenant data version + if (FAILEDx(GET_MIN_DATA_VERSION(target_tenant_id, data_version))) { + LOG_WARN("fail to get user tenant data version", K(ret), K(target_tenant_id)); + } else if (data_version < DATA_VERSION_4_2_1_0) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("Tenant COMPATIBLE is below 4.2.1.0, recover table is not supported", K(ret), K(data_version)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "Tenant COMPATIBLE is below 4.2.1.0, recover table is"); + } + return ret; +} \ No newline at end of file diff --git a/src/share/restore/ob_recover_table_util.h b/src/share/restore/ob_recover_table_util.h new file mode 100644 index 0000000000..c42e0d4eb2 --- /dev/null +++ b/src/share/restore/ob_recover_table_util.h @@ -0,0 +1,29 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_RECOVER_TABLE_UTIL_H +#define OCEANBASE_SHARE_RECOVER_TABLE_UTIL_H +#include "lib/ob_define.h" +namespace oceanbase +{ +namespace share +{ +class ObRecoverTableUtil final +{ +public: + static int check_compatible(const uint64_t target_tenant_id); +}; + +} +} + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_remap_schema_item.h b/src/share/restore/ob_remap_schema_item.h new file mode 100644 index 0000000000..7a58f3ffde --- /dev/null +++ b/src/share/restore/ob_remap_schema_item.h @@ -0,0 +1,122 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_REMAP_SCHEMA_ITEM_H +#define OCEANBASE_SHARE_REMAP_SCHEMA_ITEM_H + +#include "share/restore/ob_import_table_item.h" +#include "share/restore/ob_import_partition_item.h" + +namespace oceanbase +{ +namespace share +{ + +template +struct ObRemapSchemaItem : public ObIImportItem +{ + OB_UNIS_VERSION(1); +public: + ObRemapSchemaItem() : ObIImportItem(ItemType::REMAP) + {} + + virtual void reset() override; + virtual bool is_valid() const override; + virtual int64_t get_format_serialize_size() const override; + virtual int format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const override; + virtual int deep_copy(common::ObIAllocator &allocator, const ObIImportItem &src) override; + virtual bool operator==(const ObIImportItem &other) const override; + + using ObIImportItem::format_serialize; + int assign(const ObRemapSchemaItem &other); + + TO_STRING_KV(K_(src), K_(target)); + +public: + S src_; + T target_; +}; + + + +template +class ObRemapSchemaItemArray : public ObIImportItemFormatProvider, public ObImportItemHexFormatImpl +{ + OB_UNIS_VERSION(1); +public: + ObRemapSchemaItemArray() {} + virtual int64_t get_format_serialize_size() const override; + virtual int format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const override; + + virtual void reset(); + + int assign(const ObRemapSchemaItemArray &other); + int add_item(const ObRemapSchemaItem &item); + bool is_empty() const { return remap_items_.empty(); } + bool is_src_exist(const S &src) const; + bool is_src_exist(const S &src, const S *&out) const; + bool is_target_exist(const T &target) const; + bool is_target_exist(const T &target, const T *&out) const; + const common::ObSArray> &get_remap_items() const { return remap_items_; } + bool is_remap_target_exist(const S &src, const T *&out) const; + + template + int foreach(Operator &op) const + { + int ret = common::OB_SUCCESS; + ARRAY_FOREACH(remap_items_, idx) { + const ObRemapSchemaItem &remap_item = remap_items_.at(idx); + if (OB_FAIL(op(remap_item))) { + SHARE_LOG(WARN, "fail to do operator", K(ret), K(remap_item)); + } + } + return ret; + } + + TO_STRING_KV(K_(remap_items)); + +protected: + common::ObArenaAllocator allocator_; + +private: + common::ObSArray> remap_items_; +}; + +using ObRemapDatabaseItem = ObRemapSchemaItem; +using ObRemapTableItem = ObRemapSchemaItem; +using ObRemapPartitionItem = ObRemapSchemaItem; +using ObRemapTablespaceItem = ObRemapSchemaItem; +using ObRemapTablegroupItem = ObRemapSchemaItem; + + +using ObRemapDatabaseArray = ObRemapSchemaItemArray; +using ObRemapTableArray = ObRemapSchemaItemArray; +using ObRemapPartitionArray = ObRemapSchemaItemArray; +using ObRemapTablespaceArray = ObRemapSchemaItemArray; +using ObRemapTablegroupArray = ObRemapSchemaItemArray; + + +} +} + +#ifndef INCLUDE_OB_REMAP_SCHEMA_ITEM_HPP +#define INCLUDE_OB_REMAP_SCHEMA_ITEM_HPP +#include "ob_remap_schema_item.hpp" +#endif + +#endif \ No newline at end of file diff --git a/src/share/restore/ob_remap_schema_item.hpp b/src/share/restore/ob_remap_schema_item.hpp new file mode 100644 index 0000000000..4ff81124b9 --- /dev/null +++ b/src/share/restore/ob_remap_schema_item.hpp @@ -0,0 +1,376 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef INCLUDE_OB_REMAP_SCHEMA_ITEM_HPP +#define INCLUDE_OB_REMAP_SCHEMA_ITEM_HPP +#include "share/restore/ob_remap_schema_item.h" +#endif + + +namespace oceanbase +{ +namespace share +{ + +// ObRemapSchemaItem +template +int64_t ObRemapSchemaItem::get_serialize_size() const +{ + int64_t size = 0; + size += src_.get_serialize_size(); + size += target_.get_serialize_size(); + return size; +} + +template +int ObRemapSchemaItem::serialize(SERIAL_PARAMS) const +{ + int ret = OB_SUCCESS; + int64_t new_pos = pos; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (OB_FAIL(src_.serialize(buf, buf_len, new_pos))) { + SHARE_LOG(WARN, "serialize src item failed", K(ret)); + } else if (OB_FAIL(target_.serialize(buf, buf_len, new_pos))) { + SHARE_LOG(WARN, "serialize target item failed", K(ret)); + } else { + pos = new_pos; + } + return ret; +} + +template +int ObRemapSchemaItem::deserialize(DESERIAL_PARAMS) +{ + int ret = OB_SUCCESS; + int64_t new_pos = pos; + if ((NULL == buf) || (data_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(data_len)); + } else if (OB_FAIL(src_.deserialize(buf, data_len, new_pos))) { + SHARE_LOG(WARN, "deserialize src item failed", K(ret)); + } else if (OB_FAIL(target_.deserialize(buf, data_len, new_pos))) { + SHARE_LOG(WARN, "deserialize target item failed", K(ret)); + } else { + pos = new_pos; + } + return ret; +} + + +template +void ObRemapSchemaItem::reset() +{ + src_.reset(); + target_.reset(); +} + +template +bool ObRemapSchemaItem::is_valid() const +{ + return src_.is_valid() && target_.is_valid(); +} + +template +int64_t ObRemapSchemaItem::get_format_serialize_size() const +{ + // Concatenate src and target with ':'. + return src_.get_format_serialize_size() + target_.get_format_serialize_size() + 1; +} + +template +int ObRemapSchemaItem::format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const +{ + int ret = OB_SUCCESS; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (!is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret)); + } else if (OB_FAIL(src_.format_serialize(buf, buf_len, pos))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(pos), K(buf_len), K_(src)); + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", ":"))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(pos), K(buf_len)); + } else if (OB_FAIL(target_.format_serialize(buf, buf_len, pos))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(pos), K(buf_len), K_(target)); + } + + return ret; +} + +template +int ObRemapSchemaItem::deep_copy( + common::ObIAllocator &allocator, + const ObIImportItem &src) +{ + int ret = OB_SUCCESS; + if (!src.is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), K(src)); + } else if (get_item_type() != src.get_item_type()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "item type not match", K(ret), "src", src.get_item_type(), "dest", get_item_type()); + } else { + const ObRemapSchemaItem &other = static_cast &>(src); + if (OB_FAIL(src_.deep_copy(allocator, other.src_))) { + SHARE_LOG(WARN, "failed to copy src", K(ret), K(src)); + } else if (OB_FAIL(target_.deep_copy(allocator, other.target_))) { + SHARE_LOG(WARN, "failed to copy target", K(ret), K(src)); + } + } + return ret; +} + +template +bool ObRemapSchemaItem::operator==(const ObIImportItem &other) const +{ + bool is_equal = false; + if (get_item_type() == other.get_item_type()) { + const ObRemapSchemaItem &the_other = static_cast &>(other); + + is_equal = src_ == the_other.src_ && target_ == the_other.target_; + } + return is_equal; +} + +template +int ObRemapSchemaItem::assign(const ObRemapSchemaItem &other) +{ + int ret = OB_SUCCESS; + if (!other.is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), K(other)); + } else if (OB_FAIL(src_.assign(other.src_))) { + SHARE_LOG(WARN, "failed to assign src", K(ret), K(other)); + } else if (OB_FAIL(target_.assign(other.target_))) { + SHARE_LOG(WARN, "failed to assign target", K(ret), K(other)); + } + + return ret; +} + + +// ObRemapSchemaItemArray +template +int64_t ObRemapSchemaItemArray::get_serialize_size(void) const +{ + return remap_items_.get_serialize_size(); +} + +template +int ObRemapSchemaItemArray::serialize(SERIAL_PARAMS) const +{ + int ret = OB_SUCCESS; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), KP(buf), K(buf_len), K(pos)); + } else if (OB_FAIL(remap_items_.serialize(buf, buf_len, pos))) { + SHARE_LOG(WARN, "failed to serialize", K(ret), KP(buf), K(buf_len), K(pos)); + } + return ret; +} + +template +int ObRemapSchemaItemArray::deserialize(DESERIAL_PARAMS) +{ + int ret = OB_SUCCESS; + common::ObSArray> tmp_remap_items; + if ((NULL == buf) || (data_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), KP(buf), K(data_len), K(pos)); + } else if (OB_FALSE_IT(reset())) { + } else if (OB_FAIL(tmp_remap_items.deserialize(buf, data_len, pos))) { + SHARE_LOG(WARN, "failed to deserialize", K(ret), K(buf), K(data_len), K(pos)); + } + ARRAY_FOREACH(tmp_remap_items, idx) { + const ObRemapSchemaItem &item = tmp_remap_items.at(idx); + if (OB_FAIL(add_item(item))) { + SHARE_LOG(WARN, "failed to add item", K(ret), K(idx), K(item)); + } + } + return ret; +} + + +template +int64_t ObRemapSchemaItemArray::get_format_serialize_size() const +{ + int64_t size = 1; // include '\0' + ARRAY_FOREACH_NORET(remap_items_, idx) { + const ObRemapSchemaItem &item = remap_items_.at(idx); + // item is concatenated with ',' + size += 0 == idx ? 0 : 1; + size += item.get_format_serialize_size(); + } + + return size; +} + +template +int ObRemapSchemaItemArray::format_serialize( + char *buf, + const int64_t buf_len, + int64_t &pos) const +{ + int ret = OB_SUCCESS; + int64_t new_pos = pos; + if ((NULL == buf) || (buf_len <= 0)) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid arguments.", K(ret), KP(buf), K(buf_len)); + } else if (remap_items_.empty()) { + buf[new_pos++] = 0; + } + + ARRAY_FOREACH(remap_items_, idx) { + const ObRemapSchemaItem &remap_item = remap_items_.at(idx); + // concatenated items with ',' + if (0 < idx && OB_FAIL(databuff_printf(buf, buf_len, new_pos, "%s", ","))) { + SHARE_LOG(WARN, "fail to append str", K(ret), K(new_pos), K(buf_len)); + } else if (OB_FAIL(remap_item.format_serialize(buf, buf_len, new_pos))) { + SHARE_LOG(WARN, "fail to format str", K(ret), K(new_pos), K(buf_len)); + } + } + + if (OB_SUCC(ret)) { + pos = new_pos; + } + + return ret; +} + +template +void ObRemapSchemaItemArray::reset() +{ + remap_items_.reset(); + allocator_.reset(); +} + +template +int ObRemapSchemaItemArray::assign(const ObRemapSchemaItemArray &other) +{ + int ret = OB_SUCCESS; + const common::ObSArray> &remap_items = other.get_remap_items(); + + reset(); + ARRAY_FOREACH(remap_items, idx) { + const ObRemapSchemaItem &item = remap_items.at(idx); + if (OB_FAIL(add_item(item))) { + SHARE_LOG(WARN, "failed to add item", K(ret), K(idx), K(item)); + } + } + + return ret; +} + +template +bool ObRemapSchemaItemArray::is_src_exist(const S &src) const +{ + bool is_exist = false; + ARRAY_FOREACH_NORET(remap_items_, idx) { + const ObRemapSchemaItem &tmp = remap_items_.at(idx); + if (tmp.src_ == src) { + is_exist = true; + break; + } + } + return is_exist; +} + +template +bool ObRemapSchemaItemArray::is_src_exist( + const S &src, + const S *&out) const +{ + bool is_exist = false; + ARRAY_FOREACH_NORET(remap_items_, idx) { + const ObRemapSchemaItem &tmp = remap_items_.at(idx); + if (tmp.src_ == src) { + is_exist = true; + out = &tmp.src_; + break; + } + } + return is_exist; +} + +template +bool ObRemapSchemaItemArray::is_target_exist(const T &target) const +{ + bool is_exist = false; + ARRAY_FOREACH_NORET(remap_items_, idx) { + const ObRemapSchemaItem &tmp = remap_items_.at(idx); + if (tmp.target_ == target) { + is_exist = true; + break; + } + } + return is_exist; +} + +template +bool ObRemapSchemaItemArray::is_target_exist( + const T &target, + const T *&out) const +{ + bool is_exist = false; + ARRAY_FOREACH_NORET(remap_items_, idx) { + const ObRemapSchemaItem &tmp = remap_items_.at(idx); + if (tmp.target_ == target) { + is_exist = true; + out = &tmp.target_; + break; + } + } + return is_exist; +} + +template +bool ObRemapSchemaItemArray::is_remap_target_exist(const S &src, const T *&out) const +{ + bool is_exist = false; + ARRAY_FOREACH_NORET(remap_items_, idx) { + const ObRemapSchemaItem &tmp = remap_items_.at(idx); + if (tmp.src_ == src) { + out = &tmp.target_; + is_exist = true; + break; + } + } + + return is_exist; +} + +template +int ObRemapSchemaItemArray::add_item(const ObRemapSchemaItem &item) +{ + int ret = OB_SUCCESS; + ObRemapSchemaItem remap_item; + if (!item.is_valid()) { + ret = OB_INVALID_ARGUMENT; + SHARE_LOG(WARN, "invalid argument", K(ret), K(item)); + } else if (OB_FAIL(remap_item.deep_copy(allocator_, item))) { + SHARE_LOG(WARN, "failed to copy item", K(ret), K(item)); + } else if (OB_FAIL(remap_items_.push_back(remap_item))) { + SHARE_LOG(WARN, "failed to push back item", K(ret), K(item)); + } else { + SHARE_LOG(INFO, "add one remap item", K(item)); + } + return ret; +} + +} +} \ No newline at end of file diff --git a/src/sql/engine/cmd/ob_alter_system_executor.cpp b/src/sql/engine/cmd/ob_alter_system_executor.cpp index b5aae79082..8233baf973 100644 --- a/src/sql/engine/cmd/ob_alter_system_executor.cpp +++ b/src/sql/engine/cmd/ob_alter_system_executor.cpp @@ -2574,6 +2574,27 @@ int ObCheckpointSlogExecutor::execute(ObExecContext &ctx, ObCheckpointSlogStmt & return ret; } +int ObRecoverTableExecutor::execute(ObExecContext &ctx, ObRecoverTableStmt &stmt) +{ + int ret = OB_SUCCESS; + ObTaskExecutorCtx *task_exec_ctx = nullptr; + ObCommonRpcProxy *common_proxy = nullptr; + ObAddr server; + if (OB_ISNULL(task_exec_ctx = GET_TASK_EXECUTOR_CTX(ctx))) { + ret = OB_NOT_INIT; + LOG_WARN("get task executor failed"); + } else if (OB_ISNULL(common_proxy = task_exec_ctx->get_common_rpc())) { + ret = OB_NOT_INIT; + LOG_WARN("get common rpc proxy failed"); + } else if (OB_FAIL(common_proxy->recover_table(stmt.get_rpc_arg()))) { + LOG_WARN("failed to send recover table rpc", K(ret)); + } else { + const obrpc::ObRecoverTableArg &recover_table_rpc_arg = stmt.get_rpc_arg(); + LOG_INFO("send recover table rpc finish", K(recover_table_rpc_arg)); + } + return ret; +} + int ObCancelRestoreExecutor::execute(ObExecContext &ctx, ObCancelRestoreStmt &stmt) { int ret = OB_SUCCESS; diff --git a/src/sql/engine/cmd/ob_alter_system_executor.h b/src/sql/engine/cmd/ob_alter_system_executor.h index 691673d2c4..bfa58a745c 100644 --- a/src/sql/engine/cmd/ob_alter_system_executor.h +++ b/src/sql/engine/cmd/ob_alter_system_executor.h @@ -129,6 +129,7 @@ DEF_SIMPLE_EXECUTOR(ObBackupSetEncryption); DEF_SIMPLE_EXECUTOR(ObBackupSetDecryption); DEF_SIMPLE_EXECUTOR(ObAddRestoreSource); DEF_SIMPLE_EXECUTOR(ObClearRestoreSource); +DEF_SIMPLE_EXECUTOR(ObRecoverTable); DEF_SIMPLE_EXECUTOR(ObTableTTL); DEF_SIMPLE_EXECUTOR(ObSetRegionBandwidth); diff --git a/src/sql/engine/cmd/ob_ddl_executor_util.cpp b/src/sql/engine/cmd/ob_ddl_executor_util.cpp index d0a19ded2b..d3ef5fba13 100644 --- a/src/sql/engine/cmd/ob_ddl_executor_util.cpp +++ b/src/sql/engine/cmd/ob_ddl_executor_util.cpp @@ -47,7 +47,7 @@ int ObDDLExecutorUtil::handle_session_exception(ObSQLSessionInfo &session) int ObDDLExecutorUtil::wait_ddl_finish( const uint64_t tenant_id, const int64_t task_id, - ObSQLSessionInfo &session, + ObSQLSessionInfo *session, obrpc::ObCommonRpcProxy *common_rpc_proxy, const bool is_support_cancel) { @@ -93,7 +93,7 @@ int ObDDLExecutorUtil::wait_ddl_finish( } if (OB_FAIL(ret)) { - } else if (OB_FAIL(handle_session_exception(session))) { + } else if (nullptr != session && OB_FAIL(handle_session_exception(*session))) { LOG_WARN("session exeception happened", K(ret), K(is_support_cancel)); if (is_support_cancel && OB_TMP_FAIL(cancel_ddl_task(tenant_id, common_rpc_proxy))) { LOG_WARN("cancel ddl task failed", K(tmp_ret)); diff --git a/src/sql/engine/cmd/ob_ddl_executor_util.h b/src/sql/engine/cmd/ob_ddl_executor_util.h index a90f4f2eb7..dda17da1f6 100644 --- a/src/sql/engine/cmd/ob_ddl_executor_util.h +++ b/src/sql/engine/cmd/ob_ddl_executor_util.h @@ -55,7 +55,7 @@ public: static int wait_ddl_finish( const uint64_t tenant_id, const int64_t task_id, - ObSQLSessionInfo &session, + ObSQLSessionInfo *session, obrpc::ObCommonRpcProxy *common_rpc_proxy, const bool is_support_cancel = true); static int wait_ddl_retry_task_finish( diff --git a/src/sql/engine/cmd/ob_index_executor.cpp b/src/sql/engine/cmd/ob_index_executor.cpp index d073b03bde..57cd42e9db 100644 --- a/src/sql/engine/cmd/ob_index_executor.cpp +++ b/src/sql/engine/cmd/ob_index_executor.cpp @@ -98,7 +98,7 @@ int ObCreateIndexExecutor::execute(ObExecContext &ctx, ObCreateIndexStmt &stmt) ret = OB_ERR_ADD_INDEX; LOG_WARN("index table id is invalid", KR(ret)); } - } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(create_index_arg.tenant_id_, res.task_id_, *my_session, common_rpc_proxy))) { + } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(create_index_arg.tenant_id_, res.task_id_, my_session, common_rpc_proxy))) { LOG_WARN("failed to wait ddl finish", K(ret)); } } diff --git a/src/sql/engine/cmd/ob_restore_executor.cpp b/src/sql/engine/cmd/ob_restore_executor.cpp index 810bfc3e80..8898f1ad67 100644 --- a/src/sql/engine/cmd/ob_restore_executor.cpp +++ b/src/sql/engine/cmd/ob_restore_executor.cpp @@ -83,19 +83,12 @@ int ObPhysicalRestoreTenantExecutor::execute( LOG_WARN("failed to remove user variable", KR(tmp_ret)); } } - if (OB_FAIL(ret)) { } else if (OB_FAIL(sync_wait_tenant_created_(ctx, restore_tenant_arg.tenant_name_, job_id))) { LOG_WARN("failed to sync wait tenant created", K(ret)); } - - } else { - // TODO(chongrong.th): fix restore preview in 4.3 - ret = OB_NOT_SUPPORTED; - LOG_WARN("restore preview is not support now", K(ret)); - // if (OB_FAIL(physical_restore_preview(ctx, stmt))) { - // LOG_WARN("failed to do physical restore preview", K(ret)); - // } + } else if (OB_FAIL(physical_restore_preview(ctx, stmt))) { + LOG_WARN("failed to do physical restore preview", K(ret)); } } return ret; @@ -189,6 +182,7 @@ int ObPhysicalRestoreTenantExecutor::physical_restore_preview( { int ret = OB_SUCCESS; ObSqlString set_backup_dest_sql; + ObSqlString set_scn_sql; ObSqlString set_timestamp_sql; sqlclient::ObISQLConnection *conn = NULL; observer::ObInnerSQLConnectionPool *pool = NULL; @@ -196,7 +190,6 @@ int ObPhysicalRestoreTenantExecutor::physical_restore_preview( ObSQLSessionInfo *session_info = ctx.get_my_session(); const obrpc::ObPhysicalRestoreTenantArg &restore_tenant_arg = stmt.get_rpc_arg(); int64_t affected_rows = 0; - if (OB_ISNULL(session_info)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid args", K(ret), KP(session_info)); @@ -212,18 +205,21 @@ int ObPhysicalRestoreTenantExecutor::physical_restore_preview( } else if (OB_FAIL(set_backup_dest_sql.assign_fmt("set @%s = '%.*s'", OB_RESTORE_PREVIEW_BACKUP_DEST_SESSION_STR, restore_tenant_arg.uri_.length(), restore_tenant_arg.uri_.ptr()))) { LOG_WARN("failed to set backup dest", KR(ret), K(set_backup_dest_sql)); - } else if (OB_FAIL(set_timestamp_sql.assign_fmt("set @%s = '%lu'", - OB_RESTORE_PREVIEW_TIMESTAMP_SESSION_STR, restore_tenant_arg.restore_scn_.get_val_for_inner_table_field()))) { - LOG_WARN("failed to set timestamp", KR(ret), K(set_timestamp_sql)); + } else if (OB_FAIL(set_scn_sql.assign_fmt("set @%s = '%ld'", + OB_RESTORE_PREVIEW_SCN_SESSION_STR, restore_tenant_arg.with_restore_scn_ ? restore_tenant_arg.restore_scn_.get_val_for_inner_table_field() : 0))) { + LOG_WARN("failed to set timestamp", KR(ret), K(set_scn_sql)); + } else if (OB_FAIL(set_scn_sql.assign_fmt("set @%s = '%.*s'", + OB_RESTORE_PREVIEW_TIMESTAMP_SESSION_STR, restore_tenant_arg.restore_timestamp_.length(), restore_tenant_arg.uri_.ptr()))) { + LOG_WARN("failed to set timestamp", KR(ret), K(set_scn_sql)); } else if (OB_FAIL(pool->acquire(session_info, conn))) { LOG_WARN("failed to get conn", K(ret)); } else if (OB_FAIL(conn->execute_write(session_info->get_effective_tenant_id(), set_backup_dest_sql.ptr(), affected_rows))) { LOG_WARN("failed to set backup dest", K(ret), K(set_backup_dest_sql)); } else if (OB_FAIL(conn->execute_write(session_info->get_effective_tenant_id(), - set_timestamp_sql.ptr(), affected_rows))) { - LOG_WARN("failed to set restore timestamp", K(ret), K(set_timestamp_sql)); - } + set_scn_sql.ptr(), affected_rows))) { + LOG_WARN("failed to set restore timestamp", K(ret), K(set_scn_sql)); + } return ret; } diff --git a/src/sql/engine/cmd/ob_table_executor.cpp b/src/sql/engine/cmd/ob_table_executor.cpp index 84c3366240..92293044ca 100644 --- a/src/sql/engine/cmd/ob_table_executor.cpp +++ b/src/sql/engine/cmd/ob_table_executor.cpp @@ -752,7 +752,7 @@ int ObAlterTableExecutor::alter_table_rpc_v2( ObIArray &ddl_ress = res.ddl_res_array_; for (int64_t i = 0; OB_SUCC(ret) && i < ddl_ress.count(); ++i) { ObDDLRes &ddl_res = ddl_ress.at(i); - if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(ddl_res.tenant_id_, ddl_res.task_id_, *my_session, common_rpc_proxy, is_support_cancel))) { + if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(ddl_res.tenant_id_, ddl_res.task_id_, my_session, common_rpc_proxy, is_support_cancel))) { LOG_WARN("wait drop index finish", K(ret)); } } @@ -1255,7 +1255,7 @@ int ObAlterTableExecutor::execute(ObExecContext &ctx, ObAlterTableStmt &stmt) // do nothing, don't check if data is valid } else if (OB_FAIL(refresh_schema_for_table(tenant_id))) { LOG_WARN("refresh_schema_for_table failed", K(ret)); - } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(tenant_id, res.task_id_, *my_session, common_rpc_proxy))) { + } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(tenant_id, res.task_id_, my_session, common_rpc_proxy))) { LOG_WARN("wait check constraint finish", K(ret)); } } @@ -1266,7 +1266,7 @@ int ObAlterTableExecutor::execute(ObExecContext &ctx, ObAlterTableStmt &stmt) } else { if (OB_FAIL(refresh_schema_for_table(tenant_id))) { LOG_WARN("refresh_schema_for_table failed", K(ret)); - } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(tenant_id, res.task_id_, *my_session, common_rpc_proxy))) { + } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(tenant_id, res.task_id_, my_session, common_rpc_proxy))) { LOG_WARN("wait fk constraint finish", K(ret)); } } @@ -1279,8 +1279,7 @@ int ObAlterTableExecutor::execute(ObExecContext &ctx, ObAlterTableStmt &stmt) int64_t affected_rows = 0; if (OB_FAIL(refresh_schema_for_table(alter_table_arg.exec_tenant_id_))) { LOG_WARN("refresh_schema_for_table failed", K(ret)); - } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(tenant_id, res.task_id_, - *my_session, common_rpc_proxy))) { + } else if (OB_FAIL(ObDDLExecutorUtil::wait_ddl_finish(tenant_id, res.task_id_, my_session, common_rpc_proxy))) { LOG_WARN("fail to wait ddl finish", K(ret), K(tenant_id), K(res)); } } diff --git a/src/sql/engine/table/ob_table_scan_op.cpp b/src/sql/engine/table/ob_table_scan_op.cpp index 208f609a21..6ac64ac052 100644 --- a/src/sql/engine/table/ob_table_scan_op.cpp +++ b/src/sql/engine/table/ob_table_scan_op.cpp @@ -21,6 +21,7 @@ #include "lib/profile/ob_perf_event.h" #include "lib/geo/ob_s2adapter.h" #include "lib/geo/ob_geo_utils.h" +#include "share/ob_ddl_common.h" #include "share/ob_ddl_checksum.h" #include "storage/access/ob_table_scan_iterator.h" #include "observer/ob_server_struct.h" @@ -2596,7 +2597,7 @@ int ObTableScanOp::add_ddl_column_checksum() // } else if (OB_FAIL(corrupt_obj(store_datum))) { // LOG_WARN("failed to corrupt obj", K(ret)); #endif - } else if (col_need_reshape_[i] && OB_FAIL(reshape_ddl_column_obj(store_datum, e->obj_meta_))) { + } else if (col_need_reshape_[i] && OB_FAIL(ObDDLUtil::reshape_ddl_column_obj(store_datum, e->obj_meta_))) { LOG_WARN("reshape ddl column obj failed", K(ret)); } else { column_checksum_[i] += store_datum.checksum(0); @@ -2642,7 +2643,7 @@ int ObTableScanOp::add_ddl_column_checksum_batch(const int64_t row_count) // } else if (OB_FAIL(corrupt_obj(store_datum))) { // LOG_WARN("failed to corrupt obj", K(ret)); #endif - } else if (col_need_reshape_[i] && OB_FAIL(reshape_ddl_column_obj(store_datum, e->obj_meta_))) { + } else if (col_need_reshape_[i] && OB_FAIL(ObDDLUtil::reshape_ddl_column_obj(store_datum, e->obj_meta_))) { LOG_WARN("reshape ddl column obj failed", K(ret)); } else { column_checksum_[i] += store_datum.checksum(0); @@ -2661,39 +2662,6 @@ int ObTableScanOp::add_ddl_column_checksum_batch(const int64_t row_count) return ret; } -int ObTableScanOp::reshape_ddl_column_obj(ObDatum &datum, const ObObjMeta &obj_meta) -{ - int ret = OB_SUCCESS; - if (datum.is_null()) { - // do not need to reshape - } else if (obj_meta.is_lob_storage()) { - ObLobLocatorV2 lob(datum.get_string(), obj_meta.has_lob_header()); - ObString disk_loc; - if (!lob.is_valid()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("invalid lob locator", K(ret)); - } else if (!lob.is_lob_disk_locator() && !lob.is_persist_lob()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("invalid lob locator, should be persist lob", K(ret), K(lob)); - } else if (OB_FAIL(lob.get_disk_locator(disk_loc))) { - LOG_WARN("get disk locator failed", K(ret), K(lob)); - } - if (OB_SUCC(ret)) { - datum.set_string(disk_loc); - } - } else if (OB_UNLIKELY(!obj_meta.is_fixed_len_char_type())) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("no need to reshape non-char", K(ret)); - } else { - const char *ptr = datum.ptr_; - int32_t len = datum.len_; - int32_t trunc_len_byte = static_cast(ObCharset::strlen_byte_no_sp( - obj_meta.get_collation_type(), ptr, len)); - datum.set_string(ObString(trunc_len_byte, ptr)); - } - return ret; -} - int ObTableScanOp::report_ddl_column_checksum() { int ret = OB_SUCCESS; diff --git a/src/sql/engine/table/ob_table_scan_op.h b/src/sql/engine/table/ob_table_scan_op.h index 2441493811..30064eb5ee 100644 --- a/src/sql/engine/table/ob_table_scan_op.h +++ b/src/sql/engine/table/ob_table_scan_op.h @@ -466,7 +466,6 @@ protected: int add_ddl_column_checksum(); int add_ddl_column_checksum_batch(const int64_t row_count); static int corrupt_obj(ObObj &obj); - int reshape_ddl_column_obj(common::ObDatum &datum, const ObObjMeta &obj_meta); int report_ddl_column_checksum(); int get_next_batch_with_das(int64_t &count, int64_t capacity); void replace_bnlj_param(int64_t batch_idx); diff --git a/src/sql/executor/ob_cmd_executor.cpp b/src/sql/executor/ob_cmd_executor.cpp index f5ce63e8dd..bc295571f3 100644 --- a/src/sql/executor/ob_cmd_executor.cpp +++ b/src/sql/executor/ob_cmd_executor.cpp @@ -905,6 +905,10 @@ int ObCmdExecutor::execute(ObExecContext &ctx, ObICmd &cmd) DEFINE_EXECUTE_CMD(ObCancelRestoreStmt, ObCancelRestoreExecutor); break; } + case stmt::T_RECOVER_TABLE: { + DEFINE_EXECUTE_CMD(ObRecoverTableStmt, ObRecoverTableExecutor); + break; + } case stmt::T_BACKUP_MANAGE: { DEFINE_EXECUTE_CMD(ObBackupManageStmt, ObBackupManageExecutor); break; diff --git a/src/sql/parser/non_reserved_keywords_mysql_mode.c b/src/sql/parser/non_reserved_keywords_mysql_mode.c index f3fedf4645..5208bc54ec 100644 --- a/src/sql/parser/non_reserved_keywords_mysql_mode.c +++ b/src/sql/parser/non_reserved_keywords_mysql_mode.c @@ -676,6 +676,7 @@ static const NonReservedKeyword Mysql_none_reserved_keywords[] = {"relaylog", RELAYLOG}, {"release", RELEASE}, {"reload", RELOAD}, + {"remap", REMAP}, {"remove", REMOVE}, {"rename", RENAME}, {"reorganize", REORGANIZE}, diff --git a/src/sql/parser/ob_parser.cpp b/src/sql/parser/ob_parser.cpp index f85cf1792c..d188e37b12 100644 --- a/src/sql/parser/ob_parser.cpp +++ b/src/sql/parser/ob_parser.cpp @@ -959,6 +959,7 @@ int ObParser::parse_sql(const ObString &stmt, K(parse_result.yycolumn_), K(parse_result.yylineno_), K(parse_result.extra_errno_), + K(parse_result.is_for_remap_), K(err_charge_sql_mode), K(sql_mode_), K(parse_result.sql_mode_), diff --git a/src/sql/parser/ob_sql_parser.cpp b/src/sql/parser/ob_sql_parser.cpp index 510a473fda..1fced349e4 100644 --- a/src/sql/parser/ob_sql_parser.cpp +++ b/src/sql/parser/ob_sql_parser.cpp @@ -69,6 +69,7 @@ int ObSQLParser::parse_and_gen_sqlid(void *malloc_pool, parse_result->minus_ctx_.pos_ = -1; parse_result->minus_ctx_.raw_sql_offset_ = -1; parse_result->is_for_trigger_ = false; + parse_result->is_for_remap_ = false; parse_result->is_dynamic_sql_ = false; parse_result->is_batched_multi_enabled_split_ = false; parse_result->may_bool_value_ = false; diff --git a/src/sql/parser/parse_node.h b/src/sql/parser/parse_node.h index b25120ea19..3fc9149705 100644 --- a/src/sql/parser/parse_node.h +++ b/src/sql/parser/parse_node.h @@ -313,6 +313,7 @@ typedef struct uint32_t is_normal_ps_prepare_ : 1; uint32_t is_multi_values_parser_ : 1; uint32_t is_for_udr_ : 1; + uint32_t is_for_remap_ : 1; }; ParseNode *result_tree_; diff --git a/src/sql/parser/sql_parser_base.h b/src/sql/parser/sql_parser_base.h index 9199fd2dfa..fc39f3424f 100644 --- a/src/sql/parser/sql_parser_base.h +++ b/src/sql/parser/sql_parser_base.h @@ -663,6 +663,7 @@ do { #define IS_NEED_PARAMETERIZE ((ParseResult *)yyextra)->need_parameterize_ #define IS_FOR_TRIGGER ((ParseResult *)yyextra)->is_for_trigger_ #define IF_FOR_PREPROCESS ((ParseResult *)yyextra)->is_for_preprocess_ +#define IS_FOR_REMAP ((ParseResult *)yyextra)->is_for_remap_ #define COPY_STRING(src, src_len, dst) \ do { \ diff --git a/src/sql/parser/sql_parser_mysql_mode.l b/src/sql/parser/sql_parser_mysql_mode.l index 62352ee10c..42857144d9 100755 --- a/src/sql/parser/sql_parser_mysql_mode.l +++ b/src/sql/parser/sql_parser_mysql_mode.l @@ -1485,12 +1485,267 @@ BEGIN(in_c_comment); } } + + + +{identifier}":"{identifier} { + ParseResult *p = (ParseResult *)yyextra; + check_value(yylval); + if (IS_FOR_REMAP) { + char *colon_p = strstr(yytext, ":"); + if (NULL == colon_p) { + YY_UNEXPECTED_ERROR("invalid remap table name\n"); + } else { + size_t src_len = colon_p - yytext; + ParseNode *node = NULL, *src_node = NULL, *target_node = NULL; + malloc_new_node(node, p->malloc_pool_, T_RELATION_FACTOR, 2); + yylval->node = node; + malloc_new_node(src_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_node, p->malloc_pool_, T_INT, 0); + src_node->str_value_ = parse_strndup(yytext, src_len, p->malloc_pool_); + src_node->str_len_ = src_len; + check_malloc(src_node->str_value_); + target_node->str_value_ = parse_strdup(colon_p + 1, p->malloc_pool_, &(target_node->str_len_)); + check_malloc(target_node->str_value_); + node->children_[0] = src_node; + node->children_[1] = target_node; + return REMAP_TABLE_VAL; + } + } else { + YY_UNEXPECTED_ERROR("Named binding is only for remap\n"); + } +} + +{identifier}"."{identifier}":"{identifier} { + ParseResult *p = (ParseResult *)yyextra; + check_value(yylval); + if (IS_FOR_REMAP) { + char *colon_p = strstr(yytext, ":"); + char *point_p = strstr(yytext, "."); + if (NULL == colon_p || NULL == point_p) { + YY_UNEXPECTED_ERROR("invalid remap table name\n"); + } else { + size_t src_db_len = point_p - yytext; + size_t src_table_len = colon_p - point_p - 1; + ParseNode *node = NULL, *src_db_node = NULL, *src_table_node = NULL, *target_table_node = NULL; + malloc_new_node(node, p->malloc_pool_, T_RELATION_FACTOR, 5); + yylval->node = node; + malloc_new_node(src_db_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(src_table_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_table_node, p->malloc_pool_, T_INT, 0); + src_db_node->str_value_ = parse_strndup(yytext, src_db_len, p->malloc_pool_); + src_db_node->str_len_ = src_db_len; + check_malloc(src_db_node->str_value_); + src_table_node->str_value_ = parse_strndup(point_p + 1, src_table_len, p->malloc_pool_); + src_table_node->str_len_ = src_table_len; + check_malloc(src_table_node->str_value_); + target_table_node->str_value_ = parse_strdup(colon_p + 1, p->malloc_pool_, &(target_table_node->str_len_)); + check_malloc(target_table_node->str_value_); + node->children_[0] = src_db_node; + node->children_[1] = src_table_node; + node->children_[4] = target_table_node; + return REMAP_TABLE_VAL; + } + } else { + YY_UNEXPECTED_ERROR("Named binding is only for remap\n"); + } +} + +{identifier}"."{identifier}":"{identifier}"."{identifier} { + ParseResult *p = (ParseResult *)yyextra; + check_value(yylval); + if (IS_FOR_REMAP) { + char *first_point_p = strstr(yytext, "."); + char *colon_p = strstr(yytext, ":"); + char *final_point_p = strrchr(yytext, '.'); + if (NULL == first_point_p || NULL == colon_p || NULL == final_point_p) { + YY_UNEXPECTED_ERROR("invalid remap table name\n"); + } else { + size_t src_db_len = first_point_p - yytext; + size_t src_table_len = colon_p - first_point_p - 1; + size_t target_db_len = final_point_p - colon_p - 1; + ParseNode *node = NULL, *src_db_node = NULL, *src_table_node = NULL, *target_db_node = NULL, *target_table_node = NULL; + malloc_new_node(node, p->malloc_pool_, T_RELATION_FACTOR, 5); + yylval->node = node; + malloc_new_node(src_db_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(src_table_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_db_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_table_node, p->malloc_pool_, T_INT, 0); + src_db_node->str_value_ = parse_strndup(yytext, src_db_len, p->malloc_pool_); + src_db_node->str_len_ = src_db_len; + check_malloc(src_db_node->str_value_); + src_table_node->str_value_ = parse_strndup(first_point_p + 1, src_table_len, p->malloc_pool_); + src_table_node->str_len_ = src_table_len; + check_malloc(src_table_node->str_value_); + target_db_node->str_value_ = parse_strndup(colon_p + 1, target_db_len, p->malloc_pool_); + target_db_node->str_len_ = target_db_len; + check_malloc(target_db_node->str_value_); + target_table_node->str_value_ = parse_strdup(final_point_p + 1, p->malloc_pool_, &(target_table_node->str_len_)); + check_malloc(target_table_node->str_value_); + node->children_[0] = src_db_node; + node->children_[1] = src_table_node; + node->children_[3] = target_db_node; + node->children_[4] = target_table_node; + return REMAP_TABLE_VAL; + } + } else { + YY_UNEXPECTED_ERROR("Named binding is only for remap\n"); + } +} + +{identifier}".*:"{identifier}".*" { + ParseResult *p = (ParseResult *)yyextra; + check_value(yylval); + if (IS_FOR_REMAP) { + char *first_point_p = strstr(yytext, "."); + char *colon_p = strstr(yytext, ":"); + char *final_point_p = strrchr(yytext, '.'); + if (NULL == first_point_p || NULL == colon_p || NULL == final_point_p) { + YY_UNEXPECTED_ERROR("invalid remap table name\n"); + } else { + size_t src_db_len = first_point_p - yytext; + size_t target_db_len = final_point_p - colon_p - 1; + ParseNode *node = NULL, *src_db_node = NULL, *target_db_node = NULL; + malloc_new_node(node, p->malloc_pool_, T_RELATION_FACTOR, 5); + yylval->node = node; + malloc_new_node(src_db_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_db_node, p->malloc_pool_, T_INT, 0); + src_db_node->str_value_ = parse_strndup(yytext, src_db_len, p->malloc_pool_); + src_db_node->str_len_ = src_db_len; + check_malloc(src_db_node->str_value_); + target_db_node->str_value_ = parse_strndup(colon_p + 1, target_db_len, p->malloc_pool_); + target_db_node->str_len_ = target_db_len; + check_malloc(target_db_node->str_value_); + node->children_[0] = src_db_node; + node->children_[3] = target_db_node; + return REMAP_TABLE_VAL; + } + } else { + YY_UNEXPECTED_ERROR("Named binding is only for remap\n"); + } +} + +{identifier}"."{identifier}":"{identifier}":"{identifier}"."{identifier} { + ParseResult *p = (ParseResult *)yyextra; + check_value(yylval); + if (IS_FOR_REMAP) { + char *first_point_p = strstr(yytext, "."); + char *first_colon_p = strstr(yytext, ":"); + char *final_colon_p = strrchr(yytext, ':'); + char *final_point_p = strrchr(yytext, '.'); + if (NULL == first_point_p || NULL == first_colon_p || NULL == final_colon_p || NULL == final_point_p) { + YY_UNEXPECTED_ERROR("invalid remap table name\n"); + } else { + size_t src_db_len = first_point_p - yytext; + size_t src_table_len = first_colon_p - first_point_p - 1; + size_t src_part_len = final_colon_p - first_colon_p - 1; + size_t target_db_len = final_point_p - final_colon_p - 1; + ParseNode *node = NULL, *src_db_node = NULL, *src_table_node = NULL, *src_part_node = NULL; + ParseNode *target_db_node = NULL, *target_table_node = NULL; + malloc_new_node(node, p->malloc_pool_, T_RELATION_FACTOR, 5); + yylval->node = node; + malloc_new_node(src_db_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(src_table_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(src_part_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_db_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_table_node, p->malloc_pool_, T_INT, 0); + src_db_node->str_value_ = parse_strndup(yytext, src_db_len, p->malloc_pool_); + src_db_node->str_len_ = src_db_len; + check_malloc(src_db_node->str_value_); + src_table_node->str_value_ = parse_strndup(first_point_p + 1, src_table_len, p->malloc_pool_); + src_table_node->str_len_ = src_table_len; + check_malloc(src_table_node->str_value_); + src_part_node->str_value_ = parse_strndup(first_colon_p + 1, src_part_len, p->malloc_pool_); + src_part_node->str_len_ = src_part_len; + check_malloc(src_part_node->str_value_); + target_db_node->str_value_ = parse_strndup(final_colon_p + 1, target_db_len, p->malloc_pool_); + target_db_node->str_len_ = target_db_len; + check_malloc(target_db_node->str_value_); + target_table_node->str_value_ = parse_strdup(final_point_p + 1, p->malloc_pool_, &(target_table_node->str_len_)); + check_malloc(target_table_node->str_value_); + node->children_[0] = src_db_node; + node->children_[1] = src_table_node; + node->children_[2] = src_part_node; + node->children_[3] = target_db_node; + node->children_[4] = target_table_node; + return REMAP_TABLE_VAL; + } + } else { + YY_UNEXPECTED_ERROR("Named binding is only for remap\n"); + } +} + +{identifier}"."{identifier}":"{identifier}":"{identifier} { + ParseResult *p = (ParseResult *)yyextra; + check_value(yylval); + if (IS_FOR_REMAP) { + char *first_point_p = strstr(yytext, "."); + char *first_colon_p = strstr(yytext, ":"); + char *final_colon_p = strrchr(yytext, ':'); + if (NULL == first_point_p || NULL == first_colon_p || NULL == final_colon_p) { + YY_UNEXPECTED_ERROR("invalid remap table name\n"); + } else { + size_t src_db_len = first_point_p - yytext; + size_t src_table_len = first_colon_p - first_point_p - 1; + size_t src_part_len = final_colon_p - first_colon_p - 1; + ParseNode *node = NULL, *src_db_node = NULL, *src_table_node = NULL, *src_part_node = NULL; + ParseNode *target_table_node = NULL; + malloc_new_node(node, p->malloc_pool_, T_RELATION_FACTOR, 5); + yylval->node = node; + malloc_new_node(src_db_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(src_table_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(src_part_node, p->malloc_pool_, T_INT, 0); + malloc_new_node(target_table_node, p->malloc_pool_, T_INT, 0); + src_db_node->str_value_ = parse_strndup(yytext, src_db_len, p->malloc_pool_); + src_db_node->str_len_ = src_db_len; + check_malloc(src_db_node->str_value_); + src_table_node->str_value_ = parse_strndup(first_point_p + 1, src_table_len, p->malloc_pool_); + src_table_node->str_len_ = src_table_len; + check_malloc(src_table_node->str_value_); + src_part_node->str_value_ = parse_strndup(first_colon_p + 1, src_part_len, p->malloc_pool_); + src_part_node->str_len_ = src_part_len; + check_malloc(src_part_node->str_value_); + target_table_node->str_value_ = parse_strdup(final_colon_p + 1, p->malloc_pool_, &(target_table_node->str_len_)); + check_malloc(target_table_node->str_value_); + node->children_[0] = src_db_node; + node->children_[1] = src_table_node; + node->children_[2] = src_part_node; + node->children_[4] = target_table_node; + return REMAP_TABLE_VAL; + } + } else { + YY_UNEXPECTED_ERROR("Named binding is only for remap\n"); + } +} + + ":"{identifier}"."{identifier} { ParseResult *p = (ParseResult *)yyextra; check_value(yylval); char *header = str_toupper(yytext, 4); bool is_for_trigger = (0 == strncmp(":NEW", header, 4)) || (0 == strncmp(":OLD", header, 4)); - if (is_for_trigger) { + if (IS_FOR_REMAP) { + bool is_contain_colon = (0 == strncmp(":", yytext, 1)); + char *dot_p = strstr(yytext, "."); + if (!is_contain_colon || NULL == dot_p) { + YY_UNEXPECTED_ERROR("invalid remap table name\n"); + } else { + size_t db_len = dot_p - yytext - 1; + ParseNode *node = NULL, *db_node = NULL, *tb_node = NULL; + malloc_new_node(node, p->malloc_pool_, T_IDENT, 2); + yylval->node = node; + malloc_new_node(db_node, p->malloc_pool_, T_IDENT, 0); + malloc_new_node(tb_node, p->malloc_pool_, T_IDENT, 0); + db_node->str_value_ = parse_strndup(yytext + 1, db_len, p->malloc_pool_); + db_node->str_len_ = db_len; + check_malloc(db_node->str_value_); + tb_node->str_value_ = parse_strdup(dot_p + 1, p->malloc_pool_, &(tb_node->str_len_)); + check_malloc(tb_node->str_value_); + node->children_[0] = db_node; + node->children_[1] = tb_node; + return REMAP_TABLE_VAL; + } + } else if (is_for_trigger) { malloc_new_node(yylval->node, p->malloc_pool_, T_QUESTIONMARK, 0); yylval->node->value_ = get_question_mark(&p->question_mark_ctx_, p->malloc_pool_, yytext); if (IS_FAST_PARAMETERIZE) { diff --git a/src/sql/parser/sql_parser_mysql_mode.y b/src/sql/parser/sql_parser_mysql_mode.y index 827dce20da..840932e584 100755 --- a/src/sql/parser/sql_parser_mysql_mode.y +++ b/src/sql/parser/sql_parser_mysql_mode.y @@ -69,6 +69,7 @@ extern void obsql_oracle_parse_fatal_error(int32_t errcode, yyscan_t yyscanner, %token CLIENT_VERSION %token MYSQL_DRIVER %token HEX_STRING_VALUE +%token REMAP_TABLE_VAL %token OUTLINE_DEFAULT_TOKEN/*use for outline parser to just filter hint of query_sql*/ /*empty_query:: @@ -101,6 +102,7 @@ extern void obsql_oracle_parse_fatal_error(int32_t errcode, yyscan_t yyscanner, %nonassoc KILL_EXPR %nonassoc CONNECTION QUERY %nonassoc LOWER_COMMA +%nonassoc REMAP %nonassoc ',' WITH %left UNION EXCEPT MINUS %left INTERSECT @@ -322,7 +324,7 @@ END_P SET_VAR DELIMITER QUARTER QUERY QUERY_RESPONSE_TIME QUEUE_TIME QUICK REBUILD RECOVER RECOVERY_WINDOW RECYCLE REDO_BUFFER_SIZE REDOFILE REDUNDANCY REDUNDANT REFRESH REGION RELAY RELAYLOG - RELAY_LOG_FILE RELAY_LOG_POS RELAY_THREAD RELOAD REMOVE REORGANIZE REPAIR REPEATABLE REPLICA + RELAY_LOG_FILE RELAY_LOG_POS RELAY_THREAD RELOAD REMAP REMOVE REORGANIZE REPAIR REPEATABLE REPLICA REPLICA_NUM REPLICA_TYPE REPLICATION REPORT RESET RESOURCE RESOURCE_POOL_LIST RESPECT RESTART RESTORE RESUME RETURNED_SQLSTATE RETURNS RETURNING REVERSE ROLLBACK ROLLUP ROOT ROOTTABLE ROOTSERVICE ROOTSERVICE_LIST ROUTINE ROW ROLLING ROW_COUNT ROW_FORMAT ROWS RTREE RUN @@ -500,7 +502,8 @@ END_P SET_VAR DELIMITER %type alter_tablespace_stmt %type permanent_tablespace permanent_tablespace_options permanent_tablespace_option alter_tablespace_actions alter_tablespace_action opt_force_purge %type opt_sql_throttle_for_priority opt_sql_throttle_using_cond sql_throttle_one_or_more_metrics sql_throttle_metric -%type opt_copy_id opt_backup_dest opt_preview opt_backup_backup_dest opt_tenant_info opt_with_active_piece get_format_unit opt_backup_tenant_list opt_backup_to opt_description policy_name opt_recovery_window opt_redundancy opt_backup_copies opt_restore_until opt_backup_key_info opt_encrypt_key +%type opt_copy_id opt_backup_dest opt_backup_backup_dest opt_tenant_info opt_with_active_piece get_format_unit opt_backup_tenant_list opt_backup_to opt_description policy_name opt_recovery_window opt_redundancy opt_backup_copies opt_restore_until opt_backup_key_info opt_encrypt_key +%type opt_recover_tenant recover_table_list recover_table_relation_name restore_remap_list remap_relation_name table_relation_name opt_recover_remap_item_list restore_remap_item_list restore_remap_item remap_item %type new_or_old new_or_old_column_ref diagnostics_info_ref %type on_empty on_error json_on_response opt_returning_type opt_on_empty_or_error json_value_expr opt_ascii opt_truncate_clause %type ws_nweights opt_ws_as_char opt_ws_levels ws_level_flag_desc ws_level_flag_reverse ws_level_flags ws_level_list ws_level_list_item ws_level_number ws_level_range ws_level_list_or_range @@ -15311,16 +15314,21 @@ ALTER SYSTEM CLEAR RESTORE SOURCE malloc_terminal_node($$, result->malloc_pool_, T_CLEAR_RESTORE_SOURCE); } | -ALTER SYSTEM RESTORE table_list FOR relation_name opt_backup_dest opt_restore_until WITH STRING_VALUE opt_encrypt_key opt_backup_key_info opt_description +ALTER SYSTEM RECOVER TABLE recover_table_list opt_recover_tenant opt_backup_dest opt_restore_until WITH STRING_VALUE opt_encrypt_key opt_backup_key_info opt_recover_remap_item_list opt_description { ParseNode *tables = NULL; - merge_nodes(tables, result, T_TABLE_LIST, $4); - malloc_non_terminal_node($$, result->malloc_pool_, T_PHYSICAL_RESTORE_TENANT, 8, $6, $7, $8, $10, $11, $12, $13, tables); + merge_nodes(tables, result, T_TABLE_LIST, $5); + malloc_non_terminal_node($$, result->malloc_pool_, T_RECOVER_TABLE, 9, $6, $7, $8, $10, tables, $11, $12, $13, $14); } | -ALTER SYSTEM RESTORE relation_name opt_backup_dest opt_restore_until WITH STRING_VALUE opt_encrypt_key opt_backup_key_info opt_description opt_preview +ALTER SYSTEM RESTORE FROM STRING_VALUE opt_restore_until PREVIEW { - malloc_non_terminal_node($$, result->malloc_pool_, T_PHYSICAL_RESTORE_TENANT, 8, $4, $5, $6, $8, $9, $10, $11, $12); + malloc_non_terminal_node($$, result->malloc_pool_, T_PHYSICAL_RESTORE_TENANT, 2, $5, $6); +} +| +ALTER SYSTEM RESTORE relation_name opt_backup_dest opt_restore_until WITH STRING_VALUE opt_encrypt_key opt_backup_key_info opt_description +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_PHYSICAL_RESTORE_TENANT, 7, $4, $5, $6, $8, $9, $10, $11); } | ALTER SYSTEM CHANGE TENANT change_tenant_name_or_tenant_id @@ -15515,6 +15523,11 @@ ALTER SYSTEM CANCEL RESTORE relation_name malloc_non_terminal_node($$, result->malloc_pool_, T_CANCEL_RESTORE, 1, $5); } | +ALTER SYSTEM CANCEL RECOVER TABLE relation_name +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_CANCEL_RECOVER_TABLE, 1, $6); +} +| ALTER SYSTEM SUSPEND BACKUP { ParseNode *type = NULL; @@ -16170,7 +16183,7 @@ STRING_VALUE ; opt_backup_dest: -/*empry*/ +/*empty*/ { $$ = NULL; } @@ -16180,6 +16193,126 @@ opt_backup_dest: } ; +recover_table_list: +recover_table_relation_name +{ + $$ = $1; +} +| recover_table_list ',' recover_table_relation_name +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_LINK_NODE, 2, $1, $3); +} +; + +recover_table_relation_name: +'*' '.' '*' +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_RELATION_FACTOR, 3, NULL, NULL, NULL); +} +| relation_name '.' table_relation_name +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_RELATION_FACTOR, 3, $1, $3, NULL); +} +| relation_name '.' '*' +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_RELATION_FACTOR, 3, $1, NULL, NULL); +} +| { result->is_for_remap_ = 1; } REMAP_TABLE_VAL +{ + $$ = $2; +} +; + +opt_recover_tenant: +TO tenant_name +{ + $$ = $2; +} +; + +opt_recover_remap_item_list: +/*empty*/ +{ + $$ = NULL; +} +| restore_remap_item_list +{ + ParseNode *remap_items = NULL; + merge_nodes(remap_items, result, T_TABLE_LIST, $1); + $$ = remap_items; +} +; + +restore_remap_item_list: +restore_remap_item +{ + $$ = $1; +} +| restore_remap_item_list restore_remap_item +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_LINK_NODE, 2, $1, $2); +} +; + +restore_remap_item: +REMAP { result->is_for_remap_ = 1 } remap_item +{ + $$ = $3; +} +; + +remap_item: +TABLE restore_remap_list +{ + ParseNode *tables = NULL; + merge_nodes(tables, result, T_REMAP_TABLE, $2); + $$ = tables; +} +| TABLEGROUP restore_remap_list +{ + ParseNode *tablegroups = NULL; + merge_nodes(tablegroups, result, T_REMAP_TABLEGROUP, $2); + $$ = tablegroups; +} +| TABLESPACE restore_remap_list +{ + ParseNode *tablespaces = NULL; + merge_nodes(tablespaces, result, T_REMAP_TABLESPACE, $2); + $$ = tablespaces; +} +; + +restore_remap_list: +remap_relation_name +{ + $$ = $1; +} +| restore_remap_list ',' REMAP_TABLE_VAL +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_LINK_NODE, 2, $1, $3); +} +; + +remap_relation_name: +REMAP_TABLE_VAL +{ + $$ = $1; +} +; + +table_relation_name: +relation_name +{ + $$ = $1; +} +| mysql_reserved_keyword +{ + ParseNode *table_name = NULL; + get_non_reserved_node(table_name, result->malloc_pool_, @1.first_column, @1.last_column); + $$ = table_name; +} +; + opt_backup_backup_dest: /*empty*/ { @@ -16331,17 +16464,6 @@ BACKUP_COPIES opt_equal_mark INTNUM } ; -opt_preview: -PREVIEW -{ - malloc_terminal_node($$, result->malloc_pool_, T_PREVIEW); -} -| -{ - $$ = NULL; -} -; - server_or_zone: ip_port { @@ -17174,7 +17296,8 @@ opt_description: ; opt_restore_until: -/*EMPTY*/ { $$ = NULL; } +/*EMPTY*/ +{ $$ = NULL; } | UNTIL TIME COMP_EQ STRING_VALUE { ParseNode *is_scn = NULL; @@ -18452,6 +18575,7 @@ ACCOUNT | RELAY_LOG_POS | RELAY_THREAD | RELOAD +| REMAP | REMOVE | REORGANIZE | REPAIR diff --git a/src/sql/resolver/cmd/ob_alter_system_resolver.cpp b/src/sql/resolver/cmd/ob_alter_system_resolver.cpp index 42d7fc8536..7ee8ef5663 100644 --- a/src/sql/resolver/cmd/ob_alter_system_resolver.cpp +++ b/src/sql/resolver/cmd/ob_alter_system_resolver.cpp @@ -26,7 +26,8 @@ #include "observer/omt/ob_tenant_config_mgr.h" #include "share/ob_zone_table_operation.h" #include "share/backup/ob_backup_struct.h" - +#include "share/restore/ob_import_table_struct.h" +#include "share/restore/ob_recover_table_util.h" #include "sql/session/ob_sql_session_info.h" #include "sql/resolver/cmd/ob_alter_system_stmt.h" #include "sql/resolver/cmd/ob_system_cmd_stmt.h" @@ -2725,24 +2726,47 @@ int ObAdminRollingUpgradeCmdResolver::resolve(const ParseNode &parse_tree) int ObPhysicalRestoreTenantResolver::resolve(const ParseNode &parse_tree) { int ret = OB_SUCCESS; + ObPhysicalRestoreTenantStmt *stmt = nullptr; if (OB_UNLIKELY(T_PHYSICAL_RESTORE_TENANT != parse_tree.type_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("type is not T_PHYSICAL_RESTORE_TENANT", "type", get_type_name(parse_tree.type_)); } else if (OB_UNLIKELY(NULL == parse_tree.children_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("children should not be null"); - } else if (OB_UNLIKELY(8 != parse_tree.num_child_)) { + } else if (OB_UNLIKELY(7 != parse_tree.num_child_ && 2 != parse_tree.num_child_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("num of children not match", K(ret), "child_num", parse_tree.num_child_); + } else if (OB_ISNULL(stmt = create_stmt())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_ERROR("create ObPhysicalRestoreTenantStmt failed"); + } else if (OB_FALSE_IT(stmt_ = stmt)) { + } else if (2 == parse_tree.num_child_) { + // parse preview + stmt->set_is_preview(true); + stmt->get_rpc_arg().initiator_tenant_id_ = OB_SYS_TENANT_ID; + const ParseNode *time_node = parse_tree.children_[1]; + if (OB_FAIL(Util::resolve_string(parse_tree.children_[0], stmt->get_rpc_arg().uri_))) { + LOG_WARN("resolve string failed", K(ret)); + } else if (OB_ISNULL(parse_tree.children_[1])) { + stmt->get_rpc_arg().with_restore_scn_ = false; + } else if (0/*timestamp*/ == time_node->children_[0]->value_) { + stmt->get_rpc_arg().restore_timestamp_.assign_ptr(time_node->children_[1]->str_value_, time_node->children_[1]->str_len_); + stmt->get_rpc_arg().with_restore_scn_ = false; + } else if (1/*timestamp*/ == time_node->children_[0]->value_) { + if (share::OB_BASE_SCN_TS_NS >= time_node->children_[1]->value_) { + ret = OB_INVALID_ARGUMENT; + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "until scn, it should be positive integer"); + LOG_WARN("until scn, it should be positive integer", KR(ret), K(time_node->children_[1]->value_)); + } else if (OB_FAIL(stmt->get_rpc_arg().restore_scn_.convert_for_sql(time_node->children_[1]->value_))) { + LOG_WARN("failed to convert scn", K(ret)); + } else { + stmt->get_rpc_arg().with_restore_scn_ = true; + } + } } else { - ObPhysicalRestoreTenantStmt *stmt = create_stmt(); - if (NULL == stmt) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_ERROR("create ObPhysicalRestoreTenantStmt failed"); - } else { - stmt_ = stmt; - if (OB_FAIL(Util::resolve_relation_name(parse_tree.children_[0], - stmt->get_rpc_arg().tenant_name_))) { + stmt->set_is_preview(false); + stmt->get_rpc_arg().initiator_tenant_id_ = OB_SYS_TENANT_ID; + if(OB_FAIL(Util::resolve_relation_name(parse_tree.children_[0], stmt->get_rpc_arg().tenant_name_))) { LOG_WARN("resolve tenant_name failed", K(ret)); } else { const ObString &tenant_name = stmt->get_rpc_arg().tenant_name_; @@ -2805,85 +2829,25 @@ int ObPhysicalRestoreTenantResolver::resolve(const ParseNode &parse_tree) LOG_WARN("failed to resolve restore source array", K(ret)); } else { // resolve datetime + const ParseNode *time_node = parse_tree.children_[2]; if (OB_ISNULL(parse_tree.children_[2])) { stmt->get_rpc_arg().with_restore_scn_ = false; - } else { - const ParseNode *time_node = parse_tree.children_[2]; - if (OB_UNLIKELY(T_PHYSICAL_RESTORE_UNTIL != time_node->type_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("type is not T_PHYSICAL_RESTORE_UNTIL", "type", get_type_name(time_node->type_)); - } else if (OB_ISNULL(time_node->children_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("children should not be null"); - } else if (OB_UNLIKELY(2 != time_node->num_child_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("num of children not match", K(ret), "child_num", time_node->num_child_); - } else if (OB_ISNULL(time_node->children_[0]) || OB_ISNULL(time_node->children_[1])) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("children should not be null", KP(time_node->children_[0]), KP(time_node->children_[1])); - } else if (0/*timestamp*/ == time_node->children_[0]->value_) { - stmt->get_rpc_arg().restore_timestamp_.assign_ptr(time_node->children_[1]->str_value_, time_node->children_[1]->str_len_); - stmt->get_rpc_arg().with_restore_scn_ = false; - } else if (1/*timestamp*/ == time_node->children_[0]->value_) { - if (share::OB_BASE_SCN_TS_NS >= time_node->children_[1]->value_) { - ret = OB_INVALID_ARGUMENT; - LOG_USER_ERROR(OB_INVALID_ARGUMENT, "until scn, it should be positive integer"); - LOG_WARN("until scn, it should be positive integer", KR(ret), K(time_node->children_[1]->value_)); - } else if (OB_FAIL(stmt->get_rpc_arg().restore_scn_.convert_for_sql(time_node->children_[1]->value_))) { - LOG_WARN("failed to convert scn", K(ret)); - } else { - stmt->get_rpc_arg().with_restore_scn_ = true; - } - } - } - } - - if (OB_SUCC(ret)) { - if (8 == parse_tree.num_child_) { // resolve table_list - const ParseNode *node = parse_tree.children_[7]; - if (OB_ISNULL(node)) { - stmt->set_is_preview(false); + } else if (0/*timestamp*/ == time_node->children_[0]->value_) { + stmt->get_rpc_arg().restore_timestamp_.assign_ptr(time_node->children_[1]->str_value_, time_node->children_[1]->str_len_); + stmt->get_rpc_arg().with_restore_scn_ = false; + } else if (1/*timestamp*/ == time_node->children_[0]->value_) { + if (share::OB_BASE_SCN_TS_NS >= time_node->children_[1]->value_) { + ret = OB_INVALID_ARGUMENT; + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "until scn, it should be positive integer"); + LOG_WARN("until scn, it should be positive integer", KR(ret), K(time_node->children_[1]->value_)); + } else if (OB_FAIL(stmt->get_rpc_arg().restore_scn_.convert_for_sql(time_node->children_[1]->value_))) { + LOG_WARN("failed to convert scn", K(ret)); } else { - if (T_TABLE_LIST == node->type_) { - // TODO(chongrong.th) table list restore not support, fix this in 4.3 - ret = OB_NOT_SUPPORTED; - LOG_USER_ERROR(OB_NOT_SUPPORTED, "table list restore is"); - // store database_name/table_name with case sensitive. - // compare database_name/table_name with tenant's name_case_mode. - lib::CompatModeGuard g(lib::Worker::CompatMode::ORACLE); - for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; i++) { - const ParseNode *table_node = node->children_[i]; - if (OB_ISNULL(table_node)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("table_node is null", KR(ret)); - } else { - ObString table_name; - ObString database_name; - obrpc::ObTableItem table_item; - if (OB_FAIL(resolve_table_relation_node(table_node, - table_name, - database_name))) { - LOG_WARN("failed to resolve table name", KR(ret), K(table_item)); - } else { - table_item.table_name_ = table_name; - table_item.database_name_ = database_name; - if (OB_FAIL(stmt->get_rpc_arg().add_table_item(table_item))) { - LOG_WARN("failed to add table item", KR(ret), K(table_item)); - } - } - } - } - } else if (T_PREVIEW == node->type_) { - stmt->set_is_preview(true); - } else { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("node type is not right", K(ret), K(node)); - } + stmt->get_rpc_arg().with_restore_scn_ = true; } } } } - } return ret; } @@ -4254,6 +4218,49 @@ int ObCancelRestoreResolver::resolve(const ParseNode &parse_tree) return ret; } +int ObCancelRecoverTableResolver::resolve(const ParseNode &parse_tree) +{ + int ret = OB_SUCCESS; + uint64_t session_tenant_id = session_info_->get_effective_tenant_id(); + uint64_t tenant_id = OB_INVALID_TENANT_ID; + ObRecoverTableStmt *stmt = nullptr; + ObSchemaGetterGuard schema_guard; + ObString tenant_name; + if (OB_UNLIKELY(T_CANCEL_RECOVER_TABLE != parse_tree.type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("type is not T_CANCEL_RESTORE", "type", get_type_name(parse_tree.type_)); + } else if (OB_UNLIKELY(NULL == parse_tree.children_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("children should not be null", K(ret)); + } else if (OB_UNLIKELY(1 != parse_tree.num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("children num not match", K(ret), "num_child", parse_tree.num_child_); + } else if (!is_sys_tenant(session_tenant_id)) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("user tenant cancel recover table is not allowed", K(ret), K(session_tenant_id)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "user tenant cancel recover table is"); + } else if (OB_ISNULL(stmt = create_stmt())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_ERROR("failed to create stmt", K(ret)); + } else if (OB_UNLIKELY(T_IDENT != parse_tree.children_[0]->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid node", K(ret)); + } else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(session_tenant_id, schema_guard))) { + LOG_WARN("failed to get_tenant_schema_guard", KR(ret)); + } else if (OB_FALSE_IT(tenant_name.assign_ptr(parse_tree.children_[0]->str_value_, parse_tree.children_[0]->str_len_))) { + } else if (OB_FAIL(schema_guard.get_tenant_id(tenant_name, tenant_id))) { + LOG_WARN("failed to get tenant id from schema guard", KR(ret), K(tenant_name)); + } else if (OB_FAIL(ObRecoverTableUtil::check_compatible(tenant_id))) { + LOG_WARN("check recover table compatible failed", K(ret)); + } else { + stmt->get_rpc_arg().tenant_id_ = tenant_id; + stmt->get_rpc_arg().tenant_name_ = tenant_name; + stmt->get_rpc_arg().action_ = ObRecoverTableArg::CANCEL; + } + + return ret; +} + int ObAlterSystemResolverUtil::get_tenant_ids(const ParseNode &t_node, ObIArray &tenant_ids) { int ret = OB_SUCCESS; @@ -4925,6 +4932,601 @@ int ObCheckpointSlogResolver::resolve(const ParseNode &parse_tree) return ret; } +int ObRecoverTableResolver::resolve(const ParseNode &parse_tree) +{ + int ret = OB_NOT_SUPPORTED; + if (OB_UNLIKELY(T_RECOVER_TABLE != parse_tree.type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("type is not T_RECOVER_TABLE", "type", get_type_name(parse_tree.type_)); + } else if (OB_UNLIKELY(9 != parse_tree.num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("children num not match", K(ret), "num_child", parse_tree.num_child_); + } else { + ObRecoverTableStmt *stmt = create_stmt(); + Worker::CompatMode compat_mode = Worker::CompatMode::INVALID; + ObNameCaseMode case_mode = ObNameCaseMode::OB_NAME_CASE_INVALID; + if (OB_ISNULL(stmt)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_ERROR("create ObRecoverTableStmt failed", K(ret)); + } else if (OB_FAIL(resolve_tenant_( + parse_tree.children_[0], stmt->get_rpc_arg().tenant_id_, stmt->get_rpc_arg().tenant_name_, compat_mode, case_mode))) { + LOG_WARN("failed to resolve tenant id", K(ret)); + } else if (OB_FAIL(ObRecoverTableUtil::check_compatible(stmt->get_rpc_arg().tenant_id_))) { + LOG_WARN("check recover table compatible failed", K(ret)); + } else if (OB_FAIL(Util::resolve_string(parse_tree.children_[1], stmt->get_rpc_arg().restore_tenant_arg_.uri_))) { + LOG_WARN("failed to resolve backup dest", K(ret)); + } else if (OB_FAIL(resolve_scn_(parse_tree.children_[2], stmt->get_rpc_arg().restore_tenant_arg_))) { + LOG_WARN("failed to resolve restore scn", K(ret)); + } else if (OB_FAIL(Util::resolve_string(parse_tree.children_[3], stmt->get_rpc_arg().restore_tenant_arg_.restore_option_))) { + LOG_WARN("failed to resolve restore option", K(ret)); + } else if (OB_FAIL(resolve_recover_tables_( + parse_tree.children_[4], compat_mode, case_mode, stmt->get_rpc_arg().import_arg_.get_import_table_arg()))) { + LOG_WARN("failed to resolve recover table list", K(ret)); + } else if (OB_NOT_NULL(parse_tree.children_[5]) + && OB_FAIL(Util::resolve_string(parse_tree.children_[5], stmt->get_rpc_arg().restore_tenant_arg_.encrypt_key_))) { + LOG_WARN("failed to resolve encrypt key", K(ret)); + } else if (OB_NOT_NULL(parse_tree.children_[6])) { + ParseNode *kms_node = parse_tree.children_[6]; + if (2 != kms_node->num_child_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("num of children not match", K(ret), "child_num", kms_node->num_child_); + } else if (OB_ISNULL(kms_node->children_[0])) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("kms uri should not be NULL", K(ret)); + } else if (OB_FAIL(Util::resolve_string(kms_node->children_[0], + stmt->get_rpc_arg().restore_tenant_arg_.kms_uri_))) { + LOG_WARN("failed to resolve kms uri", K(ret)); + } else if (OB_NOT_NULL(kms_node->children_[1]) + && OB_FAIL(Util::resolve_string(kms_node->children_[1], + stmt->get_rpc_arg().restore_tenant_arg_.kms_encrypt_key_))) { + LOG_WARN("failed to resolve kms encrypt key", K(ret)); + } + } + + if (OB_FAIL(ret)) { + } else if (OB_NOT_NULL(parse_tree.children_[7]) + && OB_FAIL(resolve_remap_(parse_tree.children_[7], compat_mode, case_mode, stmt->get_rpc_arg().import_arg_.get_remap_table_arg()))) { + LOG_WARN("failed to resolve remap", K(ret)); + } else if (OB_NOT_NULL(parse_tree.children_[8]) + && OB_FAIL(Util::resolve_string(parse_tree.children_[8], stmt->get_rpc_arg().restore_tenant_arg_.description_))) { + LOG_WARN("failed to resolve desc", K(ret)); +#ifndef OB_BUILD_TDE_SECURITY + } else if (OB_FAIL(resolve_kms_info_( + stmt->get_rpc_arg().restore_tenant_arg_.restore_option_, stmt->get_rpc_arg().restore_tenant_arg_.kms_info_))) { + LOG_WARN("failed to resolve kms info", K(ret)); +#endif + } else if (OB_FAIL(resolve_backup_set_pwd_(stmt->get_rpc_arg().restore_tenant_arg_.passwd_array_))) { + LOG_WARN("failed to resolve backup set pwd", K(ret)); + } else if (OB_FAIL(resolve_restore_source_(stmt->get_rpc_arg().restore_tenant_arg_.multi_uri_))) { + LOG_WARN("failed to resolve restore source", K(ret)); + } + + if (OB_SUCC(ret)) { + stmt->get_rpc_arg().action_ = ObRecoverTableArg::INITIATE; + } + } + return ret; +} + +int ObRecoverTableResolver::resolve_remap_( + const ParseNode *node, + const lib::Worker::CompatMode &compat_mode, + const ObNameCaseMode &case_mode, + share::ObImportRemapArg &remap_arg) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("node must not be null", K(ret)); + } else if (node->num_child_ > 3) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("children num not match", K(ret), "num_child", node->num_child_); + } else { + bool parsed_remap_table = false; + bool parsed_remap_tablegroup = false; + bool parsed_remap_tablespace = false; + const int DEFAULT_ERROR_MSG_LEN = 16; + for (int64_t i = 0; i < node->num_child_ && OB_SUCC(ret); i++) { + const ParseNode *child_node = node->children_[i]; + if (T_REMAP_TABLE == child_node->type_) { + if (parsed_remap_table) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("duplicate remap table is not allowed", K(ret)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "duplicate REMAP TABLE is"); + } else if (OB_FAIL(resolve_remap_tables_(child_node, compat_mode, case_mode, remap_arg))) { + LOG_WARN("failed to resolve remap tables", K(ret)); + } else { + parsed_remap_table = true; + } + } else if (T_REMAP_TABLEGROUP == child_node->type_) { + if (parsed_remap_tablegroup) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("duplicate remap tablegroup is not allowed", K(ret)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "duplicate REMAP TABLEGROUP is"); + } else if (OB_FAIL(resolve_remap_tablegroups_(child_node, remap_arg))) { + LOG_WARN("failed to resolve remap tablegroups", K(ret)); + } else { + parsed_remap_tablegroup = true; + } + } else if (T_REMAP_TABLESPACE == child_node->type_) { + if (parsed_remap_tablespace) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("duplicate remap tablespace is not allowed", K(ret)); + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "duplicate REMAP TABLESPACE is"); + } else if (OB_FAIL(resolve_remap_tablespaces_(child_node, remap_arg))) { + LOG_WARN("failed to resolve remap tablespaces", K(ret)); + } else { + parsed_remap_tablespace = true; + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid remap", K(ret)); + } + } + } + return ret; +} + +int ObRecoverTableResolver::resolve_restore_source_(common::ObString &restore_source) +{ + int ret = OB_SUCCESS; + ObObj value; + if (OB_ISNULL(session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("session info must not be nullptr", K(ret)); + } else if (session_info_->user_variable_exists(OB_RESTORE_SOURCE_NAME_SESSION_STR)) { + if (OB_FAIL(session_info_->get_user_variable_value(OB_RESTORE_SOURCE_NAME_SESSION_STR, value))) { + LOG_WARN("failed to get user variable value", K(ret)); + } else { + restore_source = value.get_char(); + LOG_INFO("succeed to resolve restore source", K(restore_source)); + } + } + return ret; +} + +int ObRecoverTableResolver::resolve_remap_tablespaces_( + const ParseNode *node, share::ObImportRemapArg &remap_arg) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(node)) {// no need to remap tablegroups + } else if (OB_UNLIKELY(T_REMAP_TABLESPACE != node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("type is not T_REMAP_TABLESPACE", "type", get_type_name(node->type_)); + } else { + share::ObRemapTablespaceItem item; + const ObNameCaseMode case_mode = OB_ORIGIN_AND_SENSITIVE; + for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; ++i) { + const ParseNode *remap_ts_node = node->children_[i]; + if (OB_ISNULL(remap_ts_node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("remap table group node must not be null", K(ret)); + } else if (OB_UNLIKELY(T_RELATION_FACTOR != remap_ts_node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid parse node type", K(ret)); + } else if (OB_UNLIKELY(2 != remap_ts_node->num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid child num", K(ret)); + } else { + const ParseNode *src = remap_ts_node->children_[0]; + const ParseNode *dst = remap_ts_node->children_[1]; + if (OB_ISNULL(src) || OB_ISNULL(dst)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("remap tablegroup must not be null", K(ret)); + } else { + ObString src_name(src->str_len_, src->str_value_), dst_name(dst->str_len_, dst->str_value_); + item.reset(); + if (src_name.length() > OB_MAX_TABLESPACE_NAME_LENGTH || src_name.length() <= 0) { + ret = OB_ERR_WRONG_VALUE; + LOG_WARN("invalid src name or dst name", K(ret), K(src_name)); + LOG_USER_ERROR(OB_ERR_WRONG_VALUE, "REMAP TABLESPACE", to_cstring(src_name)); + } else if (dst_name.length() > OB_MAX_TABLESPACE_NAME_LENGTH || dst_name.length() <= 0) { + ret = OB_ERR_WRONG_VALUE; + LOG_WARN("invalid src name or dst name", K(ret), K(dst_name)); + LOG_USER_ERROR(OB_ERR_WRONG_VALUE, "REMAP TABLESPACE", to_cstring(dst_name)); + } else if (OB_FALSE_IT(item.src_.mode_ = case_mode)) { + } else if (OB_FALSE_IT(item.target_.mode_ = case_mode)) { + } else if (OB_FALSE_IT(item.src_.name_.assign_ptr(src_name.ptr(), src_name.length()))) { + } else if (OB_FALSE_IT(item.target_.name_.assign_ptr(dst_name.ptr(), dst_name.length()))) { + LOG_WARN("failed to assign", K(dst_name)); + } else if (OB_FAIL(remap_arg.add_remap_tablespace(item))) { + LOG_WARN("failed to add remap tablespace", K(ret), K(item)); + } + } + } + } + } + return ret; +} + +int ObRecoverTableResolver::resolve_remap_tablegroups_( + const ParseNode *node, share::ObImportRemapArg &remap_arg) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(node)) {// no need to remap tablegroups + } else if (OB_UNLIKELY(T_REMAP_TABLEGROUP != node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("type is not T_REMAP_TABLEGROUP", "type", get_type_name(node->type_)); + } else { + share::ObRemapTablegroupItem item; + const ObNameCaseMode case_mode = OB_ORIGIN_AND_SENSITIVE; + for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; ++i) { + const ParseNode *remap_tg_node = node->children_[i]; + if (OB_ISNULL(remap_tg_node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("remap table group node must not be null", K(ret)); + } else if (OB_UNLIKELY(T_RELATION_FACTOR != remap_tg_node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid parse node type", K(ret)); + } else if (OB_UNLIKELY(2 != remap_tg_node->num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid child num", K(ret)); + } else { + const ParseNode *src = remap_tg_node->children_[0]; + const ParseNode *dst = remap_tg_node->children_[1]; + if (OB_ISNULL(src) || OB_ISNULL(dst)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("remap tablegroup must not be null", K(ret)); + } else { + ObString src_name(src->str_len_, src->str_value_), dst_name(dst->str_len_, dst->str_value_); + item.reset(); + if (src_name.length() > OB_MAX_TABLEGROUP_NAME_LENGTH || src_name.length() <= 0) { + ret = OB_ERR_WRONG_VALUE; + LOG_WARN("invalid src name", K(ret), K(src_name)); + LOG_USER_ERROR(OB_ERR_WRONG_VALUE, "REMAP TABLEGROUP", to_cstring(src_name)); + } else if (dst_name.length() > OB_MAX_TABLEGROUP_NAME_LENGTH || dst_name.length() <= 0) { + ret = OB_ERR_WRONG_VALUE; + LOG_WARN("invalid dst name", K(ret), K(dst_name)); + LOG_USER_ERROR(OB_ERR_WRONG_VALUE, "REMAP TABLEGROUP", to_cstring(dst_name)); + } else if (OB_FALSE_IT(item.src_.mode_ = case_mode)) { + } else if (OB_FALSE_IT(item.target_.mode_ = case_mode)) { + } else if (OB_FALSE_IT(item.src_.name_.assign_ptr(src_name.ptr(), src_name.length()))) { + } else if (OB_FALSE_IT(item.target_.name_.assign_ptr(dst_name.ptr(), dst_name.length()))) { + LOG_WARN("failed to assign", K(dst_name)); + } else if (OB_FAIL(remap_arg.add_remap_tablegroup(item))) { + LOG_WARN("failed to add remap tablegroup", K(ret), K(item)); + } + } + } + } + } + return ret; +} + +int ObRecoverTableResolver::resolve_remap_tables_( + const ParseNode *node, const lib::Worker::CompatMode &compat_mode, const ObNameCaseMode &case_mode, + share::ObImportRemapArg &remap_arg) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(node)) { // no need to remap tables + } else if (OB_UNLIKELY(T_REMAP_TABLE != node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("type is not T_REMAP_TABLE", "type", get_type_name(node->type_)); + } else { + share::ObRemapDatabaseItem remap_db_item; + share::ObRemapTableItem remap_table_item; + ObCollationType cs_type = CS_TYPE_INVALID; + bool perserve_lettercase = Worker::CompatMode::ORACLE == compat_mode ? true : (case_mode != OB_LOWERCASE_AND_INSENSITIVE); + // No matter what name case mode is of target tenant, the names of remap tables are case sensitive. + const ObNameCaseMode sensitive_case_mode = OB_ORIGIN_AND_SENSITIVE; + for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; ++i) { + bool is_remap_db = false; + + remap_db_item.reset(); + remap_table_item.reset(); + remap_db_item.src_.mode_ = sensitive_case_mode; + remap_db_item.target_.mode_ = sensitive_case_mode; + remap_table_item.src_.mode_ = sensitive_case_mode; + remap_table_item.target_.mode_ = sensitive_case_mode; + const ParseNode *remap_table_node = node->children_[i]; + if (OB_ISNULL(remap_table_node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("remap table group node must not be null", K(ret)); + } else if (OB_UNLIKELY(T_RELATION_FACTOR != remap_table_node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid parse node type", K(ret)); + } else if (5 == remap_table_node->num_child_) { + // remap table + const ParseNode *src_db_node = remap_table_node->children_[0]; + const ParseNode *src_tb_node = remap_table_node->children_[1]; + const ParseNode *src_pt_node = remap_table_node->children_[2]; + const ParseNode *dst_db_node = remap_table_node->children_[3]; + const ParseNode *dst_tb_node = remap_table_node->children_[4]; + + ObString src_db_name, src_tb_name, src_pt_name, dst_db_name, dst_tb_name; + + if (OB_NOT_NULL(src_db_node) && OB_FALSE_IT(src_db_name.assign_ptr(src_db_node->str_value_, src_db_node->str_len_))) { + } else if (OB_NOT_NULL(src_tb_node) && OB_FALSE_IT(src_tb_name.assign_ptr(src_tb_node->str_value_, src_tb_node->str_len_))) { + } else if (OB_NOT_NULL(src_pt_node) && OB_FALSE_IT(src_pt_name.assign_ptr(src_pt_node->str_value_, src_pt_node->str_len_))) { + } else if (OB_NOT_NULL(dst_db_node) && OB_FALSE_IT(dst_db_name.assign_ptr(dst_db_node->str_value_, dst_db_node->str_len_))) { + } else if (OB_NOT_NULL(dst_tb_node) && OB_FALSE_IT(dst_tb_name.assign_ptr(dst_tb_node->str_value_, dst_tb_node->str_len_))) { + } + + if (!src_db_name.empty() && OB_FAIL(ObSQLUtils::check_and_convert_db_name(cs_type, perserve_lettercase, src_db_name))) { + LOG_WARN("failed to check and convert db name", K(ret), K(cs_type), K(perserve_lettercase), K(src_db_name)); + } else if (!src_tb_name.empty() && OB_FAIL(ObSQLUtils::check_and_convert_table_name(cs_type, perserve_lettercase, src_tb_name))) { + LOG_WARN("failed to check and convert table name", K(ret), K(cs_type), K(perserve_lettercase), K(src_tb_name)); + } else if (!dst_db_name.empty() && OB_FAIL(ObSQLUtils::check_and_convert_db_name(cs_type, perserve_lettercase, dst_db_name))) { + LOG_WARN("failed to check and convert db name", K(ret), K(cs_type), K(perserve_lettercase), K(dst_db_name)); + } else if (!dst_tb_name.empty() && OB_FAIL(ObSQLUtils::check_and_convert_table_name(cs_type, perserve_lettercase, dst_tb_name))) { + LOG_WARN("failed to check and convert table name", K(ret), K(cs_type), K(perserve_lettercase), K(dst_tb_name)); + } else if (!src_pt_name.empty() && src_pt_name.length() > OB_MAX_PARTITION_NAME_LENGTH) { + ret = OB_ERR_WRONG_VALUE; + LOG_WARN("invalid partition name", K(ret)); + LOG_USER_ERROR(OB_ERR_WRONG_VALUE, "INVALID PARTITION NAME", to_cstring(src_pt_name)); + } + + if (OB_FAIL(ret)) { + } else if (OB_NOT_NULL(src_db_node) && OB_ISNULL(src_tb_node) && OB_ISNULL(src_pt_node) && OB_NOT_NULL(dst_db_node) && OB_ISNULL(dst_tb_node)) { + // db_name.'*':new_db_name.'*'; + remap_db_item.src_.name_.assign_ptr(src_db_name.ptr(), src_db_name.length()); + remap_db_item.target_.name_.assign_ptr(dst_db_name.ptr(), dst_db_name.length()); + is_remap_db = true; + } else if (OB_NOT_NULL(src_db_node) && OB_NOT_NULL(src_tb_node) && OB_ISNULL(src_pt_node) && OB_ISNULL(dst_db_node) && OB_NOT_NULL(dst_tb_node)) { + // db_name.tb_name:new_tb_name; + remap_table_item.src_.database_name_.assign_ptr(src_db_name.ptr(), src_db_name.length()); + remap_table_item.src_.table_name_.assign_ptr(src_tb_name.ptr(), src_tb_name.length()); + remap_table_item.target_.database_name_.assign_ptr(src_db_name.ptr(), src_db_name.length()); + remap_table_item.target_.table_name_.assign_ptr(dst_tb_name.ptr(), dst_tb_name.length()); + } else if (OB_NOT_NULL(src_db_node) && OB_NOT_NULL(src_tb_node) && OB_ISNULL(src_pt_node) && OB_NOT_NULL(dst_db_node) && OB_NOT_NULL(dst_tb_node)) { + // db_name.tb_name:new_db_name.new_tb_name + remap_table_item.src_.database_name_.assign_ptr(src_db_name.ptr(), src_db_name.length()); + remap_table_item.src_.table_name_.assign_ptr(src_tb_name.ptr(), src_tb_name.length()); + remap_table_item.target_.database_name_.assign_ptr(dst_db_name.ptr(), dst_db_name.length()); + remap_table_item.target_.table_name_.assign_ptr(dst_tb_name.ptr(), dst_tb_name.length()); + } else if (OB_NOT_NULL(src_db_node) && OB_NOT_NULL(src_tb_node) && OB_NOT_NULL(src_pt_node) && OB_ISNULL(dst_db_node) && OB_NOT_NULL(dst_tb_node)) { + // db_name.tb_name:part_name:new_tb_name; + int ret = OB_NOT_SUPPORTED; + LOG_WARN("remap partition is not supported", K(ret)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "remap partition"); + } else if (OB_NOT_NULL(src_db_node) && OB_NOT_NULL(src_tb_node) && OB_NOT_NULL(src_pt_node) && OB_NOT_NULL(dst_db_node) && OB_NOT_NULL(dst_tb_node)) { + int ret = OB_NOT_SUPPORTED; + LOG_WARN("remap partition is not supported", K(ret)); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid remap tables", K(ret)); + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid remap tables", K(ret)); + } + + if (OB_FAIL(ret)) { + } else if (!is_remap_db && OB_FAIL(remap_arg.add_remap_table(remap_table_item))) { + LOG_WARN("fail to push backup", K(ret), K(remap_table_item)); + } else if (is_remap_db && OB_FAIL(remap_arg.add_remap_database(remap_db_item))) { + LOG_WARN("fail to push backup", K(ret), K(remap_db_item)); + } + } + } + return ret; +} + +int ObRecoverTableResolver::resolve_backup_set_pwd_(common::ObString &pwd) +{ + int ret = OB_SUCCESS; + ObObj value; + if (OB_ISNULL(session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("session is null" , K(ret)); + } else if (!session_info_->user_variable_exists(OB_BACKUP_DECRYPTION_PASSWD_ARRAY_SESSION_STR)) { + LOG_INFO("no decryption passwd is specified"); + pwd.reset(); + } else if (OB_FAIL(session_info_->get_user_variable_value( + OB_BACKUP_DECRYPTION_PASSWD_ARRAY_SESSION_STR, value))) { + LOG_WARN("fail to get user variable", K(ret)); + } else { + pwd = value.get_varchar(); + LOG_INFO("succeed to resolve_decryption_passwd", "passwd", pwd); + } + return ret; +} + +#ifndef OB_BUILD_TDE_SECURITY +int ObRecoverTableResolver::resolve_kms_info_(const common::ObString &restore_option, common::ObString &kms_info) +{ + int ret = OB_SUCCESS; + const char *encrypt_option_str = "kms_encrypt=true"; + ObString kms_var("kms_encrypt_info"); + int64_t encrypt_opt_str_len = strlen(encrypt_option_str); + bool is_kms_encrypt = false; + for (int i = 0; i <= restore_option.length() - encrypt_opt_str_len; ++i) { + if (0 == STRNCASECMP(restore_option.ptr() + i, encrypt_option_str, encrypt_opt_str_len)) { + is_kms_encrypt = true; + break; + } + } + if (is_kms_encrypt) { + ObObj value; + if (OB_ISNULL(session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("session is null" , K(ret)); + } else if (OB_FAIL(session_info_->get_user_variable_value(kms_var, value))) { + LOG_WARN("fail to get user variable", K(ret)); + } else { + kms_info = value.get_varchar(); + if (kms_info.length() > 4000 || kms_info.length() < 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("kms info is not valid", K(ret)); + } + } + } + return ret; +} +#endif + +int ObRecoverTableResolver::resolve_recover_tables_( + const ParseNode *node, const lib::Worker::CompatMode &compat_mode, const ObNameCaseMode &case_mode, + share::ObImportTableArg &import_arg) +{ + int ret = OB_SUCCESS; + ObCollationType cs_type = CS_TYPE_INVALID; + bool perserve_lettercase = Worker::CompatMode::ORACLE == compat_mode ? true : (case_mode != OB_LOWERCASE_AND_INSENSITIVE); + // No matter what name case mode is of target tenant, the names of recover tables are case sensitive. + const ObNameCaseMode sensitive_case_mode = OB_ORIGIN_AND_SENSITIVE; + if (OB_ISNULL(node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table list node must not be null", K(ret)); + } else if (OB_FAIL(session_info_->get_collation_connection(cs_type))) { + LOG_WARN("failed to get collation connection", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; ++i) { + const ParseNode *table_relation_node = node->children_[i]; + if (OB_ISNULL(table_relation_node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table relation node must not be null", K(ret)); + } else if (OB_UNLIKELY(T_RELATION_FACTOR != table_relation_node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid parse node type", K(ret)); + } else if (3 == table_relation_node->num_child_) { + const ParseNode *db_node = table_relation_node->children_[0]; + const ParseNode *tb_node = table_relation_node->children_[1]; + const ParseNode *pt_node = table_relation_node->children_[2]; + if (OB_ISNULL(db_node) && OB_ISNULL(tb_node) && OB_ISNULL(pt_node)) { + // '*'.'*' recover all table of all all database. + if (OB_FAIL(import_arg.set_import_all())) { + LOG_WARN("failed to set import all", K(ret)); + } + } else if (OB_NOT_NULL(db_node) && OB_ISNULL(tb_node) && OB_ISNULL(pt_node)) { + // db_name.'*' recover all tables of db_name + ObString db_name(db_node->str_len_, db_node->str_value_); + share::ObImportDatabaseItem db_item(sensitive_case_mode, db_node->str_value_, db_node->str_len_); + if (OB_FAIL(ObSQLUtils::check_and_convert_db_name(cs_type, perserve_lettercase, db_name))) { + LOG_WARN("failed to check and convert db name", K(ret), K(cs_type), K(perserve_lettercase), K(db_name)); + } else if (OB_FAIL(import_arg.add_database(db_item))) { + LOG_WARN("failed to add database", K(ret), K(db_item)); + } + } else if (OB_NOT_NULL(db_node) && OB_NOT_NULL(tb_node) && OB_ISNULL(pt_node)) { + // db_name.tb_name recover tb_name of db_name + ObString db_name(db_node->str_len_, db_node->str_value_), tb_name(tb_node->str_len_, tb_node->str_value_); + if (OB_FAIL(ObSQLUtils::check_and_convert_table_name(cs_type, perserve_lettercase, tb_name))) { + LOG_WARN("failed to check and convert table name", K(ret), K(cs_type), K(perserve_lettercase), K(tb_name)); + } else if (OB_FAIL(ObSQLUtils::check_and_convert_db_name(cs_type, perserve_lettercase, db_name))) { + LOG_WARN("failed to check and convert db name", K(ret), K(cs_type), K(perserve_lettercase), K(db_name)); + } else { + share::ObImportTableItem table_item(sensitive_case_mode, db_name.ptr(), db_name.length(), tb_name.ptr(), tb_name.length()); + if (OB_FAIL(import_arg.add_table(table_item))) { + LOG_WARN("failed to add table", K(ret), K(table_item)); + } + } + } else if (OB_NOT_NULL(db_node) && OB_NOT_NULL(tb_node) && OB_NOT_NULL(pt_node)) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("recover partition is not support", K(ret)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "recover partition"); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid recover table list", K(ret)); + LOG_USER_ERROR(OB_ERR_UNEXPECTED, "invalid recover table list"); + } + } else if (5 == table_relation_node->num_child_) { + const ParseNode *db_node = table_relation_node->children_[0]; + const ParseNode *tb_node = table_relation_node->children_[1]; + const ParseNode *pt_node = table_relation_node->children_[4]; + if (OB_NOT_NULL(db_node) && OB_NOT_NULL(tb_node) && OB_NOT_NULL(pt_node)) { + // db_name.tb_name:pt_name reocver the partion of tb_name + ret = OB_NOT_SUPPORTED; + LOG_WARN("recover partition is not supported", K(ret)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "recover partition"); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid recover table list", K(ret)); + LOG_USER_ERROR(OB_ERR_UNEXPECTED, "invalid recover table list"); + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid recover table list", K(ret)); + LOG_USER_ERROR(OB_ERR_UNEXPECTED, "invalid recover table list"); + } + } + + return ret; +} + +int ObRecoverTableResolver::resolve_scn_( + const ParseNode *node, obrpc::ObPhysicalRestoreTenantArg &arg) +{ + int ret = OB_SUCCESS; + const ParseNode *time_node = nullptr; + if (OB_ISNULL(node)) { //restore to latest, scn = 0; + } else if (OB_FALSE_IT(time_node = node)) { + } else if (OB_UNLIKELY(T_PHYSICAL_RESTORE_UNTIL != time_node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("type is not T_PHYSICAL_RESTORE_UNTIL", "type", get_type_name(time_node->type_)); + } else if (OB_UNLIKELY(2 != time_node->num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("num of children not match", K(ret), "child_num", time_node->num_child_); + } else if (0/*timestamp*/ == time_node->children_[0]->value_) { + arg.restore_timestamp_.assign_ptr(time_node->children_[1]->str_value_, time_node->children_[1]->str_len_); + arg.with_restore_scn_ = false; + } else if (1/*scn*/ == time_node->children_[0]->value_) { + if (share::OB_BASE_SCN_TS_NS >= time_node->children_[1]->value_) { + ret = OB_INVALID_ARGUMENT; + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "until scn, it should be positive integer"); + LOG_WARN("until scn, it should be positive integer", KR(ret), K(time_node->children_[1]->value_)); + } else if (OB_FAIL(arg.restore_scn_.convert_for_sql(time_node->children_[1]->value_))) { + LOG_WARN("failed to convert scn", K(ret)); + } else { + arg.with_restore_scn_ = true; + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_INFO("invalid until", K(ret)); + LOG_USER_ERROR(OB_ERR_UNEXPECTED, "invalid until type"); + } + return ret; +} + +int ObRecoverTableResolver::resolve_tenant_( + const ParseNode *node, uint64_t &tenant_id, common::ObString &tenant_name, lib::Worker::CompatMode &compat_mode, + ObNameCaseMode &case_mode) +{ + int ret = OB_SUCCESS; + uint64_t session_tenant_id = OB_INVALID_TENANT_ID; + if (OB_ISNULL(session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("session is NULL", K(ret)); + } else if (OB_FALSE_IT(session_tenant_id = session_info_->get_effective_tenant_id())) { + } else if (is_sys_tenant(session_tenant_id) && OB_NOT_NULL(node)) { + // System tenant initiates recover table. + if (OB_UNLIKELY(T_TENANT_NAME != node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("type is not T_TENANT_NAME", "type", get_type_name(node->type_)); + } else if (OB_UNLIKELY(1 != node->num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tenant name node must not be null", K(ret)); + } else { + ObSchemaGetterGuard schema_guard; + ObString tmp_tenant_name(node->children_[0]->str_len_, node->children_[0]->str_value_); + if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(session_tenant_id, schema_guard))) { + LOG_WARN("failed to get_tenant_schema_guard", KR(ret)); + } else if (OB_FAIL(schema_guard.get_tenant_id(tmp_tenant_name, tenant_id))) { + LOG_WARN("failed to get tenant id from schema guard", KR(ret), K(tmp_tenant_name)); + } else { + tenant_name = tmp_tenant_name; + } + } + } else { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("can't initiate table recover", K(ret), K(session_tenant_id)); + if (is_sys_tenant(session_tenant_id)) { + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "no target tenant specified"); + } else { + LOG_USER_ERROR(OB_OP_NOT_ALLOW, "user tenant initiate recover table is"); + } + } + if (OB_SUCC(ret)) { + ObSchemaGetterGuard schema_guard; + if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) { + LOG_WARN("failed to get tenant schema guard", K(ret), K(tenant_id)); + } else if (OB_FAIL(schema_guard.get_tenant_compat_mode(tenant_id, compat_mode))) { + LOG_WARN("failed to get compat mode", K(ret), K(tenant_id)); + } else if (OB_FAIL(schema_guard.get_tenant_name_case_mode(tenant_id, case_mode))) { + LOG_WARN("failed to get name case mode", K(ret), K(tenant_id)); + } + } + return ret; +} + int ObRecoverTenantResolver::resolve(const ParseNode &parse_tree) { int ret = OB_SUCCESS; diff --git a/src/sql/resolver/cmd/ob_alter_system_resolver.h b/src/sql/resolver/cmd/ob_alter_system_resolver.h index 882e67075e..b83f0fa758 100644 --- a/src/sql/resolver/cmd/ob_alter_system_resolver.h +++ b/src/sql/resolver/cmd/ob_alter_system_resolver.h @@ -250,6 +250,38 @@ DEF_SIMPLE_CMD_RESOLVER(ObEnableSqlThrottleResolver); DEF_SIMPLE_CMD_RESOLVER(ObDisableSqlThrottleResolver); DEF_SIMPLE_CMD_RESOLVER(ObSetRegionBandwidthResolver); DEF_SIMPLE_CMD_RESOLVER(ObCancelRestoreResolver); +DEF_SIMPLE_CMD_RESOLVER(ObCancelRecoverTableResolver); + +class ObRecoverTableResolver : public ObSystemCmdResolver +{ +public: + ObRecoverTableResolver(ObResolverParams ¶ms) : ObSystemCmdResolver(params) {} + virtual ~ObRecoverTableResolver() {} + virtual int resolve(const ParseNode &parse_tree); + +private: + int resolve_tenant_(const ParseNode *node, uint64_t &tenant_id, common::ObString &tenant_name, + lib::Worker::CompatMode &compat_mode, ObNameCaseMode &case_mode); + int resolve_scn_(const ParseNode *node, obrpc::ObPhysicalRestoreTenantArg &arg); + int resolve_recover_tables_( + const ParseNode *node, const lib::Worker::CompatMode &compat_mode, const ObNameCaseMode &case_mode, + share::ObImportTableArg &import_arg); + int resolve_remap_(const ParseNode *node, const lib::Worker::CompatMode &compat_mode, const ObNameCaseMode &case_mode, + share::ObImportRemapArg &remap_arg); + int resolve_remap_tables_( + const ParseNode *node, const lib::Worker::CompatMode &compat_mode, const ObNameCaseMode &case_mode, + share::ObImportRemapArg &remap_arg); + int resolve_remap_tablegroups_( + const ParseNode *node, share::ObImportRemapArg &remap_arg); + int resolve_remap_tablespaces_( + const ParseNode *node, share::ObImportRemapArg &remap_arg); +#ifndef OB_BUILD_TDE_SECURITY + int resolve_kms_info_(const common::ObString &restore_option, common::ObString &kms_info); +#endif + int resolve_backup_set_pwd_(common::ObString &pwd); + int resolve_restore_source_(common::ObString &restore_source); +}; + DEF_SIMPLE_CMD_RESOLVER(ObTableTTLResolver); #undef DEF_SIMPLE_CMD_RESOLVER diff --git a/src/sql/resolver/cmd/ob_alter_system_stmt.h b/src/sql/resolver/cmd/ob_alter_system_stmt.h index 0c1dc0feb6..cb103a4e8b 100644 --- a/src/sql/resolver/cmd/ob_alter_system_stmt.h +++ b/src/sql/resolver/cmd/ob_alter_system_stmt.h @@ -1320,6 +1320,18 @@ private: obrpc::ObRecoverTenantArg rpc_arg_; }; +class ObRecoverTableStmt : public ObSystemCmdStmt +{ +public: + ObRecoverTableStmt() + : ObSystemCmdStmt(stmt::T_RECOVER_TABLE), rpc_arg_() {} + virtual ~ObRecoverTableStmt() {} + obrpc::ObRecoverTableArg &get_rpc_arg() { return rpc_arg_; } +private: + obrpc::ObRecoverTableArg rpc_arg_; +}; + + } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/resolver/cmd/ob_show_resolver.cpp b/src/sql/resolver/cmd/ob_show_resolver.cpp index 8b46fe21fe..308febd0d9 100644 --- a/src/sql/resolver/cmd/ob_show_resolver.cpp +++ b/src/sql/resolver/cmd/ob_show_resolver.cpp @@ -1420,9 +1420,19 @@ int ObShowResolver::resolve(const ParseNode &parse_tree) break; } case T_SHOW_RESTORE_PREVIEW: { - // TODO(chongrong.th): fix restore preview in 4.1 - ret = OB_NOT_SUPPORTED; - LOG_USER_ERROR(OB_NOT_SUPPORTED, "show restore preview is"); + if (OB_UNLIKELY(parse_tree.num_child_ != 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("parse tree is wrong", K(ret), K(parse_tree.num_child_)); + } else if (!is_sys_tenant(real_tenant_id)) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("the tenant has no priv to show restore preview", K(ret), K(real_tenant_id)); + } else { + show_resv_ctx.stmt_type_ = stmt::T_SHOW_RESTORE_PREVIEW; + GEN_SQL_STEP_1(ObShowSqlSet::SHOW_RESTORE_PREVIEW); + GEN_SQL_STEP_2(ObShowSqlSet::SHOW_RESTORE_PREVIEW, + OB_SYS_DATABASE_NAME, + OB_TENANT_VIRTUAL_SHOW_RESTORE_PREVIEW_TNAME); + } break; } default: diff --git a/src/sql/resolver/ob_resolver.cpp b/src/sql/resolver/ob_resolver.cpp index 6949650736..10f63935a9 100644 --- a/src/sql/resolver/ob_resolver.cpp +++ b/src/sql/resolver/ob_resolver.cpp @@ -1056,10 +1056,18 @@ int ObResolver::resolve(IsPrepared if_prepared, const ParseNode &parse_tree, ObS REGISTER_STMT_RESOLVER(CancelRestore); break; } + case T_CANCEL_RECOVER_TABLE: { + REGISTER_STMT_RESOLVER(CancelRecoverTable); + break; + } case T_BACKUP_KEY: { REGISTER_STMT_RESOLVER(BackupKey); break; } + case T_RECOVER_TABLE: { + REGISTER_STMT_RESOLVER(RecoverTable); + break; + } case T_BACKUP_MANAGE: { REGISTER_STMT_RESOLVER(BackupManage); break; diff --git a/src/sql/resolver/ob_stmt_type.h b/src/sql/resolver/ob_stmt_type.h index cc8e30cf38..36485a76d0 100644 --- a/src/sql/resolver/ob_stmt_type.h +++ b/src/sql/resolver/ob_stmt_type.h @@ -284,8 +284,8 @@ OB_STMT_TYPE_DEF_UNKNOWN_AT(T_BACKUP_KEY, get_sys_tenant_alter_system_priv, 284) OB_STMT_TYPE_DEF_UNKNOWN_AT(T_CREATE_STANDBY_TENANT, get_sys_tenant_super_priv, 285) OB_STMT_TYPE_DEF_UNKNOWN_AT(T_CANCEL_RESTORE, get_sys_tenant_alter_system_priv, 286) OB_STMT_TYPE_DEF_UNKNOWN_AT(T_TABLE_TTL, get_sys_tenant_alter_system_priv, 287) -//OB_STMT_TYPE_DEF_UNKNOWN_AT(T_RECOVER_TABLE, get_sys_tenant_alter_system_priv, 288) -//OB_STMT_TYPE_DEF_UNKNOWN_AT(T_CANCEL_RECOVER_TABLE, get_sys_tenant_alter_system_priv, 289) +OB_STMT_TYPE_DEF_UNKNOWN_AT(T_RECOVER_TABLE, get_sys_tenant_alter_system_priv, 288) +OB_STMT_TYPE_DEF_UNKNOWN_AT(T_CANCEL_RECOVER_TABLE, get_sys_tenant_alter_system_priv, 289) OB_STMT_TYPE_DEF_UNKNOWN_AT(T_MAX, err_stmt_type_priv, 500) #endif diff --git a/src/storage/backup/ob_backup_data_store.cpp b/src/storage/backup/ob_backup_data_store.cpp index 8aafee66d0..a37416cd32 100755 --- a/src/storage/backup/ob_backup_data_store.cpp +++ b/src/storage/backup/ob_backup_data_store.cpp @@ -954,7 +954,7 @@ int ObBackupDataStore::do_get_backup_set_array_(const common::ObString &passwd_a } for (int64_t i = 0; OB_SUCC(ret) && i < backup_set_desc_array.count(); ++i) { - const share::ObBackupSetDesc &backup_set_desc = backup_set_desc_array.at(i); + share::ObBackupSetDesc &backup_set_desc = backup_set_desc_array.at(i); backup_desc_.backup_set_id_ = backup_set_desc.backup_set_id_; backup_desc_.backup_type_.type_ = backup_set_desc.backup_type_.type_; backup_set_dest_.reset(); @@ -982,6 +982,8 @@ int ObBackupDataStore::do_get_backup_set_array_(const common::ObString &passwd_a || share::ObBackupFileStatus::STATUS::BACKUP_FILE_AVAILABLE != backup_set_file.file_status_) { LOG_WARN("invalid status backup set can not be used to restore", K(backup_set_file)); } else if (backup_set_file.backup_type_.is_full_backup()) { + backup_set_desc.min_restore_scn_ = backup_set_file.min_restore_scn_; + backup_set_desc.total_bytes_ = backup_set_file.stats_.output_bytes_; if (OB_FAIL(backup_set_map.clear())) { LOG_WARN("fail to clear backup set map", K(ret)); } else if (OB_FAIL(backup_set_map.set_refactored(backup_set_file.backup_set_id_, backup_set_desc))) { @@ -993,6 +995,8 @@ int ObBackupDataStore::do_get_backup_set_array_(const common::ObString &passwd_a } else if (backup_set_file.backup_type_.is_inc_backup()) { share::ObBackupSetDesc value; value.backup_set_id_ = backup_set_file.prev_full_backup_set_id_; + backup_set_desc.min_restore_scn_ = backup_set_file.min_restore_scn_; + backup_set_desc.total_bytes_ = backup_set_file.stats_.output_bytes_; if (OB_FAIL(backup_set_map.get_refactored(backup_set_file.prev_full_backup_set_id_, value))) { if (OB_ENTRY_NOT_EXIST == ret) { ret = OB_SUCCESS; diff --git a/src/storage/ddl/ob_build_index_task.cpp b/src/storage/ddl/ob_build_index_task.cpp index 947332e455..008af56866 100644 --- a/src/storage/ddl/ob_build_index_task.cpp +++ b/src/storage/ddl/ob_build_index_task.cpp @@ -87,7 +87,7 @@ int ObUniqueIndexChecker::calc_column_checksum( const common::ObIArray &need_reshape, const ObColDescIArray &cols_desc, const ObIArray &output_projector, - ObIStoreRowIterator &iterator, + ObLocalScan &iterator, common::ObIArray &column_checksum, int64_t &row_count) { @@ -102,13 +102,14 @@ int ObUniqueIndexChecker::calc_column_checksum( STORAGE_LOG(WARN, "fail to reserve column", K(ret), K(column_cnt)); } else { const ObDatumRow *row = NULL; + const ObDatumRow *unused_row = nullptr; for (int64_t i = 0; OB_SUCC(ret) && i < column_cnt; ++i) { if (OB_FAIL(column_checksum.push_back(0))) { STORAGE_LOG(WARN, "fail to push back column checksum", K(ret)); } } while (OB_SUCC(ret)) { - if (OB_FAIL(iterator.get_next_row(row))) { + if (OB_FAIL(iterator.get_next_row(row, unused_row))) { if (OB_ITER_END == ret) { ret = OB_SUCCESS; break; diff --git a/src/storage/ddl/ob_build_index_task.h b/src/storage/ddl/ob_build_index_task.h index c03dbd7bbc..2927cf4f61 100644 --- a/src/storage/ddl/ob_build_index_task.h +++ b/src/storage/ddl/ob_build_index_task.h @@ -18,6 +18,7 @@ #include "storage/ob_store_row_comparer.h" #include "storage/ob_parallel_external_sort.h" #include "storage/meta_mem/ob_tablet_handle.h" +#include "storage/ddl/ob_complement_data_task.h" namespace oceanbase { @@ -96,7 +97,7 @@ private: const common::ObIArray &need_reshape, const ObColDescIArray &cols_desc, const common::ObIArray &output_projector, - ObIStoreRowIterator &iterator, + ObLocalScan &iterator, common::ObIArray &column_checksum, int64_t &row_count); int report_column_checksum(const common::ObIArray &column_checksum, diff --git a/src/storage/ddl/ob_complement_data_task.cpp b/src/storage/ddl/ob_complement_data_task.cpp index 029b0762ac..70dea53a8a 100755 --- a/src/storage/ddl/ob_complement_data_task.cpp +++ b/src/storage/ddl/ob_complement_data_task.cpp @@ -16,11 +16,13 @@ #include "logservice/ob_log_service.h" #include "observer/ob_server_struct.h" #include "share/ob_dml_sql_splicer.h" +#include "share/ob_ddl_common.h" #include "share/ob_ddl_checksum.h" #include "share/ob_ddl_error_message_table_operator.h" #include "share/ob_freeze_info_proxy.h" #include "share/ob_get_compat_mode.h" #include "share/schema/ob_table_dml_param.h" +#include "share/schema/ob_part_mgr_util.h" #include "sql/engine/px/ob_granule_util.h" #include "sql/ob_sql_utils.h" #include "share/scheduler/ob_dag_warning_history_mgr.h" @@ -55,70 +57,102 @@ using namespace blocksstable; namespace storage { + int ObComplementDataParam::init(const ObDDLBuildSingleReplicaRequestArg &arg) { int ret = OB_SUCCESS; - ObSchemaGetterGuard schema_guard; const ObTenantSchema *tenant_schema = nullptr; - const ObTableSchema *data_table_schema = nullptr; - const ObTableSchema *hidden_table_schema = nullptr; - MAKE_TENANT_SWITCH_SCOPE_GUARD(guard); - const bool check_formal = arg.dest_schema_id_ > OB_MAX_CORE_TABLE_ID; //avoid circular dependencies - const uint64_t tenant_id = arg.tenant_id_; + const ObTableSchema *orig_table_schema = nullptr; + const ObTableSchema *dest_table_schema = nullptr; + const uint64_t orig_tenant_id = arg.tenant_id_; + const uint64_t dest_tenant_id = arg.dest_tenant_id_; + const int64_t orig_table_id = arg.source_table_id_; + const int64_t dest_table_id = arg.dest_schema_id_; + const int64_t orig_schema_version = arg.schema_version_; + const int64_t dest_schema_version = arg.dest_schema_version_; + ObSchemaGetterGuard src_tenant_schema_guard; + ObSchemaGetterGuard dst_tenant_schema_guard; if (OB_UNLIKELY(is_inited_)) { ret = OB_INIT_TWICE; LOG_WARN("ObComplementDataParam has been inited before", K(ret)); } else if (OB_UNLIKELY(!arg.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arg", K(ret), K(arg)); - } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( - tenant_id, schema_guard, arg.schema_version_))) { - LOG_WARN("fail to get tenant schema guard", K(ret), K(arg)); - } else if (check_formal && OB_FAIL(schema_guard.check_formal_guard())) { - LOG_WARN("schema_guard is not formal", K(ret), K(arg)); - } else if (OB_FAIL(schema_guard.get_tenant_info(tenant_id, tenant_schema))) { - LOG_WARN("fail to get tenant info", K(ret), K(arg)); - } else if (OB_ISNULL(tenant_schema)) { - ret = OB_TABLE_NOT_EXIST; - LOG_WARN("tenant not exist", K(ret), K(arg)); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, - arg.dest_schema_id_, hidden_table_schema))) { - LOG_WARN("fail to get hidden table schema", K(ret), K(arg)); - } else if (OB_ISNULL(hidden_table_schema)) { - ret = OB_TABLE_NOT_EXIST; - LOG_WARN("hidden table schema not exist", K(ret), K(arg)); - } else if (OB_UNLIKELY(hidden_table_schema->get_association_table_id() != arg.source_table_id_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected error", K(ret), K(arg), K(hidden_table_schema->get_association_table_id())); - } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, - arg.source_table_id_, data_table_schema))) { - LOG_WARN("fail to get data table schema", K(ret), K(arg)); - } else if (OB_ISNULL(data_table_schema)) { - ret = OB_TABLE_NOT_EXIST; - LOG_WARN("data table schema not exist", K(ret), K(arg)); - } else if (OB_FAIL(guard.switch_to(tenant_id))) { - LOG_WARN("fail to switch to tenant", K(ret), K(arg)); - } else if (FALSE_IT(snapshot_version_ = arg.snapshot_version_)) { - } else if (OB_FAIL(split_task_ranges(arg.ls_id_, arg.source_tablet_id_, data_table_schema->get_tablet_size(), arg.parallelism_))) { - LOG_WARN("fail to init concurrent params", K(ret), K(arg)); - } else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(tenant_id, arg.source_table_id_, compat_mode_))) { - LOG_WARN("failed to get compat mode", K(ret), K(arg)); } else { + MTL_SWITCH (OB_SYS_TENANT_ID) { + if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( + orig_tenant_id, src_tenant_schema_guard, orig_schema_version))) { + LOG_WARN("fail to get tenant schema guard", K(ret), K(orig_tenant_id), K(orig_schema_version)); + } else if (OB_FAIL(src_tenant_schema_guard.get_tenant_info(orig_tenant_id, tenant_schema))) { + LOG_WARN("fail to get tenant info", K(ret), K(orig_tenant_id)); + } else if (OB_ISNULL(tenant_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("tenant not exist", K(ret), K(orig_tenant_id), K(orig_schema_version)); + } else if (OB_FAIL(src_tenant_schema_guard.get_table_schema(orig_tenant_id, orig_table_id, orig_table_schema))) { + LOG_WARN("fail to get table schema", K(ret), K(orig_tenant_id), K(orig_table_id)); + } else if (OB_ISNULL(orig_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table not exist", K(ret), K(orig_tenant_id), K(orig_table_id), K(orig_schema_version)); + } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( + dest_tenant_id, dst_tenant_schema_guard, dest_schema_version))) { + LOG_WARN("fail to get tenant schema guard", K(ret), K(dest_tenant_id), K(dest_schema_version)); + } else if (OB_FAIL(dst_tenant_schema_guard.get_tenant_info(dest_tenant_id, tenant_schema))) { + LOG_WARN("fail to get tenant info", K(ret), K(dest_tenant_id)); + } else if (OB_ISNULL(tenant_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("tenant not exist", K(ret), K(dest_tenant_id), K(dest_schema_version)); + } else if (OB_FAIL(dst_tenant_schema_guard.get_table_schema(dest_tenant_id, dest_table_id, dest_table_schema))) { + LOG_WARN("fail to get table schema", K(ret), K(dest_tenant_id), K(dest_table_id)); + } else if (OB_ISNULL(dest_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table not exist", K(ret), K(dest_tenant_id), K(dest_table_id), K(dest_schema_version)); + } else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(orig_tenant_id, arg.source_table_id_, compat_mode_))) { + LOG_WARN("failed to get compat mode", K(ret), K(arg)); + } else { + snapshot_version_ = arg.snapshot_version_; + } + } + + if (OB_SUCC(ret)) { + if (orig_tenant_id == dest_tenant_id) { + if (OB_UNLIKELY(dest_table_schema->get_association_table_id() != arg.source_table_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error", K(ret), K(arg), K(dest_table_schema->get_association_table_id())); + } else if (OB_FAIL(split_task_ranges(arg.ls_id_, arg.source_tablet_id_, orig_table_schema->get_tablet_size(), arg.parallelism_))) { + LOG_WARN("fail to init concurrent params", K(ret), K(arg)); + } + } else { + // TODO yiren, support parallel for remote scan. + // 1. support split task range even if data in other nodes. + // 2. support query the data by adding rowkey range(hidden_column_visible for heap table). + // recover restore table ddl task. + ObStoreRange whole_range; + whole_range.set_whole_range(); + if (OB_FAIL(ranges_.push_back(whole_range))) { + LOG_WARN("push back failed", K(ret)); + } else { + concurrent_cnt_ = 1; + } + } + } + } + if (OB_SUCC(ret)) { is_inited_ = true; - tenant_id_ = tenant_id; - ls_id_ = arg.ls_id_; - source_table_id_ = arg.source_table_id_; - dest_table_id_ = arg.dest_schema_id_; - source_tablet_id_ = arg.source_tablet_id_; + orig_tenant_id_ = orig_tenant_id; + dest_tenant_id_ = dest_tenant_id; + orig_table_id_ = orig_table_id; + dest_table_id_ = dest_table_id; + orig_schema_version_ = orig_schema_version; + dest_schema_version_ = dest_schema_version; + orig_tablet_id_ = arg.source_tablet_id_; dest_tablet_id_ = arg.dest_tablet_id_; - schema_version_ = arg.schema_version_; + orig_ls_id_ = arg.ls_id_; + dest_ls_id_ = arg.dest_ls_id_; task_id_ = arg.task_id_; execution_id_ = arg.execution_id_; tablet_task_id_ = arg.tablet_task_id_; data_format_version_ = arg.data_format_version_; - FLOG_INFO("succeed to init ObComplementDataParam", K(ret), K(is_inited_), K(tenant_id_), K(ls_id_), - K(source_tablet_id_), K(dest_tablet_id_), K(schema_version_), K(task_id_), K(arg), K(concurrent_cnt_), - K(data_format_version_)); + FLOG_INFO("succeed to init ObComplementDataParam", K(ret), KPC(this)); } return ret; } @@ -234,9 +268,9 @@ int ObComplementDataContext::init(const ObComplementDataParam ¶m, const ObDa } else if (OB_UNLIKELY(!param.is_valid() || !desc.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(param), K(desc)); - } else if (OB_FAIL(ObTabletDDLUtil::check_and_get_major_sstable(param.ls_id_, param.dest_tablet_id_, first_major_sstable, table_store_wrapper))) { + } else if (OB_FAIL(ObTabletDDLUtil::check_and_get_major_sstable(param.dest_ls_id_, param.dest_tablet_id_, first_major_sstable, table_store_wrapper))) { LOG_WARN("check if major sstable exist failed", K(ret), K(param)); - } else if (OB_FAIL(data_sstable_redo_writer_.init(param.ls_id_, + } else if (OB_FAIL(data_sstable_redo_writer_.init(param.dest_ls_id_, param.dest_tablet_id_))) { LOG_WARN("fail to init data sstable redo writer", K(ret), K(param)); } else if (nullptr != index_builder_) { @@ -381,7 +415,6 @@ int ObComplementDataDag::create_first_task() } else if (OB_FAIL(add_task(*merge_task))) { LOG_WARN("add task failed"); } - return ret; } @@ -405,16 +438,16 @@ int ObComplementDataDag::prepare_context() ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected", K(ret), K(param_)); } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( - param_.tenant_id_, schema_guard, param_.schema_version_))) { + param_.dest_tenant_id_, schema_guard, param_.dest_schema_version_))) { LOG_WARN("fail to get tenant schema guard", K(ret), K(param_)); - } else if (OB_FAIL(schema_guard.get_table_schema(param_.tenant_id_, + } else if (OB_FAIL(schema_guard.get_table_schema(param_.dest_tenant_id_, param_.dest_table_id_, hidden_table_schema))) { LOG_WARN("fail to get hidden table schema", K(ret), K(param_)); } else if (OB_ISNULL(hidden_table_schema)) { ret = OB_TABLE_NOT_EXIST; LOG_WARN("hidden table schema not exist", K(ret), K(param_)); } else if (OB_FAIL(data_desc.init_as_index(*hidden_table_schema, - param_.ls_id_, + param_.dest_ls_id_, param_.dest_tablet_id_, MAJOR_MERGE, param_.snapshot_version_, @@ -435,9 +468,10 @@ int64_t ObComplementDataDag::hash() const tmp_ret = OB_ERR_SYS; LOG_ERROR("table schema must not be NULL", K(tmp_ret), K(is_inited_), K(param_)); } else { - hash_val = param_.tenant_id_ + param_.ls_id_.hash() - + param_.source_table_id_ + param_.dest_table_id_ - + param_.source_tablet_id_.hash() + param_.dest_tablet_id_.hash() + ObDagType::DAG_TYPE_DDL; + hash_val = param_.orig_tenant_id_ + param_.dest_tenant_id_ + + param_.orig_table_id_ + param_.dest_table_id_ + + param_.orig_ls_id_.hash() + param_.dest_ls_id_.hash() + + param_.orig_tablet_id_.hash() + param_.dest_tablet_id_.hash() + ObDagType::DAG_TYPE_DDL; } return hash_val; } @@ -454,9 +488,10 @@ bool ObComplementDataDag::operator==(const ObIDag &other) const tmp_ret = OB_ERR_SYS; LOG_ERROR("invalid argument", K(tmp_ret), K(param_), K(dag.param_)); } else { - is_equal = (param_.tenant_id_ == dag.param_.tenant_id_) && (param_.ls_id_ == dag.param_.ls_id_) && - (param_.source_table_id_ == dag.param_.source_table_id_) && (param_.dest_table_id_ == dag.param_.dest_table_id_) && - (param_.source_tablet_id_ == dag.param_.source_tablet_id_) && (param_.dest_tablet_id_ == dag.param_.dest_tablet_id_); + is_equal = (param_.orig_tenant_id_ == dag.param_.orig_tenant_id_) && (param_.dest_tenant_id_ == dag.param_.dest_tenant_id_) && + (param_.orig_table_id_ == dag.param_.orig_table_id_) && (param_.dest_table_id_ == dag.param_.dest_table_id_) && + (param_.orig_ls_id_ == dag.param_.orig_ls_id_) && (param_.dest_ls_id_ == dag.param_.dest_ls_id_) && + (param_.orig_tablet_id_ == dag.param_.orig_tablet_id_) && (param_.dest_tablet_id_ == dag.param_.dest_tablet_id_); } } return is_equal; @@ -482,18 +517,22 @@ int ObComplementDataDag::report_replica_build_status() #endif obrpc::ObDDLBuildSingleReplicaResponseArg arg; ObAddr rs_addr; - arg.tenant_id_ = param_.tenant_id_; - arg.ls_id_ = param_.ls_id_; - arg.tablet_id_ = param_.source_tablet_id_; - arg.source_table_id_ = param_.source_table_id_; + arg.tenant_id_ = param_.orig_tenant_id_; + arg.dest_tenant_id_ = param_.dest_tenant_id_; + arg.ls_id_ = param_.orig_ls_id_; + arg.dest_ls_id_ = param_.dest_ls_id_; + arg.tablet_id_ = param_.orig_tablet_id_; + arg.source_table_id_ = param_.orig_table_id_; arg.dest_schema_id_ = param_.dest_table_id_; arg.ret_code_ = context_.complement_data_ret_; arg.snapshot_version_ = param_.snapshot_version_; - arg.schema_version_ = param_.schema_version_; + arg.schema_version_ = param_.orig_schema_version_; + arg.dest_schema_version_ = param_.dest_schema_version_; arg.task_id_ = param_.task_id_; arg.execution_id_ = param_.execution_id_; arg.row_scanned_ = context_.row_scanned_; arg.row_inserted_ = context_.row_inserted_; + arg.server_addr_ = GCTX.self_addr(); FLOG_INFO("send replica build status response to RS", K(ret), K(context_), K(arg)); if (OB_FAIL(ret)) { } else if (OB_ISNULL(GCTX.rs_rpc_proxy_) || OB_ISNULL(GCTX.rs_mgr_)) { @@ -516,12 +555,13 @@ int ObComplementDataDag::fill_info_param(compaction::ObIBasicInfoParam *&out_par ret = OB_NOT_INIT; LOG_WARN("ObComplementDataDag has not been initialized", K(ret)); } else if (OB_FAIL(ADD_DAG_WARN_INFO_PARAM(out_param, allocator, get_type(), - param_.ls_id_.id(), - static_cast(param_.source_tablet_id_.id()), + param_.orig_ls_id_.id(), + static_cast(param_.orig_tablet_id_.id()), static_cast(param_.dest_tablet_id_.id()), - static_cast(param_.source_table_id_), + static_cast(param_.orig_table_id_), static_cast(param_.dest_table_id_), - param_.schema_version_, param_.snapshot_version_))) { + param_.orig_schema_version_, + param_.snapshot_version_))) { LOG_WARN("failed to fill info param", K(ret)); } return ret; @@ -537,7 +577,7 @@ int ObComplementDataDag::fill_dag_key(char *buf, const int64_t buf_len) const ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid params", K(ret), K(param_)); } else if (OB_FAIL(databuff_printf(buf, buf_len, "logstream_id=%ld source_tablet_id=%ld dest_tablet_id=%ld", - param_.ls_id_.id(), param_.source_tablet_id_.id(), param_.dest_tablet_id_.id()))) { + param_.orig_ls_id_.id(), param_.orig_tablet_id_.id(), param_.dest_tablet_id_.id()))) { LOG_WARN("fill dag key for ddl table merge dag failed", K(ret), K(param_)); } return ret; @@ -625,9 +665,9 @@ int ObComplementWriteTask::init(const int64_t task_id, ObComplementDataParam &pa ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(task_id), K(param), K(context)); } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( - param.tenant_id_, schema_guard, param.schema_version_))) { + param.dest_tenant_id_, schema_guard, param.dest_schema_version_))) { LOG_WARN("fail to get tenant schema guard", K(ret), K(param)); - } else if (OB_FAIL(schema_guard.get_table_schema(param.tenant_id_, + } else if (OB_FAIL(schema_guard.get_table_schema(param.dest_tenant_id_, param.dest_table_id_, hidden_table_schema))) { LOG_WARN("fail to get hidden table schema", K(ret), K(param)); } else if (OB_ISNULL(hidden_table_schema)) { @@ -660,14 +700,15 @@ int ObComplementWriteTask::process() } else if (OB_SUCCESS != (context_->complement_data_ret_)) { LOG_WARN("complement data has already failed", "ret", context_->complement_data_ret_); } else if (context_->is_major_sstable_exist_) { - } else if (OB_FAIL(guard.switch_to(param_->tenant_id_))) { - LOG_WARN("switch to tenant failed", K(ret), K(param_->tenant_id_)); - } else if (OB_FAIL(local_scan_by_range())) { - LOG_WARN("fail to do local scan by range", K(ret), K(task_id_)); - } else { - LOG_INFO("finish the complement write task", K(ret)); + } else if (OB_FAIL(guard.switch_to(param_->dest_tenant_id_, false))) { + LOG_WARN("switch to tenant failed", K(ret), K(param_->dest_tenant_id_)); + } else if (param_->dest_tenant_id_ == param_->orig_tenant_id_) { + if (OB_FAIL(local_scan_by_range())) { + LOG_WARN("local scan and append row for column redefinition failed", K(ret), K(task_id_)); + } + } else if (OB_FAIL(remote_scan())) { + LOG_WARN("remote scan for recover restore table ddl failed", K(ret)); } - if (OB_FAIL(ret) && OB_NOT_NULL(context_)) { context_->complement_data_ret_ = ret; ret = OB_SUCCESS; @@ -719,76 +760,82 @@ int ObComplementWriteTask::generate_col_param() org_col_ids_.reuse(); output_projector_.reuse(); ObArray tmp_col_ids; - ObSchemaGetterGuard schema_guard; + 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; const ObTableSchema *data_table_schema = nullptr; const ObTableSchema *hidden_table_schema = nullptr; - if (OB_UNLIKELY(!is_inited_)) { - ret = OB_NOT_INIT; - LOG_WARN("not init", K(ret)); - } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( - param_->tenant_id_, schema_guard, param_->schema_version_))) { - LOG_WARN("fail to get tenant schema guard", K(ret), KPC(param_)); - } else if (OB_FAIL(schema_guard.get_table_schema(param_->tenant_id_, - param_->source_table_id_, data_table_schema))) { - LOG_WARN("fail to get data table schema", K(ret), K(arg)); - } else if (OB_ISNULL(data_table_schema)) { - ret = OB_TABLE_NOT_EXIST; - LOG_WARN("data table schema not exist", K(ret), K(arg)); - } else if (OB_FAIL(schema_guard.get_table_schema(param_->tenant_id_, - param_->dest_table_id_, hidden_table_schema))) { - LOG_WARN("fail to get hidden table schema", K(ret), KPC(param_)); - } else if (OB_ISNULL(hidden_table_schema)) { - ret = OB_TABLE_NOT_EXIST; - LOG_WARN("hidden table schema not exist", K(ret), KPC(param_)); - } else if (OB_FAIL(hidden_table_schema->get_store_column_ids(tmp_col_ids, false))) { - LOG_WARN("fail to get column ids", K(ret), KPC(hidden_table_schema)); - } else if (OB_FAIL(org_col_ids_.assign(tmp_col_ids))) { - LOG_WARN("fail to assign col descs", K(ret), K(tmp_col_ids)); - } else { - // generate col_ids - for (int64_t i = 0; OB_SUCC(ret) && i < tmp_col_ids.count(); i++) { - const uint64_t hidden_column_id = tmp_col_ids.at(i).col_id_; - const ObColumnSchemaV2 *hidden_column_schema = hidden_table_schema->get_column_schema(hidden_column_id); - if (OB_ISNULL(hidden_column_schema)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected null column schema", K(ret), K(hidden_column_id)); - } else { - const ObString &hidden_column_name = hidden_column_schema->get_column_name_str(); - const ObColumnSchemaV2 *data_column_schema = data_table_schema->get_column_schema(hidden_column_name); - ObColDesc tmp_col_desc = tmp_col_ids.at(i); - if (nullptr == data_column_schema) { - // may be newly added column, can not find in data table. - } else if (FALSE_IT(tmp_col_desc.col_id_ = data_column_schema->get_column_id())) { - } else if (OB_FAIL(col_ids_.push_back(tmp_col_desc))) { - LOG_WARN("fail to push back col desc", K(ret), K(tmp_col_ids.at(i)), K(tmp_col_desc)); - } - } - } - } - // generate output_projector. - if (OB_FAIL(ret)) { - } else { - // notice that, can not find newly added column, get the row firstly, and then resolve it. - for (int64_t i = 0; OB_SUCC(ret) && i < tmp_col_ids.count(); i++) { - const ObColumnSchemaV2 *hidden_column_schema = hidden_table_schema->get_column_schema(tmp_col_ids.at(i).col_id_); - const ObString &hidden_column_name = hidden_column_schema->get_column_name_str(); - for (int64_t j = 0; OB_SUCC(ret) && j < col_ids_.count(); j++) { - const ObColumnSchemaV2 *data_column_schema = data_table_schema->get_column_schema(col_ids_.at(j).col_id_); - if (nullptr == data_column_schema) { - // may be newly added column. - } else if (hidden_column_name == data_column_schema->get_column_name_str()) { - if (OB_FAIL(output_projector_.push_back(static_cast(j)))) { - LOG_WARN("fail to push back output projector", K(ret)); + MTL_SWITCH (OB_SYS_TENANT_ID) { + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(param_->orig_tenant_id_, param_->dest_tenant_id_, + hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard, + src_tenant_schema_guard, dst_tenant_schema_guard))) { + LOG_WARN("get tenant schema guard failed", K(ret), KPC(param_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(param_->orig_tenant_id_, + param_->orig_table_id_, data_table_schema))) { + LOG_WARN("fail to get data table schema", K(ret), K(arg)); + } else if (OB_ISNULL(data_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("data table schema not exist", K(ret), K(arg)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(param_->dest_tenant_id_, + param_->dest_table_id_, hidden_table_schema))) { + LOG_WARN("fail to get hidden table schema", K(ret), KPC(param_)); + } else if (OB_ISNULL(hidden_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("hidden table schema not exist", K(ret), KPC(param_)); + } else if (OB_FAIL(hidden_table_schema->get_store_column_ids(tmp_col_ids, false))) { + LOG_WARN("fail to get column ids", K(ret), KPC(hidden_table_schema)); + } else if (OB_FAIL(org_col_ids_.assign(tmp_col_ids))) { + LOG_WARN("fail to assign col descs", K(ret), K(tmp_col_ids)); + } else { + // generate col_ids + for (int64_t i = 0; OB_SUCC(ret) && i < tmp_col_ids.count(); i++) { + const uint64_t hidden_column_id = tmp_col_ids.at(i).col_id_; + const ObColumnSchemaV2 *hidden_column_schema = hidden_table_schema->get_column_schema(hidden_column_id); + if (OB_ISNULL(hidden_column_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null column schema", K(ret), K(hidden_column_id)); + } else { + const ObString &hidden_column_name = hidden_column_schema->get_column_name_str(); + const ObColumnSchemaV2 *data_column_schema = data_table_schema->get_column_schema(hidden_column_name); + ObColDesc tmp_col_desc = tmp_col_ids.at(i); + if (nullptr == data_column_schema) { + // may be newly added column, can not find in data table. + } else if (FALSE_IT(tmp_col_desc.col_id_ = data_column_schema->get_column_id())) { + } else if (OB_FAIL(col_ids_.push_back(tmp_col_desc))) { + LOG_WARN("fail to push back col desc", K(ret), K(tmp_col_ids.at(i)), K(tmp_col_desc)); } - break; } } } - } - if (OB_FAIL(ret)) { - } else if (OB_UNLIKELY(col_ids_.count() != output_projector_.count())) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected error", K(ret), K_(col_ids), K_(output_projector)); + // generate output_projector. + if (OB_FAIL(ret)) { + } else { + // notice that, can not find newly added column, get the row firstly, and then resolve it. + for (int64_t i = 0; OB_SUCC(ret) && i < tmp_col_ids.count(); i++) { + const ObColumnSchemaV2 *hidden_column_schema = hidden_table_schema->get_column_schema(tmp_col_ids.at(i).col_id_); + const ObString &hidden_column_name = hidden_column_schema->get_column_name_str(); + for (int64_t j = 0; OB_SUCC(ret) && j < col_ids_.count(); j++) { + const ObColumnSchemaV2 *data_column_schema = data_table_schema->get_column_schema(col_ids_.at(j).col_id_); + if (nullptr == data_column_schema) { + // may be newly added column. + } else if (hidden_column_name == data_column_schema->get_column_name_str()) { + if (OB_FAIL(output_projector_.push_back(static_cast(j)))) { + LOG_WARN("fail to push back output projector", K(ret)); + } + break; + } + } + } + } + if (OB_FAIL(ret)) { + } else if (OB_UNLIKELY(col_ids_.count() != output_projector_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error", K(ret), K_(col_ids), K_(output_projector)); + } } return ret; } @@ -841,15 +888,16 @@ int ObComplementWriteTask::do_local_scan() ObSSTable *sstable = nullptr; ObTransService *trans_service = nullptr; ObSEArray sstables; - const uint64_t tenant_id = param_->tenant_id_; + const uint64_t tenant_id = param_->dest_tenant_id_; + const int64_t schema_version = param_->dest_schema_version_; ObTxDesc *read_tx_desc = nullptr; // for reading lob column from aux_lob_table by table_scan - if (OB_FAIL(MTL(ObLSService *)->get_ls(param_->ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { + if (OB_FAIL(MTL(ObLSService *)->get_ls(param_->orig_ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { LOG_WARN("fail to get log stream", K(ret), KPC(param_)); } else if (OB_UNLIKELY(nullptr == ls_handle.get_ls())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ls is null", K(ret), K(ls_handle)); - } else if (OB_FAIL(ls_handle.get_ls()->get_tablet_svr()->get_read_tables(param_->source_tablet_id_, + } else if (OB_FAIL(ls_handle.get_ls()->get_tablet_svr()->get_read_tables(param_->orig_tablet_id_, param_->snapshot_version_, iterator, allow_not_ready))) { if (OB_REPLICA_NOT_READABLE == ret) { ret = OB_EAGAIN; @@ -867,15 +915,19 @@ int ObComplementWriteTask::do_local_scan() ObSchemaGetterGuard schema_guard; const ObTableSchema *data_table_schema = nullptr; const ObTableSchema *hidden_table_schema = nullptr; - if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( - tenant_id, schema_guard, param_->schema_version_))) { + if (OB_UNLIKELY(param_->orig_tenant_id_ != param_->dest_tenant_id_ + || param_->orig_schema_version_ != param_->dest_schema_version_)) { + ret = OB_ERR_SYS; + LOG_WARN("err sys", K(ret), KPC(param_)); + } else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( + tenant_id, schema_guard, schema_version))) { LOG_WARN("fail to get tenant schema guard", K(ret), KPC(param_)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, - param_->source_table_id_, data_table_schema))) { - LOG_WARN("fail to get data table schema", K(ret), K(arg)); + param_->orig_table_id_, data_table_schema))) { + LOG_WARN("fail to get data table schema", K(ret), KPC(param_)); } else if (OB_ISNULL(data_table_schema)) { ret = OB_TABLE_NOT_EXIST; - LOG_WARN("data table schema not exist", K(ret), K(arg)); + LOG_WARN("data table schema not exist", K(ret), KPC(param_)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, param_->dest_table_id_, hidden_table_schema))) { LOG_WARN("fail to get hidden table schema", K(ret), KPC(param_)); @@ -892,19 +944,18 @@ int ObComplementWriteTask::do_local_scan() false/*output all columns of hidden table*/))) { LOG_WARN("fail to init local scan param", K(ret), K(*param_)); } else if (OB_FAIL(ObInsertLobColumnHelper::start_trans( - param_->ls_id_, true/*is_for_read*/, INT64_MAX, read_tx_desc))) { + param_->orig_ls_id_, true/*is_for_read*/, INT64_MAX, read_tx_desc))) { LOG_WARN("fail to get tx_desc", K(ret)); } else if (OB_FAIL(local_scan.table_scan(*data_table_schema, - param_->ls_id_, - param_->source_tablet_id_, + param_->orig_ls_id_, + param_->orig_tablet_id_, iterator, query_flag, datum_range, read_tx_desc))) { LOG_WARN("fail to do table scan", K(ret)); } } - - if (FAILEDx(append_row(local_scan))) { + if (FAILEDx(append_row(&local_scan))) { LOG_WARN("append row failed", K(ret)); } @@ -920,7 +971,47 @@ int ObComplementWriteTask::do_local_scan() return ret; } -int ObComplementWriteTask::add_extra_rowkey(const int64_t rowkey_cnt, const int64_t extra_rowkey_cnt, const blocksstable::ObDatumRow &row) +int ObComplementWriteTask::remote_scan() +{ + int ret = OB_SUCCESS; + const int64_t start_time = ObTimeUtility::current_time(); + if (OB_ISNULL(param_) || OB_UNLIKELY(!param_->is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments", K(ret), K(idx), KP(param_)); + } else if (OB_FAIL(generate_col_param())) { + LOG_WARN("fail to get column ids", K(ret)); + } else if (OB_FAIL(do_remote_scan())) { + LOG_WARN("fail to do remote scan", K_(task_id), KPC(param_)); + } else { + LOG_INFO("finish remote scan", K(ret), "cost_time", ObTimeUtility::current_time() - start_time , K_(task_id)); + } + return ret; +} + +int ObComplementWriteTask::do_remote_scan() +{ + int ret = OB_SUCCESS; + SMART_VAR(ObRemoteScan, remote_scan) { + remote_scan.reset(); + if (OB_FAIL(remote_scan.init(param_->orig_tenant_id_, + param_->orig_table_id_, + param_->dest_tenant_id_, + param_->dest_table_id_, + param_->orig_schema_version_, + param_->dest_schema_version_, + param_->orig_tablet_id_))) { + LOG_WARN("fail to remote_scan init", K(ret), KPC(param_)); + } else if (OB_FAIL(append_row(&remote_scan))) { + LOG_WARN("append row remote scan failed", K(ret)); + } + } + return ret; +} + +int ObComplementWriteTask::add_extra_rowkey(const int64_t rowkey_cnt, + const int64_t extra_rowkey_cnt, + const blocksstable::ObDatumRow &row, + const int64_t sql_no) { int ret = OB_SUCCESS; int64_t rowkey_column_count = rowkey_cnt; @@ -943,7 +1034,7 @@ int ObComplementWriteTask::add_extra_rowkey(const int64_t rowkey_cnt, const int6 return ret; } -int ObComplementWriteTask::append_row(ObLocalScan &local_scan) +int ObComplementWriteTask::append_row(ObScan *scan) { int ret = OB_SUCCESS; ObDataStoreDesc data_desc; @@ -975,24 +1066,24 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("ObComplementWriteTask is not inited", K(ret)); - } else if (OB_ISNULL(param_) || OB_UNLIKELY(!param_->is_valid()) || OB_ISNULL(context_)) { + } else if (OB_ISNULL(param_) || OB_ISNULL(scan) || OB_UNLIKELY(!param_->is_valid()) || OB_ISNULL(context_)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid arguments", K(param_), KP(context_), K(ret)); + LOG_WARN("invalid arguments", K(ret), KPC(param_), KPC(context_)); } else if (OB_FAIL(macro_start_seq.set_parallel_degree(task_id_))) { LOG_WARN("set parallel degree failed", K(ret), K(task_id_)); } else { const ObTableSchema *hidden_table_schema = nullptr; if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( - param_->tenant_id_, schema_guard, param_->schema_version_))) { + param_->dest_tenant_id_, schema_guard, param_->dest_schema_version_))) { LOG_WARN("fail to get tenant schema guard", K(ret), KPC(param_)); - } else if (OB_FAIL(schema_guard.get_table_schema(param_->tenant_id_, + } else if (OB_FAIL(schema_guard.get_table_schema(param_->dest_tenant_id_, param_->dest_table_id_, hidden_table_schema))) { LOG_WARN("fail to get hidden table schema", K(ret), KPC(param_)); } else if (OB_ISNULL(hidden_table_schema)) { ret = OB_TABLE_NOT_EXIST; - LOG_WARN("table not exist", K(ret), K(param_->tenant_id_), K(param_->dest_table_id_)); + LOG_WARN("table not exist", K(ret), K(param_->dest_tenant_id_), K(param_->dest_table_id_)); } else if (OB_FAIL(data_desc.init(*hidden_table_schema, - param_->ls_id_, + param_->dest_ls_id_, param_->dest_tablet_id_, MAJOR_MERGE, param_->snapshot_version_, @@ -1005,7 +1096,7 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) } else if (OB_UNLIKELY(!hidden_table_key.is_valid())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("hidden table key is invalid", K(ret), K(hidden_table_key)); - } else if (OB_FAIL(sstable_redo_writer.init(param_->ls_id_, param_->dest_tablet_id_))) { + } else if (OB_FAIL(sstable_redo_writer.init(param_->dest_ls_id_, param_->dest_tablet_id_))) { LOG_WARN("fail to init sstable redo writer", K(ret)); } else if (OB_UNLIKELY(nullptr == static_cast(get_dag()))) { ret = OB_ERR_UNEXPECTED; @@ -1019,7 +1110,6 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) } else { rowkey_column_cnt = hidden_table_schema->get_rowkey_column_num(); } - ObTableSchemaParam schema_param(allocator); // Hack to prevent row reshaping from converting empty string to null. // @@ -1044,17 +1134,20 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) LOG_WARN("failed to init datum row", K(ret), K(data_desc.get_full_stored_col_descs())); } } - while (OB_SUCC(ret)) { //get each row from row_iter const ObDatumRow *tmp_row = nullptr; + const ObDatumRow *reshape_row_only_for_remote_scan = nullptr; ObStoreRow tmp_store_row; ObColumnChecksumCalculator *checksum_calculator = nullptr; t1 = ObTimeUtility::current_time(); dag_yield(); - if (OB_FAIL(local_scan.get_next_row(tmp_row))) { + if (OB_FAIL(scan->get_next_row(tmp_row, reshape_row_only_for_remote_scan))) { if (OB_UNLIKELY(OB_ITER_END != ret)) { LOG_WARN("fail to get next row", K(ret)); } + } else if (OB_ISNULL(tmp_row)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tmp_row is nullptr", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < org_col_ids_.count(); i++) { ObStorageDatum &datum = tmp_row->storage_datums_[i]; @@ -1064,7 +1157,8 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) lob_cnt++; const int64_t timeout_ts = ObTimeUtility::current_time() + 60000000; // 60s if (OB_FAIL(ObInsertLobColumnHelper::insert_lob_column( - lob_allocator, param_->ls_id_, param_->dest_tablet_id_, org_col_ids_.at(i), datum, timeout_ts, true))) { + lob_allocator, param_->dest_ls_id_, param_->dest_tablet_id_, + org_col_ids_.at(i), datum, timeout_ts, true, param_->orig_tenant_id_))) { LOG_WARN("fail to insert_lob_col", K(ret), K(datum)); } } @@ -1091,12 +1185,23 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) } } if (OB_FAIL(ret)) { - } else if (OB_ISNULL(checksum_calculator = local_scan.get_checksum_calculator())) { + } else if (OB_ISNULL(checksum_calculator = scan->get_checksum_calculator())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("checksum calculator is nullptr", K(ret), KP(checksum_calculator)); - } else if (OB_FAIL(checksum_calculator->calc_column_checksum(data_desc.get_full_stored_col_descs(), &write_row_, nullptr, nullptr))) { - LOG_WARN("fail to calc column checksum", K(ret), K(write_row_)); - } else { + } else if (param_->orig_tenant_id_ == param_->dest_tenant_id_) { + if (OB_FAIL(checksum_calculator->calc_column_checksum(data_desc.get_full_stored_col_descs(), &write_row_, nullptr, nullptr))) { + LOG_WARN("fail to calc column checksum", K(ret), K(write_row_)); + } + } else if (param_->orig_tenant_id_ != param_->dest_tenant_id_) { + if (OB_ISNULL(reshape_row_only_for_remote_scan)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected err", K(ret), K(write_row_)); + } else if (OB_FAIL(checksum_calculator->calc_column_checksum(org_col_ids_/*without extra rowkey*/, + reshape_row_only_for_remote_scan, nullptr, nullptr))) { + LOG_WARN("fail to calc column checksum", K(ret), K(write_row_), KPC(reshape_row_only_for_remote_scan)); + } + } + if (OB_SUCC(ret)) { t3 = ObTimeUtility::current_time(); append_row_time += t3 - t2; context_->row_inserted_++; @@ -1122,10 +1227,16 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) } } if (OB_FAIL(ret)) { - } else if (OB_FAIL(local_scan.get_origin_table_checksum(report_col_checksums, report_col_ids))) { + } else if (OB_FAIL(scan->get_origin_table_checksum(report_col_checksums, report_col_ids))) { LOG_WARN("fail to get origin table columns checksum", K(ret)); - } else if (OB_FAIL(ObDDLChecksumOperator::update_checksum(param_->tenant_id_, - param_->source_table_id_, + } + /** + * For DDL_RESTORE_TABLE, orig tenant id is differen to dest tenant id. + * Meanwhile, the original tenant is a backup tenant, can not support write operation, + * report its' checksum under the dest tenant, and origin_table_id + ddl_task_id will aviod the conflict. + */ + else if (OB_FAIL(ObDDLChecksumOperator::update_checksum(param_->dest_tenant_id_, + param_->orig_table_id_, param_->task_id_, report_col_checksums, report_col_ids, @@ -1133,7 +1244,9 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan) param_->tablet_task_id_ << 48 | task_id_, *GCTX.sql_proxy_))) { LOG_WARN("fail to report origin table checksum", K(ret)); - } else {/* do nothing. */} + } else { + LOG_INFO("update checksum successfully", K(param_->orig_table_id_), K(report_col_checksums), K(param_->orig_tablet_id_)); + } } return ret; } @@ -1180,13 +1293,13 @@ int ObComplementMergeTask::process() } else if (FALSE_IT(dag = static_cast(tmp_dag))) { } else if (OB_SUCCESS != (context_->complement_data_ret_)) { LOG_WARN("complement data has already failed", "ret", context_->complement_data_ret_); - } else if (OB_FAIL(guard.switch_to(param_->tenant_id_))) { - LOG_WARN("switch to tenant failed", K(ret), K(param_->tenant_id_)); + } else if (OB_FAIL(guard.switch_to(param_->dest_tenant_id_, false))) { + LOG_WARN("switch to tenant failed", K(ret), K(param_->dest_tenant_id_)); } else if (context_->is_major_sstable_exist_) { ObTabletMemberWrapper table_store_wrapper; const ObSSTable *first_major_sstable = nullptr; ObSSTableMetaHandle sst_meta_hdl; - if (OB_FAIL(MTL(ObLSService *)->get_ls(param_->ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { + if (OB_FAIL(MTL(ObLSService *)->get_ls(param_->dest_ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { LOG_WARN("failed to get log stream", K(ret), K(*param_)); } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, param_->dest_tablet_id_, @@ -1207,7 +1320,7 @@ int ObComplementMergeTask::process() LOG_WARN("unexpected error, major sstable shoud not be null", K(ret), K(*param_)); } else if (OB_FAIL(first_major_sstable->get_meta(sst_meta_hdl))) { LOG_WARN("fail to get sstable meta handle", K(ret)); - } else if (OB_FAIL(ObTabletDDLUtil::report_ddl_checksum(param_->ls_id_, + } else if (OB_FAIL(ObTabletDDLUtil::report_ddl_checksum(param_->dest_ls_id_, param_->dest_tablet_id_, param_->dest_table_id_, 1 /* execution_id */, @@ -1215,7 +1328,7 @@ int ObComplementMergeTask::process() sst_meta_hdl.get_sstable_meta().get_col_checksum(), sst_meta_hdl.get_sstable_meta().get_col_checksum_cnt()))) { LOG_WARN("report ddl column checksum failed", K(ret), K(*param_)); - } else if (OB_FAIL(GCTX.ob_service_->submit_tablet_update_task(param_->tenant_id_, param_->ls_id_, param_->dest_tablet_id_))) { + } else if (OB_FAIL(GCTX.ob_service_->submit_tablet_update_task(param_->dest_tenant_id_, param_->dest_ls_id_, param_->dest_tablet_id_))) { LOG_WARN("fail to submit tablet update task", K(ret), K(*param_)); } } @@ -1238,6 +1351,7 @@ int ObComplementMergeTask::process() int ObComplementMergeTask::add_build_hidden_table_sstable() { int ret = OB_SUCCESS; + ObLSHandle ls_handle; ObITable::TableKey hidden_table_key; SCN commit_scn; if (OB_UNLIKELY(!is_inited_)) { @@ -1248,10 +1362,12 @@ int ObComplementMergeTask::add_build_hidden_table_sstable() || OB_UNLIKELY(!param_->is_valid())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("error unexpected", K(ret), KP(param_), KP(context_)); + } else if (OB_FAIL(MTL(ObLSService *)->get_ls(param_->dest_ls_id_, ls_handle, ObLSGetMod::DDL_MOD))) { + LOG_WARN("failed to get log stream", K(ret), K(param_->dest_ls_id_)); } else if (OB_FAIL(param_->get_hidden_table_key(hidden_table_key))) { LOG_WARN("fail to get hidden table key", K(ret), K(hidden_table_key)); } else if (OB_FAIL(context_->data_sstable_redo_writer_.end_ddl_redo_and_create_ddl_sstable( - param_->ls_id_, hidden_table_key, param_->dest_table_id_, param_->execution_id_, param_->task_id_))) { + param_->dest_ls_id_, hidden_table_key, param_->dest_table_id_, param_->execution_id_, param_->task_id_))) { LOG_WARN("failed to end ddl redo", K(ret)); } return ret; @@ -1261,7 +1377,7 @@ int ObComplementMergeTask::add_build_hidden_table_sstable() * -----------------------------------ObLocalScan----------------------------------------- */ -ObLocalScan::ObLocalScan() : is_inited_(false), tenant_id_(OB_INVALID_TENANT_ID), source_table_id_(OB_INVALID_ID), +ObLocalScan::ObLocalScan() : is_inited_(false), tenant_id_(OB_INVALID_TENANT_ID), table_id_(OB_INVALID_ID), dest_table_id_(OB_INVALID_ID), schema_version_(0), extended_gc_(), snapshot_version_(common::OB_INVALID_VERSION), txs_(nullptr), default_row_(), tmp_row_(), row_iter_(nullptr), scan_merge_(nullptr), ctx_(), access_param_(), access_ctx_(), get_table_param_(), allocator_("ObLocalScan"), calc_buf_(ObModIds::OB_SQL_EXPR_CALC), @@ -1339,7 +1455,7 @@ int ObLocalScan::init( LOG_WARN("failed to deep copy default row", K(ret)); } else { tenant_id_ = data_table_schema.get_tenant_id(); - source_table_id_ = data_table_schema.get_table_id(); + table_id_ = data_table_schema.get_table_id(); dest_table_id_ = hidden_table_schema.get_table_id(); schema_version_ = hidden_table_schema.get_schema_version(); is_inited_ = true; @@ -1354,6 +1470,7 @@ int ObLocalScan::get_output_columns( ObIArray &col_ids) { int ret = OB_SUCCESS; + col_ids.reset(); if (output_org_cols_only_) { if (OB_FAIL(col_ids.assign(extended_gc_.extended_col_ids_))) { LOG_WARN("assign tmp col ids failed", K(ret)); @@ -1630,11 +1747,11 @@ int ObLocalScan::get_origin_table_checksum( tenant_id_, schema_guard, schema_version_))) { LOG_WARN("fail to get tenant schema guard", K(ret), K(tenant_id_), K(schema_version_)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, - source_table_id_, data_table_schema))) { - LOG_WARN("get data table schema failed", K(ret), K(tenant_id_), K(source_table_id_)); + table_id_, data_table_schema))) { + LOG_WARN("get data table schema failed", K(ret), K(tenant_id_), K(table_id_)); } else if (OB_ISNULL(data_table_schema)) { ret = OB_TABLE_NOT_EXIST; - LOG_WARN("data table not exist", K(ret), K(tenant_id_), K(source_table_id_)); + LOG_WARN("data table not exist", K(ret), K(tenant_id_), K(table_id_)); } else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, dest_table_id_, hidden_table_schema))) { LOG_WARN("fail to get hidden table schema", K(ret), K(tenant_id_), K(dest_table_id_)); @@ -1670,8 +1787,11 @@ int ObLocalScan::get_origin_table_checksum( return ret; } -int ObLocalScan::get_next_row(const ObDatumRow *&tmp_row) +int ObLocalScan::get_next_row( + const ObDatumRow *&tmp_row, + const ObDatumRow *&tmp_row_after_reshape) { + UNUSED(tmp_row_after_reshape); int ret = OB_SUCCESS; tmp_row = nullptr; calc_buf_.reuse(); @@ -1691,7 +1811,7 @@ int ObLocalScan::get_next_row(const ObDatumRow *&tmp_row) ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected error", K(ret), K(j), K(extended_gc_.extended_col_ids_.count())); } else { - tmp_row_.storage_datums_[i] = row->storage_datums_[j++]; + tmp_row_.storage_datums_[i] = row->storage_datums_[j++]; } } else { // the column is newly added, thus fill with default value. @@ -1705,5 +1825,407 @@ int ObLocalScan::get_next_row(const ObDatumRow *&tmp_row) return ret; } +ObRemoteScan::ObRemoteScan() + : is_inited_(false), + current_(0), + tenant_id_(OB_INVALID_ID), + table_id_(OB_INVALID_ID), + dest_tenant_id_(OB_INVALID_ID), + dest_table_id_(OB_INVALID_ID), + schema_version_(0), + dest_schema_version_(0), + row_without_reshape_(), + row_with_reshape_(), + res_(), + result_(nullptr), + allocator_("DDLRemoteScan"), + org_col_ids_(), + column_names_(), + checksum_calculator_() +{ +} + +ObRemoteScan::~ObRemoteScan() +{ + reset(); +} + +void ObRemoteScan::reset() +{ + is_inited_ = false; + current_ = 0; + tenant_id_ = OB_INVALID_ID; + table_id_ = OB_INVALID_ID; + dest_tenant_id_ = OB_INVALID_ID; + dest_table_id_ = OB_INVALID_ID; + schema_version_ = 0; + dest_schema_version_ = 0; + row_without_reshape_.reset(); + row_with_reshape_.reset(); + res_.reset(); + result_ = nullptr; + org_col_ids_.reset(); + column_names_.reset(); + allocator_.reset(); +} + +int ObRemoteScan::init(const uint64_t tenant_id, + const int64_t table_id, + const uint64_t dest_tenant_id, + const int64_t dest_table_id, + const int64_t schema_version, + const int64_t dest_schema_version, + const ObTabletID &src_tablet_id) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", K(ret)); + } else if (OB_INVALID_ID == tenant_id || OB_INVALID_ID == table_id + || OB_INVALID_ID == dest_tenant_id || OB_INVALID_ID == dest_table_id + || schema_version <= 0 || dest_schema_version <= 0 || !src_tablet_id.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(tenant_id), K(table_id), + K(dest_tenant_id), K(dest_table_id), K(schema_version), K(dest_schema_version), K(src_tablet_id)); + } else { + ObSqlString sql_string; + ObSchemaGetterGuard schema_guard; + const ObTableSchema *hidden_table_schema = nullptr; + bool is_oracle_mode = false; + if (OB_FAIL((ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard( + dest_tenant_id, schema_guard, dest_schema_version)))) { + LOG_WARN("fail to get tenant schema guard", K(ret), K(dest_tenant_id), K(dest_schema_version)); + } else if (OB_FAIL(schema_guard.get_table_schema(dest_tenant_id, dest_table_id, hidden_table_schema))) { + LOG_WARN("fail to get table schema", K(ret), K(tenant_id), K(table_id)); + } else if (OB_ISNULL(hidden_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table not exist", K(ret), K(dest_tenant_id), K(dest_table_id)); + } else if (OB_FAIL(hidden_table_schema->get_store_column_ids(org_col_ids_))) { + LOG_WARN("fail to get store column ids", K(ret)); + } else if (OB_FAIL(hidden_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { + LOG_WARN("Failed to check oralce mode", K(ret)); + } else if (OB_UNLIKELY(org_col_ids_.count() <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("org col ids count is 0", K(ret)); + } else if (OB_FAIL(row_without_reshape_.init(allocator_, org_col_ids_.count()))) { + LOG_WARN("fail to init tmp_row", K(ret), K(org_col_ids_.count())); + } else if (OB_FAIL(row_with_reshape_.init(allocator_, org_col_ids_.count()))) { + LOG_WARN("fail to init tmp_row", K(ret), K(org_col_ids_.count())); + } else if (OB_FAIL(checksum_calculator_.init(org_col_ids_.count()))) { + LOG_WARN("fail to init checksum_calculator", K(ret)); + } else { + tenant_id_ = tenant_id; + table_id_ = table_id; + dest_tenant_id_ = dest_tenant_id; + dest_table_id_ = dest_table_id; + schema_version_ = schema_version; + dest_schema_version_ = dest_schema_version; + src_tablet_id_ = src_tablet_id; + row_without_reshape_.row_flag_.set_flag(ObDmlFlag::DF_INSERT); + row_with_reshape_.row_flag_.set_flag(ObDmlFlag::DF_INSERT); + if (OB_FAIL(generate_build_select_sql(sql_string))) { + LOG_WARN("fail to generate build replica sql", K(ret), K(sql_string)); + } else if (is_oracle_mode && OB_FAIL(prepare_iter(sql_string, GCTX.ddl_oracle_sql_proxy_))) { + LOG_WARN("prepare iter under oracle mode failed", K(ret), K(sql_string)); + } else if (!is_oracle_mode && OB_FAIL(prepare_iter(sql_string, GCTX.ddl_sql_proxy_))) { + LOG_WARN("prepare iter under mysql mode failed", K(ret), K(sql_string)); + } else { + is_inited_ = true; + } + } + } + return ret; +} + +int ObRemoteScan::generate_build_select_sql(ObSqlString &sql_string) +{ + int ret = OB_SUCCESS; + sql_string.reset(); + 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; + bool is_oracle_mode = false; + ObArray dest_column_ids; + const ObDatabaseSchema *orig_db_schema = nullptr; + const share::schema::ObTableSchema *orig_table_schema = nullptr; + const share::schema::ObTableSchema *dest_table_schema = nullptr; + MTL_SWITCH (OB_SYS_TENANT_ID) { + if (OB_UNLIKELY(is_inited_)) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", K(ret)); + } else if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(tenant_id_, dest_tenant_id_, + hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard, + src_tenant_schema_guard, dst_tenant_schema_guard))) { + LOG_WARN("get tenant schema guard failed", K(ret), K(tenant_id_), K(dest_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(tenant_id_, table_id_, orig_table_schema))) { + LOG_WARN("fail to get table schema", K(ret), K(tenant_id_), K(table_id_)); + } else if (OB_ISNULL(orig_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table not exist", K(ret), K(tenant_id_), K(table_id_)); + } else if (OB_FAIL(orig_table_schema->check_if_oracle_compat_mode(is_oracle_mode))) { + LOG_WARN("Failed to check oralce mode", K(ret)); + } else if (OB_FAIL(src_tenant_schema_guard->get_database_schema(tenant_id_, orig_table_schema->get_database_id(), orig_db_schema))) { + LOG_WARN("fail to get database schema", K(ret), K(tenant_id_)); + } else if (OB_ISNULL(orig_db_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("error unexpected, database schema must not be nullptr", K(ret), KPC(orig_table_schema)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(dest_tenant_id_, dest_table_id_, dest_table_schema))) { + LOG_WARN("fail to get table schema", K(ret), K(dest_tenant_id_), K(dest_table_id_)); + } else if (OB_ISNULL(dest_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table not exist", K(ret), K(dest_tenant_id_), K(dest_table_id_)); + } else if (OB_FAIL(dest_table_schema->get_store_column_ids(dest_column_ids, false))) { + LOG_WARN("get store column ids failed", K(ret), KPC(dest_table_schema)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < dest_column_ids.count(); i++) { + const uint64_t dest_column_id = dest_column_ids.at(i).col_id_; + const ObColumnSchemaV2 *dest_column_schema = dest_table_schema->get_column_schema(dest_column_id); + if (OB_ISNULL(dest_column_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null column schema", K(ret), K(dest_column_id)); + } else { + const ObString &dest_column_name = dest_column_schema->get_column_name_str(); + const ObColumnSchemaV2 *orig_column_schema = orig_table_schema->get_column_schema(dest_column_name); + if (OB_ISNULL(orig_column_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("column not exist", K(ret), K(dest_column_name), KPC(dest_table_schema)); + } else if (OB_FAIL(column_names_.push_back(ObColumnNameInfo(dest_column_name, dest_column_id >= OB_MIN_SHADOW_COLUMN_ID, + orig_column_schema->is_enum_or_set())))) { + LOG_WARN("fail to push back column name failed", K(ret)); + } + } + } + + if (OB_SUCC(ret)) { + ObSqlString query_column_sql_string; + if (OB_FAIL(ObDDLUtil::generate_column_name_str(column_names_, is_oracle_mode, true, true, false/*use_heap_table_ddl_plan*/, query_column_sql_string))) { + LOG_WARN("fail to generate column name str", K(ret)); + } else { + ObString orig_database_name_with_escape; + ObString orig_table_name_with_escape; + ObSqlString query_partition_sql; + const bool is_part_table = orig_table_schema->is_partitioned_table(); + const char *split_char = is_oracle_mode ? "\"" : "`"; + if (OB_FAIL(sql::ObSQLUtils::generate_new_name_with_escape_character( + allocator_, orig_db_schema->get_database_name_str(), orig_database_name_with_escape, is_oracle_mode))) { + LOG_WARN("generate new name failed", K(ret)); + } else if (OB_FAIL(sql::ObSQLUtils::generate_new_name_with_escape_character( + allocator_, orig_table_schema->get_table_name_str(), orig_table_name_with_escape, is_oracle_mode))) { + LOG_WARN("generate new name failed", K(ret)); + } else if (is_part_table) { + ObString partition_name_with_escape; + const ObBasePartition *source_partition = nullptr; + if (OB_FAIL(fetch_source_part_info(src_tablet_id_, *orig_table_schema, source_partition))) { + LOG_WARN("fetch source part info failed", K(ret)); + } else if (OB_FAIL(sql::ObSQLUtils::generate_new_name_with_escape_character( + allocator_, source_partition->get_part_name(), partition_name_with_escape, is_oracle_mode))) { + LOG_WARN("generate new name failed", K(ret), KPC(source_partition)); + } else if (OB_FAIL(query_partition_sql.assign_fmt("%s%.*s%s.%s%.*s%s partition (%s%.*s%s)", + split_char, static_cast(orig_database_name_with_escape.length()), orig_database_name_with_escape.ptr(), split_char, + split_char, static_cast(orig_table_name_with_escape.length()), orig_table_name_with_escape.ptr(), split_char, + split_char, static_cast(partition_name_with_escape.length()), partition_name_with_escape.ptr(), split_char))) { + LOG_WARN("add specified query partition failed", K(ret), + K(orig_database_name_with_escape), K(orig_table_name_with_escape), K(partition_name_with_escape)); + } + } else if (OB_FAIL(query_partition_sql.assign_fmt("%s%.*s%s.%s%.*s%s", + split_char, static_cast(orig_database_name_with_escape.length()), orig_database_name_with_escape.ptr(), split_char, + split_char, static_cast(orig_table_name_with_escape.length()), orig_table_name_with_escape.ptr(), split_char))) { + LOG_WARN("add specified query partition failed", K(ret), + K(orig_database_name_with_escape), K(orig_table_name_with_escape)); + } + + if (FAILEDx(sql_string.assign_fmt("SELECT /*+opt_param('hidden_column_visible' 'true')*/ %.*s from %.*s", + static_cast(query_column_sql_string.length()), query_column_sql_string.ptr(), + static_cast(query_partition_sql.length()), query_partition_sql.ptr()))) { + LOG_WARN("fail to assign sql string", K(ret), K(query_column_sql_string), K(query_partition_sql)); + } + } + } + } + } + LOG_TRACE("generate query sql finished", K(ret), K(sql_string)); + return ret; +} + +int ObRemoteScan::fetch_source_part_info( + const common::ObTabletID &src_tablet_id, + const share::schema::ObTableSchema &src_table_schema, + const ObBasePartition*& source_partition) +{ + int ret = OB_SUCCESS; + source_partition = nullptr; + const ObPartition *part = nullptr; + const ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL; + const bool has_subpart = src_table_schema.get_part_level() == share::schema::PARTITION_LEVEL_TWO; + if (OB_UNLIKELY(!src_tablet_id.is_valid() || !src_table_schema.is_valid() || (!src_table_schema.is_partitioned_table()))) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid args", K(ret), K(src_tablet_id), K(src_table_schema)); + } else { + ObPartIterator iter(src_table_schema, check_partition_mode); + while (OB_SUCC(ret) && nullptr == source_partition) { + if (OB_FAIL(iter.next(part))) { + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + break; + } else { + LOG_WARN("iter partition failed", K(ret)); + } + } else if (OB_ISNULL(part)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get null partition", K(ret)); + } else if (!has_subpart) { + if (src_tablet_id == part->get_tablet_id()) { + source_partition = part; + } + } else { + const ObSubPartition *subpart = nullptr; + ObSubPartIterator sub_iter(src_table_schema, *part, check_partition_mode); + while (OB_SUCC(ret) && nullptr == source_partition) { + if (OB_FAIL(sub_iter.next(subpart))) { + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + break; + } else { + LOG_WARN("iter sub part failed", K(ret)); + } + } else if (OB_ISNULL(subpart)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get null subpartition", K(ret)); + } else if (src_tablet_id == subpart->get_tablet_id()) { + source_partition = subpart; + } + } + } + } + } + if (OB_SUCC(ret) && nullptr == source_partition) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected err", K(ret), K(src_tablet_id), K(src_table_schema)); + } + return ret; +} + +int ObRemoteScan::get_next_row( + const blocksstable::ObDatumRow *&tmp_row_without_reshape, + const blocksstable::ObDatumRow *&tmp_row_with_reshape) +{ + int ret = OB_SUCCESS; + tmp_row_without_reshape = nullptr; + tmp_row_with_reshape = nullptr; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else if (OB_ISNULL(result_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("result is nullptr", K(ret)); + } else if (OB_FAIL(result_->next())) { + if (OB_UNLIKELY(OB_ITER_END != ret)) { + LOG_WARN("result next failed", K(ret)); + } + } else { + row_without_reshape_.reuse(); + row_with_reshape_.reuse(); + for (int64_t i = 0; OB_SUCC(ret) && i < org_col_ids_.count(); i++) { + ObObj obj; + if (OB_FAIL(result_->get_obj(i, obj))) { + LOG_WARN("failed to get object", K(ret), "column_id", org_col_ids_.at(i).col_id_); + } else if (OB_FAIL(row_without_reshape_.storage_datums_[i].from_obj_enhance(obj))) { + LOG_WARN("failed to from obj enhance", K(ret)); + } + /** + * For fix-length type column, the select result is padded to full char length, and row needs to + * be reshaped to keep the same format as lines written into macro block, avoiding checksum error. + */ + else if (OB_FAIL(row_with_reshape_.storage_datums_[i].from_obj_enhance(obj))) { + LOG_WARN("failed to from obj enhance", K(ret)); + } else if (obj.is_fixed_len_char_type() + && OB_FAIL(ObDDLUtil::reshape_ddl_column_obj(row_with_reshape_.storage_datums_[i], obj.get_meta()))) { + LOG_WARN("reshape failed", K(ret), K(obj)); + } + } + if (OB_SUCC(ret)) { + tmp_row_without_reshape = &row_without_reshape_; + tmp_row_with_reshape = &row_with_reshape_; + } + } + return ret; +} + +int ObRemoteScan::prepare_iter(const ObSqlString &sql_string, common::ObCommonSqlProxy *sql_proxy) +{ + int ret = OB_SUCCESS; + ObSessionParam session_param; + ObSQLMode sql_mode = SMO_STRICT_ALL_TABLES | SMO_PAD_CHAR_TO_FULL_LENGTH; + session_param.sql_mode_ = reinterpret_cast(&sql_mode); + session_param.tz_info_wrap_ = nullptr; + session_param.ddl_info_.set_is_ddl(true); + session_param.ddl_info_.set_source_table_hidden(false); + session_param.ddl_info_.set_dest_table_hidden(false); + if (OB_ISNULL(sql_proxy)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret)); + } else if (OB_FAIL(sql_proxy->read(res_, tenant_id_, sql_string.ptr(), &session_param))) { + LOG_WARN("fail to execute sql", K(ret), K_(tenant_id), K(sql_string)); + } else if (OB_ISNULL(result_ = res_.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("ObMySQLResult is nullptr", K(ret), K(sql_string)); + } + return ret; +} + +int ObRemoteScan::get_origin_table_checksum(ObArray &report_col_checksums, ObArray &report_col_ids) +{ + int ret = OB_SUCCESS; + report_col_checksums.reuse(); + report_col_ids.reuse(); + ObArray tmp_col_ids; + 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; + const share::schema::ObTableSchema *dest_table_schema = nullptr; + const share::schema::ObTableSchema *orig_table_schema = nullptr; + MTL_SWITCH (OB_SYS_TENANT_ID) { + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else if (OB_FAIL(ObDDLUtil::get_tenant_schema_guard(tenant_id_, dest_tenant_id_, + hold_buf_src_tenant_schema_guard, hold_buf_dst_tenant_schema_guard, + src_tenant_schema_guard, dst_tenant_schema_guard))) { + LOG_WARN("get tenant schema guard failed", K(ret), K(tenant_id_), K(dest_tenant_id_)); + } else if (OB_FAIL(src_tenant_schema_guard->get_table_schema(tenant_id_, table_id_, orig_table_schema))) { + LOG_WARN("fail to get table schema", K(ret), K(tenant_id_), K(table_id_)); + } else if (OB_ISNULL(orig_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("table not exist", K(ret), K(tenant_id_), K(table_id_)); + } else if (OB_FAIL(dst_tenant_schema_guard->get_table_schema(dest_tenant_id_, dest_table_id_, dest_table_schema))) { + LOG_WARN("fail to get table schema", K(ret), K_(dest_tenant_id), K_(dest_table_id)); + } else if (OB_ISNULL(dest_table_schema)) { + ret = OB_TABLE_NOT_EXIST; + LOG_WARN("dest table schema is nullptr", K(ret)); + } else if (OB_FAIL(dest_table_schema->get_store_column_ids(tmp_col_ids, false))) { + LOG_WARN("fail to get store column ids", K(ret), K(tmp_col_ids)); + } else { + const int64_t rowkey_cols_cnt = dest_table_schema->get_rowkey_column_num(); + for (int64_t i = 0; OB_SUCC(ret) && i < tmp_col_ids.size(); i++) { + const ObColumnSchemaV2 *dest_col_schema = dest_table_schema->get_column_schema(tmp_col_ids.at(i).col_id_); + const ObString &dest_column_name = dest_col_schema->get_column_name_str(); + const ObColumnSchemaV2 *orig_col_schema = orig_table_schema->get_column_schema(dest_column_name); + const int64_t index_in_array = i; + if (OB_ISNULL(orig_col_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("data column schema should not be null", K(ret), K(dest_column_name)); + } else if (OB_FAIL(report_col_ids.push_back(orig_col_schema->get_column_id()))) { + LOG_WARN("fail to push back col id", K(ret), KPC(orig_col_schema)); + } else if (OB_FAIL(report_col_checksums.push_back(checksum_calculator_.get_column_checksum()[index_in_array]))) { + LOG_WARN("fail to push back col checksum", K(ret)); + } else {/* do nothing */} + } + } + } + return ret; +} + } //end namespace stroage } //end namespace oceanbase diff --git a/src/storage/ddl/ob_complement_data_task.h b/src/storage/ddl/ob_complement_data_task.h index d4a7055271..5b18f21850 100644 --- a/src/storage/ddl/ob_complement_data_task.h +++ b/src/storage/ddl/ob_complement_data_task.h @@ -31,45 +31,51 @@ struct ObTempExpr; } namespace storage { +class ObScan; class ObLocalScan; +class ObRemoteScan; class ObMultipleScanMerge; struct ObComplementDataParam final { public: static const int64_t DEFAULT_COMPLEMENT_DATA_MEMORY_LIMIT = 128L * 1024L * 1024L; ObComplementDataParam(): - is_inited_(false), tenant_id_(common::OB_INVALID_TENANT_ID), ls_id_(share::ObLSID::INVALID_LS_ID), - source_table_id_(common::OB_INVALID_ID), dest_table_id_(common::OB_INVALID_ID), - source_tablet_id_(ObTabletID::INVALID_TABLET_ID), dest_tablet_id_(ObTabletID::INVALID_TABLET_ID), allocator_("CompleteDataPar"), - row_store_type_(common::ENCODING_ROW_STORE), schema_version_(0), snapshot_version_(0), - concurrent_cnt_(0), task_id_(0), execution_id_(-1), tablet_task_id_(0), - compat_mode_(lib::Worker::CompatMode::INVALID), data_format_version_(0) + is_inited_(false), orig_tenant_id_(common::OB_INVALID_TENANT_ID), dest_tenant_id_(common::OB_INVALID_TENANT_ID), + orig_ls_id_(share::ObLSID::INVALID_LS_ID), dest_ls_id_(share::ObLSID::INVALID_LS_ID), orig_table_id_(common::OB_INVALID_ID), + dest_table_id_(common::OB_INVALID_ID), orig_tablet_id_(ObTabletID::INVALID_TABLET_ID), dest_tablet_id_(ObTabletID::INVALID_TABLET_ID), + allocator_("CompleteDataPar"), row_store_type_(common::ENCODING_ROW_STORE), orig_schema_version_(0), dest_schema_version_(0), + snapshot_version_(0), concurrent_cnt_(0), task_id_(0), execution_id_(-1), tablet_task_id_(0), compat_mode_(lib::Worker::CompatMode::INVALID), data_format_version_(0) {} ~ObComplementDataParam() { destroy(); } int init(const ObDDLBuildSingleReplicaRequestArg &arg); int split_task_ranges(const share::ObLSID &ls_id, const common::ObTabletID &tablet_id, const int64_t tablet_size, const int64_t hint_parallelism); + bool is_valid() const { - return common::OB_INVALID_TENANT_ID != tenant_id_ && ls_id_.is_valid() - && common::OB_INVALID_ID != source_table_id_ && common::OB_INVALID_ID != dest_table_id_ - && source_tablet_id_.is_valid() && dest_tablet_id_.is_valid() - && 0 != concurrent_cnt_ && snapshot_version_ > 0 && compat_mode_ != lib::Worker::CompatMode::INVALID - && execution_id_ >= 0 && tablet_task_id_ > 0 && data_format_version_ > 0; + return common::OB_INVALID_TENANT_ID != orig_tenant_id_ && common::OB_INVALID_TENANT_ID != dest_tenant_id_ + && orig_ls_id_.is_valid() && dest_ls_id_.is_valid() && common::OB_INVALID_ID != orig_table_id_ + && common::OB_INVALID_ID != dest_table_id_ && orig_tablet_id_.is_valid() && dest_tablet_id_.is_valid() && 0 != concurrent_cnt_ + && snapshot_version_ > 0 && compat_mode_ != lib::Worker::CompatMode::INVALID && execution_id_ >= 0 && tablet_task_id_ > 0 + && data_format_version_ > 0; } + int get_hidden_table_key(ObITable::TableKey &table_key) const; void destroy() { is_inited_ = false; - tenant_id_ = common::OB_INVALID_TENANT_ID; - ls_id_.reset(); - source_table_id_ = common::OB_INVALID_ID; + orig_tenant_id_ = common::OB_INVALID_TENANT_ID; + dest_tenant_id_ = common::OB_INVALID_TENANT_ID; + orig_ls_id_.reset(); + dest_ls_id_.reset(); + orig_table_id_ = common::OB_INVALID_ID; dest_table_id_ = common::OB_INVALID_ID; - source_tablet_id_.reset(); + orig_tablet_id_.reset(); dest_tablet_id_.reset(); ranges_.reset(); allocator_.reset(); row_store_type_ = common::ENCODING_ROW_STORE; - schema_version_ = 0; + orig_schema_version_ = 0; + dest_schema_version_ = 0; snapshot_version_ = 0; concurrent_cnt_ = 0; task_id_ = 0; @@ -78,21 +84,24 @@ public: compat_mode_ = lib::Worker::CompatMode::INVALID; data_format_version_ = 0; } - TO_STRING_KV(K_(is_inited), K_(tenant_id), K_(ls_id), K_(source_table_id), K_(dest_table_id), - K_(source_tablet_id), K_(dest_tablet_id), K_(schema_version), K_(tablet_task_id), - K_(snapshot_version), K_(concurrent_cnt), K_(task_id), K_(execution_id), K_(compat_mode), - K_(data_format_version)); + TO_STRING_KV(K_(is_inited), K_(orig_tenant_id), K_(dest_tenant_id), K_(orig_ls_id), K_(dest_ls_id), + K_(orig_table_id), K_(dest_table_id), K_(orig_tablet_id), K_(dest_tablet_id), K_(orig_schema_version), + K_(tablet_task_id), K_(dest_schema_version), K_(snapshot_version), K_(concurrent_cnt), K_(task_id), + K_(execution_id), K_(compat_mode), K_(data_format_version), K_(ranges)); public: bool is_inited_; - uint64_t tenant_id_; - share::ObLSID ls_id_; - uint64_t source_table_id_; + uint64_t orig_tenant_id_; + uint64_t dest_tenant_id_; + share::ObLSID orig_ls_id_; + share::ObLSID dest_ls_id_; + uint64_t orig_table_id_; uint64_t dest_table_id_; - ObTabletID source_tablet_id_; + ObTabletID orig_tablet_id_; ObTabletID dest_tablet_id_; common::ObArenaAllocator allocator_; common::ObRowStoreType row_store_type_; - int64_t schema_version_; + int64_t orig_schema_version_; + int64_t dest_schema_version_; int64_t snapshot_version_; int64_t concurrent_cnt_; int64_t task_id_; @@ -115,7 +124,7 @@ public: int init(const ObComplementDataParam ¶m, const ObDataStoreDesc &desc); void destroy(); int write_start_log(const ObComplementDataParam ¶m); - TO_STRING_KV(K_(is_inited), K_(complement_data_ret), K_(concurrent_cnt), KP_(index_builder), K_(row_scanned), K_(row_inserted)); + TO_STRING_KV(K_(is_inited), K_(complement_data_ret), K_(concurrent_cnt), KP_(index_builder), K_(ddl_kv_mgr_handle), K_(row_scanned), K_(row_inserted)); public: bool is_inited_; bool is_major_sstable_exist_; @@ -190,9 +199,14 @@ private: int generate_next_task(share::ObITask *&next_task); int generate_col_param(); int local_scan_by_range(); + int remote_scan(); int do_local_scan(); - int append_row(ObLocalScan &local_scan); - int add_extra_rowkey(const int64_t rowkey_cnt, const int64_t extra_rowkey_cnt, const blocksstable::ObDatumRow &row); + int do_remote_scan(); + int append_row(ObScan *scan); + int add_extra_rowkey(const int64_t rowkey_cnt, + const int64_t extra_rowkey_cnt, + const blocksstable::ObDatumRow &row, + const int64_t sql_no = 0); private: static const int64_t RETRY_INTERVAL = 100 * 1000; // 100ms @@ -240,7 +254,19 @@ public: K_(dependent_exprs), K_(output_projector)); }; -class ObLocalScan : public ObIStoreRowIterator +class ObScan +{ +public: + ObScan() = default; + virtual ~ObScan() = default; + virtual int get_next_row( + const blocksstable::ObDatumRow *&tmp_row, + const blocksstable::ObDatumRow *&tmp_row_after_reshape) = 0; + virtual compaction::ObColumnChecksumCalculator *get_checksum_calculator() = 0; + virtual int get_origin_table_checksum(ObArray &report_col_checksums, ObArray &report_col_ids) = 0; +}; + +class ObLocalScan : public ObScan { public: ObLocalScan(); @@ -260,11 +286,14 @@ public: common::ObQueryFlag &query_flag, blocksstable::ObDatumRange &range, transaction::ObTxDesc *tx_desc); - virtual int get_next_row(const blocksstable::ObDatumRow *&tmp_row) override; + virtual int get_next_row( + const blocksstable::ObDatumRow *&tmp_row, + const blocksstable::ObDatumRow *&tmp_row_after_reshape) override; int get_origin_table_checksum( ObArray &report_col_checksums, - ObArray &report_col_ids); - compaction::ObColumnChecksumCalculator *get_checksum_calculator() {return &checksum_calculator_;} + ObArray &report_col_ids) override; + compaction::ObColumnChecksumCalculator *get_checksum_calculator() override + {return &checksum_calculator_;} private: int get_output_columns( const share::schema::ObTableSchema &hidden_table_schema, @@ -288,7 +317,7 @@ private: private: bool is_inited_; uint64_t tenant_id_; - uint64_t source_table_id_; + uint64_t table_id_; uint64_t dest_table_id_; int64_t schema_version_; ObExtendedGCParam extended_gc_; @@ -312,6 +341,59 @@ private: bool output_org_cols_only_; }; +class ObRemoteScan : public ObScan +{ +public: + ObRemoteScan(); + virtual ~ObRemoteScan(); + int init(const uint64_t tenant_id, + const int64_t table_id, + const uint64_t dest_tenant_id, + const int64_t dest_table_id, + const int64_t schema_version, + const int64_t dest_schema_version, + const common::ObTabletID &src_tablet_id); + void reset(); + virtual int get_next_row( + const blocksstable::ObDatumRow *&tmp_row, + const blocksstable::ObDatumRow *&tmp_row_after_reshape) override; + compaction::ObColumnChecksumCalculator *get_checksum_calculator() override + {return &checksum_calculator_;} + int get_origin_table_checksum(ObArray &report_col_checksums, ObArray &report_col_ids) override; +private: + int prepare_iter(const ObSqlString &sql_string, common::ObCommonSqlProxy *sql_proxy); + int generate_build_select_sql(ObSqlString &sql_string); + // to fetch partiton/subpartition name for select sql. + int fetch_source_part_info( + const common::ObTabletID &src_tablet_id, + const share::schema::ObTableSchema &src_table_schema, + const ObBasePartition*& source_partition); + int generate_column_name_str(const bool is_oracle_mode, + ObSqlString &sql_string); + int generate_column_name_str(const ObString &column_name_info, + const bool with_comma, + const bool is_oracle_mode, + ObSqlString &sql_string); +private: + bool is_inited_; + int64_t current_; + uint64_t tenant_id_; + int64_t table_id_; + uint64_t dest_tenant_id_; + int64_t dest_table_id_; + int64_t schema_version_; + int64_t dest_schema_version_; + common::ObTabletID src_tablet_id_; + blocksstable::ObDatumRow row_without_reshape_; + blocksstable::ObDatumRow row_with_reshape_; // for checksum calculate only. + ObMySQLProxy::MySQLResult res_; + sqlclient::ObMySQLResult *result_; + common::ObArenaAllocator allocator_; + ObArray org_col_ids_; + common::ObArray column_names_; + compaction::ObColumnChecksumCalculator checksum_calculator_; +}; + } // end namespace storage } // end namespace oceanbase #endif // OCEANBASE_STORAGE_OB_COMPLEMENT_DATA_TASK_H diff --git a/src/storage/ddl/ob_ddl_redo_log_writer.cpp b/src/storage/ddl/ob_ddl_redo_log_writer.cpp index c5b829ad5e..e7cf32d14e 100755 --- a/src/storage/ddl/ob_ddl_redo_log_writer.cpp +++ b/src/storage/ddl/ob_ddl_redo_log_writer.cpp @@ -1551,7 +1551,7 @@ int ObDDLRedoLogWriterCallback::init(const ObDDLMacroBlockType block_type, LOG_WARN("ObDDLSSTableRedoWriter has been inited twice", K(ret)); } else if (OB_UNLIKELY(!table_key.is_valid() || nullptr == ddl_writer || DDL_MB_INVALID_TYPE == block_type || 0 == task_id || !ddl_kv_mgr_handle.is_valid())) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid arguments", K(ret), K(table_key), K(block_type)); + LOG_WARN("invalid arguments", K(ret), K(table_key), K(block_type), K(task_id)); } else if (OB_FAIL(MTL(ObLSService *)->get_ls(ddl_kv_mgr_handle.get_obj()->get_ls_id(), ls_handle, ObLSGetMod::DDL_MOD))) { LOG_WARN("failed to get log stream", K(ret), KPC(ddl_kv_mgr_handle.get_obj())); } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, diff --git a/src/storage/ddl/ob_ddl_server_client.cpp b/src/storage/ddl/ob_ddl_server_client.cpp index af8a7e0e75..4561eaea93 100644 --- a/src/storage/ddl/ob_ddl_server_client.cpp +++ b/src/storage/ddl/ob_ddl_server_client.cpp @@ -28,9 +28,37 @@ namespace oceanbase namespace storage { -int ObDDLServerClient::create_hidden_table(const obrpc::ObCreateHiddenTableArg &arg, obrpc::ObCreateHiddenTableRes &res, int64_t &snapshot_version, sql::ObSQLSessionInfo &session) +int ObDDLServerClient::execute_recover_restore_table(const obrpc::ObRecoverRestoreTableDDLArg &arg) { int ret = OB_SUCCESS; + ObAddr rs_leader_addr; + obrpc::ObCommonRpcProxy *common_rpc_proxy = GCTX.rs_rpc_proxy_; + const uint64_t dst_tenant_id = arg.target_schema_.get_tenant_id(); + const int64_t ddl_task_id = arg.ddl_task_id_; + if (OB_UNLIKELY(!arg.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(arg)); + } else if (OB_ISNULL(common_rpc_proxy)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("common rpc proxy is null", K(ret)); + } else if (OB_FAIL(GCTX.rs_mgr_->get_master_root_server(rs_leader_addr))) { + LOG_WARN("fail to rootservice address", K(ret)); + } else if (OB_FAIL(common_rpc_proxy->to(rs_leader_addr).recover_restore_table_ddl(arg))) { + LOG_WARN("fail to create not major sstable table", K(ret), K(arg)); + } else if (OB_FAIL(OB_DDL_HEART_BEAT_TASK_CONTAINER.set_register_task_id(ddl_task_id, dst_tenant_id))) { + LOG_WARN("failed to set register task id", K(ret)); + } + return ret; +} + +int ObDDLServerClient::create_hidden_table( + const obrpc::ObCreateHiddenTableArg &arg, + obrpc::ObCreateHiddenTableRes &res, + int64_t &snapshot_version, + sql::ObSQLSessionInfo &session) +{ + int ret = OB_SUCCESS; + ObAddr rs_leader_addr; const int64_t retry_interval = 100 * 1000L; obrpc::ObCommonRpcProxy *common_rpc_proxy = GCTX.rs_rpc_proxy_; if (OB_UNLIKELY(!arg.is_valid())) { @@ -39,10 +67,12 @@ int ObDDLServerClient::create_hidden_table(const obrpc::ObCreateHiddenTableArg & } else if (OB_ISNULL(common_rpc_proxy)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("common rpc proxy is null", K(ret)); + } else if (OB_FAIL(GCTX.rs_mgr_->get_master_root_server(rs_leader_addr))) { + LOG_WARN("fail to rootservice address", K(ret)); } while (OB_SUCC(ret)) { - if (OB_FAIL(common_rpc_proxy->timeout(GCONF._ob_ddl_timeout).create_hidden_table(arg, res))) { + if (OB_FAIL(common_rpc_proxy->to(rs_leader_addr).timeout(GCONF._ob_ddl_timeout).create_hidden_table(arg, res))) { LOG_WARN("failed to create hidden table", KR(ret), K(arg)); } else { break; @@ -60,7 +90,7 @@ int ObDDLServerClient::create_hidden_table(const obrpc::ObCreateHiddenTableArg & LOG_WARN("failed to set register task id", K(ret), K(res)); } if (OB_SUCC(ret)) { - if (OB_FAIL(wait_task_reach_pending(arg.tenant_id_, res.task_id_, snapshot_version, *GCTX.sql_proxy_, session))) { + if (OB_FAIL(wait_task_reach_pending(arg.tenant_id_, res.task_id_, snapshot_version, *GCTX.sql_proxy_))) { LOG_WARN("failed to wait table lock. remove register task id and abort redef table task.", K(ret), K(arg), K(res)); } #ifdef ERRSIM @@ -74,7 +104,7 @@ int ObDDLServerClient::create_hidden_table(const obrpc::ObCreateHiddenTableArg & obrpc::ObAbortRedefTableArg abort_redef_table_arg; abort_redef_table_arg.task_id_ = res.task_id_; abort_redef_table_arg.tenant_id_ = arg.tenant_id_; - if (OB_TMP_FAIL(abort_redef_table(abort_redef_table_arg, session))) { + if (OB_TMP_FAIL(abort_redef_table(abort_redef_table_arg, &session))) { LOG_WARN("failed to abort redef table", K(tmp_ret), K(abort_redef_table_arg)); } // abort_redef_table() function last step must remove heart_beat task, so there is no need to call heart_beat_clear() @@ -86,6 +116,7 @@ int ObDDLServerClient::create_hidden_table(const obrpc::ObCreateHiddenTableArg & int ObDDLServerClient::start_redef_table(const obrpc::ObStartRedefTableArg &arg, obrpc::ObStartRedefTableRes &res, sql::ObSQLSessionInfo &session) { int ret = OB_SUCCESS; + ObAddr rs_leader_addr; obrpc::ObCommonRpcProxy *common_rpc_proxy = GCTX.rs_rpc_proxy_; int64_t unused_snapshot_version = OB_INVALID_VERSION; if (OB_UNLIKELY(!arg.is_valid())) { @@ -94,24 +125,29 @@ int ObDDLServerClient::start_redef_table(const obrpc::ObStartRedefTableArg &arg, } else if (OB_ISNULL(common_rpc_proxy)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("common rpc proxy is null", K(ret)); - } else if (OB_FAIL(common_rpc_proxy->start_redef_table(arg, res))) { + } else if (OB_FAIL(GCTX.rs_mgr_->get_master_root_server(rs_leader_addr))) { + LOG_WARN("fail to rootservice address", K(ret)); + } else if (OB_FAIL(common_rpc_proxy->to(rs_leader_addr).start_redef_table(arg, res))) { LOG_WARN("failed to start redef table", KR(ret), K(arg)); } else if (OB_FAIL(OB_DDL_HEART_BEAT_TASK_CONTAINER.set_register_task_id(res.task_id_, res.tenant_id_))) { LOG_WARN("failed to set register task id", K(ret), K(res)); - } else if (OB_FAIL(wait_task_reach_pending(arg.orig_tenant_id_, res.task_id_, unused_snapshot_version, *GCTX.sql_proxy_, session))) { + } else if (OB_FAIL(wait_task_reach_pending(arg.orig_tenant_id_, res.task_id_, unused_snapshot_version, *GCTX.sql_proxy_))) { LOG_WARN("failed to wait table lock. remove register task id and abort redef table task.", K(ret), K(arg), K(res)); int tmp_ret = OB_SUCCESS; obrpc::ObAbortRedefTableArg abort_redef_table_arg; abort_redef_table_arg.task_id_ = res.task_id_; abort_redef_table_arg.tenant_id_ = arg.orig_tenant_id_; - if (OB_TMP_FAIL(abort_redef_table(abort_redef_table_arg, session))) { + if (OB_TMP_FAIL(abort_redef_table(abort_redef_table_arg, &session))) { LOG_WARN("failed to abort redef table", K(tmp_ret), K(abort_redef_table_arg)); } // abort_redef_table() function last step must remove heart_beat task, so there is no need to call heart_beat_clear() } return ret; } -int ObDDLServerClient::copy_table_dependents(const obrpc::ObCopyTableDependentsArg &arg, sql::ObSQLSessionInfo &session) + +int ObDDLServerClient::copy_table_dependents( + const obrpc::ObCopyTableDependentsArg &arg, + sql::ObSQLSessionInfo &session) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; @@ -130,12 +166,12 @@ int ObDDLServerClient::copy_table_dependents(const obrpc::ObCopyTableDependentsA if (OB_FAIL(check_need_stop(tenant_id))) { LOG_WARN("fail to basic check", K(ret), K(tenant_id)); } else if (OB_FAIL(ObDDLExecutorUtil::handle_session_exception(session))) { - LOG_WARN("session execption happened", K(ret)); + LOG_WARN("fail to handle session exception", K(ret)); if (OB_TMP_FAIL(ObDDLExecutorUtil::cancel_ddl_task(tenant_id, common_rpc_proxy))) { - LOG_WARN("cancel ddl task failed", K(tmp_ret)); + LOG_WARN("cancel ddl task failed", K(tmp_ret), K(tenant_id)); } } else if (OB_TMP_FAIL(GCTX.rs_mgr_->get_master_root_server(rs_leader_addr))) { - LOG_WARN("fail to rootservice address", K(tmp_ret)); + LOG_WARN("fail to get rootservice address", K(ret)); } else if (OB_FAIL(common_rpc_proxy->to(rs_leader_addr).copy_table_dependents(arg))) { LOG_WARN("copy table dependents failed", K(ret), K(arg)); if (OB_ENTRY_NOT_EXIST == ret) { @@ -158,7 +194,7 @@ int ObDDLServerClient::copy_table_dependents(const obrpc::ObCopyTableDependentsA return ret; } -int ObDDLServerClient::abort_redef_table(const obrpc::ObAbortRedefTableArg &arg, sql::ObSQLSessionInfo &session) +int ObDDLServerClient::abort_redef_table(const obrpc::ObAbortRedefTableArg &arg, sql::ObSQLSessionInfo *session) { int ret = OB_SUCCESS; const uint64_t tenant_id = arg.tenant_id_; @@ -265,7 +301,7 @@ int ObDDLServerClient::finish_redef_table(const obrpc::ObFinishRedefTableArg &fi if (OB_FAIL(ret)) { } else if (OB_FAIL(build_ddl_single_replica_response(build_single_arg))) { LOG_WARN("build ddl single replica response", K(ret), K(build_single_arg)); - } else if (OB_FAIL(sql::ObDDLExecutorUtil::wait_ddl_finish(finish_redef_arg.tenant_id_, finish_redef_arg.task_id_, session, common_rpc_proxy))) { + } else if (OB_FAIL(sql::ObDDLExecutorUtil::wait_ddl_finish(finish_redef_arg.tenant_id_, finish_redef_arg.task_id_, &session, common_rpc_proxy))) { LOG_WARN("failed to wait ddl finish", K(ret), K(finish_redef_arg.tenant_id_), K(finish_redef_arg.task_id_)); } if (OB_TMP_FAIL(heart_beat_clear(finish_redef_arg.task_id_))) { @@ -291,7 +327,7 @@ int ObDDLServerClient::build_ddl_single_replica_response(const obrpc::ObDDLBuild return ret; } -int ObDDLServerClient::wait_task_reach_pending(const uint64_t tenant_id, const int64_t task_id, int64_t &snapshot_version, ObMySQLProxy &sql_proxy, sql::ObSQLSessionInfo &session) +int ObDDLServerClient::wait_task_reach_pending(const uint64_t tenant_id, const int64_t task_id, int64_t &snapshot_version, ObMySQLProxy &sql_proxy) { int ret = OB_SUCCESS; const int64_t retry_interval = 100 * 1000; diff --git a/src/storage/ddl/ob_ddl_server_client.h b/src/storage/ddl/ob_ddl_server_client.h index 5f85be5819..a9f19db80b 100644 --- a/src/storage/ddl/ob_ddl_server_client.h +++ b/src/storage/ddl/ob_ddl_server_client.h @@ -23,16 +23,30 @@ namespace storage class ObDDLServerClient final { public: + /** + * For recover restore table ddl task, including: + * 1. create a user hidden table under the target tenant(dest tenant). + * 2. import the backup tenant's data into the target tenant by the table redefinition task. + */ + static int execute_recover_restore_table(const obrpc::ObRecoverRestoreTableDDLArg &arg); + /** + * for load data. + */ static int create_hidden_table(const obrpc::ObCreateHiddenTableArg &arg, obrpc::ObCreateHiddenTableRes &res, int64_t &snapshot_version, sql::ObSQLSessionInfo &session); static int start_redef_table(const obrpc::ObStartRedefTableArg &arg, obrpc::ObStartRedefTableRes &res, sql::ObSQLSessionInfo &session); static int copy_table_dependents(const obrpc::ObCopyTableDependentsArg &arg, sql::ObSQLSessionInfo &session); static int finish_redef_table(const obrpc::ObFinishRedefTableArg &finish_redef_arg, const obrpc::ObDDLBuildSingleReplicaResponseArg &build_single_arg, sql::ObSQLSessionInfo &session); - static int abort_redef_table(const obrpc::ObAbortRedefTableArg &arg, sql::ObSQLSessionInfo &session); + static int finish_redef_table(const obrpc::ObFinishRedefTableArg &finish_redef_arg); + static int abort_redef_table(const obrpc::ObAbortRedefTableArg &arg, sql::ObSQLSessionInfo *session = nullptr); static int build_ddl_single_replica_response(const obrpc::ObDDLBuildSingleReplicaResponseArg &arg); private: - static int wait_task_reach_pending(const uint64_t tenant_id, const int64_t task_id, int64_t &snapshot_version, ObMySQLProxy &sql_proxy, sql::ObSQLSessionInfo &session); + static int wait_task_reach_pending( + const uint64_t tenant_id, + const int64_t task_id, + int64_t &snapshot_version, + ObMySQLProxy &sql_proxy); static int heart_beat_clear(const int64_t task_id); static int check_need_stop(const uint64_t tenant_id); }; diff --git a/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp b/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp index 8bb50c1ff2..9d5e216270 100755 --- a/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp +++ b/src/storage/ddl/ob_direct_insert_sstable_ctx.cpp @@ -351,7 +351,8 @@ int ObSSTableInsertSliceWriter::append_row(const ObNewRow &row_val) ObTimeUtility::current_time() + ObInsertLobColumnHelper::LOB_ACCESS_TX_TIMEOUT; bool has_lob_header = store_row_.row_val_.cells_[i].has_lob_header(); if (OB_FAIL(ObInsertLobColumnHelper::insert_lob_column( - lob_allocator_, ls_id_, tablet_id_, col_descs_->at(i), datum, timeout_ts, has_lob_header))) { + lob_allocator_, ls_id_, tablet_id_, col_descs_->at(i), datum, timeout_ts, has_lob_header, + MTL_ID()))) { LOG_WARN("fail to insert_lob_col", KR(ret), K(datum)); } } diff --git a/src/storage/lob/ob_lob_manager.cpp b/src/storage/lob/ob_lob_manager.cpp index f7051f0c93..a6196dfc06 100644 --- a/src/storage/lob/ob_lob_manager.cpp +++ b/src/storage/lob/ob_lob_manager.cpp @@ -415,14 +415,15 @@ int ObLobManager::is_remote(ObLobAccessParam& param, bool& is_remote, common::Ob } else if (!lob_locator->is_persist_lob()) { is_remote = false; } else { - int64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; if (OB_FAIL(get_ls_leader(param, tenant_id, param.ls_id_, dst_addr))) { LOG_WARN("failed to get ls leader", K(ret), K(tenant_id), K(param.ls_id_)); } else { - is_remote = (dst_addr != self_addr); + // lob from other tenant also should read by rpc + is_remote = (dst_addr != self_addr) || (tenant_id != MTL_ID()); if (param.from_rpc_ == true && is_remote) { ret = OB_NOT_MASTER; - LOG_WARN("call from rpc, but remote again", K(ret), K(dst_addr), K(self_addr)); + LOG_WARN("call from rpc, but remote again", K(ret), K(dst_addr), K(self_addr), K(tenant_id), K(MTL_ID())); } } } @@ -446,7 +447,7 @@ int ObLobManager::query_remote(ObLobAccessParam& param, common::ObAddr& dst_addr } else { SMART_VAR(ObLobQueryArg, arg) { // build arg - arg.tenant_id_ = MTL_ID(); + arg.tenant_id_ = param.tenant_id_; arg.offset_ = param.offset_; arg.len_ = param.len_; arg.cs_type_ = param.coll_type_; @@ -465,7 +466,7 @@ int ObLobManager::query_remote(ObLobAccessParam& param, common::ObAddr& dst_addr .timeout(timeout) .lob_query(arg, rpc_buffer, handle); if (OB_FAIL(ret)) { - LOG_WARN("failed to do remote query", K(ret)); + LOG_WARN("failed to do remote query", K(ret), K(arg)); } else { ObLobQueryBlock block; ObString block_data; @@ -1187,6 +1188,7 @@ int ObLobManager::append( data.assign_buffer(buf + cur_handle_size, append_lob_len); SMART_VAR(ObLobAccessParam, read_param) { read_param.tx_desc_ = param.tx_desc_; + read_param.tenant_id_ = param.src_tenant_id_; if (OB_FAIL(build_lob_param(read_param, *param.allocator_, param.coll_type_, 0, UINT64_MAX, param.timeout_, lob))) { LOG_WARN("fail to build read param", K(ret), K(lob)); @@ -1311,6 +1313,7 @@ int ObLobManager::append( if (OB_SUCC(ret)) { SMART_VAR(ObLobAccessParam, read_param) { read_param.tx_desc_ = param.tx_desc_; + read_param.tenant_id_ = param.src_tenant_id_; if (OB_FAIL(build_lob_param(read_param, *param.allocator_, param.coll_type_, 0, UINT64_MAX, param.timeout_, lob))) { LOG_WARN("fail to build read param", K(ret), K(lob)); @@ -1831,7 +1834,7 @@ int ObLobManager::getlength_remote(ObLobAccessParam& param, common::ObAddr& dst_ } else { SMART_VAR(ObLobQueryArg, arg) { // build arg - arg.tenant_id_ = MTL_ID(); + arg.tenant_id_ = param.tenant_id_; arg.offset_ = param.offset_; arg.len_ = param.len_; arg.cs_type_ = param.coll_type_; @@ -3349,7 +3352,7 @@ int ObLobQueryIter::open(ObLobAccessParam ¶m, common::ObAddr dst_addr) obrpc::ObStorageRpcProxy *svr_rpc_proxy = ls_service->get_storage_rpc_proxy(); const int64_t cluster_id = GCONF.cluster_id; // build arg - query_arg_.tenant_id_ = MTL_ID(); + query_arg_.tenant_id_ = param.tenant_id_; query_arg_.offset_ = param.offset_; query_arg_.len_ = param.len_; query_arg_.cs_type_ = param.coll_type_; diff --git a/src/storage/lob/ob_lob_persistent_adaptor.cpp b/src/storage/lob/ob_lob_persistent_adaptor.cpp index 7032f75c59..a70a69477f 100644 --- a/src/storage/lob/ob_lob_persistent_adaptor.cpp +++ b/src/storage/lob/ob_lob_persistent_adaptor.cpp @@ -77,7 +77,7 @@ int ObPersistentLobApator::scan_lob_meta( if (OB_FAIL(get_lob_tablets(param, data_tablet, lob_meta_tablet, lob_piece_tablet))) { LOG_WARN("failed to get tablets.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; // 2. prepare tablet scan param scan_param.tablet_id_ = lob_meta_tablet.get_obj()->get_tablet_meta().tablet_id_; scan_param.schema_version_ = lob_meta_tablet.get_obj()->get_tablet_meta().max_sync_storage_schema_version_; @@ -149,7 +149,7 @@ int ObPersistentLobApator::get_lob_data( if (OB_FAIL(get_lob_tablets(param, data_tablet, lob_meta_tablet, lob_piece_tablet))) { LOG_WARN("failed to get tablets.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; // 2. prepare tablet scan param ObTableScanParam scan_param; scan_param.table_param_ = param.piece_tablet_param_; @@ -239,7 +239,7 @@ int ObPersistentLobApator::fetch_lob_id(ObLobAccessParam& param, uint64_t &lob_i if (OB_FAIL(get_lob_tablets_id(param, lob_meta_tablet_id, lob_piece_tablet_id))) { LOG_WARN("get lob tablet id failed.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; share::ObTabletAutoincrementService &auto_inc = share::ObTabletAutoincrementService::get_instance(); if (OB_FAIL(auto_inc.get_autoinc_seq(tenant_id, lob_meta_tablet_id, lob_id))) { LOG_WARN("get lob_id fail", K(ret), K(tenant_id), K(lob_meta_tablet_id)); @@ -386,7 +386,7 @@ int ObPersistentLobApator::erase_lob_meta(ObLobAccessParam ¶m, ObLobMetaInfo ret = OB_ERR_NULL_VALUE; LOG_WARN("get tx desc null.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; if (OB_FAIL(prepare_lob_meta_dml(param, tenant_id, data_tablet, lob_meta_tablet))) { LOG_WARN("failed to prepare lob meta dml", K(ret)); } else { @@ -437,7 +437,7 @@ int ObPersistentLobApator::erase_lob_piece_tablet(ObLobAccessParam& param, ObLob ret = OB_ERR_NULL_VALUE; LOG_WARN("get tx desc null.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; ObDMLBaseParam dml_base_param; share::schema::ObTableDMLParam table_dml_param(*param.allocator_); @@ -499,7 +499,7 @@ int ObPersistentLobApator::write_lob_meta(ObLobAccessParam& param, ObLobMetaInfo ret = OB_ERR_NULL_VALUE; LOG_WARN("get tx desc null.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; if (OB_FAIL(prepare_lob_meta_dml(param, tenant_id, data_tablet, lob_meta_tablet))) { LOG_WARN("failed to prepare lob meta dml.", K(ret)); } else { @@ -550,7 +550,7 @@ int ObPersistentLobApator::update_lob_meta(ObLobAccessParam& param, ObLobMetaInf ret = OB_ERR_NULL_VALUE; LOG_WARN("get tx desc null.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; ObSEArray update_column_ids; for (int i = 2; OB_SUCC(ret) && i < ObLobMetaUtil::LOB_META_COLUMN_CNT; ++i) { @@ -667,7 +667,7 @@ int ObPersistentLobApator::write_lob_piece_tablet(ObLobAccessParam& param, ObLob ret = OB_ERR_NULL_VALUE; LOG_WARN("get tx desc null.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; ObDMLBaseParam dml_base_param; share::schema::ObTableDMLParam table_dml_param(*param.allocator_); @@ -730,7 +730,7 @@ int ObPersistentLobApator::update_lob_piece_tablet(ObLobAccessParam& param, ObLo ret = OB_ERR_NULL_VALUE; LOG_WARN("get tx desc null.", K(ret), K(param)); } else { - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; ObDMLBaseParam dml_base_param; share::schema::ObTableDMLParam table_dml_param(*param.allocator_); @@ -810,7 +810,7 @@ int ObPersistentLobApator::build_common_scan_param( ObTableScanParam& scan_param) { int ret = OB_SUCCESS; - uint64_t tenant_id = MTL_ID(); + uint64_t tenant_id = param.tenant_id_; scan_param.ls_id_ = param.ls_id_; ObQueryFlag query_flag(ObQueryFlag::Forward, // scan_order diff --git a/src/storage/lob/ob_lob_util.cpp b/src/storage/lob/ob_lob_util.cpp index bbc38875cf..ef097acb60 100644 --- a/src/storage/lob/ob_lob_util.cpp +++ b/src/storage/lob/ob_lob_util.cpp @@ -109,7 +109,8 @@ int ObInsertLobColumnHelper::insert_lob_column(ObIAllocator &allocator, const ObColDesc &column, blocksstable::ObStorageDatum &datum, const int64_t timeout_ts, - const bool has_lob_header) + const bool has_lob_header, + const uint64_t src_tenant_id) { int ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS; @@ -152,6 +153,7 @@ int ObInsertLobColumnHelper::insert_lob_column(ObIAllocator &allocator, } else { // 4.0 text tc compatiable ObLobAccessParam lob_param; + lob_param.src_tenant_id_ = src_tenant_id; lob_param.tx_desc_ = tx_desc; lob_param.snapshot_ = snapshot; lob_param.sql_mode_ = SMO_DEFAULT; @@ -191,7 +193,7 @@ int ObInsertLobColumnHelper::insert_lob_column(ObIAllocator &allocator, int ret = OB_SUCCESS; ObStorageDatum datum; datum.from_obj(obj); - if (OB_SUCC(insert_lob_column(allocator, ls_id, tablet_id, column, datum, timeout_ts, obj.has_lob_header()))) { + if (OB_SUCC(insert_lob_column(allocator, ls_id, tablet_id, column, datum, timeout_ts, obj.has_lob_header(), MTL_ID()))) { obj.set_lob_value(obj.get_type(), datum.get_string().ptr(), datum.get_string().length()); } return ret; diff --git a/src/storage/lob/ob_lob_util.h b/src/storage/lob/ob_lob_util.h index fcc4598644..1ac2b4d8b1 100644 --- a/src/storage/lob/ob_lob_util.h +++ b/src/storage/lob/ob_lob_util.h @@ -36,8 +36,9 @@ struct ObLobAccessParam { dml_base_param_(nullptr), column_ids_(), meta_table_schema_(nullptr), piece_table_schema_(nullptr), main_tablet_param_(nullptr), meta_tablet_param_(nullptr), piece_tablet_param_(nullptr), - ls_id_(), tablet_id_(), coll_type_(), lob_locator_(nullptr), lob_common_(nullptr), - lob_data_(nullptr), byte_size_(0), handle_size_(0), timeout_(0), + tenant_id_(MTL_ID()), src_tenant_id_(MTL_ID()), + ls_id_(), tablet_id_(), coll_type_(), lob_locator_(nullptr), + lob_common_(nullptr), lob_data_(nullptr), byte_size_(0), handle_size_(0), timeout_(0), fb_snapshot_(), scan_backward_(false), asscess_ptable_(false), offset_(0), len_(0), parent_seq_no_(), seq_no_st_(), used_seq_cnt_(0), total_seq_cnt_(0), checksum_(0), update_len_(0), @@ -51,9 +52,10 @@ struct ObLobAccessParam { } public: int set_lob_locator(common::ObLobLocatorV2 *lob_locator); - TO_STRING_KV(K_(ls_id), K_(tablet_id), KPC_(lob_locator), KPC_(lob_common), KPC_(lob_data), K_(byte_size), K_(handle_size), - K_(coll_type), K_(scan_backward), K_(offset), K_(len), K_(parent_seq_no), K_(seq_no_st), K_(used_seq_cnt), K_(total_seq_cnt), - K_(checksum), K_(update_len), K_(op_type), K_(is_fill_zero), K_(from_rpc), K_(snapshot), K_(tx_id), K_(inrow_read_nocopy)); + TO_STRING_KV(K_(tenant_id), K_(src_tenant_id), K_(ls_id), K_(tablet_id), KPC_(lob_locator), KPC_(lob_common), + KPC_(lob_data), K_(byte_size), K_(handle_size), K_(coll_type), K_(scan_backward), K_(offset), K_(len), + K_(parent_seq_no), K_(seq_no_st), K_(used_seq_cnt), K_(total_seq_cnt), K_(checksum), K_(update_len), K_(op_type), + K_(is_fill_zero), K_(from_rpc), K_(snapshot), K_(tx_id), K_(inrow_read_nocopy)); public: transaction::ObTxDesc *tx_desc_; // for write/update/delete transaction::ObTxReadSnapshot snapshot_; // for read @@ -68,6 +70,10 @@ public: share::schema::ObTableParam *main_tablet_param_; // for test share::schema::ObTableParam *meta_tablet_param_; // for test share::schema::ObTableParam *piece_tablet_param_; // for test + uint64_t tenant_id_; + // some lob manager func will access other lob for data + // other lob can read from other tenant + uint64_t src_tenant_id_; share::ObLSID ls_id_; common::ObTabletID tablet_id_; common::ObCollationType coll_type_; @@ -194,7 +200,8 @@ public: const share::schema::ObColDesc &column, blocksstable::ObStorageDatum &datum, const int64_t timeout_ts, - const bool has_lob_header); + const bool has_lob_header, + const uint64_t src_tenant_id); static int insert_lob_column(ObIAllocator &allocator, const share::ObLSID ls_id, const common::ObTabletID tablet_id, diff --git a/src/storage/ls/ob_ls.cpp b/src/storage/ls/ob_ls.cpp index 560705323b..a69a967acc 100755 --- a/src/storage/ls/ob_ls.cpp +++ b/src/storage/ls/ob_ls.cpp @@ -39,7 +39,7 @@ #include "rootserver/ob_recovery_ls_service.h" #include "rootserver/ob_tenant_transfer_service.h" // ObTenantTransferService #include "rootserver/ob_tenant_balance_service.h" -#include "rootserver/restore/ob_restore_scheduler.h" +#include "rootserver/restore/ob_restore_service.h" #include "share/ob_tenant_info_proxy.h" #include "share/leak_checker/obj_leak_checker.h" #include "share/ob_ls_id.h" diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result index 8c2764203d..caf2d6bda0 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result @@ -243,6 +243,7 @@ writing_throttling_trigger_percentage zone _advance_checkpoint_timeout _audit_mode +_auto_drop_recovering_auxiliary_tenant _backup_idle_time _backup_task_keep_alive_interval _backup_task_keep_alive_timeout diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result index 428c0b1983..7a52de6ebc 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result @@ -254,6 +254,12 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 472 __wr_control 0 201001 1 473 __all_tenant_event_history 0 201001 1 474 __all_tenant_scheduler_job_class 0 201001 1 +475 __all_recover_table_job 0 201001 1 +476 __all_recover_table_job_history 0 201001 1 +477 __all_import_table_job 0 201001 1 +478 __all_import_table_job_history 0 201001 1 +479 __all_import_table_task 0 201001 1 +480 __all_import_table_task_history 0 201001 1 10001 __tenant_virtual_all_table 2 201001 1 10002 __tenant_virtual_table_column 2 201001 1 10003 __tenant_virtual_table_index 2 201001 1 @@ -328,6 +334,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 11098 __all_virtual_audit_action 2 201001 1 11099 __all_virtual_dag_warning_history 2 201001 1 11100 __all_virtual_tablet_encrypt_info 2 201001 1 +11102 __tenant_virtual_show_restore_preview 2 201001 1 11104 __all_virtual_master_key_version_info 2 201001 1 11105 __all_virtual_dag 2 201001 1 11106 __all_virtual_dag_scheduler 2 201001 1 @@ -645,6 +652,12 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 12417 __all_virtual_balance_group_ls_stat 2 201001 1 12420 __all_virtual_flt_config 2 201001 1 12421 __all_virtual_tenant_scheduler_job_class 2 201001 1 +12422 __all_virtual_recover_table_job 2 201001 1 +12423 __all_virtual_recover_table_job_history 2 201001 1 +12424 __all_virtual_import_table_job 2 201001 1 +12425 __all_virtual_import_table_job_history 2 201001 1 +12426 __all_virtual_import_table_task 2 201001 1 +12427 __all_virtual_import_table_task_history 2 201001 1 20001 GV$OB_PLAN_CACHE_STAT 1 201001 1 20002 GV$OB_PLAN_CACHE_PLAN_STAT 1 201001 1 20003 SCHEMATA 1 201002 1 @@ -994,6 +1007,18 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 21447 DBA_OB_TENANT_EVENT_HISTORY 1 201001 1 21448 CDB_OB_TENANT_EVENT_HISTORY 1 201001 1 21449 GV$OB_FLT_TRACE_CONFIG 1 201001 1 +21463 CDB_OB_RECOVER_TABLE_JOBS 1 201001 1 +21464 DBA_OB_RECOVER_TABLE_JOBS 1 201001 1 +21465 CDB_OB_RECOVER_TABLE_JOB_HISTORY 1 201001 1 +21466 DBA_OB_RECOVER_TABLE_JOB_HISTORY 1 201001 1 +21467 CDB_OB_IMPORT_TABLE_JOBS 1 201001 1 +21468 DBA_OB_IMPORT_TABLE_JOBS 1 201001 1 +21469 CDB_OB_IMPORT_TABLE_JOB_HISTORY 1 201001 1 +21470 DBA_OB_IMPORT_TABLE_JOB_HISTORY 1 201001 1 +21471 CDB_OB_IMPORT_TABLE_TASKS 1 201001 1 +21472 DBA_OB_IMPORT_TABLE_TASKS 1 201001 1 +21473 CDB_OB_IMPORT_TABLE_TASK_HISTORY 1 201001 1 +21474 DBA_OB_IMPORT_TABLE_TASK_HISTORY 1 201001 1 21477 GV$OB_TENANT_RUNTIME_INFO 1 201001 1 21478 V$OB_TENANT_RUNTIME_INFO 1 201001 1 check sys table count and table_id range success diff --git a/unittest/rootserver/CMakeLists.txt b/unittest/rootserver/CMakeLists.txt index 68dab03d21..c46c0f54b1 100644 --- a/unittest/rootserver/CMakeLists.txt +++ b/unittest/rootserver/CMakeLists.txt @@ -20,4 +20,5 @@ rs_unittest(test_primary_ls_service) #rs_unittest(test_zone_merge_manager) #rs_unittest(test_freeze_info_manager) rs_unittest(test_archive_checkpoint) -rs_unittest(test_heartbeat_service) +rs_unittest(test_import_table) +rs_unittest(test_heartbeat_service) \ No newline at end of file diff --git a/unittest/rootserver/test_import_table.cpp b/unittest/rootserver/test_import_table.cpp new file mode 100644 index 0000000000..66627cca92 --- /dev/null +++ b/unittest/rootserver/test_import_table.cpp @@ -0,0 +1,471 @@ +/** + * 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 +#include +#include +#include "share/ob_errno.h" +#include "share/restore/ob_import_arg.h" + + +using namespace oceanbase; +using namespace common; +using namespace share; + +class ImportTableTest : public testing::Test +{ +public: + ImportTableTest() {} + virtual ~ImportTableTest(){} + virtual void SetUp() {}; + virtual void TearDown() {} + virtual void TestBody() {} +}; + +TEST_F(ImportTableTest, test_import_database) +{ + ObImportDatabaseArray import_db_array; + ObImportDatabaseItem db1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem db2(OB_ORIGIN_AND_SENSITIVE, "db2", strlen("db2")); + ObImportDatabaseItem db3(OB_ORIGIN_AND_SENSITIVE, "db3", strlen("db3")); + + ASSERT_EQ(OB_SUCCESS, import_db_array.add_item(db1)); + ASSERT_EQ(OB_SUCCESS, import_db_array.add_item(db2)); + ASSERT_EQ(OB_SUCCESS, import_db_array.add_item(db3)); + + const char *format_str = "`db1`,`db2`,`db3`"; + char format_buff[1024]; + memset(format_buff, 0, 1024); + int64_t pos = 0; + ASSERT_EQ(OB_SUCCESS, import_db_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + char hex_format_buff[1024]; + pos = 0; + memset(hex_format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, import_db_array.hex_format_serialize(hex_format_buff, 1024, pos)); + ASSERT_EQ(pos, import_db_array.get_hex_format_serialize_size()); + + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_db_array.hex_format_deserialize(hex_format_buff, import_db_array.get_hex_format_serialize_size(), pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_db_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + pos = 0; + char buff[1024]; + ASSERT_EQ(OB_SUCCESS, import_db_array.serialize(buff, 1024, pos)); + ASSERT_EQ(import_db_array.get_serialize_size(), pos); + + pos = 0; + ObImportDatabaseArray import_db_array_bak; + ASSERT_EQ(OB_SUCCESS, import_db_array_bak.deserialize(buff, 1024, pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_db_array_bak.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); +} + +TEST_F(ImportTableTest, test_import_table) +{ + ObImportTableArray import_table_array; + ObImportTableItem tb1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb1", strlen("tb1")); + ObImportTableItem tb2(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb2", strlen("tb2")); + ObImportTableItem tb3(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb,3", strlen("tb,3")); + + ASSERT_EQ(OB_SUCCESS, import_table_array.add_item(tb1)); + ASSERT_EQ(OB_SUCCESS, import_table_array.add_item(tb2)); + ASSERT_EQ(OB_SUCCESS, import_table_array.add_item(tb3)); + + const char *format_str = "`db1`.`tb1`,`db1`.`tb2`,`db1`.`tb,3`"; + char format_buff[1024]; + int64_t pos = 0; + memset(format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, import_table_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + char hex_format_buff[1024]; + pos = 0; + memset(hex_format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, import_table_array.hex_format_serialize(hex_format_buff, 1024, pos)); + ASSERT_EQ(pos, import_table_array.get_hex_format_serialize_size()); + + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_table_array.hex_format_deserialize(hex_format_buff, import_table_array.get_hex_format_serialize_size(), pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_table_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + pos = 0; + char buff[1024]; + ASSERT_EQ(OB_SUCCESS, import_table_array.serialize(buff, 1024, pos)); + ASSERT_EQ(import_table_array.get_serialize_size(), pos); + + pos = 0; + ObImportTableArray import_table_array_bak; + ASSERT_EQ(OB_SUCCESS, import_table_array_bak.deserialize(buff, 1024, pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_table_array_bak.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); +} + + +TEST_F(ImportTableTest, test_import_partition) +{ + ObImportPartitionArray import_part_array; + ObImportPartitionItem p1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb1", strlen("tb1"), "p1", strlen("p1")); + ObImportPartitionItem p2(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb2", strlen("tb2"), "p1", strlen("p1")); + ObImportPartitionItem p3(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb,3", strlen("tb,3"), "p1", strlen("p1")); + + ASSERT_EQ(OB_SUCCESS, import_part_array.add_item(p1)); + ASSERT_EQ(OB_SUCCESS, import_part_array.add_item(p2)); + ASSERT_EQ(OB_SUCCESS, import_part_array.add_item(p3)); + + const char *format_str = "`db1`.`tb1`:`p1`,`db1`.`tb2`:`p1`,`db1`.`tb,3`:`p1`"; + char format_buff[1024]; + int64_t pos = 0; + memset(format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, import_part_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + char hex_format_buff[1024]; + pos = 0; + memset(hex_format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, import_part_array.hex_format_serialize(hex_format_buff, 1024, pos)); + ASSERT_EQ(pos, import_part_array.get_hex_format_serialize_size()); + + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_part_array.hex_format_deserialize(hex_format_buff, import_part_array.get_hex_format_serialize_size(), pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_part_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + pos = 0; + char buff[1024]; + ASSERT_EQ(OB_SUCCESS, import_part_array.serialize(buff, 1024, pos)); + ASSERT_EQ(import_part_array.get_serialize_size(), pos); + + pos = 0; + ObImportPartitionArray import_part_array_bak; + ASSERT_EQ(OB_SUCCESS, import_part_array_bak.deserialize(buff, 1024, pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, import_part_array_bak.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); +} + + +TEST_F(ImportTableTest, test_remap_database) +{ + ObRemapDatabaseArray remap_db_array; + ObImportDatabaseItem src_db1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem target_db1(OB_ORIGIN_AND_SENSITIVE, "db1_bak", strlen("db1_bak")); + ObImportDatabaseItem src_db2(OB_ORIGIN_AND_SENSITIVE, "db2", strlen("db2")); + ObImportDatabaseItem target_db2(OB_ORIGIN_AND_SENSITIVE, "db2_bak", strlen("db2_bak")); + ObImportDatabaseItem src_db3(OB_ORIGIN_AND_SENSITIVE, "db3", strlen("db3")); + ObImportDatabaseItem target_db3(OB_ORIGIN_AND_SENSITIVE, "db3_bak", strlen("db3_bak")); + + ObRemapDatabaseItem remap_db1; + remap_db1.src_ = src_db1; + remap_db1.target_ = target_db1; + + ObRemapDatabaseItem remap_db2; + remap_db2.src_ = src_db2; + remap_db2.target_ = target_db2; + + ObRemapDatabaseItem remap_db3; + remap_db3.src_ = src_db3; + remap_db3.target_ = target_db3; + + ASSERT_EQ(OB_SUCCESS, remap_db_array.add_item(remap_db1)); + ASSERT_EQ(OB_SUCCESS, remap_db_array.add_item(remap_db2)); + ASSERT_EQ(OB_SUCCESS, remap_db_array.add_item(remap_db3)); + + const char *format_str = "`db1`:`db1_bak`,`db2`:`db2_bak`,`db3`:`db3_bak`"; + char format_buff[1024]; + memset(format_buff, 0, 1024); + int64_t pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_db_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + char hex_format_buff[1024]; + memset(hex_format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_db_array.hex_format_serialize(hex_format_buff, 1024, pos)); + ASSERT_EQ(pos, remap_db_array.get_hex_format_serialize_size()); + + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_db_array.hex_format_deserialize(hex_format_buff, remap_db_array.get_hex_format_serialize_size(), pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_db_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + pos = 0; + char buff[1024]; + ASSERT_EQ(OB_SUCCESS, remap_db_array.serialize(buff, 1024, pos)); + + pos = 0; + ObRemapDatabaseArray remap_db_array_bak; + ASSERT_EQ(OB_SUCCESS, remap_db_array_bak.deserialize(buff, 1024, pos)); + pos = 0; + memset(format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, remap_db_array_bak.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_buff)); +} + + +TEST_F(ImportTableTest, test_remap_table) +{ + ObRemapTableArray remap_table_array; + ObImportTableItem src_tb1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb1", strlen("tb1")); + ObImportTableItem target_tb1(OB_ORIGIN_AND_SENSITIVE, "db1_bak", strlen("db1_bak"), "tb1_bak", strlen("tb1_bak")); + ObImportTableItem src_tb2(OB_ORIGIN_AND_SENSITIVE, "db2", strlen("db2"), "tb1", strlen("tb1")); + ObImportTableItem target_tb2(OB_ORIGIN_AND_SENSITIVE, "db2_bak", strlen("db2_bak"), "tb1_bak", strlen("tb1_bak")); + ObImportTableItem src_tb3(OB_ORIGIN_AND_SENSITIVE, "db3", strlen("db3"), "tb1,", strlen("tb1,")); + ObImportTableItem target_tb3(OB_ORIGIN_AND_SENSITIVE, "db3_bak", strlen("db3_bak"), "tb1,_bak", strlen("tb1,_bak")); + + ObRemapTableItem remap_tb1; + remap_tb1.src_ = src_tb1; + remap_tb1.target_ = target_tb1; + + ObRemapTableItem remap_tb2; + remap_tb2.src_ = src_tb2; + remap_tb2.target_ = target_tb2; + + ObRemapTableItem remap_tb3; + remap_tb3.src_ = src_tb3; + remap_tb3.target_ = target_tb3; + + ASSERT_EQ(OB_SUCCESS, remap_table_array.add_item(remap_tb1)); + ASSERT_EQ(OB_SUCCESS, remap_table_array.add_item(remap_tb2)); + ASSERT_EQ(OB_SUCCESS, remap_table_array.add_item(remap_tb3)); + + const char *format_str = "`db1`.`tb1`:`db1_bak`.`tb1_bak`,`db2`.`tb1`:`db2_bak`.`tb1_bak`,`db3`.`tb1,`:`db3_bak`.`tb1,_bak`"; + char format_buff[1024]; + memset(format_buff, 0, 1024); + int64_t pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_table_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + char hex_format_buff[1024]; + memset(hex_format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_table_array.hex_format_serialize(hex_format_buff, 1024, pos)); + ASSERT_EQ(pos, remap_table_array.get_hex_format_serialize_size()); + + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_table_array.hex_format_deserialize(hex_format_buff, remap_table_array.get_hex_format_serialize_size(), pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_table_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + pos = 0; + char buff[1024]; + ASSERT_EQ(OB_SUCCESS, remap_table_array.serialize(buff, 1024, pos)); + + pos = 0; + ObRemapTableArray remap_table_array_bak; + ASSERT_EQ(OB_SUCCESS, remap_table_array_bak.deserialize(buff, 1024, pos)); + pos = 0; + memset(format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, remap_table_array_bak.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_buff)); +} + + + +TEST_F(ImportTableTest, test_remap_partition) +{ + ObRemapPartitionArray remap_part_array; + ObImportPartitionItem src_part1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb1", strlen("tb1"), "p1", strlen("p1")); + ObImportTableItem target_tb1(OB_ORIGIN_AND_SENSITIVE, "db1_bak", strlen("db1_bak"), "tb1_p1", strlen("tb1_p1")); + ObImportPartitionItem src_part2(OB_ORIGIN_AND_SENSITIVE, "db2", strlen("db2"), "tb1", strlen("tb1"), "p1", strlen("p1")); + ObImportTableItem target_tb2(OB_ORIGIN_AND_SENSITIVE, "db2_bak", strlen("db2_bak"), "tb1_p1", strlen("tb1_p1")); + ObImportPartitionItem src_part3(OB_ORIGIN_AND_SENSITIVE, "db3", strlen("db3"), "tb1,", strlen("tb1,"), "p1", strlen("p1")); + ObImportTableItem target_tb3(OB_ORIGIN_AND_SENSITIVE, "db3_bak", strlen("db3_bak"), "tb1,_p1", strlen("tb1,_p1")); + + ObRemapPartitionItem remap_p1; + remap_p1.src_ = src_part1; + remap_p1.target_ = target_tb1; + + ObRemapPartitionItem remap_p2; + remap_p2.src_ = src_part2; + remap_p2.target_ = target_tb2; + + ObRemapPartitionItem remap_p3; + remap_p3.src_ = src_part3; + remap_p3.target_ = target_tb3; + + ASSERT_EQ(OB_SUCCESS, remap_part_array.add_item(remap_p1)); + ASSERT_EQ(OB_SUCCESS, remap_part_array.add_item(remap_p2)); + ASSERT_EQ(OB_SUCCESS, remap_part_array.add_item(remap_p3)); + + const char *format_str = "`db1`.`tb1`:`p1`:`db1_bak`.`tb1_p1`,`db2`.`tb1`:`p1`:`db2_bak`.`tb1_p1`,`db3`.`tb1,`:`p1`:`db3_bak`.`tb1,_p1`"; + char format_buff[1024]; + memset(format_buff, 0, 1024); + int64_t pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_part_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + char hex_format_buff[1024]; + memset(hex_format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_part_array.hex_format_serialize(hex_format_buff, 1024, pos)); + ASSERT_EQ(pos, remap_part_array.get_hex_format_serialize_size()); + + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_part_array.hex_format_deserialize(hex_format_buff, remap_part_array.get_hex_format_serialize_size(), pos)); + memset(format_buff, 0, 1024); + pos = 0; + ASSERT_EQ(OB_SUCCESS, remap_part_array.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_str)); + + pos = 0; + char buff[1024]; + ASSERT_EQ(OB_SUCCESS, remap_part_array.serialize(buff, 1024, pos)); + + pos = 0; + ObRemapPartitionArray remap_part_array_bak; + ASSERT_EQ(OB_SUCCESS, remap_part_array_bak.deserialize(buff, 1024, pos)); + pos = 0; + memset(format_buff, 0, 1024); + ASSERT_EQ(OB_SUCCESS, remap_part_array_bak.format_serialize(format_buff, 1024, pos)); + ASSERT_EQ(0, strcmp(format_buff, format_buff)); +} + + +TEST_F(ImportTableTest, test_import_conflict) +{ + ObImportArg arg; + ObImportDatabaseItem db1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem db2(OB_ORIGIN_AND_SENSITIVE, "db2", strlen("db2")); + ObImportDatabaseItem db3(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1")); + + ASSERT_EQ(OB_SUCCESS, arg.add_import_database(db1)); + ASSERT_EQ(OB_SUCCESS, arg.add_import_database(db2)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_import_database(db3)); + + ObImportTableItem tb1(OB_ORIGIN_AND_SENSITIVE, "db3", strlen("db3"), "tb1", strlen("tb1")); + ObImportTableItem tb2(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb2", strlen("tb2")); + ASSERT_EQ(OB_SUCCESS, arg.add_import_table(tb1)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_import_table(tb2)); +} + +TEST_F(ImportTableTest, test_remap_conflict) +{ + ObImportArg arg; + ObImportDatabaseItem src_db1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem target_db1(OB_ORIGIN_AND_SENSITIVE, "db1_bak", strlen("db1_bak")); + ObImportDatabaseItem src_db2(OB_ORIGIN_AND_SENSITIVE, "db2", strlen("db2")); + ObImportDatabaseItem target_db2(OB_ORIGIN_AND_SENSITIVE, "db2_bak", strlen("db2_bak")); + ObImportDatabaseItem src_db3(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem target_db3(OB_ORIGIN_AND_SENSITIVE, "db3_bak", strlen("db3_bak")); + ObImportDatabaseItem src_db4(OB_ORIGIN_AND_SENSITIVE, "db4", strlen("db4")); + ObImportDatabaseItem target_db4(OB_ORIGIN_AND_SENSITIVE, "db2_bak", strlen("db2_bak")); + + ObRemapDatabaseItem remap_db1; + remap_db1.src_ = src_db1; + remap_db1.target_ = target_db1; + + ObRemapDatabaseItem remap_db2; + remap_db2.src_ = src_db2; + remap_db2.target_ = target_db2; + + ObRemapDatabaseItem remap_db3; + remap_db3.src_ = src_db3; + remap_db3.target_ = target_db3; + + ObRemapDatabaseItem remap_db4; + remap_db4.src_ = src_db4; + remap_db4.target_ = target_db4; + + ASSERT_EQ(OB_SUCCESS, arg.add_remap_database(remap_db1)); + ASSERT_EQ(OB_SUCCESS, arg.add_remap_database(remap_db2)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_remap_database(remap_db3)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_remap_database(remap_db4)); + + + ObImportTableItem src_tb1(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb1", strlen("tb1")); + ObImportTableItem target_tb1(OB_ORIGIN_AND_SENSITIVE, "db1_bak", strlen("db1_bak"), "tb1_bak", strlen("tb1_bak")); + ObImportTableItem src_tb2(OB_ORIGIN_AND_SENSITIVE, "db1", strlen("db1"), "tb1", strlen("tb1")); + ObImportTableItem target_tb2(OB_ORIGIN_AND_SENSITIVE, "db2_bak", strlen("db2_bak"), "tb1_bak", strlen("tb1_bak")); + ObImportTableItem src_tb3(OB_ORIGIN_AND_SENSITIVE, "db3", strlen("db3"), "tb1", strlen("tb1")); + ObImportTableItem target_tb3(OB_ORIGIN_AND_SENSITIVE, "db1_bak", strlen("db1_bak"), "tb1_bak", strlen("tb1_bak")); + + ObRemapTableItem remap_tb1; + remap_tb1.src_ = src_tb1; + remap_tb1.target_ = target_tb1; + + ObRemapTableItem remap_tb2; + remap_tb2.src_ = src_tb2; + remap_tb2.target_ = target_tb2; + + ObRemapTableItem remap_tb3; + remap_tb3.src_ = src_tb3; + remap_tb3.target_ = target_tb3; + + ASSERT_EQ(OB_SUCCESS, arg.add_remap_table(remap_tb1)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_remap_table(remap_tb2)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_remap_table(remap_tb3)); +} + +TEST_F(ImportTableTest, test_namecase_mode) +{ + ObImportArg arg; + ObImportDatabaseItem db1(OB_LOWERCASE_AND_INSENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem db2(OB_LOWERCASE_AND_INSENSITIVE, "db2", strlen("db2")); + ObImportDatabaseItem db3(OB_LOWERCASE_AND_INSENSITIVE, "DB1", strlen("DB1")); + + ASSERT_EQ(OB_SUCCESS, arg.add_import_database(db1)); + ASSERT_EQ(OB_SUCCESS, arg.add_import_database(db2)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_import_database(db3)); + + ObImportTableItem tb1(OB_LOWERCASE_AND_INSENSITIVE, "Db1", strlen("Db1"), "tb1", strlen("tb1")); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_import_table(tb1)); + + + ObImportDatabaseItem src_db1(OB_LOWERCASE_AND_INSENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem target_db1(OB_LOWERCASE_AND_INSENSITIVE, "db1_bak", strlen("db1_bak")); + ObImportDatabaseItem src_db2(OB_LOWERCASE_AND_INSENSITIVE, "db1", strlen("db1")); + ObImportDatabaseItem target_db2(OB_LOWERCASE_AND_INSENSITIVE, "db1_Bak", strlen("db1_Bak")); + + ObRemapDatabaseItem remap_db1; + remap_db1.src_ = src_db1; + remap_db1.target_ = target_db1; + + ObRemapDatabaseItem remap_db2; + remap_db2.src_ = src_db2; + remap_db2.target_ = target_db2; + + ASSERT_EQ(OB_SUCCESS, arg.add_remap_database(remap_db1)); + ASSERT_EQ(OB_BACKUP_CONFLICT_VALUE, arg.add_remap_database(remap_db2)); +} + + +int main(int argc, char **argv) +{ + system("rm -f test_import_table.log*"); + ObLogger &logger = ObLogger::get_logger(); + logger.set_file_name("test_import_table.log", true); + logger.set_log_level(OB_LOG_LEVEL_DEBUG); + + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} \ No newline at end of file