提交 5c15eecc 编写于 作者: G godyangfight 提交者: wangzelin.wzl

4.0 ls migration handler should make sure migration task finish before ls destroy

上级 00b7bf04
......@@ -842,7 +842,8 @@ ObIDagNet::ObIDagNet(
type_(type),
add_time_(0),
start_time_(0),
dag_record_map_()
dag_record_map_(),
is_cancel_(false)
{
}
......@@ -851,18 +852,21 @@ int ObIDagNet::add_dag_into_dag_net(ObIDag &dag)
int ret = OB_SUCCESS;
void *buf = nullptr;
WEAK_BARRIER();
const bool is_stopped = is_stopped_;
ObDagRecord *dag_record = nullptr;
int hash_ret = OB_SUCCESS;
ObMutexGuard guard(lock_);
if (OB_NOT_NULL(dag.get_dag_net())) {
ret = OB_INVALID_ARGUMENT;
COMMON_LOG(WARN, "dag already belongs to a dag_net", K(ret), K(dag));
} else if (is_stopped) {
} else if (is_stopped_) {
ret = OB_INNER_STAT_ERROR;
LOG_WARN("dag_net is in stop state, not allowed to add dag", K(ret), K(is_stopped_));
} else if (is_cancel_) {
ret = OB_CANCELED;
LOG_WARN("dag net is cancel, do not allow to add new dag", K(ret), K(is_cancel_));
} else {
ObMutexGuard guard(lock_);
if (!dag_record_map_.created() && OB_FAIL(dag_record_map_.create(DEFAULT_DAG_BUCKET, "DagRecordMap"))) {
COMMON_LOG(WARN, "failed to create dag record map", K(ret), K(dag));
} else if (OB_HASH_NOT_EXIST != (hash_ret = dag_record_map_.get_refactored(&dag, dag_record))) {
......@@ -1033,6 +1037,18 @@ int ObIDagNet::set_dag_id(const ObDagId &dag_id)
return ret;
}
void ObIDagNet::set_cancel()
{
ObMutexGuard guard(lock_);
is_cancel_ = true;
}
bool ObIDagNet::is_cancel()
{
ObMutexGuard guard(lock_);
return is_cancel_;
}
void ObIDagNet::gene_dag_info(ObDagInfo &info, const char *list_info)
{
ObMutexGuard guard(lock_);
......@@ -2239,10 +2255,6 @@ int ObTenantDagScheduler::deal_with_finish_task(ObITask &task, ObTenantDagWorker
}
}
if (OB_SUCC(ret) && nullptr != erase_dag_net) {
if (OB_TMP_FAIL(erase_dag_net->clear_dag_net_ctx())) {
COMMON_LOG(WARN, "failed to clear dag net ctx", K(tmp_ret), KPC(erase_dag_net));
}
if (OB_FAIL(finish_dag_net(erase_dag_net))) {
COMMON_LOG(WARN, "failed to finish dag net", K(ret));
}
......@@ -2291,7 +2303,12 @@ int ObTenantDagScheduler::finish_task_in_dag(ObITask &task, ObIDag &dag, ObIDagN
int ObTenantDagScheduler::finish_dag_net(ObIDagNet *dag_net)
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
if (OB_NOT_NULL(dag_net)) {
if (OB_TMP_FAIL(dag_net->clear_dag_net_ctx())) {
COMMON_LOG(WARN, "failed to clear dag net ctx", K(tmp_ret), KPC(dag_net));
}
{
ObMutexGuard guard(dag_net_map_lock_);
if (OB_FAIL(dag_net_map_[RUNNING_DAG_NET_MAP].erase_refactored(dag_net))) {
......@@ -2658,7 +2675,10 @@ int ObTenantDagScheduler::pop_task_from_ready_list(
move_dag_to_waiting_list = true;
} else if (ObIDag::DAG_STATUS_READY == dag_status
|| ObIDag::DAG_STATUS_RETRY == dag_status) { // first schedule this dag
if (!cur->check_can_schedule()) { // cur dag can't be scheduled now
ObIDag::ObDagStatus next_dag_status = dag_status;
if (OB_NOT_NULL(cur->get_dag_net()) && cur->get_dag_net()->is_cancel()) {
next_dag_status = ObIDag::DAG_STATUS_NODE_FAILED;
} else if (!cur->check_can_schedule()) { // cur dag can't be scheduled now
move_dag_to_waiting_list = true;
} else { // dag can be scheduled
if (ObIDag::DAG_STATUS_READY == dag_status) {
......@@ -2669,11 +2689,12 @@ int ObTenantDagScheduler::pop_task_from_ready_list(
LOG_WARN("failed to generate next dag", K(ret), K(cur));
}
}
cur->set_dag_status(ObIDag::DAG_STATUS_NODE_RUNNING);
cur->update_status_in_dag_net();
next_dag_status = ObIDag::DAG_STATUS_NODE_RUNNING;
}
cur->set_dag_status(next_dag_status);
cur->update_status_in_dag_net();
cur->start_time_ = ObTimeUtility::current_time(); // dag start running
COMMON_LOG(DEBUG, "dag start running", K(ret), KP(cur));
COMMON_LOG(DEBUG, "dag start running", K(ret), KPC(cur));
} else if (ObIDag::DAG_STATUS_NODE_FAILED == dag_status
&& 0 == cur->get_running_task_count()) { // no task running failed dag, need free
tmp_dag = cur;
......@@ -3078,6 +3099,7 @@ int ObTenantDagScheduler::check_dag_net_exist(
{
int ret = OB_SUCCESS;
const ObIDagNet *dag_net = nullptr;
ObMutexGuard guard(dag_net_map_lock_);
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
......@@ -3223,6 +3245,47 @@ int ObTenantDagScheduler::try_move_child_to_ready_list(
return ret;
}
int ObTenantDagScheduler::cancel_dag_net(const ObDagId &dag_id)
{
int ret = OB_SUCCESS;
const ObIDagNet *dag_net_key = nullptr;
ObIDagNet *dag_net = nullptr;
ObArray<ObIDag*> dag_array;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
COMMON_LOG(WARN, "ObTenantDagScheduler is not inited", K(ret));
} else if (dag_id.is_invalid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("cancel dag net get invalid argument", K(ret), K(dag_id));
} else {
{
ObMutexGuard dag_net_guard(dag_net_map_lock_);
if (OB_FAIL(dag_net_id_map_.get_refactored(dag_id, dag_net_key))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("failed to get dag id from dag net", K(ret), K(dag_id));
}
} else if (OB_ISNULL(dag_net_key)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("dag net key should not be NULL", K(ret), K(dag_id), KP(dag_net));
} else if (OB_FAIL(dag_net_map_[RUNNING_DAG_NET_MAP].get_refactored(dag_net_key, dag_net))) {
LOG_WARN("failed to get dag net", K(ret), KPC(dag_net_key));
} else {
dag_net->set_cancel();
if (OB_FAIL(dag_net->deal_with_cancel())) {
LOG_WARN("failed to deal with cancel", K(ret), KPC(dag_net));
}
}
}
if (OB_SUCC(ret)) {
notify();
}
}
return ret;
}
int ObFakeTask::process()
{
......
......@@ -460,6 +460,12 @@ public:
{
return OB_SUCCESS;
}
void set_cancel();
bool is_cancel();
virtual int deal_with_cancel()
{
return OB_SUCCESS;
}
public:
friend class ObTenantDagScheduler;
......@@ -484,6 +490,7 @@ private:
int64_t start_time_;
DagRecordMap dag_record_map_;
ObDagId dag_id_;
bool is_cancel_;
};
struct ObDagInfo
......@@ -793,6 +800,8 @@ public:
int check_ls_compaction_dag_exist(const ObLSID &ls_id, bool &exist);
int check_dag_net_exist(
const ObDagId &dag_id, bool &exist);
int cancel_dag_net(const ObDagId &dag_id);
private:
typedef common::ObDList<ObIDag> DagList;
typedef common::ObDList<ObIDagNet> DagNetList;
......@@ -918,7 +927,6 @@ private:
int tg_id_;
};
// ATTENTION! when alloc task success, the task is already added into task_list_!!!
template <typename T>
int ObIDag::alloc_task(T *&task)
......
......@@ -326,7 +326,11 @@ int ObLSCompleteMigrationDagNet::update_migration_status_(ObLS *ls)
ObMigrationStatus current_migration_status = ObMigrationStatus::OB_MIGRATION_STATUS_MAX;
ObMigrationStatus new_migration_status = ObMigrationStatus::OB_MIGRATION_STATUS_MAX;
if (scheduler->has_set_stop()) {
if (ls->is_stopped()) {
ret = OB_NOT_RUNNING;
LOG_WARN("ls is not running, stop migration dag net", K(ret), K(ctx_));
break;
} else if (scheduler->has_set_stop()) {
ret = OB_SERVER_IS_STOPPING;
LOG_WARN("tenant dag scheduler has set stop, stop migration dag net", K(ret), K(ctx_));
break;
......@@ -370,6 +374,21 @@ int ObLSCompleteMigrationDagNet::update_migration_status_(ObLS *ls)
return ret;
}
int ObLSCompleteMigrationDagNet::deal_with_cancel()
{
int ret = OB_SUCCESS;
const int32_t result = OB_CANCELED;
const bool need_retry = false;
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("ls complete migration dag net do not init", K(ret));
} else if (OB_FAIL(ctx_.set_result(result, need_retry))) {
LOG_WARN("failed to set result", K(ret), KPC(this));
}
return ret;
}
/******************ObCompleteMigrationDag*********************/
ObCompleteMigrationDag::ObCompleteMigrationDag(const ObStorageHADagType sub_type)
: ObStorageHADag(ObDagType::DAG_TYPE_MIGRATE, sub_type)
......@@ -905,6 +924,9 @@ int ObStartCompleteMigrationTask::wait_log_sync_()
if (ctx_->is_failed()) {
ret = OB_CANCELED;
STORAGE_LOG(WARN, "group task has error, cancel subtask", K(ret));
} else if (ls->is_stopped()) {
ret = OB_NOT_RUNNING;
LOG_WARN("ls is not running, stop migration dag net", K(ret), K(ctx_));
} else if (OB_FAIL(SYS_TASK_STATUS_MGR.is_task_cancel(get_dag()->get_dag_id(), is_cancel))) {
STORAGE_LOG(ERROR, "failed to check is task canceled", K(ret), K(*this));
} else if (is_cancel) {
......@@ -1003,6 +1025,9 @@ int ObStartCompleteMigrationTask::wait_log_replay_sync_()
if (ctx_->is_failed()) {
ret = OB_CANCELED;
STORAGE_LOG(WARN, "group task has error, cancel subtask", K(ret));
} else if (ls->is_stopped()) {
ret = OB_NOT_RUNNING;
LOG_WARN("ls is not running, stop migration dag net", K(ret), K(ctx_));
} else if (OB_FAIL(SYS_TASK_STATUS_MGR.is_task_cancel(get_dag()->get_dag_id(), is_cancel))) {
STORAGE_LOG(ERROR, "failed to check is task canceled", K(ret), K(*this));
} else if (is_cancel) {
......@@ -1239,7 +1264,10 @@ int ObStartCompleteMigrationTask::check_tablet_ready_(
while (OB_SUCC(ret)) {
ObTabletHandle tablet_handle;
ObTablet *tablet = nullptr;
if (OB_FAIL(SYS_TASK_STATUS_MGR.is_task_cancel(get_dag()->get_dag_id(), is_cancel))) {
if (ls->is_stopped()) {
ret = OB_NOT_RUNNING;
LOG_WARN("ls is not running, stop migration dag net", K(ret), K(ctx_));
} else if (OB_FAIL(SYS_TASK_STATUS_MGR.is_task_cancel(get_dag()->get_dag_id(), is_cancel))) {
STORAGE_LOG(ERROR, "failed to check is task canceled", K(ret), K(*this));
} else if (is_cancel) {
ret = OB_CANCELED;
......@@ -1321,6 +1349,9 @@ int ObStartCompleteMigrationTask::wait_log_replay_to_max_minor_end_scn_()
if (ctx_->is_failed()) {
ret = OB_CANCELED;
STORAGE_LOG(WARN, "ls migration task is failed, cancel wait ls check point ts push", K(ret));
} else if (ls->is_stopped()) {
ret = OB_NOT_RUNNING;
LOG_WARN("ls is not running, stop migration dag net", K(ret), K(ctx_));
} else if (OB_FAIL(SYS_TASK_STATUS_MGR.is_task_cancel(get_dag()->get_dag_id(), is_cancel))) {
STORAGE_LOG(ERROR, "failed to check is task canceled", K(ret), K(*this));
} else if (is_cancel) {
......
......@@ -80,7 +80,8 @@ public:
virtual int64_t hash() const override;
virtual int fill_comment(char *buf, const int64_t buf_len) const override;
virtual int fill_dag_net_key(char *buf, const int64_t buf_len) const override;
virtual int clear_dag_net_ctx();
virtual int clear_dag_net_ctx() override;
virtual int deal_with_cancel() override;
ObLSCompleteMigrationCtx *get_ctx() { return &ctx_; }
const share::ObLSID &get_ls_id() const { return ctx_.arg_.ls_id_; }
......
......@@ -412,6 +412,21 @@ int ObMigrationDagNet::clear_dag_net_ctx()
return ret;
}
int ObMigrationDagNet::deal_with_cancel()
{
int ret = OB_SUCCESS;
const int32_t result = OB_CANCELED;
const bool need_retry = false;
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("ls migration dag net do not init", K(ret));
} else if (OB_FAIL(ctx_->set_result(result, need_retry))) {
LOG_WARN("failed to set result", K(ret), KPC(this));
}
return ret;
}
/******************ObMigrationDag*********************/
ObMigrationDag::ObMigrationDag(const ObStorageHADagType sub_type)
: ObStorageHADag(ObDagType::DAG_TYPE_MIGRATE, sub_type)
......
......@@ -128,7 +128,8 @@ public:
virtual int64_t hash() const override;
virtual int fill_comment(char *buf, const int64_t buf_len) const override;
virtual int fill_dag_net_key(char *buf, const int64_t buf_len) const override;
virtual int clear_dag_net_ctx();
virtual int clear_dag_net_ctx() override;
virtual int deal_with_cancel() override;
ObMigrationCtx *get_migration_ctx() { return ctx_; }
common::ObInOutBandwidthThrottle *get_bandwidth_throttle() { return bandwidth_throttle_; }
......
......@@ -161,7 +161,8 @@ ObLSMigrationHandler::ObLSMigrationHandler()
task_list_(),
lock_(),
status_(ObLSMigrationHandlerStatus::INIT),
result_(OB_SUCCESS)
result_(OB_SUCCESS),
is_stop_(false)
{
}
......@@ -396,6 +397,9 @@ int ObLSMigrationHandler::add_ls_migration_task(
if (!task_list_.empty()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls already has migration task", K(ret), K(task_list_), K(arg), K(task_id));
} else if (is_stop_) {
ret = OB_IN_STOP_STATE;
LOG_WARN("ls migration handler is int stop status", K(ret), K(task_id), K(arg));
} else {
ObLSMigrationTask task;
task.task_id_ = task_id;
......@@ -1152,5 +1156,50 @@ int ObLSMigrationHandler::get_ls_info_(
return ret;
}
void ObLSMigrationHandler::stop()
{
int ret = OB_SUCCESS;
ObTenantDagScheduler *scheduler = nullptr;
common::SpinWLockGuard guard(lock_);
is_stop_ = true;
result_ = OB_SUCCESS != result_ ? result_ : OB_IN_STOP_STATE;
if (task_list_.empty()) {
} else if (task_list_.count() > 1) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("ls migration task count is unexpected", K(ret), K(task_list_));
} else {
ObLSMigrationTask &task = task_list_.at(0);
if (OB_ISNULL(scheduler = MTL(ObTenantDagScheduler*))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("failed to get ObTenantDagScheduler from MTL", K(ret), KPC(ls_));
} else if (OB_FAIL(scheduler->cancel_dag_net(task.task_id_))) {
LOG_ERROR("failed to cancel dag net", K(ret), K(task), KPC(ls_));
}
}
}
int ObLSMigrationHandler::safe_to_destroy(
bool &is_safe_to_destroy)
{
int ret = OB_SUCCESS;
is_safe_to_destroy = false;
ObLSMigrationTask task;
if (OB_FAIL(get_ls_migration_task_(task))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS;
is_safe_to_destroy = true;
} else {
LOG_WARN("failed to get ls migration task", K(ret), KPC(ls_));
}
} else {
is_safe_to_destroy = false;
wakeup_();
}
return ret;
}
}
}
......@@ -82,6 +82,8 @@ public:
int switch_next_stage(const int32_t result);
int check_task_exist(const share::ObTaskId &task_id, bool &is_exist);
void destroy();
void stop();
int safe_to_destroy(bool &is_safe_to_destroy);
private:
void reuse_();
......@@ -144,6 +146,7 @@ private:
common::SpinRWLock lock_;
ObLSMigrationHandlerStatus status_;
int32_t result_;
bool is_stop_;
DISALLOW_COPY_AND_ASSIGN(ObLSMigrationHandler);
};
......
......@@ -286,6 +286,21 @@ int ObLSPrepareMigrationDagNet::clear_dag_net_ctx()
return ret;
}
int ObLSPrepareMigrationDagNet::deal_with_cancel()
{
int ret = OB_SUCCESS;
const int32_t result = OB_CANCELED;
const bool need_retry = false;
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("ls prepare migration dag net do not init", K(ret));
} else if (OB_FAIL(ctx_.set_result(result, need_retry))) {
LOG_WARN("failed to set result", K(ret), KPC(this));
}
return ret;
}
/******************ObPrepareMigrationDag*********************/
ObPrepareMigrationDag::ObPrepareMigrationDag(const ObStorageHADagType sub_type)
: ObStorageHADag(ObDagType::DAG_TYPE_MIGRATE, sub_type)
......@@ -837,7 +852,10 @@ int ObStartPrepareMigrationTask::wait_log_replay_sync_()
while (OB_SUCC(ret) && !wait_log_replay_success) {
if (ctx_->is_failed()) {
ret = OB_CANCELED;
STORAGE_LOG(WARN, "group task has error, cancel subtask", K(ret));
STORAGE_LOG(WARN, "migration task has error, cancel subtask", K(ret));
} else if (ls->is_stopped()) {
ret = OB_NOT_RUNNING;
LOG_WARN("ls is not running, stop migration dag net", K(ret), K(ctx_));
} else if (OB_FAIL(SYS_TASK_STATUS_MGR.is_task_cancel(get_dag()->get_dag_id(), is_cancel))) {
STORAGE_LOG(ERROR, "failed to check is task canceled", K(ret), K(*this));
} else if (is_cancel) {
......@@ -1028,6 +1046,9 @@ int ObStartPrepareMigrationTask::wait_ls_checkpoint_ts_push_()
if (ctx_->is_failed()) {
ret = OB_CANCELED;
STORAGE_LOG(WARN, "ls migration task is failed, cancel wait ls check point ts push", K(ret));
} else if (ls->is_stopped()) {
ret = OB_NOT_RUNNING;
LOG_WARN("ls is not running, stop migration dag net", K(ret), K(ctx_));
} else if (OB_FAIL(SYS_TASK_STATUS_MGR.is_task_cancel(get_dag()->get_dag_id(), is_cancel))) {
STORAGE_LOG(ERROR, "failed to check is task canceled", K(ret), K(*this));
} else if (is_cancel) {
......
......@@ -80,7 +80,8 @@ public:
virtual int64_t hash() const override;
virtual int fill_comment(char *buf, const int64_t buf_len) const override;
virtual int fill_dag_net_key(char *buf, const int64_t buf_len) const override;
virtual int clear_dag_net_ctx();
virtual int clear_dag_net_ctx() override;
virtual int deal_with_cancel() override;
ObLSPrepareMigrationCtx *get_ctx() { return &ctx_; }
const share::ObLSID &get_ls_id() const { return ctx_.arg_.ls_id_; }
......
......@@ -175,7 +175,7 @@ void ObLSRebuildCbImpl::wakeup_ha_service_()
int ret = OB_SUCCESS;
ObStorageHAService *ha_service = nullptr;
if (OB_ISNULL(ha_service = (MTL(ObStorageHAService *)))) {
if (OB_ISNULL(ha_service = (MTL(ObStorageHAService *)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls service should not be NULL", K(ret), KP(ha_service));
} else {
......
......@@ -417,6 +417,21 @@ int ObLSRestoreDagNet::report_result_()
return ret;
}
int ObLSRestoreDagNet::deal_with_cancel()
{
int ret = OB_SUCCESS;
const int32_t result = OB_CANCELED;
const bool need_retry = false;
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("ls restore dag net do not init", K(ret));
} else if (OB_FAIL(ctx_->set_result(result, need_retry))) {
LOG_WARN("failed to set result", K(ret), KPC(this));
}
return ret;
}
/******************ObLSRestoreDag*********************/
ObLSRestoreDag::ObLSRestoreDag(const ObStorageHADagType sub_type)
: ObStorageHADag(ObDagType::DAG_TYPE_RESTORE, sub_type)
......
......@@ -95,7 +95,8 @@ public:
virtual int64_t hash() const override;
virtual int fill_comment(char *buf, const int64_t buf_len) const override;
virtual int fill_dag_net_key(char *buf, const int64_t buf_len) const override;
virtual int clear_dag_net_ctx();
virtual int clear_dag_net_ctx() override;
virtual int deal_with_cancel() override;
ObLSRestoreCtx *get_ls_restore_ctx() { return ctx_; }
common::ObInOutBandwidthThrottle *get_bandwidth_throttle() { return bandwidth_throttle_; }
......
......@@ -333,6 +333,7 @@ int ObTabletBackfillTXDag::create_first_task()
int ObTabletBackfillTXDag::generate_next_dag(share::ObIDag *&dag)
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
dag = nullptr;
ObTenantDagScheduler *scheduler = nullptr;
common::ObTabletID next_tablet_id;
......@@ -343,6 +344,11 @@ int ObTabletBackfillTXDag::generate_next_dag(share::ObIDag *&dag)
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("tablet backfill tx dag do not init", K(ret));
} else if (ha_dag_net_ctx_->is_failed()) {
if (OB_SUCCESS != (tmp_ret = ha_dag_net_ctx_->get_result(ret))) {
LOG_WARN("failed to get result", K(tmp_ret), KPC(this));
ret = tmp_ret;
}
} else if (OB_FAIL(backfill_tx_ctx_->get_tablet_id(next_tablet_id))) {
if (OB_ITER_END == ret) {
//do nothing
......
......@@ -489,6 +489,21 @@ int ObTabletGroupRestoreDagNet::report_result_()
return ret;
}
int ObTabletGroupRestoreDagNet::deal_with_cancel()
{
int ret = OB_SUCCESS;
const int32_t result = OB_CANCELED;
const bool need_retry = false;
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("tablet group restore dag net do not init", K(ret));
} else if (OB_FAIL(ctx_->set_result(result, need_retry))) {
LOG_WARN("failed to set result", K(ret), KPC(this));
}
return ret;
}
/******************ObTabletGroupRestoreDag*********************/
ObTabletGroupRestoreDag::ObTabletGroupRestoreDag(const ObStorageHADagType sub_type)
: ObStorageHADag(ObDagType::DAG_TYPE_RESTORE, sub_type)
......
......@@ -136,7 +136,8 @@ public:
virtual int64_t hash() const override;
virtual int fill_comment(char *buf, const int64_t buf_len) const override;
virtual int fill_dag_net_key(char *buf, const int64_t buf_len) const override;
virtual int clear_dag_net_ctx();
virtual int clear_dag_net_ctx() override;
virtual int deal_with_cancel() override;
ObTabletGroupRestoreCtx *get_restore_ctx() { return ctx_; }
common::ObInOutBandwidthThrottle *get_bandwidth_throttle() { return bandwidth_throttle_; }
......
......@@ -421,6 +421,7 @@ int ObLS::stop()
if (OB_FAIL(log_handler_.stop())) {
LOG_WARN("stop log handler failed", K(ret), KPC(this));
}
ls_migration_handler_.stop();
is_stopped_ = true;
if (OB_SUCC(ret)) {
......@@ -457,6 +458,7 @@ bool ObLS::safe_to_destroy()
bool is_safe = false;
bool is_ls_restore_handler_safe = false;
bool is_tablet_service_safe = false;
bool is_ls_migration_handler_safe = false;
if (OB_FAIL(ls_tablet_svr_.safe_to_destroy(is_tablet_service_safe))) {
LOG_WARN("ls tablet service check safe to destroy failed", K(ret), KPC(this));
......@@ -467,6 +469,9 @@ bool ObLS::safe_to_destroy()
} else if (!is_ls_restore_handler_safe) {
} else if (OB_FAIL(log_handler_.safe_to_destroy())) {
LOG_WARN("log_handler_ check safe to destroy failed", K(ret), KPC(this));
} else if (OB_FAIL(ls_migration_handler_.safe_to_destroy(is_ls_migration_handler_safe))) {
LOG_WARN("ls migration handler check safe to destroy failed", K(ret), KPC(this));
} else if (!is_ls_migration_handler_safe) {
} else if (is_tablet_service_safe) {
if (1 == ref_mgr_.get_total_ref_cnt()) { // only has one ref at the safe destroy task
is_safe = true;
......@@ -478,7 +483,8 @@ bool ObLS::safe_to_destroy()
if (!is_safe) {
if (REACH_TIME_INTERVAL(60 * 1000 * 1000)) {
LOG_WARN("this ls is not safe to destroy", K(is_safe), "ls_ref", ref_mgr_.get_total_ref_cnt(),
K(is_tablet_service_safe), K(is_ls_restore_handler_safe), K(ret), KP(this), KPC(this));
K(is_tablet_service_safe), K(is_ls_restore_handler_safe), K(is_ls_migration_handler_safe),
K(ret), KP(this), KPC(this));
ref_mgr_.print();
PRINT_OBJ_LEAK(MTL_ID(), share::LEAK_CHECK_OBJ_LS_HANDLE);
}
......
......@@ -274,6 +274,7 @@ public:
ObTabletHandle &handle) const;
int flush_if_need(const bool need_flush);
bool is_stopped() const { return is_stopped_; }
TO_STRING_KV(K_(ls_meta), K_(log_handler), K_(restore_handler), K_(is_inited), K_(tablet_gc_handler));
private:
......
......@@ -1428,6 +1428,7 @@ TEST_F(TestDagScheduler, test_free_dag_func)
class ObCancelDag : public ObBasicDag
{
public:
ObCancelDag() : can_schedule_(false) {}
virtual int create_first_task() override
{
int ret = OB_SUCCESS;
......@@ -1441,8 +1442,10 @@ public:
}
virtual bool check_can_schedule() override
{
return false;
return can_schedule_;
}
bool can_schedule_;
};
class ObCancelDagNet: public ObFatherDagNet
......@@ -1500,6 +1503,37 @@ TEST_F(TestDagScheduler, test_cancel_dag_func)
EXPECT_EQ(0, ObDagWarningHistoryManager::get_instance().size());
}
TEST_F(TestDagScheduler, test_cancel_dag_net_func)
{
int ret = OB_SUCCESS;
ObTenantDagScheduler *scheduler = MTL(ObTenantDagScheduler*);
ASSERT_TRUE(nullptr != scheduler);
ObCancelDagNet *dag_net = nullptr;
EXPECT_EQ(OB_SUCCESS, scheduler->create_and_add_dag_net(nullptr, dag_net));
while (scheduler->get_cur_dag_cnt() < 3) {
usleep(100);
}
ObArray<ObIDag *> dag_array;
dag_net->get_dag_list(dag_array);
ret = scheduler->cancel_dag_net(dag_net->get_dag_id());
for (int i = 0; i < dag_array.count(); ++i) {
ObCancelDag *dag = static_cast<ObCancelDag *>(dag_array[i]);
dag->can_schedule_ = true;
}
EXPECT_EQ(OB_SUCCESS, ret);
ob_usleep(5000 * 1000);
EXPECT_EQ(true, scheduler->is_empty());
EXPECT_EQ(0, ObDagWarningHistoryManager::get_instance().size());
}
}
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册