Fix ls migration tablet meta merge with clog_checkpoint_ts

This commit is contained in:
godyangfight 2022-11-10 02:37:27 +00:00 committed by wangzelin.wzl
parent ad1c2616f6
commit 27459b3e05
13 changed files with 141 additions and 76 deletions

View File

@ -555,11 +555,9 @@ int ObStorageHATaskUtils::check_need_copy_sstable(
LOG_WARN("failed to check major sstable need copy", K(ret), K(param), K(tablet_handle));
}
} else if (param.table_key_.is_minor_sstable()) {
need_copy = true;
//TODO(muwei.ym) reuse local minor sstable
//if (OB_FAIL(check_minor_sstable_need_copy_(param, tablet_handle, need_copy))) {
// LOG_WARN("failed to check minor sstable need copy", K(ret), K(param), K(tablet_handle));
//}
if (OB_FAIL(check_minor_sstable_need_copy_(param, tablet_handle, need_copy))) {
LOG_WARN("failed to check minor sstable need copy", K(ret), K(param), K(tablet_handle));
}
} else if (param.table_key_.is_ddl_sstable()) {
if (OB_FAIL(check_ddl_sstable_need_copy_(param, tablet_handle, need_copy))) {
LOG_WARN("failed to check ddl sstable need copy", K(ret), K(param), K(tablet_handle));

View File

@ -805,6 +805,7 @@ int ObInitialTabletGroupRestoreTask::check_local_tablets_restore_status_()
ctx_->arg_.action_, action_restore_status))) {
LOG_WARN("failed to trans restore action to restore status", K(ret), KPC(ctx_));
} else {
ctx_->tablet_id_array_.reset();
bool can_change = false;
for (int64_t i = 0; OB_SUCC(ret) && i < ctx_->arg_.tablet_id_array_.count(); ++i) {
const ObTabletID &tablet_id = ctx_->arg_.tablet_id_array_.at(i);

View File

@ -2820,6 +2820,7 @@ int ObLSTabletService::build_ha_tablet_new_table_store(
ObTablet *new_tablet = nullptr;
ObTenantMetaMemMgr *t3m = nullptr;
ObMetaDiskAddr disk_addr;
ObFreezer *freezer = nullptr;
ObBucketHashWLockGuard lock_guard(bucket_lock_, tablet_id.hash());
if (IS_NOT_INIT) {
@ -2833,6 +2834,9 @@ int ObLSTabletService::build_ha_tablet_new_table_store(
LOG_WARN("t3m is null", K(ret));
} else if (OB_FAIL(direct_get_tablet(tablet_id, old_tablet_handle))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_id));
} else if (OB_ISNULL(freezer = ls_->get_freezer())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("freezer should not be NULL", K(ret), KP(freezer), KPC(ls_));
} else {
old_tablet = old_tablet_handle.get_obj();
const share::ObLSID &ls_id = ls_->get_ls_id();
@ -2840,6 +2844,9 @@ int ObLSTabletService::build_ha_tablet_new_table_store(
ObTabletTxMultiSourceDataUnit tx_data;
ObTabletBindingInfo ddl_data;
ObTabletAutoincSeq autoinc_seq;
//In order to merge tablet meta
//it is necessary to make the left side of the newly created memtable start from clog_checkpinoit_ts
//the new memtable can be stuck during the creation of the tablet, it is safe here
if (OB_FAIL(old_tablet->get_tx_data(tx_data))) {
LOG_WARN("failed to get tx data from old tablet", K(ret), K(tablet_id));
@ -2852,6 +2859,18 @@ int ObLSTabletService::build_ha_tablet_new_table_store(
} else if (FALSE_IT(new_tablet = new_tablet_handle.get_obj())) {
} else if (OB_FAIL(new_tablet->init(param, *old_tablet, tx_data, ddl_data, autoinc_seq))) {
LOG_WARN("failed to init tablet", K(ret), KPC(old_tablet));
} else if (old_tablet->get_tablet_meta().clog_checkpoint_ts_ < new_tablet->get_tablet_meta().clog_checkpoint_ts_
&& OB_FAIL(freezer->tablet_freeze(tablet_id))) {
if (OB_ENTRY_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("failed to freeze tablet", K(ret), K(tablet_id), KPC(old_tablet), KPC(new_tablet));
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(new_tablet->set_memtable_clog_checkpoint_ts(param.tablet_meta_))) {
LOG_WARN("failed to set memtable clog checkpoint ts", K(ret), KPC(old_tablet), KPC(new_tablet), K(param));
} else if (OB_FAIL(ObTabletSlogHelper::write_create_tablet_slog(new_tablet_handle, disk_addr))) {
LOG_WARN("fail to write update tablet slog", K(ret), K(new_tablet_handle), K(disk_addr));
} else if (OB_FAIL(t3m->compare_and_swap_tablet(key, disk_addr, old_tablet_handle, new_tablet_handle))) {

View File

@ -113,6 +113,7 @@ ObMemtable::ObMemtable()
state_(ObMemtableState::INVALID),
freeze_state_(ObMemtableFreezeState::INVALID),
timestamp_(0),
migration_clog_checkpoint_ts_(ObLogTsRange::MIN_TS),
is_tablet_freeze_(false),
is_force_freeze_(false),
is_flushed_(false),
@ -250,6 +251,7 @@ void ObMemtable::destroy()
unset_active_memtable_logging_blocked_ = false;
resolve_active_memtable_left_boundary_ = true;
max_end_log_ts_ = ObLogTsRange::MIN_TS;
migration_clog_checkpoint_ts_ = ObLogTsRange::MIN_TS;
rec_log_ts_ = INT64_MAX;
read_barrier_ = false;
is_tablet_freeze_ = false;
@ -1367,7 +1369,8 @@ int64_t ObMemtable::dec_write_ref()
} else {
if (0 == get_unsynced_cnt()) {
resolve_right_boundary();
if (OB_FAIL(memtable_mgr_->resolve_left_boundary_for_active_memtable(this, get_end_log_ts(), get_snapshot_version()))) {
int64_t new_start_log_ts = MAX(get_end_log_ts(), get_migration_clog_checkpoint_ts());
if (OB_FAIL(memtable_mgr_->resolve_left_boundary_for_active_memtable(this, new_start_log_ts, get_snapshot_version()))) {
TRANS_LOG(WARN, "fail to resolve left boundary for active memtable", K(ret), K(ls_id), KPC(this));
}
}
@ -1400,7 +1403,8 @@ int ObMemtable::dec_unsynced_cnt()
} else if (is_frozen && 0 == write_ref_cnt && 0 == unsynced_cnt) {
resolve_right_boundary();
TRANS_LOG(INFO, "[resolve_right_boundary] dec_unsynced_cnt", K(ls_id), KPC(this));
if (OB_FAIL(memtable_mgr_->resolve_left_boundary_for_active_memtable(this, get_end_log_ts(), get_snapshot_version()))) {
int64_t new_start_log_ts = MAX(get_end_log_ts(), get_migration_clog_checkpoint_ts());
if (OB_FAIL(memtable_mgr_->resolve_left_boundary_for_active_memtable(this, new_start_log_ts, get_snapshot_version()))) {
TRANS_LOG(WARN, "fail to set start log ts for active memtable", K(ret), K(ls_id), KPC(this));
}
TRANS_LOG(INFO, "memtable log synced", K(ret), K(ls_id), KPC(this));
@ -1434,6 +1438,23 @@ int ObMemtable::get_frozen_schema_version(int64_t &schema_version) const
return OB_NOT_SUPPORTED;
}
int ObMemtable::set_migration_clog_checkpoint_ts(const int64_t clog_checkpoint_ts)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
TRANS_LOG(WARN, "not inited", K(ret));
} else if (clog_checkpoint_ts <= ObLogTsRange::MIN_TS) {
ret = OB_LOG_TS_OUT_OF_BOUND;
TRANS_LOG(WARN, "invalid clog_checkpoint_ts", K(ret));
} else {
ATOMIC_STORE(&migration_clog_checkpoint_ts_, clog_checkpoint_ts);
}
return ret;
}
int ObMemtable::set_snapshot_version(const int64_t snapshot_version)
{
int ret = OB_SUCCESS;

View File

@ -432,6 +432,8 @@ public:
blocksstable::ObDatumRange &m_get_real_range(blocksstable::ObDatumRange &real_range,
const blocksstable::ObDatumRange &range, const bool is_reverse) const;
int get_tx_table_guard(storage::ObTxTableGuard &tx_table_guard);
int set_migration_clog_checkpoint_ts(const int64_t clog_checkpoint_ts);
int64_t get_migration_clog_checkpoint_ts() { return ATOMIC_LOAD(&migration_clog_checkpoint_ts_); }
/* multi source data operations */
virtual int get_multi_source_data_unit(ObIMultiSourceDataUnit *multi_source_data_unit, ObIAllocator *allocator);
@ -532,6 +534,7 @@ private:
int64_t state_;
int64_t freeze_state_;
int64_t timestamp_;
int64_t migration_clog_checkpoint_ts_;
bool is_tablet_freeze_;
bool is_force_freeze_;
bool is_flushed_;

View File

@ -1816,18 +1816,10 @@ int ObTablet::try_update_table_store_flag(const ObUpdateTableStoreParam &param)
int ObTablet::build_migration_tablet_param(ObMigrationTabletParam &mig_tablet_param) const
{
int ret = OB_SUCCESS;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTabletTxMultiSourceDataUnit &tx_data = mig_tablet_param.tx_data_;
const ObTabletMapKey key(tablet_meta_.ls_id_, tablet_meta_.tablet_id_);
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("not inited", K(ret), K_(is_inited));
} else if (OB_FAIL(t3m->get_tablet_pointer_tx_data(key, tx_data))) {
LOG_WARN("failed to get tx data in tablet pointer", K(ret), K(key));
} else if (OB_UNLIKELY(ObTabletStatus::MAX == tx_data.tablet_status_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected tablet status", K(ret), K(key), K(tx_data));
} else {
mig_tablet_param.ls_id_ = tablet_meta_.ls_id_;
mig_tablet_param.tablet_id_ = tablet_meta_.tablet_id_;
@ -1840,12 +1832,13 @@ int ObTablet::build_migration_tablet_param(ObMigrationTabletParam &mig_tablet_pa
mig_tablet_param.autoinc_seq_ = tablet_meta_.autoinc_seq_;
mig_tablet_param.compat_mode_ = tablet_meta_.compat_mode_;
mig_tablet_param.ha_status_ = tablet_meta_.ha_status_;
mig_tablet_param.report_status_ = tablet_meta_.report_status_;
mig_tablet_param.tx_data_ = tablet_meta_.tx_data_;
mig_tablet_param.table_store_flag_ = tablet_meta_.table_store_flag_;
mig_tablet_param.ddl_checkpoint_ts_ = tablet_meta_.ddl_checkpoint_ts_;
mig_tablet_param.ddl_start_log_ts_ = tablet_meta_.ddl_start_log_ts_;
mig_tablet_param.ddl_snapshot_version_ = tablet_meta_.ddl_snapshot_version_;
mig_tablet_param.max_sync_storage_schema_version_ = tablet_meta_.max_sync_storage_schema_version_;
mig_tablet_param.report_status_.reset();
if (OB_FAIL(mig_tablet_param.storage_schema_.init(mig_tablet_param.allocator_, storage_schema_))) {
LOG_WARN("failed to copy storage schema", K(ret), K_(tablet_meta));
@ -2458,8 +2451,6 @@ int ObTablet::set_tx_log_ts(
tx_data.tx_log_ts_ = log_ts;
if (OB_FAIL(save_multi_source_data_unit(&tx_data, log_ts, for_replay, memtable::MemtableRefOp::DEC_REF, true/*is_callback*/))) {
LOG_WARN("failed to save tx data", K(ret), K(tx_data), K(log_ts));
} else if (OB_FAIL(set_tx_data_in_tablet_pointer(tx_data))) {
LOG_WARN("failed to set tx data in tablet pointer", K(ret), K(tx_data));
}
}
@ -2482,8 +2473,6 @@ int ObTablet::set_tablet_final_status(
LOG_WARN("invalid args", K(ret), K(tx_data), K(memtable_log_ts), K(for_replay));
} else if (OB_FAIL(set_multi_data_for_commit(tx_data, memtable_log_ts, for_replay, ref_op))) {
LOG_WARN("failed to save tx data", K(ret), K(tx_data), K(memtable_log_ts), K(for_replay), K(ref_op));
} else if (OB_FAIL(set_tx_data_in_tablet_pointer(tx_data))) {
LOG_WARN("failed to set tx data in tablet pointer", K(ret), K(tx_data));
}
return ret;
@ -2492,7 +2481,6 @@ int ObTablet::set_tablet_final_status(
int ObTablet::set_tx_data(
const ObTabletTxMultiSourceDataUnit &tx_data,
const bool for_replay,
const bool update_cache,
const MemtableRefOp ref_op,
const bool is_callback)
{
@ -2506,9 +2494,7 @@ int ObTablet::set_tx_data(
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(tx_data));
} else if (OB_FAIL(save_multi_source_data_unit(&tx_data, log_ts, for_replay, ref_op, is_callback))) {
LOG_WARN("failed to save tx data", K(ret), K(tx_data), K(log_ts), K(for_replay), K(update_cache), K(ref_op), K(is_callback));
} else if (update_cache && OB_FAIL(set_tx_data_in_tablet_pointer(tx_data))) {
LOG_WARN("failed to set tx data in tablet pointer", K(ret), K(tx_data));
LOG_WARN("failed to save tx data", K(ret), K(tx_data), K(log_ts), K(for_replay), K(ref_op), K(is_callback));
}
return ret;
@ -2518,7 +2504,6 @@ int ObTablet::set_tx_data(
const ObTabletTxMultiSourceDataUnit &tx_data,
const int64_t memtable_log_ts,
const bool for_replay,
const bool update_cache,
const memtable::MemtableRefOp ref_op,
const bool is_callback)
{
@ -2532,9 +2517,7 @@ int ObTablet::set_tx_data(
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(tx_data), K(memtable_log_ts));
} else if (OB_FAIL(save_multi_source_data_unit(&tx_data, memtable_log_ts, for_replay, ref_op, is_callback))) {
LOG_WARN("failed to save tx data", K(ret), K(tx_data), K(memtable_log_ts), K(for_replay), K(update_cache), K(ref_op), K(is_callback));
} else if (update_cache && OB_FAIL(set_tx_data_in_tablet_pointer(tx_data))) {
LOG_WARN("failed to set tx data in tablet pointer", K(ret), K(tx_data));
LOG_WARN("failed to save tx data", K(ret), K(tx_data), K(memtable_log_ts), K(for_replay), K(ref_op), K(is_callback));
}
return ret;
@ -2698,23 +2681,8 @@ int ObTablet::get_msd_from_memtable(memtable::ObIMultiSourceDataUnit &msd) const
bool exist_on_memtable = false;
if (is_ls_inner_tablet()) {
exist_on_memtable = false;
} else if (OB_FAIL(memtable_mgr_->get_multi_source_data_unit(&msd))) {
if (OB_ENTRY_NOT_EXIST == ret) {
LOG_DEBUG("multi source data does not exist on memtable", K(ret), K(ls_id), K(tablet_id), K(tablet_meta_));
ret = OB_SUCCESS;
exist_on_memtable = false;
} else {
LOG_WARN("failed to get multi source data", K(ret), K(ls_id), K(tablet_id));
}
}
// NOTICE: memtable may be released after minor freezing, so here we may find that there is no memtable in memtable mgr.
// In this situation, we should get memtables from table store, and try to read tx data again
if (OB_FAIL(ret)) {
} else if (is_ls_inner_tablet()) {
// won't do anything for ls inner tablet
} else if (!exist_on_memtable) {
} else {
ObSEArray<ObITable*, MAX_MEMSTORE_CNT> memtable_array;
if (OB_FAIL(get_memtables(memtable_array, true/*need_active*/))) {
LOG_WARN("failed to get memtables", K(ret));
@ -2767,14 +2735,11 @@ int ObTablet::set_tx_data_in_tablet_pointer()
int ret = OB_SUCCESS;
const ObLSID &ls_id = tablet_meta_.ls_id_;
const ObTabletID &tablet_id = tablet_meta_.tablet_id_;
ObTabletTxMultiSourceDataUnit tx_data;
bool exist_on_memtable = true;
ObTabletTxMultiSourceDataUnit &tx_data = tablet_meta_.tx_data_;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not inited", K(ret), K_(is_inited));
} else if (OB_FAIL(inner_get_tx_data(tx_data, exist_on_memtable))) {
LOG_WARN("failed to get tx data", K(ret), K(ls_id), K(tablet_id));
} else if (OB_FAIL(set_tx_data_in_tablet_pointer(tx_data))) {
LOG_WARN("failed to set tx data in tablet pointer", K(ret), K(ls_id), K(tablet_id), K(tx_data));
}
@ -2826,5 +2791,42 @@ int ObTablet::check_max_sync_schema_version() const
return ret;
}
int ObTablet::set_memtable_clog_checkpoint_ts(
const ObMigrationTabletParam *tablet_meta)
{
int ret = OB_SUCCESS;
ObIMemtableMgr *memtable_mgr = nullptr;
ObTableHandleV2 handle;
memtable::ObMemtable *memtable = nullptr;
if (OB_ISNULL(tablet_meta)) {
//no need to set memtable clog checkpoint ts
} else if (tablet_meta->clog_checkpoint_ts_ <= tablet_meta_.clog_checkpoint_ts_) {
//do nothing
} else if (OB_FAIL(get_memtable_mgr(memtable_mgr))) {
LOG_WARN("failed to get memtable mgr", K(ret));
} else if (tablet_meta_.tablet_id_.is_ls_inner_tablet()) {
if (memtable_mgr->has_memtable()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls inner tablet should not has memtable", K(ret), KPC(tablet_meta));
}
} else if (OB_FAIL(memtable_mgr->get_active_memtable(handle))) {
if (OB_ENTRY_NOT_EXIST != ret) {
LOG_WARN("failed to get active memtable for tablet", K(ret), KPC(this), KPC(tablet_meta));
} else {
ret = OB_SUCCESS;
}
} else if (OB_FAIL(handle.get_data_memtable(memtable))) {
LOG_WARN("failed to get memtalbe", K(ret), K(handle));
} else if (OB_ISNULL(memtable)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null memtable", K(ret), KPC(memtable));
} else if (OB_FAIL(memtable->set_migration_clog_checkpoint_ts(tablet_meta->clog_checkpoint_ts_))) {
LOG_WARN("failed to set migration clog checkpoint ts", K(ret), K(handle), KPC(this));
}
return ret;
}
} // namespace storage
} // namespace oceanbase

View File

@ -346,10 +346,11 @@ public:
const ObTabletTxMultiSourceDataUnit &tx_data,
const int64_t memtable_log_ts,
const bool for_replay,
const bool update_cache,
const memtable::MemtableRefOp ref_op = memtable::MemtableRefOp::NONE,
const bool is_callback = false);
int set_tx_data_in_tablet_pointer();
int set_memtable_clog_checkpoint_ts(
const ObMigrationTabletParam *tablet_meta);
TO_STRING_KV(KP(this), K_(wash_score), K_(ref_cnt), K_(tablet_meta), K_(table_store), K_(storage_schema));
private:
@ -447,13 +448,13 @@ private:
int set_tx_data(
const ObTabletTxMultiSourceDataUnit &tx_data,
const bool for_replay,
const bool update_cache,
const memtable::MemtableRefOp ref_op = memtable::MemtableRefOp::NONE,
const bool is_callback = false);
int get_msd_from_memtable(memtable::ObIMultiSourceDataUnit &msd) const;
int set_tx_data_in_tablet_pointer(const ObTabletTxMultiSourceDataUnit &tx_data);
int get_max_sync_storage_schema_version(int64_t &max_schema_version) const;
int check_max_sync_schema_version() const;
private:
static const int32_t TABLET_VERSION = 1;
private:

View File

@ -869,11 +869,11 @@ int ObTabletBindingHelper::lock_and_set_tx_data(ObTabletHandle &handle, ObTablet
need_update = false;
} else {
ret = OB_EAGAIN;
LOG_WARN("tablet binding locked by others", K(ret), "tablet_id", tablet->get_tablet_meta().tablet_id_, K(tx_data), K(old_tx_data));
LOG_WARN("tablet binding locked by others", K(ret), K(tablet_id), K(tx_data), K(old_tx_data));
}
if (OB_FAIL(ret)) {
} else if (need_update && OB_FAIL(tablet->set_tx_data(tx_data, log_ts, for_replay,
false/*update_cache*/, ref_op, false/*is_callback*/))) {
ref_op, false/*is_callback*/))) {
LOG_WARN("failed to save msd", K(ret), K(tx_data), K(log_ts), K(for_replay), K(ref_op));
} else if (OB_FAIL(t3m->insert_pinned_tablet(key))) {
LOG_WARN("failed to insert in tx tablet", K(ret), K(key));
@ -900,7 +900,6 @@ int ObTabletBindingHelper::lock_tablet_binding(ObTabletHandle &handle, const ObM
const int64_t old_log_ts = tx_data.tx_log_ts_;
bool need_update = true;
const int64_t memtable_log_ts = for_replay ? log_ts : INT64_MAX;
const bool update_cache = for_replay; // if for replay is true, we should update cache in tablet pointer
const MemtableRefOp ref_op = for_replay ? MemtableRefOp::NONE : MemtableRefOp::INC_REF;
if (!old_tx_id.is_valid()) {
tx_data.tx_id_ = tx_id;
@ -913,7 +912,7 @@ int ObTabletBindingHelper::lock_tablet_binding(ObTabletHandle &handle, const ObM
}
if (OB_FAIL(ret)) {
} else if (need_update && OB_FAIL(tablet->set_tx_data(tx_data, memtable_log_ts, for_replay,
update_cache, ref_op, false/*is_callback*/))) {
ref_op, false/*is_callback*/))) {
LOG_WARN("failed to save tx data", K(ret), K(tx_data), K(log_ts), K(for_replay), K(ref_op));
} else if (OB_FAIL(t3m->insert_pinned_tablet(key))) {
LOG_WARN("failed to insert in tx tablet", K(ret), K(key));
@ -964,7 +963,7 @@ int ObTabletBindingHelper::set_log_ts(ObTabletHandle &handle, const ObMulSourceD
LOG_WARN("failed to get data", K(ret));
} else if (OB_UNLIKELY(data.tx_id_ != tx_id)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cannot set log ts for unlocked tablet", K(ret), K(tx_id), K(data), K(tablet->get_tablet_meta().tablet_id_));
LOG_WARN("cannot set log ts for unlocked tablet", K(ret), K(tx_id), K(data), "tablet_id", tablet->get_tablet_meta().tablet_id_);
} else if (OB_UNLIKELY(OB_INVALID_TIMESTAMP == data.tx_log_ts_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid log ts", K(ret), K(tx_id), K(log_ts), K(data));
@ -972,9 +971,7 @@ int ObTabletBindingHelper::set_log_ts(ObTabletHandle &handle, const ObMulSourceD
LOG_WARN("log ts already set, may be bug or retry", K(ret), K(tx_id), K(log_ts), K(data));
} else {
data.tx_log_ts_ = log_ts;
const bool update_cache = true;
if (OB_FAIL(tablet->set_tx_data(data, log_ts, for_replay,
update_cache, memtable::MemtableRefOp::DEC_REF, true/*is_callback*/))) {
if (OB_FAIL(tablet->set_tx_data(data, log_ts, for_replay, memtable::MemtableRefOp::DEC_REF, true/*is_callback*/))) {
LOG_WARN("failed to save msd", K(ret), K(data), K(log_ts), K(for_replay));
}
}

View File

@ -296,11 +296,10 @@ int ObTabletCreateDeleteHelper::handle_special_tablets_for_replay(
tx_data.tx_id_ = trans_flags.tx_id_;
tx_data.tx_log_ts_ = trans_flags.log_ts_;
tx_data.tablet_status_ = ObTabletStatus::CREATING;
const bool update_cache = false;
const MemtableRefOp ref_op = MemtableRefOp::NONE;
if (OB_FAIL(tablet_handle.get_obj()->set_tx_data(tx_data, trans_flags.for_replay_,
update_cache, MemtableRefOp::NONE/*ref_op*/))) {
LOG_WARN("failed to set tx data", K(ret), K(tx_data), K(trans_flags));
if (OB_FAIL(tablet_handle.get_obj()->set_tx_data(tx_data, trans_flags.for_replay_, ref_op))) {
LOG_WARN("failed to set tx data", K(ret), K(tx_data), K(trans_flags), K(ref_op));
} else if (OB_FAIL(t3m->insert_pinned_tablet(key))) {
LOG_WARN("failed to insert in tx tablet", K(ret));
}
@ -2291,10 +2290,9 @@ int ObTabletCreateDeleteHelper::do_create_tablet(
tx_data.tx_id_ = tx_id;
tx_data.tx_log_ts_ = log_ts;
tx_data.tablet_status_ = ObTabletStatus::CREATING;
const bool update_cache = trans_flags.for_replay_; // if for replay is true, we should update cache in tablet pointer
if (OB_FAIL(tablet->set_tx_data(tx_data, trans_flags.for_replay_, update_cache, ref_op))) {
LOG_WARN("failed to set tx data", K(ret), K(tx_data), K(trans_flags), K(update_cache), K(ref_op));
if (OB_FAIL(tablet->set_tx_data(tx_data, trans_flags.for_replay_, ref_op))) {
LOG_WARN("failed to set tx data", K(ret), K(tx_data), K(trans_flags), K(ref_op));
} else if (OB_FAIL(t3m->insert_pinned_tablet(key))) {
LOG_WARN("failed to insert in tx tablet", K(ret), K(key));
}

View File

@ -217,7 +217,8 @@ int ObTabletMemtableMgr::create_memtable(const int64_t clog_checkpoint_ts,
last_frozen_memtable->resolve_right_boundary();
TRANS_LOG(INFO, "[resolve_right_boundary] create_memtable", K(for_replay), K(ls_id), KPC(last_frozen_memtable));
if (memtable != last_frozen_memtable) {
memtable->resolve_left_boundary(last_frozen_memtable->get_end_log_ts());
int64_t new_start_log_ts = MAX(last_frozen_memtable->get_end_log_ts(), last_frozen_memtable->get_migration_clog_checkpoint_ts());
memtable->resolve_left_boundary(new_start_log_ts);
}
}
// there is no frozen memtable and new sstable will not be generated,

View File

@ -257,11 +257,28 @@ int ObTabletMeta::init(
const int64_t max_sync_storage_schema_version = OB_ISNULL(tablet_meta) ?
old_tablet_meta.max_sync_storage_schema_version_ : MIN(old_tablet_meta.max_sync_storage_schema_version_,
tablet_meta->max_sync_storage_schema_version_);
const int64_t clog_checkpoint_ts = OB_ISNULL(tablet_meta) ?
old_tablet_meta.clog_checkpoint_ts_ : MAX(old_tablet_meta.clog_checkpoint_ts_, tablet_meta->clog_checkpoint_ts_);
ObTabletTableStoreFlag table_store_flag = old_tablet_meta.table_store_flag_;
if (!table_store_flag.with_major_sstable()) {
table_store_flag = OB_ISNULL(tablet_meta) ? table_store_flag : tablet_meta->table_store_flag_;
}
if (OB_NOT_NULL(tablet_meta) && tablet_meta->clog_checkpoint_ts_ > old_tablet_meta.clog_checkpoint_ts_) {
if (OB_FAIL(autoinc_seq_.assign(tablet_meta->autoinc_seq_))) {
LOG_WARN("failed to assign autoinc seq", K(ret));
} else {
tx_data_ = tablet_meta->tx_data_;
}
} else {
if (OB_FAIL(autoinc_seq_.assign(autoinc_seq))) {
LOG_WARN("failed to assign autoinc seq", K(ret));
} else {
tx_data_ = tx_data;
}
}
version_ = TABLET_META_VERSION;
ls_id_ = old_tablet_meta.ls_id_;
tablet_id_ = old_tablet_meta.tablet_id_;
@ -269,17 +286,13 @@ int ObTabletMeta::init(
ref_tablet_id_ = old_tablet_meta.ref_tablet_id_;
create_scn_ = old_tablet_meta.create_scn_;
start_scn_ = old_tablet_meta.start_scn_;
clog_checkpoint_ts_ = old_tablet_meta.clog_checkpoint_ts_;
clog_checkpoint_ts_ = clog_checkpoint_ts;
ddl_checkpoint_ts_ = old_tablet_meta.ddl_checkpoint_ts_;
snapshot_version_ = snapshot_version;
multi_version_start_ = multi_version_start;
compat_mode_ = old_tablet_meta.compat_mode_;
if (FAILEDx(autoinc_seq_.assign(autoinc_seq))) {
LOG_WARN("failed to assign autoinc seq", K(ret));
}
ha_status_ = old_tablet_meta.ha_status_;
report_status_ = old_tablet_meta.report_status_;
tx_data_ = tx_data;
report_status_ = old_tablet_meta.report_status_; //old tablet meta report status already reset
if (FAILEDx(ddl_data_.assign(ddl_data))) {
LOG_WARN("failed to assign ddl data", K(ret));
}

View File

@ -40,7 +40,8 @@ ObTabletTxMultiSourceDataUnit::~ObTabletTxMultiSourceDataUnit()
}
ObTabletTxMultiSourceDataUnit::ObTabletTxMultiSourceDataUnit(const ObTabletTxMultiSourceDataUnit &other)
: version_(other.version_),
: ObIMultiSourceDataUnit(other),
version_(other.version_),
length_(other.length_),
tx_id_(other.tx_id_),
tx_log_ts_(other.tx_log_ts_),
@ -51,6 +52,7 @@ ObTabletTxMultiSourceDataUnit::ObTabletTxMultiSourceDataUnit(const ObTabletTxMul
ObTabletTxMultiSourceDataUnit &ObTabletTxMultiSourceDataUnit::operator=(const ObTabletTxMultiSourceDataUnit &other)
{
if (this != &other) {
ObIMultiSourceDataUnit::operator=(other);
version_ = other.version_;
length_ = other.length_;
tx_id_ = other.tx_id_;

View File

@ -1230,18 +1230,27 @@ int ObTabletTableStore::replace_ha_minor_sstables_(
LOG_WARN("failed to add need add minor tables", K(ret), K(param));
} else if (OB_FAIL(old_store.minor_tables_.get_all_tables(old_minor_tables))) {
LOG_WARN("failed to get old minor tables", K(ret), K(old_store));
} else if (OB_FAIL(check_minor_tables_continue_(need_add_minor_tables.count(), need_add_minor_tables.get_data()))) {
LOG_WARN("failed to check minor tables continue", K(ret), K(need_add_minor_tables), K(param));
} else if (OB_FAIL(check_old_store_minor_sstables_(old_minor_tables))) {
LOG_WARN("failed to check old store minor sstables", K(ret), K(old_minor_tables));
} else if (OB_FAIL(combin_ha_minor_sstables_(old_minor_tables, need_add_minor_tables, new_minor_tables))) {
LOG_WARN("failed to combin ha minor sstables", K(ret), K(old_store), K(param));
} else if (new_minor_tables.empty()) { // no minor tables
LOG_INFO("minor tables is empty, skip it", K(ret), K(new_minor_tables));
if (tablet_ptr_->get_tablet_meta().start_scn_ != tablet_ptr_->get_tablet_meta().clog_checkpoint_ts_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet meta is not match with minor sstables", K(ret), K(new_minor_tables), K(param), K(old_store));
} else {
LOG_INFO("minor tables is empty, skip it", K(ret), K(new_minor_tables));
}
} else if (OB_FAIL(ObTableStoreUtil::sort_minor_tables(new_minor_tables))) {
LOG_WARN("failed to sort minor tables", K(ret));
} else if (OB_FAIL(cut_ha_sstable_log_ts_range_(new_minor_tables))) {
LOG_WARN("failed to cut ha sstable log ts range", K(ret), K(old_store), K(param));
} else if (OB_FAIL(check_minor_tables_continue_(new_minor_tables.count(), new_minor_tables.get_data()))) {
LOG_WARN("minor tables is not continue", K(ret), K(param), K(new_minor_tables), K(old_store));
} else if (new_minor_tables.at(0)->get_start_log_ts() != tablet_ptr_->get_tablet_meta().start_scn_
|| new_minor_tables.at(new_minor_tables.count() - 1)->get_end_log_ts() != tablet_ptr_->get_tablet_meta().clog_checkpoint_ts_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet meta is not match with minor sstables", K(ret), K(new_minor_tables), K(param), K(old_store));
} else if (OB_FAIL(minor_tables_.init_and_copy(allocator, new_minor_tables, inc_pos))) {
LOG_WARN("failed to init minor_tables", K(ret));
} else {