[CP] fix trans of ddl filling lob submit after lob tablet dropped.
This commit is contained in:
@ -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)
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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_);
|
||||
}
|
||||
|
||||
@ -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<common::ObTabletID> src_tablet_ids;
|
||||
common::ObArray<common::ObTabletID> 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<common::ObTabletID, common::ObTabletID> ::const_iterator iter =
|
||||
check_dag_exit_tablets_map_.begin();
|
||||
ObArray<common::ObTabletID> 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<ObAddr> 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;
|
||||
}
|
||||
|
||||
@ -170,6 +170,7 @@ protected:
|
||||
const share::schema::ObTableSchema &dest_table_schema,
|
||||
common::hash::ObHashMap<uint64_t, uint64_t> &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<common::ObTabletID, common::ObTabletID> check_dag_exit_tablets_map_; // for dag complement data ddl only.
|
||||
};
|
||||
|
||||
} // end namespace rootserver
|
||||
|
||||
@ -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));
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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<ObComplementDataDag *>(get_dag()))) {
|
||||
} else if (OB_ISNULL(current_dag = static_cast<ObComplementDataDag *>(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);
|
||||
|
||||
@ -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_;
|
||||
|
||||
Reference in New Issue
Block a user