fix backfill false positive after transfer start abort

This commit is contained in:
WenJinyu
2023-08-18 12:43:55 +00:00
committed by ob-robot
parent 0533060191
commit 2252af091a
9 changed files with 243 additions and 152 deletions

View File

@ -943,6 +943,22 @@ int ObStartPrepareMigrationTask::prepare_backfill_tx_tablets_()
return ret;
}
int ObStartPrepareMigrationTask::build_tablet_backfill_info_(common::ObArray<ObTabletBackfillInfo> &tablet_infos)
{
int ret = OB_SUCCESS;
ObTabletBackfillInfo tablet_info;
for (int64_t i = 0; OB_SUCC(ret) && i < ctx_->tablet_id_array_.count(); ++i) {
ObTabletID tablet_id = ctx_->tablet_id_array_.at(i);
if (OB_FAIL(tablet_info.init(tablet_id, true/*is_committed*/))) {
LOG_WARN("failed to init tablet info", K(ret), K(tablet_id));
} else if (OB_FAIL(tablet_infos.push_back(tablet_info))) {
LOG_WARN("failed to push tablet info into array", K(ret), K(tablet_info));
}
}
return ret;
}
int ObStartPrepareMigrationTask::generate_prepare_migration_dags_()
{
int ret = OB_SUCCESS;
@ -952,11 +968,11 @@ int ObStartPrepareMigrationTask::generate_prepare_migration_dags_()
ObTenantDagScheduler *scheduler = nullptr;
ObIDagNet *dag_net = nullptr;
ObBackfillTXCtx *backfill_tx_ctx = nullptr;
ObTabletID tablet_id;
ObStartPrepareMigrationDag *start_prepare_migration_dag = nullptr;
ObLSHandle ls_handle;
ObLS *ls = nullptr;
common::ObArray<ObTabletBackfillInfo> tablet_infos;
storage::ObTabletBackfillInfo tablet_info;
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("start prepare migration task do not init", K(ret));
@ -979,7 +995,9 @@ int ObStartPrepareMigrationTask::generate_prepare_migration_dags_()
} else {
if (OB_FAIL(scheduler->alloc_dag(finish_backfill_tx_dag))) {
LOG_WARN("failed to alloc finish backfill tx migration dag ", K(ret));
} else if (OB_FAIL(finish_backfill_tx_dag->init(ctx_->task_id_, ctx_->arg_.ls_id_, ctx_->log_sync_scn_, ctx_->tablet_id_array_, ctx_))) {
} else if (OB_FAIL(build_tablet_backfill_info_(tablet_infos))) {
LOG_WARN("failed to build tablet backfill info", K(ret));
} else if (OB_FAIL(finish_backfill_tx_dag->init(ctx_->task_id_, ctx_->arg_.ls_id_, ctx_->log_sync_scn_, tablet_infos, ctx_))) {
LOG_WARN("failed to init data tablets migration dag", K(ret), K(*ctx_));
} else if (OB_ISNULL(backfill_tx_ctx = finish_backfill_tx_dag->get_backfill_tx_ctx())) {
ret = OB_ERR_UNEXPECTED;
@ -989,11 +1007,11 @@ int ObStartPrepareMigrationTask::generate_prepare_migration_dags_()
LOG_WARN("failed to add finish backfill tx dag as chilid", K(ret), K(*ctx_));
}
} else {
if (OB_FAIL(backfill_tx_ctx->get_tablet_id(tablet_id))) {
if (OB_FAIL(backfill_tx_ctx->get_tablet_info(tablet_info))) {
LOG_WARN("failed to get tablet id", K(ret), KPC(ctx_));
} else if (OB_FAIL(scheduler->alloc_dag(tablet_backfill_tx_dag))) {
LOG_WARN("failed to alloc tablet backfill tx migration dag ", K(ret));
} else if (OB_FAIL(tablet_backfill_tx_dag->init(ctx_->task_id_, ctx_->arg_.ls_id_, tablet_id, ctx_, backfill_tx_ctx))) {
} else if (OB_FAIL(tablet_backfill_tx_dag->init(ctx_->task_id_, ctx_->arg_.ls_id_, tablet_info, ctx_, backfill_tx_ctx))) {
LOG_WARN("failed to init tablet backfill tx dag", K(ret), K(*ctx_));
} else if (OB_FAIL(this->get_dag()->add_child(*tablet_backfill_tx_dag))) {
LOG_WARN("failed to add tablet backfill tx dag as chilid", K(ret), K(*ctx_));

View File

@ -180,6 +180,7 @@ private:
int wait_transfer_tablets_ready_();
int wait_transfer_out_tablet_ready_(
ObLS *ls, ObTablet *tablet);
int build_tablet_backfill_info_(common::ObArray<ObTabletBackfillInfo> &tablet_infos);
private:
bool is_inited_;

View File

@ -1267,6 +1267,48 @@ bool ObLSRebuildInfo::operator ==(const ObLSRebuildInfo &other) const
OB_SERIALIZE_MEMBER(ObLSRebuildInfo, status_, type_);
ObTabletBackfillInfo::ObTabletBackfillInfo()
: tablet_id_(),
is_committed_(false)
{}
int ObTabletBackfillInfo::init(const common::ObTabletID &tablet_id, bool is_committed)
{
int ret = OB_SUCCESS;
if (!tablet_id.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tablet it", K(ret), K(tablet_id));
} else {
tablet_id_ = tablet_id;
is_committed_ = is_committed;
}
return ret;
}
void ObTabletBackfillInfo::reset()
{
tablet_id_.reset();
is_committed_ = false;
}
bool ObTabletBackfillInfo::is_valid() const
{
return tablet_id_.is_valid();
}
bool ObTabletBackfillInfo::operator == (const ObTabletBackfillInfo &other) const
{
bool is_same = true;
if (this == &other) {
// same
} else if (tablet_id_ != other.tablet_id_
|| is_committed_ != other.is_committed_) {
is_same = false;
} else {
is_same = true;
}
return is_same;
}
}
}

View File

@ -405,6 +405,22 @@ public:
ObLSRebuildType type_;
};
struct ObTabletBackfillInfo final
{
OB_UNIS_VERSION(1);
public:
ObTabletBackfillInfo();
virtual ~ObTabletBackfillInfo() = default;
int init(const common::ObTabletID &tablet_id, bool is_committed);
bool is_valid() const;
void reset();
bool operator == (const ObTabletBackfillInfo &other) const;
TO_STRING_KV(
K_(tablet_id),
K_(is_committed));
common::ObTabletID tablet_id_;
bool is_committed_;
};
}

View File

