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 85704d9a96..64905ea3ab 100644 --- a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h +++ b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h @@ -836,7 +836,6 @@ PCODE_DEF(OB_DDL_CHECK_TABLET_MERGE_STATUS, 0x957) PCODE_DEF(OB_CHECK_MEMTABLE_CNT, 0x958) PCODE_DEF(OB_CHECK_MEDIUM_INFO_LIST_CNT, 0x959) PCODE_DEF(OB_SPLIT_TABLET_DATA_FINISH_REQUEST, 0x960) - PCODE_DEF(OB_MVIEW_COMPLETE_REFRESH, 0x961) PCODE_DEF(OB_CLEAR_TABLET_AUTOINC_SEQ_CACHE, 0x962) //PCODE_DEF(OB_FREEZE_SPLIT_SRC_TABLET, 0x963) diff --git a/src/observer/ob_rpc_processor_simple.cpp b/src/observer/ob_rpc_processor_simple.cpp index c6959fb5e3..8af9c7f442 100644 --- a/src/observer/ob_rpc_processor_simple.cpp +++ b/src/observer/ob_rpc_processor_simple.cpp @@ -560,6 +560,20 @@ int ObRpcBuildDDLSingleReplicaRequestP::process() return ret; } +int ObRpcCheckandCancelDDLComplementDagP::process() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(gctx_.ob_service_)) { + ret = OB_INVALID_ARGUMENT; + LOG_ERROR("invalid argument", K(ret), K(gctx_.ob_service_)); + } else { + bool is_dag_exist = true; + ret = gctx_.ob_service_->check_and_cancel_ddl_complement_data_dag(arg_, is_dag_exist); + result_ = is_dag_exist; + } + return ret; +} + int ObRpcFetchSysLSP::process() { int ret = OB_SUCCESS; diff --git a/src/observer/ob_rpc_processor_simple.h b/src/observer/ob_rpc_processor_simple.h index 022db9957d..b4d4f2f6c2 100644 --- a/src/observer/ob_rpc_processor_simple.h +++ b/src/observer/ob_rpc_processor_simple.h @@ -272,6 +272,7 @@ OB_DEFINE_PROCESSOR_S(Srv, OB_TABLET_LOCATION_BROADCAST, ObTabletLocationReceive OB_DEFINE_PROCESSOR_S(Srv, OB_CANCEL_GATHER_STATS, ObCancelGatherStatsP); OB_DEFINE_PROCESSOR_OBADMIN(Srv, OB_LOG_FORCE_SET_TENANT_LOG_DISK, ObForceSetTenantLogDiskP); OB_DEFINE_PROCESSOR_OBADMIN(Srv, OB_FORCE_DUMP_SERVER_USAGE, ObForceDumpServerUsageP); +OB_DEFINE_PROCESSOR_S(Srv, OB_CHECK_AND_CANCEL_DDL_COMPLEMENT_DAG, ObRpcCheckandCancelDDLComplementDagP); OB_DEFINE_PROCESSOR_S(Srv, OB_KILL_CLIENT_SESSION, ObKillClientSessionP); OB_DEFINE_PROCESSOR_S(Srv, OB_CLIENT_SESSION_CONNECT_TIME, ObClientSessionConnectTimeP); diff --git a/src/observer/ob_service.cpp b/src/observer/ob_service.cpp index a62a7216a9..3cb8ad87a1 100644 --- a/src/observer/ob_service.cpp +++ b/src/observer/ob_service.cpp @@ -2471,10 +2471,7 @@ int ObService::build_ddl_single_replica_request(const ObDDLBuildSingleReplicaReq ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid arguments", K(ret), K(arg)); } else { - if (DDL_DROP_COLUMN == arg.ddl_type_ - || DDL_ADD_COLUMN_OFFLINE == arg.ddl_type_ - || DDL_COLUMN_REDEFINITION == arg.ddl_type_ - || DDL_TABLE_RESTORE == arg.ddl_type_) { + if (is_complement_data_relying_on_dag(ObDDLType(arg.ddl_type_))) { int saved_ret = OB_SUCCESS; ObTenantDagScheduler *dag_scheduler = nullptr; ObComplementDataDag *dag = nullptr; @@ -2519,6 +2516,47 @@ int ObService::build_ddl_single_replica_request(const ObDDLBuildSingleReplicaReq return ret; } +int ObService::check_and_cancel_ddl_complement_data_dag(const ObDDLBuildSingleReplicaRequestArg &arg, bool &is_dag_exist) +{ + int ret = OB_SUCCESS; + is_dag_exist = true; + if (OB_UNLIKELY(!arg.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments", K(ret), K(arg)); + } else if (OB_UNLIKELY(!is_complement_data_relying_on_dag(ObDDLType(arg.ddl_type_)))) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid ddl type", K(ret), K(arg)); + } else { + 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_scheduler->check_dag_exist(dag, is_dag_exist))) { + LOG_WARN("check dag exist failed", K(ret)); + } else if (is_dag_exist && OB_FAIL(dag_scheduler->cancel_dag(dag))) { + // sync to cancel ready dag only, not including running dag. + LOG_WARN("cancel dag failed", K(ret)); + } + if (OB_NOT_NULL(dag)) { + (void) dag->handle_init_failed_ret_code(ret); + dag_scheduler->free_dag(*dag); + dag = nullptr; + } + } + if (REACH_COUNT_INTERVAL(1000L)) { + LOG_INFO("receive cancel ddl complement dag request", K(ret), K(is_dag_exist), K(arg)); + } + return ret; +} + int ObService::inner_fill_tablet_info_( const int64_t tenant_id, const ObTabletID &tablet_id, diff --git a/src/observer/ob_service.h b/src/observer/ob_service.h index 500db4c7d3..18229c6898 100644 --- a/src/observer/ob_service.h +++ b/src/observer/ob_service.h @@ -125,6 +125,7 @@ public: int switch_schema(const obrpc::ObSwitchSchemaArg &arg, obrpc::ObSwitchSchemaResult &result); int calc_column_checksum_request(const obrpc::ObCalcColumnChecksumRequestArg &arg, obrpc::ObCalcColumnChecksumRequestRes &res); int build_ddl_single_replica_request(const obrpc::ObDDLBuildSingleReplicaRequestArg &arg, obrpc::ObDDLBuildSingleReplicaRequestResult &res); + int check_and_cancel_ddl_complement_data_dag(const obrpc::ObDDLBuildSingleReplicaRequestArg &arg, bool &is_dag_exist); int stop_partition_write(const obrpc::Int64 &switchover_timestamp, obrpc::Int64 &result); int check_partition_log(const obrpc::Int64 &switchover_timestamp, obrpc::Int64 &result); int get_wrs_info(const obrpc::ObGetWRSArg &arg, obrpc::ObGetWRSResult &result); diff --git a/src/observer/ob_srv_xlator_partition.cpp b/src/observer/ob_srv_xlator_partition.cpp index a33a7a2804..51b5f82548 100644 --- a/src/observer/ob_srv_xlator_partition.cpp +++ b/src/observer/ob_srv_xlator_partition.cpp @@ -316,4 +316,7 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator *xlator) { RPC_PROCESSOR(ObKillClientSessionP, gctx_); // client session create time RPC_PROCESSOR(ObClientSessionConnectTimeP, gctx_); + + // ddl + RPC_PROCESSOR(ObRpcCheckandCancelDDLComplementDagP, gctx_); } diff --git a/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp b/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp index d47a294146..78bdfd60fd 100644 --- a/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp +++ b/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp @@ -1402,9 +1402,17 @@ int ObDDLRedefinitionTask::finish() int ObDDLRedefinitionTask::fail() { int ret = OB_SUCCESS; + bool all_complement_dag_exit = true; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("ObDDLRedefinitionTask has not been inited", K(ret)); + } else if (is_complement_data_relying_on_dag(task_type_) && + OB_FAIL(check_and_cancel_complement_data_dag(all_complement_dag_exit))) { + LOG_WARN("check and cancel complement data dag failed", K(ret)); + } else if (!all_complement_dag_exit) { + if (REACH_COUNT_INTERVAL(1000L)) { + LOG_INFO("wait all complement data dag exit", K(dst_tenant_id_), K(task_id_)); + } } else if (OB_FAIL(finish())) { LOG_WARN("finish failed", K(ret)); } else { @@ -2847,3 +2855,112 @@ int64_t ObDDLRedefinitionTask::get_build_replica_request_time() TCRLockGuard guard(lock_); return build_replica_request_time_; } + +int ObDDLRedefinitionTask::check_and_cancel_complement_data_dag(bool &all_complement_dag_exit) +{ + int ret = OB_SUCCESS; + all_complement_dag_exit = false; + const bool force_renew = true; + bool is_cache_hit = false; + const int64_t expire_renew_time = force_renew ? INT64_MAX : 0; + share::ObLocationService *location_service = GCTX.location_service_; + ObRootService *root_service = GCTX.root_service_; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not init", K(ret)); + } else if (OB_ISNULL(location_service) || OB_ISNULL(root_service)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), KP(location_service), KP(root_service)); + } else if (OB_UNLIKELY(!check_dag_exit_tablets_map_.created())) { + const int64_t CHECK_DAG_EXIT_BUCKET_NUM = 64; + common::ObArray src_tablet_ids; + common::ObArray dst_tablet_ids; + if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, object_id_, src_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_, dst_tablet_ids))) { + LOG_WARN("fail to get tablets", K(ret), K(dst_tenant_id_), K(target_object_id_)); + } else if (OB_FAIL(check_dag_exit_tablets_map_.create(CHECK_DAG_EXIT_BUCKET_NUM, lib::ObLabel("DDLChkDagMap")))) { + LOG_WARN("create hashset set failed", K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < src_tablet_ids.count(); i++) { + if (OB_FAIL(check_dag_exit_tablets_map_.set_refactored(src_tablet_ids.at(i), dst_tablet_ids.at(i)))) { + LOG_WARN("set refactored failed", K(ret)); + } + } + } + } + if (OB_SUCC(ret)) { + ObAddr unused_leader_addr; + const int64_t timeout_us = ObDDLUtil::get_default_ddl_rpc_timeout(); + common::hash::ObHashMap ::const_iterator iter = + check_dag_exit_tablets_map_.begin(); + ObArray dag_not_exist_tablets; + for (; OB_SUCC(ret) && iter != check_dag_exit_tablets_map_.end(); iter++) { + ObLSID src_ls_id; + ObLSID dst_ls_id; + const common::ObTabletID &src_tablet_id = iter->first; + const common::ObTabletID &dst_tablet_id = iter->second; + int64_t paxos_member_count = 0; + common::ObArray paxos_server_list; + if (OB_FAIL(ObDDLUtil::get_tablet_leader_addr(location_service, tenant_id_, src_tablet_id, timeout_us, src_ls_id, unused_leader_addr))) { + LOG_WARN("get src tablet leader addr failed", K(ret)); + } else if (OB_FAIL(ObDDLUtil::get_tablet_leader_addr(location_service, dst_tenant_id_, dst_tablet_id, timeout_us, dst_ls_id, unused_leader_addr))) { + LOG_WARN("get dst tablet leader addr failed", K(ret)); + } else if (OB_FAIL(ObDDLUtil::get_tablet_paxos_member_list(dst_tenant_id_, dst_tablet_id, paxos_server_list, paxos_member_count))) { + LOG_WARN("get tablet paxos member list failed", K(ret)); + } else { + bool is_dag_exist = false; + obrpc::ObDDLBuildSingleReplicaRequestArg arg; + arg.ls_id_ = src_ls_id; + arg.dest_ls_id_ = dst_ls_id; + arg.tenant_id_ = tenant_id_; + arg.dest_tenant_id_ = dst_tenant_id_; + arg.source_tablet_id_ = src_tablet_id; + arg.dest_tablet_id_ = dst_tablet_id; + arg.source_table_id_ = object_id_; + arg.dest_schema_id_ = target_object_id_; + arg.schema_version_ = schema_version_; + arg.dest_schema_version_ = dst_schema_version_; + arg.snapshot_version_ = snapshot_version_; + arg.ddl_type_ = task_type_; + arg.task_id_ = task_id_; + arg.parallelism_ = 1; // to ensure arg valid only. + arg.execution_id_ = 1; // to ensure arg valid only. + arg.data_format_version_ = 1; // to ensure arg valid only. + arg.tablet_task_id_ = 1; // to ensure arg valid only. + arg.consumer_group_id_ = 0; // to ensure arg valid only. + for (int64_t j = 0; OB_SUCC(ret) && j < paxos_server_list.count(); j++) { + int tmp_ret = OB_SUCCESS; + obrpc::Bool dag_exist_in_current_server(true); + if (OB_TMP_FAIL(root_service->get_rpc_proxy().to(paxos_server_list.at(j)) + .by(dst_tenant_id_).timeout(timeout_us).check_and_cancel_ddl_complement_dag(arg, dag_exist_in_current_server))) { + // consider as dag does not exist in this server. + LOG_WARN("check and cancel ddl complement dag failed", K(ret), K(tmp_ret), K(arg)); + } else if (dag_exist_in_current_server) { + is_dag_exist = true; + if (REACH_COUNT_INTERVAL(1000L)) { + LOG_INFO("wait dag exist", "addr", paxos_server_list.at(j), K(arg)); + } + } + } + if (OB_SUCC(ret) && !is_dag_exist) { + if (OB_FAIL(dag_not_exist_tablets.push_back(src_tablet_id))) { + LOG_WARN("push back failed", K(ret)); + } + } + } + } + if (OB_SUCC(ret)) { + for (int64_t j = 0; OB_SUCC(ret) && j < dag_not_exist_tablets.count(); j++) { + if (OB_FAIL(check_dag_exit_tablets_map_.erase_refactored(dag_not_exist_tablets.at(j)))) { + LOG_WARN("erase failed", K(ret)); + } + } + } + } + if (OB_SUCC(ret) && check_dag_exit_tablets_map_.empty()) { + // all participants have no complement data dag. + all_complement_dag_exit = true; + } + return ret; +} diff --git a/src/rootserver/ddl_task/ob_ddl_redefinition_task.h b/src/rootserver/ddl_task/ob_ddl_redefinition_task.h index 7601bc374f..2971988dc3 100644 --- a/src/rootserver/ddl_task/ob_ddl_redefinition_task.h +++ b/src/rootserver/ddl_task/ob_ddl_redefinition_task.h @@ -170,6 +170,7 @@ 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); + virtual int check_and_cancel_complement_data_dag(bool &all_complement_dag_exit); // wait dag exit before unlock table. virtual int fail(); virtual int success(); int hold_snapshot(const int64_t snapshot_version); @@ -291,6 +292,7 @@ protected: bool is_sstable_complete_task_submitted_; int64_t sstable_complete_request_time_; ObDDLSingleReplicaExecutor replica_builder_; + common::hash::ObHashMap check_dag_exit_tablets_map_; // for dag complement data ddl only. }; } // end namespace rootserver diff --git a/src/rootserver/ddl_task/ob_recover_restore_table_task.cpp b/src/rootserver/ddl_task/ob_recover_restore_table_task.cpp index 83306f19f6..1f334f8957 100755 --- a/src/rootserver/ddl_task/ob_recover_restore_table_task.cpp +++ b/src/rootserver/ddl_task/ob_recover_restore_table_task.cpp @@ -182,11 +182,18 @@ int ObRecoverRestoreTableTask::fail() obrpc::ObDropTableArg drop_table_arg; obrpc::ObDDLRes drop_table_res; bool need_cleanup = true; + bool all_complement_dag_exit = true; { ObSchemaGetterGuard dst_tenant_schema_guard; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret)); + } else if (OB_FAIL(check_and_cancel_complement_data_dag(all_complement_dag_exit))) { + LOG_WARN("check and cancel complement data dag failed", K(ret)); + } else if (!all_complement_dag_exit) { + if (REACH_COUNT_INTERVAL(1000L)) { + LOG_INFO("wait all complement data dag exit", K(dst_tenant_id_), K(task_id_)); + } } else if (OB_ISNULL(root_service)) { ret = OB_ERR_SYS; LOG_WARN("error sys, root service must not be nullptr", K(ret)); diff --git a/src/share/ob_ddl_common.h b/src/share/ob_ddl_common.h index 65c831d729..11c790ef48 100644 --- a/src/share/ob_ddl_common.h +++ b/src/share/ob_ddl_common.h @@ -285,6 +285,14 @@ static inline bool is_direct_load_task(const ObDDLType type) return DDL_DIRECT_LOAD == type || DDL_DIRECT_LOAD_INSERT == type; } +static inline bool is_complement_data_relying_on_dag(const ObDDLType type) +{ + return DDL_DROP_COLUMN == type + || DDL_ADD_COLUMN_OFFLINE == type + || DDL_COLUMN_REDEFINITION == type + || DDL_TABLE_RESTORE == type; +} + static inline bool is_invalid_ddl_type(const ObDDLType type) { return DDL_INVALID == type; diff --git a/src/share/ob_debug_sync_point.h b/src/share/ob_debug_sync_point.h index 67aaa48b5e..946b36113b 100755 --- a/src/share/ob_debug_sync_point.h +++ b/src/share/ob_debug_sync_point.h @@ -594,6 +594,9 @@ class ObString; ACT(HANG_IN_CLONE_SYS_FAILED_STATUS,)\ ACT(BEFORE_BACKUP_PREFETCH_TASK,)\ ACT(BEFORE_BACKUP_DATA_TASK,)\ + ACT(HOLD_DDL_COMPLEMENT_DAG_WHEN_APPEND_ROW,)\ + ACT(HOLD_DDL_COMPLEMENT_DAG_BEFORE_REPORT_FINISH,)\ + ACT(HOLD_DDL_COMPLEMENT_DAG_AFTER_REPORT_FINISH,)\ ACT(MAX_DEBUG_SYNC_POINT,) DECLARE_ENUM(ObDebugSyncPoint, debug_sync_point, OB_DEBUG_SYNC_POINT_DEF); diff --git a/src/share/ob_srv_rpc_proxy.h b/src/share/ob_srv_rpc_proxy.h index ca2ade00c3..4e6d5cf9f3 100755 --- a/src/share/ob_srv_rpc_proxy.h +++ b/src/share/ob_srv_rpc_proxy.h @@ -142,6 +142,7 @@ public: RPC_S(PR5 force_set_server_list, OB_FORCE_SET_SERVER_LIST, (ObForceSetServerListArg)); RPC_S(PR5 calc_column_checksum_request, OB_CALC_COLUMN_CHECKSUM_REQUEST, (ObCalcColumnChecksumRequestArg), obrpc::ObCalcColumnChecksumRequestRes); RPC_AP(PR5 build_ddl_single_replica_request, OB_DDL_BUILD_SINGLE_REPLICA_REQUEST, (obrpc::ObDDLBuildSingleReplicaRequestArg), obrpc::ObDDLBuildSingleReplicaRequestResult); + RPC_S(PR5 check_and_cancel_ddl_complement_dag, OB_CHECK_AND_CANCEL_DDL_COMPLEMENT_DAG, (ObDDLBuildSingleReplicaRequestArg), Bool); RPC_S(PR5 fetch_tablet_autoinc_seq_cache, OB_FETCH_TABLET_AUTOINC_SEQ_CACHE, (obrpc::ObFetchTabletSeqArg), obrpc::ObFetchTabletSeqRes); RPC_AP(PR5 batch_get_tablet_autoinc_seq, OB_BATCH_GET_TABLET_AUTOINC_SEQ, (obrpc::ObBatchGetTabletAutoincSeqArg), obrpc::ObBatchGetTabletAutoincSeqRes); RPC_AP(PR5 batch_set_tablet_autoinc_seq, OB_BATCH_SET_TABLET_AUTOINC_SEQ, (obrpc::ObBatchSetTabletAutoincSeqArg), obrpc::ObBatchSetTabletAutoincSeqRes); diff --git a/src/storage/ddl/ob_complement_data_task.cpp b/src/storage/ddl/ob_complement_data_task.cpp index 11eea6720c..a65cdfe1da 100644 --- a/src/storage/ddl/ob_complement_data_task.cpp +++ b/src/storage/ddl/ob_complement_data_task.cpp @@ -706,6 +706,7 @@ int ObComplementDataDag::report_replica_build_status() LOG_WARN("fail to send build ddl single replica response", K(ret), K(arg)); } } + DEBUG_SYNC(HOLD_DDL_COMPLEMENT_DAG_AFTER_REPORT_FINISH); FLOG_INFO("complement data finished", K(ret), K(context_.complement_data_ret_)); return ret; } @@ -748,6 +749,42 @@ int ObComplementDataDag::fill_dag_key(char *buf, const int64_t buf_len) const return ret; } +int ObComplementDataDag::check_and_exit_on_demand() +{ + int ret = OB_SUCCESS; + int tmp_ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("dag has not been initialized", K(ret)); + } else { + DEBUG_SYNC(HOLD_DDL_COMPLEMENT_DAG_WHEN_APPEND_ROW); + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + ObSqlString sql_string; + sqlclient::ObMySQLResult *result = nullptr; + if (OB_TMP_FAIL(sql_string.assign_fmt("SELECT status FROM %s WHERE task_id = %lu", share::OB_ALL_DDL_TASK_STATUS_TNAME, param_.task_id_))) { + LOG_WARN("assign sql string failed", K(tmp_ret), K(param_)); + } else if (OB_TMP_FAIL(GCTX.sql_proxy_->read(res, param_.dest_tenant_id_, sql_string.ptr()))) { + LOG_WARN("fail to execute sql", K(tmp_ret), K(sql_string)); + } else if (OB_ISNULL(result = res.get_result())) { + tmp_ret = OB_ERR_UNEXPECTED; + LOG_WARN("error unexpected, query result must not be NULL", K(tmp_ret)); + } else if (OB_TMP_FAIL(result->next())) { + if (OB_ENTRY_NOT_EXIST == tmp_ret) { + ret = OB_CANCELED; + } + LOG_WARN("iterate next failed", K(ret), K(tmp_ret)); + } else { + int task_status = 0; + EXTRACT_INT_FIELD_MYSQL(*result, "status", task_status, int); + if (OB_SUCC(ret)) { + ret = task_status == ObDDLTaskStatus::REDEFINITION ? ret : OB_CANCELED; + } + } + } + } + return ret; +} + ObComplementPrepareTask::ObComplementPrepareTask() : ObITask(TASK_TYPE_COMPLEMENT_PREPARE), is_inited_(false), param_(nullptr), context_(nullptr) { @@ -1286,6 +1323,9 @@ int ObComplementWriteTask::append_lob( int ObComplementWriteTask::append_row(ObScan *scan) { int ret = OB_SUCCESS; + ObComplementDataDag *current_dag = nullptr; + const int64_t CHECK_DAG_NEED_EXIT_INTERVAL = 10000; // 1w rows. + ObDataStoreDesc data_desc; HEAP_VARS_4((ObMacroBlockWriter, writer), (ObSchemaGetterGuard, schema_guard), (ObRelativeTable, relative_table), @@ -1303,6 +1343,8 @@ int ObComplementWriteTask::append_row(ObScan *scan) int64_t t2 = 0; int64_t t3 = 0; int64_t lob_cnt = 0; + int64_t row_scanned = 0; + int64_t row_inserted = 0; ObArenaAllocator lob_allocator(ObModIds::OB_LOB_ACCESS_BUFFER, OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()); ObStoreRow reshaped_row; reshaped_row.flag_.set_flag(ObDmlFlag::DF_INSERT); @@ -1357,7 +1399,7 @@ int ObComplementWriteTask::append_row(ObScan *scan) LOG_WARN("hidden table key is invalid", K(ret), K(hidden_table_key)); } 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()))) { + } else if (OB_ISNULL(current_dag = static_cast(get_dag()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("the dag of this task is null", K(ret)); } else if (OB_FAIL(tenant_direct_load_mgr->get_tablet_mgr_and_check_major( @@ -1450,7 +1492,9 @@ int ObComplementWriteTask::append_row(ObScan *scan) } else { t2 = ObTimeUtility::current_time(); get_next_row_time += t2 - t1; - context_->row_scanned_++; + if (++row_scanned % 100 == 0) { + (void) ATOMIC_AAF(&context_->row_scanned_, 100); + } if (!ddl_committed && OB_FAIL(writer.append_row(datum_row))) { LOG_WARN("fail to append row to macro block", K(ret), K(datum_row)); } @@ -1474,7 +1518,14 @@ int ObComplementWriteTask::append_row(ObScan *scan) if (OB_SUCC(ret)) { t3 = ObTimeUtility::current_time(); append_row_time += t3 - t2; - context_->row_inserted_++; + if (++row_inserted % 100 == 0) { + (void) ATOMIC_AAF(&context_->row_inserted_, 100); + } + if (row_inserted % CHECK_DAG_NEED_EXIT_INTERVAL == 0) { + if (OB_FAIL(current_dag->check_and_exit_on_demand())) { + LOG_WARN("dag check and exit on demand failed", K(ret)); + } + } } lob_allocator.reuse(); // reuse after append_row to macro block to save memory } @@ -1485,6 +1536,8 @@ int ObComplementWriteTask::append_row(ObScan *scan) LOG_WARN("close lob sstable slice failed", K(ret)); } } + (void) ATOMIC_AAF(&context_->row_scanned_, row_scanned % 100); + (void) ATOMIC_AAF(&context_->row_inserted_, row_inserted % 100); LOG_INFO("print append row to macro block cost time", K(ret), K(task_id_), K(context_->row_inserted_), K(get_next_row_time), K(append_row_time)); ObRowReshapeUtil::free_row_reshape(allocator, reshape_ptr, 1); diff --git a/src/storage/ddl/ob_complement_data_task.h b/src/storage/ddl/ob_complement_data_task.h index b9f8c2bc76..211bd82097 100644 --- a/src/storage/ddl/ob_complement_data_task.h +++ b/src/storage/ddl/ob_complement_data_task.h @@ -185,6 +185,7 @@ public: virtual bool is_ha_dag() const override { return false; } // report replica build status to RS. int report_replica_build_status(); + int check_and_exit_on_demand(); private: bool is_inited_; ObComplementDataParam param_;