提交 5a50ec19 编写于 作者: O obdev 提交者: ob-robot

delete redundant code

上级 c538314f
......@@ -20,7 +20,7 @@ namespace palf
void SCN::reset()
{
ATOMIC_SET(&val_, OB_INVALID_SCN_VAL);
val_ = OB_INVALID_SCN_VAL;
}
SCN SCN::atomic_get() const
......@@ -279,14 +279,6 @@ int SCN::convert_for_gts(int64_t ts_ns)
return ret;
}
// TODO(SCN):yaoying.yyy
int SCN::convert_tmp(int64_t ts_ns)
{
ts_ns_ = ts_ns;
v_ = SCN_VERSION;
return OB_SUCCESS;
}
int SCN::convert_for_lsn_allocator(uint64_t scn_val)
{
int ret = OB_SUCCESS;
......
......@@ -62,7 +62,6 @@ public:
// @param[in] :timestamp with us
int convert_from_ts(uint64_t ts_us);
int convert_tmp(int64_t ts_ns);
// convert time_ns generated by gts to scn. only used by gts
// @param[in] time_ns: time_ns generated by gts
int convert_for_gts(int64_t time_ns);
......
#!/bin/sh
#!/bin/bash
echo -e '#include "objit/common/ob_item_type.h"'
echo -e "const char* get_type_name(int type)\n{"
echo -e "\tswitch(type){"
......
......@@ -855,7 +855,7 @@ int ObMultipleMerge::alloc_row_store(ObTableAccessContext &context, const ObTabl
block_row_store_ = new (buf) ObBlockRowStore(context);
}
}
}
}
if (OB_SUCC(ret) && nullptr != block_row_store_) {
if (OB_FAIL(block_row_store_->init(param))) {
LOG_WARN("fail to init block row store", K(ret), K(block_row_store_));
......
......@@ -46,7 +46,7 @@ int ObSingleMerge::open(const ObDatumRowkey &rowkey)
LOG_WARN("ObSingleMerge has not been inited", K(ret), K_(get_table_param));
} else {
const ObTabletMeta &tablet_meta = get_table_param_.tablet_iter_.tablet_handle_.get_obj()->get_tablet_meta();
if (OB_FAIL(fuse_row_cache_fetcher_.init(access_param_->iter_param_.tablet_id_, access_param_->iter_param_.get_read_info(), tablet_meta.clog_checkpoint_scn_.get_val_for_gts()))) {
if (OB_FAIL(fuse_row_cache_fetcher_.init(access_param_->iter_param_.tablet_id_, access_param_->iter_param_.get_read_info(), tablet_meta.clog_checkpoint_scn_.get_val_for_tx()))) {
STORAGE_LOG(WARN, "fail to init fuse row cache fetcher", K(ret));
} else {
rowkey_ = &rowkey;
......
......@@ -716,9 +716,8 @@ int ObTabletMergeCtx::inner_init_for_minor(bool &skip_rest_operation)
} else if (is_mini_merge(param_.merge_type_)) { // OB_NO_NEED_MERGE && mini merge
int tmp_ret = OB_SUCCESS;
// then release memtable
const int64_t clog_checkpoint_log_ts = tablet->get_tablet_meta().clog_checkpoint_scn_.get_val_for_gts();
if (OB_TMP_FAIL(tablet->release_memtables(clog_checkpoint_log_ts))) {
LOG_WARN("failed to release memtable", K(tmp_ret), K(clog_checkpoint_log_ts));
if (OB_TMP_FAIL(tablet->release_memtables(tablet->get_tablet_meta().clog_checkpoint_scn_))) {
LOG_WARN("failed to release memtable", K(tmp_ret), K(tablet->get_tablet_meta().clog_checkpoint_scn_));
}
}
} else if (FALSE_IT(time_guard_.click(ObCompactionTimeGuard::COMPACTION_POLICY))) {
......
......@@ -196,9 +196,8 @@ int ObTxTableMergePrepareTask::build_merge_ctx()
} else if (is_mini_merge(ctx.param_.merge_type_)) { // OB_NO_NEED_MERGE && mini merge
int tmp_ret = OB_SUCCESS;
// then release memtable
const int64_t clog_checkpoint_log_ts = tablet->get_tablet_meta().clog_checkpoint_scn_.get_val_for_gts();
if (OB_TMP_FAIL(tablet->release_memtables(clog_checkpoint_log_ts))) {
LOG_WARN("failed to release memtable", K(tmp_ret), K(clog_checkpoint_log_ts));
if (OB_TMP_FAIL(tablet->release_memtables(tablet->get_tablet_meta().clog_checkpoint_scn_))) {
LOG_WARN("failed to release memtable", K(tmp_ret), K(tablet->get_tablet_meta().clog_checkpoint_scn_));
}
}
} else if (OB_FAIL(ctx.get_basic_info_from_result(get_merge_table_result))) {
......
......@@ -1460,7 +1460,7 @@ int ObLSTabletService::inner_table_scan(
ret = OB_SNAPSHOT_DISCARDED;
} else if (OB_FAIL(ObTabletBindingHelper::check_snapshot_readable(
tablet_handle,
store_ctx.mvcc_acc_ctx_.get_snapshot_version().get_val_for_lsn_allocator()))) {
store_ctx.mvcc_acc_ctx_.get_snapshot_version().get_val_for_tx()))) {
LOG_WARN("failed to check snapshot readable", K(ret));
} else {
if (param.need_switch_param_) {
......@@ -1558,7 +1558,7 @@ int ObLSTabletService::do_create_tablet(
table_store_flag.set_without_major_sstable();
LOG_INFO("no need to create sstable", K(ls_id), K(tablet_id), K(table_schema));
} else if (OB_FAIL(ObTabletCreateDeleteHelper::build_create_sstable_param(
table_schema, tablet_id, snapshot_version.get_val_for_lsn_allocator(), param))) {
table_schema, tablet_id, snapshot_version.get_val_for_tx(), param))) {
LOG_WARN("failed to build create sstable param", K(ret), K(tablet_id),
K(table_schema), K(snapshot_version), K(param));
} else if (OB_FAIL(ObTabletCreateDeleteHelper::create_sstable(param, table_handle))) {
......@@ -1567,7 +1567,7 @@ int ObLSTabletService::do_create_tablet(
if (OB_FAIL(ret)) {
} else if (OB_FAIL(handle.get_obj()->init(ls_id, tablet_id, data_tablet_id, lob_meta_tablet_id, lob_piece_tablet_id,
create_scn.get_val_for_lsn_allocator(), snapshot_version.get_val_for_lsn_allocator(), table_schema, compat_mode, table_store_flag, table_handle, freezer))) {
create_scn, snapshot_version.get_val_for_tx(), table_schema, compat_mode, table_store_flag, table_handle, freezer))) {
LOG_WARN("failed to init tablet", K(ret), K(ls_id), K(tablet_id), K(data_tablet_id), K(index_tablet_array),
K(create_scn), K(snapshot_version), K(table_schema), K(compat_mode), K(table_store_flag));
......@@ -4611,7 +4611,7 @@ void ObLSTabletService::dump_diag_info_for_old_row_loss(
common::ObVersionRange trans_version_rang;
trans_version_rang.base_version_ = 0;
trans_version_rang.multi_version_start_ = 0;
trans_version_rang.snapshot_version_ = store_ctx.mvcc_acc_ctx_.get_snapshot_version().get_val_for_lsn_allocator();
trans_version_rang.snapshot_version_ = store_ctx.mvcc_acc_ctx_.get_snapshot_version().get_val_for_tx();
for (int64_t i = 0; OB_SUCC(ret) && i < full_read_info.get_request_count(); i++) {
if (OB_FAIL(out_col_pros.push_back(i))) {
......
......@@ -86,7 +86,7 @@ int ObDMLRunningCtx::init(
} else if (OB_FAIL(prepare_relative_table(
dml_param_.table_param_->get_data_table(),
tablet_handle,
store_ctx_.mvcc_acc_ctx_.get_snapshot_version().get_val_for_lsn_allocator()))) {
store_ctx_.mvcc_acc_ctx_.get_snapshot_version()))) {
LOG_WARN("failed to get relative table", K(ret), K(dml_param_));
} else if (NULL != column_ids && OB_FAIL(prepare_column_info(*column_ids))) {
LOG_WARN("fail to get column descriptions and column map", K(ret), K(*column_ids));
......@@ -128,7 +128,7 @@ int ObDMLRunningCtx::prepare_column_desc(
int ObDMLRunningCtx::prepare_relative_table(
const share::schema::ObTableSchemaParam &schema,
ObTabletHandle &tablet_handle,
const int64_t read_snapshot)
const palf::SCN &read_snapshot)
{
int ret = OB_SUCCESS;
if (OB_FAIL(relative_table_.init(&schema, tablet_handle.get_obj()->get_tablet_meta().tablet_id_,
......@@ -136,7 +136,7 @@ int ObDMLRunningCtx::prepare_relative_table(
LOG_WARN("fail to init relative_table_", K(ret), K(tablet_handle), K(schema.get_index_status()));
} else if (FALSE_IT(relative_table_.tablet_iter_.tablet_handle_ = tablet_handle)) {
} else if (OB_FAIL(tablet_handle.get_obj()->get_read_tables(
read_snapshot, relative_table_.tablet_iter_, relative_table_.allow_not_ready()))) {
read_snapshot.get_val_for_tx(), relative_table_.tablet_iter_, relative_table_.allow_not_ready()))) {
LOG_WARN("failed to get relative table read tables", K(ret));
}
return ret;
......
......@@ -16,6 +16,7 @@
#include "lib/container/ob_iarray.h"
#include "storage/ob_i_store.h"
#include "storage/ob_relative_table.h"
#include "logservice/palf/scn.h"
namespace oceanbase
{
......@@ -65,7 +66,7 @@ private:
int prepare_relative_table(
const share::schema::ObTableSchemaParam &schema,
ObTabletHandle &tablet_handle,
const int64_t read_snapshot);
const palf::SCN &read_snapshot);
int check_schema_version(share::schema::ObMultiVersionSchemaService &schema_service,
const uint64_t tenant_id,
const uint64_t table_id,
......
......@@ -25,6 +25,7 @@
#include "storage/meta_mem/ob_tablet_handle.h"
#include "storage/tx_storage/ob_ls_handle.h"//ObLSHandle
#include "storage/meta_mem/ob_tenant_meta_mem_mgr.h"
#include "logservice/palf/scn.h"
namespace oceanbase
{
......@@ -369,7 +370,7 @@ int ObStorageSchemaRecorder::dec_ref_on_memtable(const bool sync_finish)
KP_(storage_schema), K_(tablet_handle));
} else {
storage_schema_->set_sync_finish(sync_finish);
if (OB_FAIL(tablet_handle_.get_obj()->save_multi_source_data_unit(storage_schema_, clog_scn_.get_val_for_tx(),
if (OB_FAIL(tablet_handle_.get_obj()->save_multi_source_data_unit(storage_schema_, clog_scn_,
false/*for_replay*/, memtable::MemtableRefOp::DEC_REF, true/*is_callback*/))) {
LOG_WARN("failed to save storage schema", K(ret), K_(tablet_id), K(storage_schema_));
}
......@@ -496,7 +497,7 @@ int ObStorageSchemaRecorder::submit_schema_log(const int64_t table_id)
} else if (FALSE_IT(ATOMIC_STORE(&logcb_finish_flag_, false))) {
} else if (FALSE_IT(storage_schema_->set_sync_finish(false))) {
} else if (OB_FAIL(tablet_handle_.get_obj()->save_multi_source_data_unit(storage_schema_,
share::ObScnRange::MAX_TS, false/*for_replay*/, memtable::MemtableRefOp::INC_REF))) {
palf::SCN::max_scn(), false/*for_replay*/, memtable::MemtableRefOp::INC_REF))) {
if (OB_BLOCK_FROZEN != ret) {
LOG_WARN("failed to inc ref for storage schema", K(ret), K_(tablet_id), K(storage_schema_));
}
......
......@@ -21,7 +21,7 @@
#include "storage/tablet/ob_tablet.h"
#include "storage/tx_storage/ob_ls_service.h"
#include "storage/tx_storage/ob_ls_handle.h" //ObLSHandle
#include "storage/tx_storage/ob_tenant_freezer.h"
#include "storage/tx_storage/ob_tenant_freezer.h"
namespace oceanbase
{
......@@ -67,7 +67,7 @@ ObStorageTableGuard::~ObStorageTableGuard()
reset();
int tmp_ret = OB_SUCCESS;
bool has_sleep = false;
while ((left_interval > 0) && need_sleep) {
//because left_interval and SLEEP_INTERVAL_PER_TIME both are greater than
//zero, so it's safe to convert to uint32_t, be careful with comparation between int and uint
......
......@@ -119,7 +119,7 @@ int ObTablet::init(
const common::ObTabletID &lob_meta_tablet_id,
const common::ObTabletID &lob_piece_tablet_id,
const SCN &create_scn,
const SCN &snapshot_version,
const int64_t snapshot_version,
const share::schema::ObTableSchema &table_schema,
const lib::Worker::CompatMode compat_mode,
const ObTabletTableStoreFlag &store_flag,
......@@ -136,7 +136,7 @@ int ObTablet::init(
|| OB_UNLIKELY(!tablet_id.is_valid())
|| OB_UNLIKELY(!data_tablet_id.is_valid())
//|| OB_UNLIKELY(create_scn <= OB_INVALID_TIMESTAMP)
|| OB_UNLIKELY(!snapshot_version.is_valid())
|| OB_UNLIKELY(snapshot_version <= OB_INVALID_TIMESTAMP)
|| OB_UNLIKELY(!table_schema.is_valid())
|| OB_UNLIKELY(lib::Worker::CompatMode::INVALID == compat_mode)
|| OB_ISNULL(freezer)) {
......@@ -152,7 +152,7 @@ int ObTablet::init(
LOG_WARN("failed to init shared params", K(ret), K(ls_id), K(tablet_id), KP(freezer));
} else if (OB_FAIL(tablet_meta_.init(*allocator_, ls_id, tablet_id, data_tablet_id,
lob_meta_tablet_id, lob_piece_tablet_id,
create_scn, snapshot_version.get_val_for_tx(), compat_mode, store_flag, table_schema.get_schema_version()))) {
create_scn, snapshot_version, compat_mode, store_flag, table_schema.get_schema_version()))) {
LOG_WARN("failed to init tablet meta", K(ret), K(ls_id), K(tablet_id), K(data_tablet_id),
K(lob_meta_tablet_id), K(lob_piece_tablet_id),
K(create_scn), K(snapshot_version), K(compat_mode), K(store_flag));
......@@ -182,34 +182,6 @@ int ObTablet::init(
return ret;
}
int ObTablet::init(
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const common::ObTabletID &data_tablet_id,
const common::ObTabletID &lob_meta_tablet_id,
const common::ObTabletID &lob_piece_tablet_id,
const int64_t create_scn,
const int64_t snapshot_version,
const share::schema::ObTableSchema &table_schema,
const lib::Worker::CompatMode compat_mode,
const ObTabletTableStoreFlag &store_flag,
ObTableHandleV2 &table_handle,
ObFreezer *freezer)
{
int ret = OB_SUCCESS;
SCN scn;
SCN snapshot_scn;
if (OB_FAIL(scn.convert_tmp(create_scn))) {
LOG_WARN("failed to convert_tmp", K(ret), K(create_scn));
} else if (OB_FAIL(snapshot_scn.convert_for_tx(snapshot_version))) {
LOG_WARN("failed to convert scn", K(ret), K(snapshot_version));
} else if (OB_FAIL(init(ls_id, tablet_id, data_tablet_id, lob_meta_tablet_id, lob_piece_tablet_id,
scn, snapshot_scn, table_schema, compat_mode, store_flag, table_handle, freezer))) {
LOG_WARN("failed to init tablet", K(ret), K(scn));
}
return ret;
}
int ObTablet::init(
const ObUpdateTableStoreParam &param,
const ObTablet &old_tablet,
......@@ -1650,23 +1622,6 @@ int ObTablet::inner_create_memtable(
return ret;
}
int ObTablet::release_memtables(const int64_t log_ts)
{
int ret = OB_SUCCESS;
ObIMemtableMgr *memtable_mgr = nullptr;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("not inited", K(ret), K_(is_inited));
} else if (OB_FAIL(get_memtable_mgr(memtable_mgr))) {
LOG_WARN("failed to get memtable mgr", K(ret));
} else if (OB_FAIL(memtable_mgr->release_memtables(log_ts))) {
LOG_WARN("failed to release memtables", K(ret), K(log_ts));
}
return ret;
}
int ObTablet::release_memtables(const palf::SCN scn)
{
int ret = OB_SUCCESS;
......@@ -2116,7 +2071,7 @@ int ObTablet::write_sync_tablet_seq_log(ObTabletAutoincSeq &autoinc_seq,
} else if (autoinc_seq.get_intervals().count() == 0 && OB_FAIL(autoinc_seq.set_autoinc_seq_value(1))) {
// need to do this to ensure the intervals list size is always 1, so the memory size is same before and after clog.
LOG_WARN("failed to set autoinc seq value", K(ret));
} else if (OB_FAIL(save_multi_source_data_unit(&autoinc_seq, share::ObScnRange::MAX_TS,
} else if (OB_FAIL(save_multi_source_data_unit(&autoinc_seq, SCN::max_scn(),
false/*for_replay*/, memtable::MemtableRefOp::INC_REF))) {
if (OB_BLOCK_FROZEN == ret) {
ret = OB_EAGAIN;
......@@ -2132,7 +2087,7 @@ int ObTablet::write_sync_tablet_seq_log(ObTabletAutoincSeq &autoinc_seq,
LOG_WARN("fail to submit sync tablet seq log", K(ret), K(buffer_size));
// rollback, dec ref
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(save_multi_source_data_unit(&autoinc_seq, share::ObScnRange::MAX_TS,
if (OB_TMP_FAIL(save_multi_source_data_unit(&autoinc_seq, SCN::max_scn(),
false/*for_replay*/, memtable::MemtableRefOp::DEC_REF, true/*is_callback*/))) {
LOG_ERROR("failed to dec ref for auto inc seq", K(tmp_ret));
ob_usleep(1000 * 1000);
......@@ -2769,10 +2724,9 @@ int ObTablet::get_tablet_status(ObTabletStatus::Status &tablet_status)
return ret;
}
int ObTablet::get_rec_log_ts(int64_t &rec_log_ts)
{
int ObTablet::get_rec_log_scn(SCN &rec_scn) {
int ret = OB_SUCCESS;
rec_log_ts = OB_MAX_SCN_TS_NS;
rec_scn = SCN::max_scn();
ObTableHandleV2 handle;
memtable::ObMemtable *mt;
if (IS_NOT_INIT) {
......@@ -2793,18 +2747,7 @@ int ObTablet::get_rec_log_ts(int64_t &rec_log_ts)
ret = OB_ERR_UNEXPECTED;
LOG_WARN("mt is NULL", KR(ret), K(handle));
} else {
rec_log_ts = mt->get_rec_scn().get_val_for_tx();
}
return ret;
}
int ObTablet::get_rec_log_scn(SCN &rec_scn) {
int ret = OB_SUCCESS;
int64_t rec_log_ts = OB_MAX_SCN_TS_NS;
if (OB_FAIL(get_rec_log_ts(rec_log_ts))) {
LOG_WARN("fail to get rec log ts", K(rec_log_ts), K(ret));
} else if (OB_FAIL(rec_scn.convert_tmp(rec_log_ts))) {
LOG_WARN("fail to convert from rec_log_ts", K(ret), K(rec_log_ts));
rec_scn = mt->get_rec_scn();
}
return ret;
}
......
......@@ -116,7 +116,6 @@ public:
int64_t dec_ref();
int64_t get_ref() const { return ATOMIC_LOAD(&ref_cnt_); }
int64_t get_wash_score() const { return ATOMIC_LOAD(&wash_score_); }
int get_rec_log_ts(int64_t &rec_log_ts);
int get_rec_log_scn(palf::SCN &rec_scn);
public:
// first time create tablet
......@@ -127,20 +126,6 @@ public:
const common::ObTabletID &lob_meta_tablet_id,
const common::ObTabletID &lob_piece_tablet_id,
const palf::SCN &create_scn,
const palf::SCN &snapshot_version,
const share::schema::ObTableSchema &table_schema,
const lib::Worker::CompatMode compat_mode,
const ObTabletTableStoreFlag &store_flag,
ObTableHandleV2 &table_handle,
ObFreezer *freezer);
int init(
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const common::ObTabletID &data_tablet_id,
const common::ObTabletID &lob_meta_tablet_id,
const common::ObTabletID &lob_piece_tablet_id,
const int64_t create_scn,
const int64_t snapshot_version,
const share::schema::ObTableSchema &table_schema,
const lib::Worker::CompatMode compat_mode,
......@@ -234,7 +219,6 @@ public:
ObIMemtableMgr *get_memtable_mgr() const { return memtable_mgr_; } // TODO(bowen.gbw): get memtable mgr from tablet pointer handle
// get the active memtable for write or replay.
int get_active_memtable(ObTableHandleV2 &handle) const;
int release_memtables(const int64_t log_ts);
int release_memtables(const palf::SCN scn);
// force release all memtables
// just for rebuild or migrate retry.
......@@ -254,15 +238,6 @@ public:
template<class T>
int set_multi_data_for_commit(T &multi_source_data_unit, const palf::SCN &log_scn, const bool for_replay, const memtable::MemtableRefOp ref_op);
template<class T>
int save_multi_source_data_unit(
const T *const msd,
const int64_t memtable_log_ts,
const bool for_replay,
const memtable::MemtableRefOp ref_op = memtable::MemtableRefOp::NONE,
const bool is_callback = false);
template<class T>
int save_multi_source_data_unit(
const T *const msd,
......@@ -324,7 +299,6 @@ public:
// other
const ObTabletMeta &get_tablet_meta() const { return tablet_meta_; }
const ObTabletTableStore &get_table_store() const { return table_store_; }
int64_t get_clog_checkpoint_ts() const { return tablet_meta_.clog_checkpoint_scn_.get_val_for_gts(); }
palf::SCN get_clog_checkpoint_scn() const { return tablet_meta_.clog_checkpoint_scn_; }
int64_t get_snapshot_version() const { return tablet_meta_.snapshot_version_; }
int64_t get_multi_version_start() const { return tablet_meta_.multi_version_start_; }
......@@ -772,24 +746,6 @@ int ObTablet::save_multi_source_data_unit(
return ret;
}
template<class T>
int ObTablet::save_multi_source_data_unit(
const T *const msd,
const int64_t log_ts,
const bool for_replay,
const memtable::MemtableRefOp ref_op,
const bool is_callback)
{
int ret = OB_SUCCESS;
palf::SCN scn;
if (OB_FAIL(scn.convert_tmp(log_ts))) {
TRANS_LOG(WARN, "failed to convert_tmp", K(ret), K(log_ts));
} else if (OB_FAIL(save_multi_source_data_unit(msd, scn, for_replay, ref_op, is_callback))) {
TRANS_LOG(WARN, "failed to init tablet", K(ret), K(scn), K(for_replay));
}
return ret;
}
} // namespace storage
} // namespace oceanbase
......
......@@ -337,7 +337,7 @@ int ObTabletCreateDeleteHelper::set_scn(
} else if (FALSE_IT(tablet = tablet_handle.get_obj())) {
} else if (OB_TMP_FAIL(tablet->set_tx_scn(tx_id, scn, for_replay))) {
LOG_WARN("failed to set tx log ts", K(tmp_ret), K(key), K(scn));
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn.get_val_for_inner_table_field()))) {
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn))) {
LOG_WARN("failed to update create scn", K(tmp_ret), K(scn));
}
}
......@@ -352,7 +352,7 @@ int ObTabletCreateDeleteHelper::set_scn(
} else if (FALSE_IT(tablet = tablet_handle.get_obj())) {
} else if (OB_TMP_FAIL(tablet->set_tx_scn(tx_id, scn, for_replay))) {
LOG_WARN("failed to set tx log ts", K(tmp_ret), K(key), K(scn));
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn.get_val_for_inner_table_field()))) {
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn))) {
LOG_WARN("failed to update create scn", K(tmp_ret), K(scn));
}
}
......@@ -367,7 +367,7 @@ int ObTabletCreateDeleteHelper::set_scn(
} else if (FALSE_IT(tablet = tablet_handle.get_obj())) {
} else if (OB_TMP_FAIL(tablet->set_tx_scn(tx_id, scn, for_replay))) {
LOG_WARN("failed to set tx log ts", K(tmp_ret), K(key), K(scn));
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn.get_val_for_inner_table_field()))) {
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn))) {
LOG_WARN("failed to update create scn", K(tmp_ret), K(scn));
}
}
......@@ -383,7 +383,7 @@ int ObTabletCreateDeleteHelper::set_scn(
} else if (FALSE_IT(tablet = tablet_handle.get_obj())) {
} else if (OB_TMP_FAIL(tablet->set_tx_scn(tx_id, scn, for_replay))) {
LOG_WARN("failed to set tx log ts", K(tmp_ret), K(key), K(scn));
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn.get_val_for_inner_table_field()))) {
} else if (OB_TMP_FAIL(tablet->tablet_meta_.update_create_scn(scn))) {
LOG_WARN("failed to update create scn", K(tmp_ret), K(scn));
}
if (OB_SUCC(ret) && OB_TMP_FAIL(tmp_ret)) {
......@@ -634,13 +634,10 @@ int ObTabletCreateDeleteHelper::do_commit_create_tablet(
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet status is not CREATING", K(ret), K(key), K(trans_flags), K(tx_data));
} else {
const int64_t tx_log_ts = trans_flags.scn_.get_val_for_lsn_allocator();
const int64_t memtable_log_ts = trans_flags.scn_.get_val_for_lsn_allocator();
if (OB_FAIL(set_tablet_final_status(tablet_handle, ObTabletStatus::NORMAL,
tx_log_ts, memtable_log_ts, trans_flags.for_replay_))) {
trans_flags.scn_, trans_flags.scn_, trans_flags.for_replay_))) {
LOG_WARN("failed to set tablet status to NORMAL", K(ret), K(tablet_handle),
K(tx_log_ts), K(memtable_log_ts), K(trans_flags));
K(trans_flags.scn_), K(trans_flags));
} else if (OB_FAIL(t3m->erase_pinned_tablet(key))) {
LOG_ERROR("failed to erase tablet handle", K(ret), K(key));
ob_usleep(1000 * 1000);
......@@ -928,13 +925,10 @@ int ObTabletCreateDeleteHelper::do_abort_create_tablet(
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet status is not CREATING", K(ret), K(tablet_id), K(trans_flags), K(tx_data));
} else {
const int64_t tx_log_ts = trans_flags.scn_.get_val_for_inner_table_field();
const int64_t memtable_log_ts = trans_flags.scn_.get_val_for_inner_table_field();
if (OB_FAIL(set_tablet_final_status(tablet_handle, ObTabletStatus::DELETED,
tx_log_ts, memtable_log_ts, trans_flags.for_replay_))) {
trans_flags.scn_, trans_flags.scn_, trans_flags.for_replay_))) {
LOG_WARN("failed to set tablet status to DELETED", K(ret), K(tablet_handle),
K(tx_log_ts), K(memtable_log_ts), K(trans_flags));
K(trans_flags.scn_), K(trans_flags));
} else if (OB_FAIL(t3m->erase_pinned_tablet(key))) {
LOG_ERROR("failed to erase tablet handle", K(ret), K(key));
ob_usleep(1000 * 1000);
......@@ -1162,13 +1156,10 @@ int ObTabletCreateDeleteHelper::do_commit_remove_tablet(
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet status is not DELETING", K(ret), K(key), K(trans_flags), K(tx_data));
} else {
const int64_t tx_log_ts = trans_flags.scn_.get_val_for_inner_table_field();
const int64_t memtable_log_ts = trans_flags.scn_.get_val_for_inner_table_field();
if (OB_FAIL(set_tablet_final_status(tablet_handle, ObTabletStatus::DELETED,
tx_log_ts, memtable_log_ts, trans_flags.for_replay_))) {
trans_flags.scn_, trans_flags.scn_, trans_flags.for_replay_))) {
LOG_WARN("failed to set tablet status to DELETED", K(ret), K(tablet_handle),
K(tx_log_ts), K(memtable_log_ts), K(trans_flags));
K(trans_flags.scn_), K(trans_flags));
} else if (OB_FAIL(t3m->erase_pinned_tablet(key))) {
LOG_ERROR("failed to erase tablet handle", K(ret), K(key));
ob_usleep(1000 * 1000);
......@@ -1292,15 +1283,12 @@ int ObTabletCreateDeleteHelper::do_abort_remove_tablet(
LOG_INFO("tablet is no valid but do nothing", K(ret), K(key), K(trans_flags));
}
} else {
const int64_t tx_log_ts = tx_data.tx_scn_.get_val_for_inner_table_field();
const int64_t memtable_log_ts = (palf::SCN::invalid_scn() == trans_flags.scn_)
? share::ObScnRange::MAX_TS
: trans_flags.scn_.get_val_for_inner_table_field();
const SCN memtable_scn = (SCN::invalid_scn() == trans_flags.scn_) ? SCN::max_scn() : trans_flags.scn_;
if (OB_FAIL(set_tablet_final_status(tablet_handle, ObTabletStatus::NORMAL,
tx_log_ts, memtable_log_ts, trans_flags.for_replay_, ref_op))) {
tx_data.tx_scn_, memtable_scn, trans_flags.for_replay_, ref_op))) {
LOG_WARN("failed to set tablet status to NORMAL", K(ret), K(tablet_handle),
K(tx_log_ts), K(memtable_log_ts), K(trans_flags), K(ref_op));
K(tx_data.tx_scn_), K(memtable_scn), K(trans_flags), K(ref_op));
} else if (OB_FAIL(t3m->erase_pinned_tablet(key))) {
LOG_ERROR("failed to erase tablet handle", K(ret), K(key));
ob_usleep(1000 * 1000);
......@@ -2273,7 +2261,8 @@ int ObTabletCreateDeleteHelper::do_create_tablet(
} else if (!need_create_empty_major_sstable) {
table_store_flag.set_without_major_sstable();
LOG_INFO("no need to create sstable", K(ls_id), K(tablet_id), K(table_schema));
} else if (OB_FAIL(build_create_sstable_param(table_schema, tablet_id, arg.major_frozen_scn_, param))) {
} else if (OB_FAIL(build_create_sstable_param(
table_schema, tablet_id, arg.major_frozen_scn_.get_val_for_tx(), param))) {
LOG_WARN("failed to build create sstable param", K(ret), K(tablet_id),
K(table_schema), K(arg), K(param));
} else if (OB_FAIL(create_sstable(param, table_handle))) {
......@@ -2285,7 +2274,7 @@ int ObTabletCreateDeleteHelper::do_create_tablet(
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error, tablet is NULL", K(ret), K(tablet_handle));
} else if (OB_FAIL(tablet->init(ls_id, tablet_id, data_tablet_id, lob_meta_tablet_id, lob_piece_tablet_id,
create_scn, arg.major_frozen_scn_, table_schema, compat_mode, table_store_flag, table_handle, freezer))) {
create_scn, arg.major_frozen_scn_.get_val_for_tx(), table_schema, compat_mode, table_store_flag, table_handle, freezer))) {
LOG_WARN("failed to init tablet", K(ret), K(ls_id), K(tablet_id), K(data_tablet_id),
K(lob_meta_tablet_id), K(lob_piece_tablet_id), K(index_tablet_array),
K(arg), K(create_scn), K(table_schema), K(compat_mode), K(table_store_flag));
......@@ -2384,69 +2373,11 @@ int ObTabletCreateDeleteHelper::build_create_sstable_param(
return ret;
}
int ObTabletCreateDeleteHelper::build_create_sstable_param(
const ObTableSchema &table_schema,
const ObTabletID &tablet_id,
const SCN snapshot_version,
ObTabletCreateSSTableParam &param)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!table_schema.is_valid()
|| !tablet_id.is_valid()
|| !snapshot_version.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(table_schema), K(snapshot_version));
} else if (OB_FAIL(table_schema.get_encryption_id(param.encrypt_id_))) {
LOG_WARN("fail to get_encryption_id", K(ret), K(table_schema));
} else {
param.master_key_id_ = table_schema.get_master_key_id();
MEMCPY(param.encrypt_key_, table_schema.get_encrypt_key_str(), table_schema.get_encrypt_key_len());
const int64_t multi_version_col_cnt = ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt();
param.table_key_.table_type_ = ObITable::TableType::MAJOR_SSTABLE;
param.table_key_.tablet_id_ = tablet_id;
param.table_key_.version_range_.snapshot_version_ = snapshot_version.get_val_for_gts();
param.max_merged_trans_version_ = snapshot_version.get_val_for_gts();
param.schema_version_ = table_schema.get_schema_version();
param.create_snapshot_version_ = 0;
param.progressive_merge_round_ = table_schema.get_progressive_merge_round();
param.progressive_merge_step_ = 0;
param.table_mode_ = table_schema.get_table_mode_struct();
param.index_type_ = table_schema.get_index_type();
param.rowkey_column_cnt_ = table_schema.get_rowkey_column_num()
+ ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt();
param.root_block_addr_.set_none_addr();
param.data_block_macro_meta_addr_.set_none_addr();
param.root_row_store_type_ = ObRowStoreType::FLAT_ROW_STORE;
param.data_index_tree_height_ = 0;
param.index_blocks_cnt_ = 0;
param.data_blocks_cnt_ = 0;
param.micro_block_cnt_ = 0;
param.use_old_macro_block_count_ = 0;
param.column_cnt_ = table_schema.get_column_count() + multi_version_col_cnt;
param.data_checksum_ = 0;
param.occupy_size_ = 0;
param.ddl_scn_.set_min();
param.filled_tx_scn_.set_min();
param.original_size_ = 0;
param.compressor_type_ = ObCompressorType::NONE_COMPRESSOR;
if (OB_FAIL(ObSSTableMergeRes::fill_column_checksum_for_empty_major(param.column_cnt_,
param.column_checksums_))) {
LOG_WARN("fail to fill column checksum for empty major", K(ret), K(param));
}
}
return ret;
}
int ObTabletCreateDeleteHelper::set_tablet_final_status(
ObTabletHandle &tablet_handle,
const ObTabletStatus::Status status,
const int64_t tx_log_ts,
const int64_t memtable_log_ts,
const SCN &tx_scn,
const SCN &memtable_scn,
const bool for_replay,
const MemtableRefOp ref_op)
{
......@@ -2468,15 +2399,14 @@ int ObTabletCreateDeleteHelper::set_tablet_final_status(
} else {
tx_data.tx_id_ = ObTabletCommon::FINAL_TX_ID;
tx_data.tablet_status_ = status;
tx_data.tx_scn_ = tx_scn;
if (OB_FAIL(tx_data.tx_scn_.convert_tmp(tx_log_ts))) {
LOG_WARN("failed to convert tmp", K(tx_log_ts), KPC(tablet), K(status));
} else if (OB_FAIL(checker.wake_up(tx_data, memtable_log_ts, for_replay, ref_op))) {
if (OB_FAIL(checker.wake_up(tx_data, memtable_scn, for_replay, ref_op))) {
LOG_WARN("failed to wake up", K(ret), KPC(tablet), K(status),
K(tx_log_ts), K(memtable_log_ts), K(for_replay), K(ref_op));
K(tx_scn), K(memtable_scn), K(for_replay), K(ref_op));
} else {
LOG_INFO("succeeded to set tablet final status", K(ret), K(ls_id), K(tablet_id), K(status),
K(tx_log_ts), K(memtable_log_ts), K(for_replay), K(ref_op));
K(tx_scn), K(memtable_scn), K(for_replay), K(ref_op));
}
}
......
......@@ -134,11 +134,6 @@ public:
const common::ObTabletID &tablet_id,
const int64_t snapshot_version,
ObTabletCreateSSTableParam &param);
static int build_create_sstable_param(
const share::schema::ObTableSchema &table_schema,
const common::ObTabletID &tablet_id,
const palf::SCN snapshot_version,
ObTabletCreateSSTableParam &param);
static int create_sstable(
const ObTabletCreateSSTableParam &param,
ObTableHandleV2 &table_handle);
......@@ -194,8 +189,8 @@ private:
static int set_tablet_final_status(
ObTabletHandle &tablet_handle,
const ObTabletStatus::Status status,
const int64_t tx_log_ts,
const int64_t memtable_log_ts,
const palf::SCN &tx_scn,
const palf::SCN &memtable_scn,
const bool for_replay,
const memtable::MemtableRefOp ref_op = memtable::MemtableRefOp::NONE);
static bool check_tablet_status(
......
......@@ -61,73 +61,6 @@ ObTabletMeta::~ObTabletMeta()
reset();
}
int ObTabletMeta::init(
ObIAllocator &allocator,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const common::ObTabletID &data_tablet_id,
const common::ObTabletID &lob_meta_tablet_id,
const common::ObTabletID &lob_piece_tablet_id,
const int64_t create_scn,
const int64_t snapshot_version,
const lib::Worker::CompatMode compat_mode,
const ObTabletTableStoreFlag &table_store_flag,
const int64_t max_sync_storage_schema_version)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(is_inited_)) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", K(ret), K_(is_inited));
} else if (OB_UNLIKELY(!ls_id.is_valid())
|| OB_UNLIKELY(!tablet_id.is_valid())
|| OB_UNLIKELY(!data_tablet_id.is_valid())
//|| OB_UNLIKELY(create_scn <= OB_INVALID_TIMESTAMP)
|| OB_UNLIKELY(OB_INVALID_VERSION == snapshot_version)
|| OB_UNLIKELY(lib::Worker::CompatMode::INVALID == compat_mode)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(ls_id), K(tablet_id), K(data_tablet_id),
K(create_scn), K(snapshot_version), K(compat_mode));
} else if (OB_FAIL(ha_status_.init_status())) {
LOG_WARN("failed to init ha status", K(ret));
} else {
version_ = TABLET_META_VERSION;
ls_id_ = ls_id;
tablet_id_ = tablet_id;
data_tablet_id_ = data_tablet_id;
create_scn_.convert_tmp(create_scn);
start_scn_.set_base();
clog_checkpoint_scn_.set_base();
ddl_checkpoint_scn_.set_base();
compat_mode_ = compat_mode;
snapshot_version_ = snapshot_version;
multi_version_start_ = snapshot_version;
table_store_flag_ = table_store_flag;
ddl_start_scn_.set_base();
ddl_snapshot_version_ = 0;
max_sync_storage_schema_version_ = max_sync_storage_schema_version;
report_status_.merge_snapshot_version_ = snapshot_version;
report_status_.cur_report_version_ = snapshot_version;
report_status_.data_checksum_ = 0;
report_status_.row_count_ = 0;
ddl_data_.lob_meta_tablet_id_ = lob_meta_tablet_id;
ddl_data_.lob_piece_tablet_id_ = lob_piece_tablet_id;
if (tablet_id_.is_ls_inner_tablet()) {
tx_data_.tablet_status_ = ObTabletStatus::NORMAL;
}
is_inited_ = true;
}
if (OB_UNLIKELY(!is_inited_)) {
reset();
}
return ret;
}
int ObTabletMeta::init(
ObIAllocator &allocator,
const share::ObLSID &ls_id,
......@@ -196,64 +129,6 @@ int ObTabletMeta::init(
return ret;
}
int ObTabletMeta::init(
common::ObIAllocator &allocator,
const ObTabletMeta &old_tablet_meta,
const int64_t snapshot_version,
const int64_t multi_version_start,
const ObTabletTxMultiSourceDataUnit &tx_data,
const ObTabletBindingInfo &ddl_data,
const ObTabletAutoincSeq &autoinc_seq,
const int64_t max_sync_storage_schema_version,
const int64_t clog_checkpoint_ts,
const int64_t ddl_checkpoint_ts,
const int64_t ddl_start_scn,
const int64_t ddl_snapshot_version)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(is_inited_)) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", K(ret), K_(is_inited));
} else if (OB_UNLIKELY(!old_tablet_meta.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(old_tablet_meta));
} else if (OB_FAIL(ddl_data_.assign(ddl_data))) {
LOG_WARN("failed to assign ddl data", K(ret));
} else if (OB_FAIL(autoinc_seq_.assign(autoinc_seq))) {
LOG_WARN("failed to assign autoinc seq", K(ret));
} else if (OB_FAIL(ddl_start_scn_.convert_for_tx(MAX(old_tablet_meta.ddl_start_scn_.get_val_for_tx(), ddl_start_scn)))) {
LOG_WARN("fail to convert scn", K(ret), K(ddl_start_scn));
} else if (OB_FAIL(clog_checkpoint_scn_.convert_for_tx(MAX(old_tablet_meta.clog_checkpoint_scn_.get_val_for_tx(), clog_checkpoint_ts)))) {
LOG_WARN("fail to convert scn", K(ret), K(clog_checkpoint_ts));
} else if (OB_FAIL(ddl_checkpoint_scn_.convert_for_tx(MAX(old_tablet_meta.ddl_checkpoint_scn_.get_val_for_tx(), ddl_checkpoint_ts)))) {
LOG_WARN("fail to convert scn", K(ret), K(ddl_checkpoint_ts));
} else {
version_ = TABLET_META_VERSION;
ls_id_ = old_tablet_meta.ls_id_;
tablet_id_ = old_tablet_meta.tablet_id_;
data_tablet_id_ = old_tablet_meta.data_tablet_id_;
ref_tablet_id_ = old_tablet_meta.ref_tablet_id_;
create_scn_ = old_tablet_meta.create_scn_;
start_scn_ = old_tablet_meta.start_scn_;
compat_mode_ = old_tablet_meta.compat_mode_;
ha_status_ = old_tablet_meta.ha_status_;
report_status_ = old_tablet_meta.report_status_;
snapshot_version_ = MAX(snapshot_version, old_tablet_meta.snapshot_version_);
multi_version_start_ = MIN(MAX(multi_version_start, old_tablet_meta.multi_version_start_), snapshot_version_);
tx_data_ = tx_data;
table_store_flag_ = old_tablet_meta.table_store_flag_;
max_sync_storage_schema_version_ = max_sync_storage_schema_version;
ddl_snapshot_version_ = MAX(old_tablet_meta.ddl_snapshot_version_, ddl_snapshot_version);
is_inited_ = true;
}
if (OB_UNLIKELY(!is_inited_)) {
reset();
}
return ret;
}
int ObTabletMeta::init(
common::ObIAllocator &allocator,
const ObTabletMeta &old_tablet_meta,
......@@ -760,23 +635,6 @@ int ObTabletMeta::update(const ObMigrationTabletParam &param)
return ret;
}
int ObTabletMeta::update_create_scn(const int64_t create_scn)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not inited", K(ret), K_(is_inited));
} else if (OB_UNLIKELY(create_scn <= OB_INVALID_TIMESTAMP)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret), K(create_scn));
} else {
create_scn_.convert_tmp(create_scn);
}
return ret;
}
int ObTabletMeta::update_create_scn(const palf::SCN create_scn)
{
int ret = OB_SUCCESS;
......
......@@ -51,18 +51,6 @@ public:
ObTabletMeta &operator=(const ObTabletMeta &other) = delete;
~ObTabletMeta();
public:
int init(
common::ObIAllocator &allocator,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
const common::ObTabletID &data_tablet_id,
const common::ObTabletID &lob_meta_tablet_id,
const common::ObTabletID &lob_piece_tablet_id,
const int64_t create_scn,
const int64_t snapshot_version,
const lib::Worker::CompatMode compat_mode,
const ObTabletTableStoreFlag &table_store_flag,
const int64_t max_sync_storage_schema_version);
int init(
common::ObIAllocator &allocator,
const share::ObLSID &ls_id,
......@@ -75,20 +63,6 @@ public:
const lib::Worker::CompatMode compat_mode,
const ObTabletTableStoreFlag &table_store_flag,
const int64_t max_sync_storage_schema_version);
int init(
common::ObIAllocator &allocator,
const ObTabletMeta &old_tablet_meta,
const int64_t snapshot_version,
const int64_t multi_version_start,
const ObTabletTxMultiSourceDataUnit &tx_data,
const ObTabletBindingInfo &ddl_data,
const share::ObTabletAutoincSeq &autoinc_seq,
const int64_t max_sync_storage_schema_version,
const int64_t clog_checkpoint_scn = 0,
const int64_t ddl_checkpoint_scn = 0,
const int64_t ddl_start_scn = 0,
const int64_t ddl_snapshot_version = 0);
int init(
common::ObIAllocator &allocator,
const ObTabletMeta &old_tablet_meta,
......@@ -126,7 +100,6 @@ public:
int64_t get_serialize_size() const;
int update(const ObMigrationTabletParam &param);
int update_create_scn(const int64_t create_scn);
int update_create_scn(const palf::SCN create_scn);
public:
static int deserialize_id(
......
......@@ -155,23 +155,6 @@ int ObTabletStatusChecker::wake_up(
return ret;
}
int ObTabletStatusChecker::wake_up(
ObTabletTxMultiSourceDataUnit &tx_data,
const int64_t memtable_log_ts,
const bool for_replay,
const memtable::MemtableRefOp ref_op)
{
int ret = OB_SUCCESS;
SCN memtable_scn;
if (OB_FAIL(memtable_scn.convert_tmp(memtable_log_ts))) {
LOG_WARN("failed to convert scn", K(memtable_log_ts), K(ret));
} else if (OB_FAIL(wake_up(tx_data, memtable_scn, for_replay, ref_op))) {
LOG_WARN("failed to wake up", K(ret));
}
return ret;
}
int ObTabletStatusChecker::do_wait(common::ObThreadCond &cond, const uint64_t time_us)
{
int ret = OB_SUCCESS;
......
......@@ -114,11 +114,6 @@ public:
const palf::SCN &memtable_scn,
const bool for_replay,
const memtable::MemtableRefOp ref_op);
int wake_up(
ObTabletTxMultiSourceDataUnit &tx_data,
const int64_t memtable_log_ts,
const bool for_replay,
const memtable::MemtableRefOp ref_op);
private:
int do_wait(common::ObThreadCond &cond, const uint64_t time_ms);
static bool is_final_status(const ObTabletStatus::Status &status);
......
......@@ -1750,13 +1750,13 @@ void ObPrintTableStore::table_to_string(
? (static_cast<ObSSTable *>(table)->get_meta().get_basic_meta().contain_uncommitted_row_ ? "true" : "false")
: "unused";
BUF_PRINTF(" %-10s %-10s %-19lu %-19lu %-19lu %-19lu %-4ld %-16s ",
BUF_PRINTF(" %-10s %-10s %-19lu %-19lu %-10s %-10s %-4ld %-16s ",
table_arr,
table_name,
table->get_upper_trans_version(),
table->get_max_merged_trans_version(),
table->get_start_scn().get_val_for_inner_table_field(),
table->get_end_scn().get_val_for_inner_table_field(),
to_cstring(table->get_start_scn()),
to_cstring(table->get_end_scn()),
table->get_ref(),
uncommit_row);
}
......
......@@ -357,11 +357,11 @@ int TestCompactionPolicy::mock_tablet(
LOG_WARN("failed to acquire tablet", K(ret), K(key));
} else if (FALSE_IT(tablet = tablet_handle.get_obj())) {
} else if (OB_FAIL(tablet->init(ls_id, tablet_id, tablet_id, empty_tablet_id, empty_tablet_id,
0, snapshot_version, table_schema, compat_mode, table_store_flag, table_handle, ls_handle.get_ls()->get_freezer()))) {
palf::SCN::min_scn(), snapshot_version, table_schema, compat_mode, table_store_flag, table_handle, ls_handle.get_ls()->get_freezer()))) {
LOG_WARN("failed to init tablet", K(ret), K(ls_id), K(tablet_id), K(snapshot_version),
K(table_schema), K(compat_mode));
} else {
tablet->tablet_meta_.clog_checkpoint_scn_.convert_tmp(clog_checkpoint_ts);
tablet->tablet_meta_.clog_checkpoint_scn_.convert_for_lsn_allocator(clog_checkpoint_ts);
tablet->tablet_meta_.snapshot_version_ = snapshot_version;
}
return ret;
......@@ -726,7 +726,7 @@ TEST_F(TestCompactionPolicy, check_mini_merge_basic)
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(3, result.handle_.get_count());
tablet_handle_.get_obj()->tablet_meta_.clog_checkpoint_scn_.convert_tmp(300);
tablet_handle_.get_obj()->tablet_meta_.clog_checkpoint_scn_.convert_for_lsn_allocator(300);
tablet_handle_.get_obj()->tablet_meta_.snapshot_version_ = 300;
result.reset();
ret = ObPartitionMergePolicy::get_mini_merge_tables(param, 0, *tablet_handle_.get_obj(), result);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册