@ -30,8 +30,8 @@ ObBackfillTXCtx::ObBackfillTXCtx()
ls_id_(),
log_sync_scn_(SCN::min_scn()),
lock_(),
tablet_id_index_(0),
tablet_id_array_()
tablet_info_index_(0),
tablet_info_array_()
{
}
@ -43,8 +43,8 @@ void ObBackfillTXCtx::reset()
{
task_id_.reset();
ls_id_.reset();
tablet_id_index_ = 0;
tablet_id_array_.reset();
tablet_info_index_ = 0;
tablet_info_array_.reset();
}
bool ObBackfillTXCtx::is_valid() const
@ -56,29 +56,29 @@ bool ObBackfillTXCtx::is_valid() const
bool ObBackfillTXCtx::inner_is_valid_() const
{
return !task_id_.is_invalid() && ls_id_.is_valid()
&& tablet_id_index_ >= 0 && !tablet_id_array_.empty()
&& tablet_id_index_ <= tablet_id_array_.count();
&& tablet_info_index_ >= 0 && !tablet_info_array_.empty()
&& tablet_info_index_ <= tablet_info_array_.count();
}
int ObBackfillTXCtx::get_tablet_id(ObTabletID &tablet_id)
int ObBackfillTXCtx::get_tablet_info(ObTabletBackfillInfo &tablet_info)
{
int ret = OB_SUCCESS;
tablet_id.reset();
tablet_info.reset();
common::SpinWLockGuard guard(lock_);
if (!inner_is_valid_()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("backfill tx ctx is invalid", K(ret), K(*this));
} else {
if (tablet_id_index_ > tablet_id_array_.count()) {
if (tablet_info_index_ > tablet_info_array_.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet id index should not bigger than tablet id array count",
K(ret), K(tablet_id_index_), K(tablet_id_array_));
} else if (tablet_id_index_ == tablet_id_array_.count()) {
LOG_WARN("tablet info index should not bigger than tablet info array count",
K(ret), K(tablet_info_index_), K(tablet_info_array_));
} else if (tablet_info_index_ == tablet_info_array_.count()) {
ret = OB_ITER_END;
} else {
tablet_id = tablet_id_array_.at(tablet_id_index_);
tablet_id_index_++;
tablet_info = tablet_info_array_.at(tablet_info_index_);
tablet_info_index_++;
}
}
return ret;
@ -88,24 +88,24 @@ int ObBackfillTXCtx::build_backfill_tx_ctx(
const share::ObTaskId &task_id,
const share::ObLSID &ls_id,
const SCN log_sync_scn,
const common::ObIArray<common::ObTabletID> &tablet_id_array)
const common::ObIArray<ObTabletBackfillInfo> &tablet_info_array)
{
int ret = OB_SUCCESS;
common::SpinWLockGuard guard(lock_);
if (!tablet_id_array_.empty()) {
if (!tablet_info_array_.empty()) {
ret = OB_INIT_TWICE;
LOG_WARN("backfill tx ctx init twice", K(ret), KPC(this));
} else if (task_id.is_invalid() || !ls_id.is_valid() || !log_sync_scn.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("build backfill tx ctx get invalid argument", K(ret), K(task_id), K(ls_id),
K(log_sync_scn), K(tablet_id_array));
} else if (!tablet_id_array.empty() && OB_FAIL(tablet_id_array_.assign(tablet_id_array))) {
LOG_WARN("failed to assign tablet id array", K(ret), K(tablet_id_array));
K(log_sync_scn), K(tablet_info_array));
} else if (!tablet_info_array.empty() && OB_FAIL(tablet_info_array_.assign(tablet_info_array))) {
LOG_WARN("failed to assign tablet info array", K(ret), K(tablet_info_array));
} else {
task_id_ = task_id;
ls_id_ = ls_id;
log_sync_scn_ = log_sync_scn;
tablet_id_index_ = 0;
tablet_info_index_ = 0;
}
return ret;
}
@ -113,22 +113,22 @@ int ObBackfillTXCtx::build_backfill_tx_ctx(
bool ObBackfillTXCtx::is_empty() const
{
common::SpinRLockGuard guard(lock_);
return tablet_id_array_.empty();
return tablet_info_array_.empty();
}
int ObBackfillTXCtx::get_tablet_id_array(
common::ObIArray<common::ObTabletID> &tablet_id_array) const
int ObBackfillTXCtx::get_tablet_info_array(
common::ObIArray<ObTabletBackfillInfo> &tablet_info_array) const
{
int ret = OB_SUCCESS;
tablet_id_array.reset();
tablet_info_array.reset();
common::SpinRLockGuard guard(lock_);
if (!inner_is_valid_()) {
ret = OB_NOT_INIT;
LOG_WARN("backfill tx ctx is not init", K(ret));
} else {
if (OB_FAIL(tablet_id_array.assign(tablet_id_array_))) {
LOG_WARN("failed to assign tablet id array", K(ret), K(tablet_id_array_));
if (OB_FAIL(tablet_info_array.assign(tablet_info_array_))) {
LOG_WARN("failed to assign tablet info array", K(ret), K(tablet_info_array_));
}
}
return ret;
@ -140,7 +140,7 @@ int ObBackfillTXCtx::check_is_same(
{
int ret = OB_SUCCESS;
is_same = true;
ObArray<ObTabletID> tablet_id_array;
ObArray<ObTabletBackfillInfo> tablet_info_array;
common::SpinRLockGuard guard(lock_);
if (!inner_is_valid_()) {
@ -151,15 +151,15 @@ int ObBackfillTXCtx::check_is_same(
LOG_WARN("check is same get invalid argument", K(ret), K(backfill_tx_ctx));
} else if (ls_id_ != backfill_tx_ctx.ls_id_) {
is_same = false;
} else if(OB_FAIL(backfill_tx_ctx.get_tablet_id_array(tablet_id_array))) {
LOG_WARN("failed to get tablet id array", K(ret), K(backfill_tx_ctx));
} else if(OB_FAIL(backfill_tx_ctx.get_tablet_info_array(tablet_info_array))) {
LOG_WARN("failed to get tablet info array", K(ret), K(backfill_tx_ctx));
} else {
if (tablet_id_array.count() != tablet_id_array_.count()) {
if (tablet_info_array.count() != tablet_info_array_.count()) {
is_same = false;
} else {
for (int64_t i = 0; i < tablet_id_array_.count() && is_same; ++i) {
if (tablet_id_array_.at(i) != tablet_id_array.at(i)) {
for (int64_t i = 0; i < tablet_info_array_.count() && is_same; ++i) {
if (!(tablet_info_array_.at(i) == tablet_info_array.at(i))) {
is_same = false;
}
}
@ -174,9 +174,9 @@ int64_t ObBackfillTXCtx::hash() const
common::SpinRLockGuard guard(lock_);
hash_value = common::murmurhash(
&ls_id_, sizeof(ls_id_), hash_value);
for (int64_t i = 0; i < tablet_id_array_.count(); ++i) {
for (int64_t i = 0; i < tablet_info_array_.count(); ++i) {
hash_value = common::murmurhash(
&tablet_id_array_.at(i), sizeof(tablet_id_array_.at(i)), hash_value);
&tablet_info_array_.at(i), sizeof(tablet_info_array_.at(i)), hash_value);
}
return hash_value;
}
@ -187,7 +187,7 @@ ObTabletBackfillTXDag::ObTabletBackfillTXDag()
is_inited_(false),
dag_net_id_(),
ls_id_(),
tablet_id_(),
tablet_info_(),
backfill_tx_ctx_(nullptr),
tablet_handle_()
{
@ -204,7 +204,7 @@ int ObTabletBackfillTXDag::fill_info_param(compaction::ObIBasicInfoParam *&out_p
ret = OB_NOT_INIT;
LOG_WARN("tablet backfill tx dag do not init", K(ret));
} else if (OB_FAIL(ADD_DAG_WARN_INFO_PARAM(out_param, allocator, get_type(),
ls_id_.id(), static_cast<int64_t>(tablet_id_.id()),
ls_id_.id(), static_cast<int64_t>(tablet_info_.tablet_id_.id()),
"dag_net_id", to_cstring(dag_net_id_)))){
LOG_WARN("failed to fill info param", K(ret));
}
@ -219,7 +219,7 @@ int ObTabletBackfillTXDag::fill_dag_key(char *buf, const int64_t buf_len) const
LOG_WARN("tablet backfill tx dag do not init", K(ret));
} else if (OB_FAIL(databuff_printf(buf, buf_len,
"ObTabletBackfillTXDag: ls_id = %s, tablet_id = %s",
to_cstring(ls_id_), to_cstring(tablet_id_)))) {
to_cstring(ls_id_), to_cstring(tablet_info_.tablet_id_)))) {
LOG_WARN("failed to fill comment", K(ret), KPC(backfill_tx_ctx_), KPC(ha_dag_net_ctx_));
}
return ret;
@ -234,7 +234,7 @@ bool ObTabletBackfillTXDag::operator == (const ObIDag &other) const
is_same = false;
} else {
const ObTabletBackfillTXDag &tablet_backfill_tx_dag = static_cast<const ObTabletBackfillTXDag&>(other);
if (tablet_backfill_tx_dag.ls_id_ != ls_id_ || tablet_backfill_tx_dag.tablet_id_ != tablet_id_) {
if (tablet_backfill_tx_dag.ls_id_ != ls_id_ || !(tablet_backfill_tx_dag.tablet_info_ == tablet_info_)) {
is_same = false;
} else {
is_same = true;
@ -255,7 +255,7 @@ int64_t ObTabletBackfillTXDag::hash() const
hash_value = common::murmurhash(
&ls_id_, sizeof(ls_id_), hash_value);
hash_value = common::murmurhash(
&tablet_id_, sizeof(tablet_id_), hash_value);
&tablet_info_.tablet_id_, sizeof(tablet_info_.tablet_id_), hash_value);
ObDagType::ObDagTypeEnum dag_type = get_type();
hash_value = common::murmurhash(
&dag_type, sizeof(dag_type), hash_value);
@ -266,7 +266,7 @@ int64_t ObTabletBackfillTXDag::hash() const
int ObTabletBackfillTXDag::init(
const share::ObTaskId &dag_net_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const ObTabletBackfillInfo &tablet_info,
ObIHADagNetCtx *ha_dag_net_ctx,
ObBackfillTXCtx *backfill_tx_ctx)
{
@ -278,10 +278,10 @@ int ObTabletBackfillTXDag::init(
if (is_inited_) {
ret = OB_INIT_TWICE;
LOG_WARN("tablet backfill tx dag init twice", K(ret));
} else if (dag_net_id.is_invalid() || !ls_id.is_valid() || !tablet_id.is_valid()
} else if (dag_net_id.is_invalid() || !ls_id.is_valid() || !tablet_info.is_valid()
|| OB_ISNULL(ha_dag_net_ctx) || OB_ISNULL(backfill_tx_ctx)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("init tablet backfill tx dag get invalid argument", K(ret), K(dag_net_id), K(ls_id), K(tablet_id),
LOG_WARN("init tablet backfill tx dag get invalid argument", K(ret), K(dag_net_id), K(ls_id), K(tablet_info),
KP(ha_dag_net_ctx), KP(backfill_tx_ctx));
} else if (OB_ISNULL(ls_service = MTL(ObLSService*))) {
ret = OB_ERR_UNEXPECTED;
@ -291,12 +291,12 @@ int ObTabletBackfillTXDag::init(
} else if (OB_ISNULL(ls = ls_handle.get_ls())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls should not be NULL", K(ret), KP(ls), K(ls_id));
} else if (OB_FAIL(ls->ha_get_tablet(tablet_id, tablet_handle_))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_id));
} else if (OB_FAIL(ls->ha_get_tablet(tablet_info.tablet_id_, tablet_handle_))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_info));
} else {
dag_net_id_ = dag_net_id;
ls_id_ = ls_id;
tablet_id_ = tablet_id;
tablet_info_ = tablet_info;
ha_dag_net_ctx_ = ha_dag_net_ctx;
backfill_tx_ctx_ = backfill_tx_ctx;
compat_mode_ = tablet_handle_.get_obj()->get_tablet_meta().compat_mode_;
@ -315,7 +315,7 @@ int ObTabletBackfillTXDag::create_first_task()
LOG_WARN("tablet backfill tx dag do not init", K(ret));
} else if (OB_FAIL(alloc_task(task))) {
LOG_WARN("Fail to alloc task", K(ret));
} else if (OB_FAIL(task->init(dag_net_id_, ls_id_, tablet_id_))) {
} else if (OB_FAIL(task->init(dag_net_id_, ls_id_, tablet_info_))) {
LOG_WARN("failed to init tablet backfill tx task", K(ret), KPC(ha_dag_net_ctx_), KPC(backfill_tx_ctx_));
} else if (OB_FAIL(add_task(*task))) {
LOG_WARN("Fail to add task", K(ret));
@ -331,7 +331,7 @@ int ObTabletBackfillTXDag::generate_next_dag(share::ObIDag *&dag)
int tmp_ret = OB_SUCCESS;
dag = nullptr;
ObTenantDagScheduler *scheduler = nullptr;
common::ObTabletID next_tablet_id;
ObTabletBackfillInfo next_tablet_info;
ObIDagNet *dag_net = nullptr;
ObTabletBackfillTXDag *tablet_backfill_tx_dag = nullptr;
bool need_set_failed_result = true;
@ -344,7 +344,7 @@ int ObTabletBackfillTXDag::generate_next_dag(share::ObIDag *&dag)
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))) {
} else if (OB_FAIL(backfill_tx_ctx_->get_tablet_info(next_tablet_info))) {
if (OB_ITER_END == ret) {
//do nothing
need_set_failed_result = false;
@ -359,7 +359,7 @@ int ObTabletBackfillTXDag::generate_next_dag(share::ObIDag *&dag)
LOG_WARN("failed to get ObTenantDagScheduler from MTL", K(ret));
} else if (OB_FAIL(scheduler->alloc_dag(tablet_backfill_tx_dag))) {
LOG_WARN("failed to alloc tablet backfill tx migration dag ", K(ret));
} else if (OB_FAIL(tablet_backfill_tx_dag->init(dag_net_id_, ls_id_, next_tablet_id, ha_dag_net_ctx_, backfill_tx_ctx_))) {
} else if (OB_FAIL(tablet_backfill_tx_dag->init(dag_net_id_, ls_id_, next_tablet_info, ha_dag_net_ctx_, backfill_tx_ctx_))) {
LOG_WARN("failed to init tablet migration dag", K(ret), KPC(ha_dag_net_ctx_), KPC(backfill_tx_ctx_));
} else {
LOG_INFO("succeed generate next dag", KPC(tablet_backfill_tx_dag));
@ -375,7 +375,7 @@ int ObTabletBackfillTXDag::generate_next_dag(share::ObIDag *&dag)
int tmp_ret = OB_SUCCESS;
const bool need_retry = false;
if (need_set_failed_result && OB_SUCCESS != (tmp_ret = ha_dag_net_ctx_->set_result(ret, need_retry, get_type()))) {
LOG_WARN("failed to set result", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to set result", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_info_));
}
}
return ret;
@ -401,7 +401,7 @@ ObTabletBackfillTXTask::ObTabletBackfillTXTask()
backfill_tx_ctx_(nullptr),
ha_dag_net_ctx_(nullptr),
ls_id_(),
tablet_id_()
tablet_info_()
{
}
@ -413,7 +413,7 @@ ObTabletBackfillTXTask::~ObTabletBackfillTXTask()
int ObTabletBackfillTXTask::init(
const share::ObTaskId &dag_net_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id)
const ObTabletBackfillInfo &tablet_info)
{
int ret = OB_SUCCESS;
ObTabletBackfillTXDag *tablet_backfill_tx_dag = nullptr;
@ -421,17 +421,17 @@ int ObTabletBackfillTXTask::init(
if (is_inited_) {
ret = OB_INIT_TWICE;
LOG_WARN("tablet backfill tx task init twice", K(ret));
} else if (dag_net_id.is_invalid() || !ls_id.is_valid() || !tablet_id.is_valid()) {
} else if (dag_net_id.is_invalid() || !ls_id.is_valid() || !tablet_info.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("init tablet backfill tx get invalid argument", K(ret), K(dag_net_id), K(ls_id), K(tablet_id));
LOG_WARN("init tablet backfill tx get invalid argument", K(ret), K(dag_net_id), K(ls_id), K(tablet_info));
} else {
tablet_backfill_tx_dag = static_cast<ObTabletBackfillTXDag *>(this->get_dag());
backfill_tx_ctx_ = tablet_backfill_tx_dag->get_backfill_tx_ctx();
ha_dag_net_ctx_ = tablet_backfill_tx_dag->get_ha_dag_net_ctx();
ls_id_ = ls_id;
tablet_id_ = tablet_id;
tablet_info_ = tablet_info;
is_inited_ = true;
LOG_INFO("succeed init st migration task", "ls id", ls_id, "tablet_id", tablet_id,
LOG_INFO("succeed init st migration task", "ls id", ls_id, "tablet_info", tablet_info,
"dag_id", *ObCurTraceId::get_trace_id(), "dag_net_id", dag_net_id);
}
@ -441,7 +441,7 @@ int ObTabletBackfillTXTask::init(
int ObTabletBackfillTXTask::process()
{
int ret = OB_SUCCESS;
LOG_INFO("start to do tablet backfill tx task", KPC(ha_dag_net_ctx_), K(tablet_id_), K(ls_id_));
LOG_INFO("start to do tablet backfill tx task", KPC(ha_dag_net_ctx_), K(tablet_info_), K(ls_id_));
if (!is_inited_) {
ret = OB_NOT_INIT;
@ -449,7 +449,7 @@ int ObTabletBackfillTXTask::process()
} else if (ha_dag_net_ctx_->is_failed()) {
//do nothing
} else if (OB_FAIL(generate_backfill_tx_task_())) {
LOG_WARN("failed to generate backfill tx task", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to generate backfill tx task", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_info_));
}
if (OB_FAIL(ret)) {
@ -459,7 +459,7 @@ int ObTabletBackfillTXTask::process()
tmp_ret = OB_ERR_UNEXPECTED;
LOG_ERROR("tablet backfill tx dag should not be NULL", K(tmp_ret), KP(tablet_backfill_tx_dag));
} else if (OB_SUCCESS != (tmp_ret = tablet_backfill_tx_dag->set_result(ret))) {
LOG_WARN("failed to set result", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to set result", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_info_));
}
}
return ret;
@ -479,18 +479,18 @@ int ObTabletBackfillTXTask::generate_backfill_tx_task_()
LOG_WARN("tablet backfill tx task do not init", K(ret));
} else if (FALSE_IT(tablet_backfill_tx_dag = static_cast<ObTabletBackfillTXDag*>(this->get_dag()))) {
} else if (OB_FAIL(tablet_backfill_tx_dag->alloc_task(finish_backfill_tx_task))) {
LOG_WARN("failed to finish backfill tx task", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_id_));
} else if (OB_FAIL(finish_backfill_tx_task->init(ls_id_, tablet_id_))) {
LOG_WARN("failed to finish backfill tx task", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_info_));
} else if (OB_FAIL(finish_backfill_tx_task->init(ls_id_, tablet_info_.tablet_id_))) {
LOG_WARN("failed to init finish backfill tx task", K(ret));
} else if (OB_FAIL(tablet_backfill_tx_dag->get_tablet_handle(tablet_handle))) {
LOG_WARN("failed to get tablet handler", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to get tablet handler", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_info_));
} else if (OB_ISNULL(tablet = tablet_handle.get_obj())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet should not be NULL", K(ret), KP(tablet));
} else if (OB_FAIL(get_all_backfill_tx_tables_(tablet, table_array))) {
LOG_WARN("get all backfill tx tabels", K(ret), KPC(tablet));
} else if (OB_FAIL(generate_table_backfill_tx_task_(finish_backfill_tx_task, table_array))) {
LOG_WARN("failed to generate minor sstables backfill tx task", K(ret), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to generate minor sstables backfill tx task", K(ret), K(ls_id_), K(tablet_info_));
} else if (OB_FAIL(dag_->add_task(*finish_backfill_tx_task))) {
LOG_WARN("failed to add copy task to dag", K(ret));
}
@ -580,15 +580,20 @@ int ObTabletBackfillTXTask::get_backfill_tx_memtables_(
} else if (table->get_start_scn() >= backfill_tx_ctx_->log_sync_scn_
&& memtable->not_empty()
&& !memtable->get_rec_scn().is_max()) {
if (tablet_info_.is_committed_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("memtable start log ts is bigger than log sync scn but not empty", K(ret), KPC(memtable), KPC_(backfill_tx_ctx));
} else {
ret = OB_EAGAIN;
LOG_WARN("memtable start log ts is bigger than log sync scn but not empty, need retry", K(ret), KPC(memtable), KPC_(backfill_tx_ctx));
}
} else if (!table->is_frozen_memtable()) {
is_memtable_ready = false;
if (OB_FAIL(ls->tablet_freeze(tablet_id_))) {
if (OB_FAIL(ls->tablet_freeze(tablet_info_.tablet_id_))) {
if (OB_EAGAIN == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("failed to force tablet freeze", K(ret), K(tablet_id_), KPC(table));
LOG_WARN("failed to force tablet freeze", K(ret), K(tablet_info_), KPC(table));
}
} else {
break;
@ -608,7 +613,7 @@ int ObTabletBackfillTXTask::get_backfill_tx_memtables_(
} else {
const int64_t current_ts = ObTimeUtility::current_time();
if (REACH_TENANT_TIME_INTERVAL(60 * 1000 * 1000)) {
LOG_INFO("tablet not ready, retry next loop", "tablet_id", tablet_id_,
LOG_INFO("tablet not ready, retry next loop", "tablet_id", tablet_info_,
"wait_tablet_start_ts", wait_memtable_start_ts,
"current_ts", current_ts);
}
@ -680,10 +685,10 @@ int ObTabletBackfillTXTask::generate_table_backfill_tx_task_(
} else if (OB_ISNULL(finish_backfill_tx_task)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("generate table backfill tx task get invalid argument",
K(ret), KP(finish_backfill_tx_task), K(ls_id_), K(tablet_id_));
K(ret), KP(finish_backfill_tx_task), K(ls_id_), K(tablet_info_));
} else if (FALSE_IT(tablet_backfill_tx_dag = static_cast<ObTabletBackfillTXDag*>(this->get_dag()))) {
} else if (OB_FAIL(tablet_backfill_tx_dag->get_tablet_handle(tablet_handle))) {
LOG_WARN("failed to get tablet handler", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to get tablet handler", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_info_));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < table_array.count(); ++i) {
ObITable *table = table_array.at(i).get_table();
@ -712,23 +717,23 @@ int ObTabletBackfillTXTask::generate_table_backfill_tx_task_(
}
if (OB_SUCC(ret) && is_add_task) {
if (OB_FAIL(tablet_backfill_tx_dag->alloc_task(table_backfill_tx_task))) {
LOG_WARN("failed to alloc table backfill tx task", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_id_));
} else if (OB_FAIL(table_backfill_tx_task->init(ls_id_, tablet_id_, tablet_handle, table_array.at(i)))) {
LOG_WARN("failed to init table backfill tx task", K(ret), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to alloc table backfill tx task", K(ret), KPC(ha_dag_net_ctx_), K(ls_id_), K(tablet_info_));
} else if (OB_FAIL(table_backfill_tx_task->init(ls_id_, tablet_info_.tablet_id_, tablet_handle, table_array.at(i)))) {
LOG_WARN("failed to init table backfill tx task", K(ret), K(ls_id_), K(tablet_info_));
} else if (OB_ISNULL(pre_table_backfill_task)) {
if (OB_FAIL(this->add_child(*table_backfill_tx_task))) {
LOG_WARN("failed to add table backfill tx task as child", K(ret), K(ls_id_), K(tablet_id_), KPC(table));
LOG_WARN("failed to add table backfill tx task as child", K(ret), K(ls_id_), K(tablet_info_), KPC(table));
}
} else {
if (OB_FAIL(pre_table_backfill_task->add_child(*table_backfill_tx_task))) {
LOG_WARN("failed to add table backfill tx task as child", K(ret), K(ls_id_), K(tablet_id_), KPC(table), KPC(pre_table_backfill_task));
LOG_WARN("failed to add table backfill tx task as child", K(ret), K(ls_id_), K(tablet_info_), KPC(table), KPC(pre_table_backfill_task));
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(table_backfill_tx_task->add_child(*finish_backfill_tx_task))) {
LOG_WARN("failed to add finish backfill tx task as child", K(ret), K(ls_id_), K(tablet_id_), KPC(table));
LOG_WARN("failed to add finish backfill tx task as child", K(ret), K(ls_id_), K(tablet_info_), KPC(table));
} else if (OB_FAIL(dag_->add_task(*table_backfill_tx_task))) {
LOG_WARN("failed to add table backfill tx task", K(ret), K(ls_id_), K(tablet_id_));
LOG_WARN("failed to add table backfill tx task", K(ret), K(ls_id_), K(tablet_info_));
} else {
pre_table_backfill_task = table_backfill_tx_task;
LOG_INFO("generate table backfill TX", KPC(table), K(i), KPC(table_backfill_tx_task));
@ -1180,7 +1185,7 @@ int ObFinishBackfillTXDag::init(
const share::ObTaskId &task_id,
const share::ObLSID &ls_id,
const SCN &log_sync_scn,
ObArray<common::ObTabletID> &tablet_id_array,
common::ObArray<ObTabletBackfillInfo> &tablet_info_array,
ObIHADagNetCtx *ha_dag_net_ctx)
{
int ret = OB_SUCCESS;
@ -1191,8 +1196,8 @@ int ObFinishBackfillTXDag::init(
} else if (task_id.is_invalid() || !ls_id.is_valid() || !log_sync_scn.is_valid() || OB_ISNULL(ha_dag_net_ctx)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("init finish backfill tx dag get invalid argument", K(ret), K(task_id), K(ls_id), K(log_sync_scn) ,KP(ha_dag_net_ctx));
} else if (OB_FAIL(backfill_tx_ctx_.build_backfill_tx_ctx(task_id, ls_id, log_sync_scn, tablet_id_array))) {
LOG_WARN("failed to build backfill tx ctx", K(ret), K(tablet_id_array));
} else if (OB_FAIL(backfill_tx_ctx_.build_backfill_tx_ctx(task_id, ls_id, log_sync_scn, tablet_info_array))) {
LOG_WARN("failed to build backfill tx ctx", K(ret), K(tablet_info_array));
} else {
ha_dag_net_ctx_ = ha_dag_net_ctx;
is_inited_ = true;

View File

@ -31,27 +31,27 @@ struct ObBackfillTXCtx
public:
ObBackfillTXCtx();
virtual ~ObBackfillTXCtx();
int get_tablet_id(common::ObTabletID &tablet_id);
int get_tablet_info(ObTabletBackfillInfo &tablet_info);
bool is_valid() const;
void reset();
int build_backfill_tx_ctx(
const share::ObTaskId &task_id,
const share::ObLSID &ls_id,
const share::SCN log_sync_scn,
const common::ObIArray<common::ObTabletID> &tablet_id_array);
const common::ObIArray<ObTabletBackfillInfo> &tablet_info_array);
bool is_empty() const;
int check_is_same(
const ObBackfillTXCtx &backfill_tx_ctx,
bool &is_same) const;
int get_tablet_id_array(common::ObIArray<common::ObTabletID> &tablet_id_array) const;
int get_tablet_info_array(common::ObIArray<ObTabletBackfillInfo> &tablet_info_array) const;
int64_t hash() const;
VIRTUAL_TO_STRING_KV(
K_(task_id),
K_(ls_id),
K_(log_sync_scn),
K_(tablet_id_index),
K_(tablet_id_array));
K_(tablet_info_index),
K_(tablet_info_array));
public:
share::ObTaskId task_id_;
share::ObLSID ls_id_;
@ -60,8 +60,8 @@ private:
bool inner_is_valid_() const;
private:
common::SpinRWLock lock_;
int64_t tablet_id_index_;
common::ObArray<common::ObTabletID> tablet_id_array_;
int64_t tablet_info_index_;
common::ObArray<ObTabletBackfillInfo> tablet_info_array_;
DISALLOW_COPY_AND_ASSIGN(ObBackfillTXCtx);
};
@ -79,7 +79,7 @@ public:
int init(
const share::ObTaskId &dag_net_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const storage::ObTabletBackfillInfo &tablet_info,
ObIHADagNetCtx *ha_dag_net_ctx,
ObBackfillTXCtx *backfill_tx_ctx);
virtual int generate_next_dag(share::ObIDag *&dag);
@ -90,7 +90,7 @@ protected:
bool is_inited_;
share::ObTaskId dag_net_id_;
share::ObLSID ls_id_;
common::ObTabletID tablet_id_;
ObTabletBackfillInfo tablet_info_;
ObBackfillTXCtx *backfill_tx_ctx_;
ObTabletHandle tablet_handle_;
DISALLOW_COPY_AND_ASSIGN(ObTabletBackfillTXDag);
@ -105,9 +105,9 @@ public:
int init(
const share::ObTaskId &dag_net_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id);
const ObTabletBackfillInfo &tablet_info);
virtual int process() override;
VIRTUAL_TO_STRING_KV(K("ObTabletBackfillTXTask"), KP(this), KPC(ha_dag_net_ctx_), K_(tablet_id));
VIRTUAL_TO_STRING_KV(K("ObTabletBackfillTXTask"), KP(this), KPC(ha_dag_net_ctx_), K_(tablet_info));
private:
int generate_backfill_tx_task_();
int generate_table_backfill_tx_task_(
@ -127,7 +127,7 @@ private:
ObBackfillTXCtx *backfill_tx_ctx_;
ObIHADagNetCtx *ha_dag_net_ctx_;
share::ObLSID ls_id_;
common::ObTabletID tablet_id_;
ObTabletBackfillInfo tablet_info_;
DISALLOW_COPY_AND_ASSIGN(ObTabletBackfillTXTask);
};
@ -203,7 +203,7 @@ public:
const share::ObTaskId &task_id,
const share::ObLSID &ls_id,
const share::SCN &log_sync_scn,
ObArray<common::ObTabletID> &tablet_id_array,
common::ObArray<ObTabletBackfillInfo> &tablet_info_array,
ObIHADagNetCtx *ha_dag_net_ctx);
ObBackfillTXCtx *get_backfill_tx_ctx() { return &backfill_tx_ctx_; }
INHERIT_TO_STRING_KV("ObStorageHADag", ObStorageHADag, KP(this));

View File

@ -109,6 +109,8 @@ int ObTransferWorkerMgr::get_need_backfill_tx_tablets_(ObTransferBackfillTXParam
user_data.reset();
tablet = nullptr;
bool is_ready = false;
bool is_committed = false;
ObTabletBackfillInfo tablet_info;
if (OB_FAIL(tablet_iter.get_next_tablet(tablet_handle))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
@ -121,12 +123,12 @@ int ObTransferWorkerMgr::get_need_backfill_tx_tablets_(ObTransferBackfillTXParam
LOG_WARN("tablet should not be NULL", K(ret), KP(tablet));
} else if (tablet->get_tablet_meta().tablet_id_.is_ls_inner_tablet()) {
//do nothing
} else if (OB_FAIL(ObTXTransferUtils::get_tablet_status(false/*get_commit*/, tablet, user_data))) {
if (OB_EMPTY_RESULT == ret || OB_ERR_SHARED_LOCK_CONFLICT == ret) { // needs to delete this judgment after ObLSTabletIterator optimization
LOG_INFO("committed tablet_status does not exist", K(ret), "tablet_id", tablet->get_tablet_meta().tablet_id_);
} else if (OB_FAIL(tablet->ObITabletMdsInterface::get_latest_tablet_status(user_data, is_committed))) {
if (OB_EMPTY_RESULT == ret) {
LOG_INFO("tablet_status does not exist", K(ret), "tablet_id", tablet->get_tablet_meta().tablet_id_);
ret = OB_SUCCESS;
} else {
LOG_WARN("failed to get tablet status", K(ret), KPC(tablet));
LOG_WARN("failed to get latest tablet status", K(ret), KPC(tablet), K(user_data));
}
} else if (ObTabletStatus::TRANSFER_IN != user_data.tablet_status_ && !in_migration) {
// do nothing
@ -177,14 +179,16 @@ int ObTransferWorkerMgr::get_need_backfill_tx_tablets_(ObTransferBackfillTXParam
"tablet_id", tablet->get_tablet_meta().tablet_id_,
"has_transfer_table", tablet->get_tablet_meta().has_transfer_table());
#endif
if (OB_FAIL(param.tablet_ids_.push_back(tablet->get_tablet_meta().tablet_id_))) {
if (OB_FAIL(tablet_info.init(tablet->get_tablet_meta().tablet_id_, is_committed))) {
} else if (OB_FAIL(param.tablet_infos_.push_back(tablet_info))) {
LOG_WARN("failed to push tablet id into array", K(ret), KPC(tablet));
} else if (src_ls_id.is_valid() && transfer_scn.is_valid()) {
if (in_migration) {
//migration will has multi transfer task tablets.
if (src_ls_id != tablet->get_tablet_meta().transfer_info_.ls_id_
|| transfer_scn != tablet->get_tablet_meta().transfer_info_.transfer_start_scn_) {
param.tablet_ids_.pop_back();
param.tablet_infos_.pop_back();
}
} else if (src_ls_id != tablet->get_tablet_meta().transfer_info_.ls_id_
|| transfer_scn != tablet->get_tablet_meta().transfer_info_.transfer_start_scn_) {
@ -323,7 +327,7 @@ int ObTransferWorkerMgr::process()
update_task_id_();
if (OB_FAIL(get_need_backfill_tx_tablets_(param))) {
LOG_WARN("failed to get need backfill tx tablets", K(ret), "ls_id", dest_ls_->get_ls_id(), K(*this));
} else if (param.tablet_ids_.empty()) {
} else if (param.tablet_infos_.empty()) {
// There are no tablets that require backfill transactions
} else if (OB_FAIL(do_transfer_backfill_tx_(param))) {
LOG_WARN("failed to do transfer backfill tx", K(ret), K(param));
@ -452,7 +456,7 @@ ObTransferBackfillTXCtx::ObTransferBackfillTXCtx()
src_ls_id_(),
dest_ls_id_(),
backfill_scn_(),
tablet_ids_()
tablet_infos_()
{
}
@ -467,7 +471,7 @@ bool ObTransferBackfillTXCtx::is_valid() const
&& src_ls_id_.is_valid()
&& dest_ls_id_.is_valid()
&& backfill_scn_.is_valid()
&& !tablet_ids_.empty();
&& !tablet_infos_.empty();
}
void ObTransferBackfillTXCtx::reset()
@ -477,7 +481,7 @@ void ObTransferBackfillTXCtx::reset()
src_ls_id_.reset();
dest_ls_id_.reset();
backfill_scn_.reset();
tablet_ids_.reset();
tablet_infos_.reset();
ObIHADagNetCtx::reset();
}
@ -504,7 +508,7 @@ void ObTransferBackfillTXCtx::reuse()
{
ObIHADagNetCtx::reuse();
backfill_scn_.reset();
tablet_ids_.reset();
tablet_infos_.reset();
}
/******************ObTransferBackfillTXParam*********************/
@ -514,7 +518,7 @@ ObTransferBackfillTXParam::ObTransferBackfillTXParam()
src_ls_id_(),
dest_ls_id_(),
backfill_scn_(),
tablet_ids_()
tablet_infos_()
{
}
@ -525,7 +529,7 @@ bool ObTransferBackfillTXParam::is_valid() const
&& dest_ls_id_.is_valid()
&& !task_id_.is_invalid()
&& backfill_scn_.is_valid()
&& !tablet_ids_.empty();
&& !tablet_infos_.empty();
}
void ObTransferBackfillTXParam::reset()
@ -535,7 +539,7 @@ void ObTransferBackfillTXParam::reset()
src_ls_id_.reset();
dest_ls_id_.reset();
backfill_scn_.reset();
tablet_ids_.reset();
tablet_infos_.reset();
}
/******************ObTransferBackfillTXDagNet*********************/
@ -562,7 +566,7 @@ int ObTransferBackfillTXDagNet::init_by_param(const ObIDagInitParam *param)
LOG_WARN("param is null or invalid", K(ret), KPC(init_param));
} else if (OB_FAIL(this->set_dag_id(init_param->task_id_))) {
LOG_WARN("failed to set dag id", K(ret), KPC(init_param));
} else if (OB_FAIL(ctx_.tablet_ids_.assign(init_param->tablet_ids_))) {
} else if (OB_FAIL(ctx_.tablet_infos_.assign(init_param->tablet_infos_))) {
LOG_WARN("failed to set transfer tablet list", K(ret), KPC(init_param));
} else {
ctx_.tenant_id_ = init_param->tenant_id_;
@ -998,7 +1002,7 @@ int ObStartTransferBackfillTXTask::generate_transfer_backfill_tx_dags_()
ObTenantDagScheduler *scheduler = nullptr;
ObIDagNet *dag_net = nullptr;
ObBackfillTXCtx *backfill_tx_ctx = nullptr;
ObTabletID tablet_id;
storage::ObTabletBackfillInfo tablet_info;
ObStartTransferBackfillTXDag *backfill_tx_dag = nullptr;
if (IS_NOT_INIT) {
@ -1020,7 +1024,7 @@ int ObStartTransferBackfillTXTask::generate_transfer_backfill_tx_dags_()
} else {
if (OB_FAIL(scheduler->alloc_dag(finish_backfill_tx_dag))) {
LOG_WARN("failed to alloc finish backfill tx transfer dag ", K(ret));
} else if (OB_FAIL(finish_backfill_tx_dag->init(ctx_->task_id_, ctx_->src_ls_id_, ctx_->backfill_scn_, ctx_->tablet_ids_, ctx_))) {
} else if (OB_FAIL(finish_backfill_tx_dag->init(ctx_->task_id_, ctx_->src_ls_id_, ctx_->backfill_scn_, ctx_->tablet_infos_, ctx_))) {
LOG_WARN("failed to init data tablets transfer dag", K(ret), K(*ctx_));
} else if (OB_ISNULL(backfill_tx_ctx = finish_backfill_tx_dag->get_backfill_tx_ctx())) {
ret = OB_ERR_UNEXPECTED;
@ -1030,11 +1034,11 @@ int ObStartTransferBackfillTXTask::generate_transfer_backfill_tx_dags_()
LOG_WARN("failed to add finish backfill tx dag as chilid", K(ret), K(*ctx_));
}
} else {
if (OB_FAIL(backfill_tx_ctx->get_tablet_id(tablet_id))) {
if (OB_FAIL(backfill_tx_ctx->get_tablet_info(tablet_info))) {
LOG_WARN("failed to get tablet id", K(ret), KPC(ctx_));
} else if (OB_FAIL(scheduler->alloc_dag(tablet_backfill_tx_dag))) {
LOG_WARN("failed to alloc tablet backfill tx dag ", K(ret));
} else if (OB_FAIL(tablet_backfill_tx_dag->init(ctx_->task_id_, ctx_->src_ls_id_, tablet_id, ctx_, backfill_tx_ctx))) {
} else if (OB_FAIL(tablet_backfill_tx_dag->init(ctx_->task_id_, ctx_->src_ls_id_, tablet_info, ctx_, backfill_tx_ctx))) {
LOG_WARN("failed to init tablet backfill tx dag", K(ret), K(*ctx_));
} else if (OB_FAIL(this->get_dag()->add_child(*tablet_backfill_tx_dag))) {
LOG_WARN("failed to add tablet backfill tx dag as chilid", K(ret), K(*ctx_));
@ -1418,7 +1422,7 @@ int ObTransferReplaceTableTask::fill_empty_minor_sstable(
int ObTransferReplaceTableTask::get_source_tablet_tables_(
const ObTablet *dest_tablet,
const common::ObTabletID &tablet_id,
const ObTabletBackfillInfo &tablet_info,
ObTableStoreIterator &sstable_iter,
ObTabletHandle &tablet_handle,
ObTabletRestoreStatus::STATUS &restore_status,
@ -1442,9 +1446,9 @@ int ObTransferReplaceTableTask::get_source_tablet_tables_(
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("transfer replace tables task do not init", K(ret));
} else if (OB_ISNULL(dest_tablet) || !tablet_id.is_valid()) {
} else if (OB_ISNULL(dest_tablet) || !tablet_info.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tablet id is invalid", K(ret), K(tablet_id));
LOG_WARN("tablet info is invalid", K(ret), K(tablet_info));
} else if (FALSE_IT(transfer_scn = dest_tablet->get_tablet_meta().transfer_info_.transfer_start_scn_)) {
} else if (OB_ISNULL(ls_service = MTL(ObLSService*))) {
ret = OB_ERR_UNEXPECTED;
@ -1454,8 +1458,8 @@ int ObTransferReplaceTableTask::get_source_tablet_tables_(
} else if (OB_ISNULL(ls = ls_handle.get_ls())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls should not be NULL", K(ret), KP(ls), "ls_id", ctx_->src_ls_id_);
} else if (OB_FAIL(ls->get_tablet(tablet_id, tablet_handle, 0, ObMDSGetTabletMode::READ_WITHOUT_CHECK))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_id));
} else if (OB_FAIL(ls->get_tablet(tablet_info.tablet_id_, tablet_handle, 0, ObMDSGetTabletMode::READ_WITHOUT_CHECK))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_info));
} else if (OB_ISNULL(tablet = tablet_handle.get_obj())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet should not be NULL", K(ret), KPC(tablet));
@ -1478,8 +1482,13 @@ int ObTransferReplaceTableTask::get_source_tablet_tables_(
K(src_user_data), K(dest_user_data), K(src_transfer_seq), K(dest_transfer_seq));
} else if (ObTabletStatus::TRANSFER_OUT != src_user_data.tablet_status_
&& ObTabletStatus::TRANSFER_OUT_DELETED != src_user_data.tablet_status_) {
if (tablet_info.is_committed_) {
ret = OB_UNEXPECTED_TABLET_STATUS;
LOG_WARN("tablet status should be TRANSFER_OUT or TRANSFER_OUT_DELETED", K(ret), KPC(tablet), K(src_user_data));
} else {
ret = OB_EAGAIN;
LOG_WARN("the transfer start transaction was rolledback and the task needs to be retried", K(ret), K(tablet_info), K(src_user_data));
}
} else if (OB_FAIL(tablet->get_tablet_meta().ha_status_.get_restore_status(restore_status))) {
LOG_WARN("failed to get tablet restore status", K(ret));
} else if (OB_FAIL(tablet->fetch_table_store(wrapper))) {
@ -1570,7 +1579,7 @@ int ObTransferReplaceTableTask::check_tablet_after_replace_(ObLS *ls, const comm
int ObTransferReplaceTableTask::transfer_replace_tables_(
ObLS *ls,
const common::ObTabletID &tablet_id,
const ObTabletBackfillInfo &tablet_info,
const ObTablet *tablet)
{
int ret = OB_SUCCESS;
@ -1605,8 +1614,8 @@ int ObTransferReplaceTableTask::transfer_replace_tables_(
} else if (!dest_wrapper.get_member()->get_major_sstables().empty()) {
ret = OB_INVALID_TABLE_STORE;
LOG_WARN("tablet should not exist major sstable", K(ret), KPC(tablet));
} else if (OB_FAIL(get_source_tablet_tables_(tablet, tablet_id, src_sstable_iter, src_tablet_handle, param.restore_status_, allocator, param.tables_handle_))) {
LOG_WARN("failed to get source tablet tables", K(ret), K(tablet_id));
} else if (OB_FAIL(get_source_tablet_tables_(tablet, tablet_info, src_sstable_iter, src_tablet_handle, param.restore_status_, allocator, param.tables_handle_))) {
LOG_WARN("failed to get source tablet tables", K(ret), K(tablet_info));
} else if (OB_FAIL(build_migration_param_(tablet, src_tablet_handle, mig_param))) {
LOG_WARN("failed to build migration param", K(ret), KPC(tablet));
} else {
@ -1618,17 +1627,17 @@ int ObTransferReplaceTableTask::transfer_replace_tables_(
SERVER_EVENT_SYNC_ADD("TRANSFER", "TRANSFER_REPLACE_TABLE_WITH_LOG_REPLAY_SKIP_CHECK",
"dest_ls_id", ls->get_ls_id(),
"migration_status", migration_status,
"tablet_id", tablet_id.id(),
"tablet_id", tablet_info.tablet_id_.id(),
"tablet_status", ObTabletStatus::get_str(user_data.tablet_status_),
"has_transfer_table", tablet->get_tablet_meta().has_transfer_table());
#endif
if (FAILEDx(ls->build_ha_tablet_new_table_store(tablet_id, param))) {
LOG_WARN("failed to build ha tablet new table store", K(ret), K(param), K(tablet_id));
} else if (OB_FAIL(check_tablet_after_replace_(ls, tablet_id))) {
LOG_WARN("failed to check tablet after replace", K(ret), K(param), K(tablet_id));
if (FAILEDx(ls->build_ha_tablet_new_table_store(tablet_info.tablet_id_, param))) {
LOG_WARN("failed to build ha tablet new table store", K(ret), K(param), K(tablet_info));
} else if (OB_FAIL(check_tablet_after_replace_(ls, tablet_info.tablet_id_))) {
LOG_WARN("failed to check tablet after replace", K(ret), K(param), K(tablet_info));
} else {
LOG_INFO("[TRANSFER_BACKFILL]succ transfer replace tables", K(ret), K(param), K(tablet_id), KPC_(ctx));
LOG_INFO("[TRANSFER_BACKFILL]succ transfer replace tables", K(ret), K(param), K(tablet_info), KPC_(ctx));
}
#ifdef ERRSIM
SERVER_EVENT_SYNC_ADD("TRANSFER", "AFTER_TRANSFER_DUMP_MDS_TABLE");
@ -1650,9 +1659,9 @@ int ObTransferReplaceTableTask::do_replace_logical_tables_(ObLS *ls)
ret = OB_INVALID_ARGUMENT;
LOG_WARN("ls is nullptr", K(ret));
} else {
for (int i = 0; OB_SUCC(ret) && i < ctx_->tablet_ids_.count(); i++) {
for (int i = 0; OB_SUCC(ret) && i < ctx_->tablet_infos_.count(); i++) {
user_data.reset();
const common::ObTabletID tablet_id = ctx_->tablet_ids_.at(i);
const ObTabletBackfillInfo tablet_info = ctx_->tablet_infos_.at(i);
bool in_migration = false;
ObMigrationStatus migration_status = ObMigrationStatus::OB_MIGRATION_STATUS_MAX;
if (ctx_->is_failed()) {
@ -1660,25 +1669,25 @@ int ObTransferReplaceTableTask::do_replace_logical_tables_(ObLS *ls)
if (OB_SUCCESS != (tmp_ret = ctx_->get_result(ret))) {
ret = tmp_ret;
}
LOG_WARN("ctx already failed", K(ret), KPC(ctx_), K(tablet_id));
LOG_WARN("ctx already failed", K(ret), KPC(ctx_), K(tablet_info));
} else if (OB_FAIL(ls->get_migration_status(migration_status))) {
LOG_WARN("failed to get migration status", K(ret), KPC(ls));
} else if (FALSE_IT(in_migration = ObMigrationStatus::OB_MIGRATION_STATUS_NONE != migration_status)) {
} else if (OB_FAIL(ls->get_tablet(tablet_id, tablet_handle, 0, ObMDSGetTabletMode::READ_WITHOUT_CHECK))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_id));
} else if (OB_FAIL(ls->get_tablet(tablet_info.tablet_id_, tablet_handle, 0, ObMDSGetTabletMode::READ_WITHOUT_CHECK))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_info));
} else if (OB_ISNULL(tablet = tablet_handle.get_obj())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet should not be NULL", K(ret), KPC(tablet));
} else if (tablet_id.is_ls_inner_tablet()) {
} else if (tablet_info.tablet_id_.is_ls_inner_tablet()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inner tablet cannot transfer", KR(ret), K(tablet_id), KPC(this));
LOG_WARN("inner tablet cannot transfer", KR(ret), K(tablet_info), KPC(this));
} else if (OB_FAIL(ObTXTransferUtils::get_tablet_status(false/*get_commit*/, tablet, user_data))) {
LOG_WARN("failed to get tablet status", K(ret), K(tablet_handle));
} else if (ObTabletStatus::TRANSFER_IN != user_data.tablet_status_ && !in_migration) {
ret = OB_UNEXPECTED_TABLET_STATUS;
LOG_WARN("tablet status should be TRANSFER_IN", K(ret), K(user_data), K(in_migration), KPC(tablet), KPC(ls));
} else if (OB_FAIL(transfer_replace_tables_(ls, tablet_id, tablet))) {
LOG_WARN("failed to transfer replace tables", K(ret), K(tablet_id), KPC(ls), KPC(tablet), KPC(ctx_));
} else if (OB_FAIL(transfer_replace_tables_(ls, tablet_info, tablet))) {
LOG_WARN("failed to transfer replace tables", K(ret), K(tablet_info), KPC(ls), KPC(tablet), KPC(ctx_));
} else {
#ifdef ERRSIM
SERVER_EVENT_ADD("TRANSFER", "REPLACE_LOGICAL_TABLE",
@ -1686,7 +1695,7 @@ int ObTransferReplaceTableTask::do_replace_logical_tables_(ObLS *ls)
"tenant_id", ctx_->tenant_id_,
"src_ls_id", ctx_->src_ls_id_.id(),
"dest_ls_id", ctx_->dest_ls_id_.id(),
"tablet_id", tablet_id.id());
"tablet_id", tablet_info.tablet_id_.id());
#endif
}
}
@ -1705,7 +1714,7 @@ int ObTransferReplaceTableTask::process()
ret = OB_NOT_INIT;
LOG_WARN("transfer replace tables task do not init", K(ret));
} else if (ctx_->is_failed()) {
LOG_INFO("[TRANSFER_BACKFILL]ctx already failed", KPC(ctx_), "tablet_list", ctx_->tablet_ids_);
LOG_INFO("[TRANSFER_BACKFILL]ctx already failed", KPC(ctx_), "tablet_list", ctx_->tablet_infos_);
} else if (OB_ISNULL(ls_service = MTL(ObLSService*))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get ObLSService from MTL", K(ret), KP(ls_service));
@ -1715,9 +1724,9 @@ int ObTransferReplaceTableTask::process()
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls should not be NULL", K(ret), KP(ls), KPC(ctx_));
} else if (OB_FAIL(do_replace_logical_tables_(ls))) {
LOG_WARN("failed to repalce logical tables", K(ret), KP(ls), KPC(ctx_), "tablet_list", ctx_->tablet_ids_);
LOG_WARN("failed to repalce logical tables", K(ret), KP(ls), KPC(ctx_), "tablet_list", ctx_->tablet_infos_);
} else {
LOG_INFO("[TRANSFER_BACKFILL]complete transfer replace task", K(ret), KPC(ctx_), "tablet_list", ctx_->tablet_ids_);
LOG_INFO("[TRANSFER_BACKFILL]complete transfer replace task", K(ret), KPC(ctx_), "tablet_list", ctx_->tablet_infos_);
}
if (OB_FAIL(ret)) {

View File

@ -29,13 +29,13 @@ public:
virtual ~ObTransferBackfillTXParam() {}
virtual bool is_valid() const override;
void reset();
VIRTUAL_TO_STRING_KV(K_(task_id), K_(src_ls_id), K_(dest_ls_id), K_(backfill_scn), K_(tablet_ids));
VIRTUAL_TO_STRING_KV(K_(task_id), K_(src_ls_id), K_(dest_ls_id), K_(backfill_scn), K_(tablet_infos));
uint64_t tenant_id_;
share::ObTaskId task_id_;
share::ObLSID src_ls_id_;
share::ObLSID dest_ls_id_;
share::SCN backfill_scn_;
ObArray<common::ObTabletID> tablet_ids_;
common::ObArray<ObTabletBackfillInfo> tablet_infos_;
};
class ObTransferWorkerMgr final
@ -85,7 +85,7 @@ public:
share::ObLSID src_ls_id_;
share::ObLSID dest_ls_id_;
share::SCN backfill_scn_;
ObArray<common::ObTabletID> tablet_ids_;
common::ObArray<ObTabletBackfillInfo> tablet_infos_;
INHERIT_TO_STRING_KV(
"ObIHADagNetCtx", ObIHADagNetCtx,
K_(tenant_id),
@ -205,11 +205,11 @@ private:
int do_replace_logical_tables_(ObLS *ls);
int transfer_replace_tables_(
ObLS *ls,
const common::ObTabletID &tablet_id,
const ObTabletBackfillInfo &tablet_info,
const ObTablet *tablet);
int get_source_tablet_tables_(
const ObTablet *dest_tablet,
const common::ObTabletID &tablet_id,
const ObTabletBackfillInfo &tablet_info,
ObTableStoreIterator &sstable_iter,
ObTabletHandle &tablet_handle,
ObTabletRestoreStatus::STATUS &restore_status,

View File

@ -1946,7 +1946,7 @@ int ObTransferHandler::stop_ls_schedule_medium_(const share::ObLSID &ls_id, bool
int ret = OB_SUCCESS;
succ_stop = false;
if (OB_FAIL(MTL(ObTenantTabletScheduler*)->stop_ls_schedule_medium(ls_id))) {
LOG_WARN("failed to resume ls schedule medium", K(ret), K(ls_id));
LOG_WARN("failed to stop ls schedule medium", K(ret), K(ls_id));
} else {
succ_stop = true;
}