remove old mds code

This commit is contained in:
hiddenbomb
2023-09-10 08:40:31 +00:00
committed by ob-robot
parent f32a2298e8
commit 0cedfb8ea9
10 changed files with 22 additions and 164 deletions

View File

@ -137,7 +137,7 @@ private:
share::ObTransferPartList &not_exist_part_list, share::ObTransferPartList &not_exist_part_list,
share::ObTransferPartList &lock_failed_part_list, share::ObTransferPartList &lock_failed_part_list,
share::ObDisplayTabletList &table_lock_tablet_list, share::ObDisplayTabletList &table_lock_tablet_list,
common::ObIArray<ObTabletID> &tablet_ids, common::ObIArray<common::ObTabletID> &tablet_ids,
transaction::tablelock::ObTableLockOwnerID &lock_owner_id); transaction::tablelock::ObTableLockOwnerID &lock_owner_id);
int unlock_table_and_part_( int unlock_table_and_part_(
ObMySQLTransaction &trans, ObMySQLTransaction &trans,
@ -151,19 +151,19 @@ private:
int get_tablet_and_partition_idx_by_object_id_( int get_tablet_and_partition_idx_by_object_id_(
share::schema::ObSimpleTableSchemaV2 &table_schema, share::schema::ObSimpleTableSchemaV2 &table_schema,
const ObObjectID &part_object_id, const ObObjectID &part_object_id,
ObTabletID &tablet_id, common::ObTabletID &tablet_id,
int64_t &part_idx); int64_t &part_idx);
int get_tablet_and_partition_idx_by_object_id_( int get_tablet_and_partition_idx_by_object_id_(
share::schema::ObSimpleTableSchemaV2 &table_schema, share::schema::ObSimpleTableSchemaV2 &table_schema,
const ObObjectID &part_object_id, const ObObjectID &part_object_id,
ObTabletID &tablet_id, common::ObTabletID &tablet_id,
int64_t &part_idx, int64_t &part_idx,
int64_t &subpart_idx); int64_t &subpart_idx);
int get_tablet_by_partition_idx_( int get_tablet_by_partition_idx_(
share::schema::ObSimpleTableSchemaV2 &table_schema, share::schema::ObSimpleTableSchemaV2 &table_schema,
const int64_t part_idx, const int64_t part_idx,
const int64_t subpart_idx, const int64_t subpart_idx,
ObTabletID &tablet_id); common::ObTabletID &tablet_id);
int check_tenant_schema_is_ready_(bool &is_ready); int check_tenant_schema_is_ready_(bool &is_ready);
int unlock_and_clear_task_( int unlock_and_clear_task_(
const share::ObTransferTaskID task_id, const share::ObTransferTaskID task_id,
@ -177,7 +177,7 @@ private:
const share::ObTransferPartInfo &part_info, const share::ObTransferPartInfo &part_info,
common::ObIAllocator &allocator, common::ObIAllocator &allocator,
share::schema::ObSimpleTableSchemaV2 *&table_schema, share::schema::ObSimpleTableSchemaV2 *&table_schema,
ObTabletID &tablet_id, common::ObTabletID &tablet_id,
int64_t &part_idx, int64_t &part_idx,
int64_t &subpart_idx); int64_t &subpart_idx);
int add_table_lock_( int add_table_lock_(
@ -191,14 +191,14 @@ private:
const transaction::tablelock::ObTableLockOwnerID &lock_owner_id, const transaction::tablelock::ObTableLockOwnerID &lock_owner_id,
share::schema::ObSimpleTableSchemaV2 &table_schema, share::schema::ObSimpleTableSchemaV2 &table_schema,
const share::ObTransferPartInfo &part_info, const share::ObTransferPartInfo &part_info,
const ObTabletID &tablet_id); const common::ObTabletID &tablet_id);
int generate_related_tablet_ids_( int generate_related_tablet_ids_(
share::schema::ObSimpleTableSchemaV2 &table_schema, share::schema::ObSimpleTableSchemaV2 &table_schema,
const int64_t part_idx, const int64_t part_idx,
const int64_t subpart_idx, const int64_t subpart_idx,
common::ObIArray<ObTabletID> &tablet_ids); common::ObIArray<common::ObTabletID> &tablet_ids);
int generate_tablet_list_( int generate_tablet_list_(
const ObIArray<ObTabletID> &tablet_ids, const common::ObIArray<common::ObTabletID> &tablet_ids,
share::ObTransferTabletList &tablet_list); share::ObTransferTabletList &tablet_list);
int unlock_table_lock_( int unlock_table_lock_(
ObMySQLTransaction &trans, ObMySQLTransaction &trans,
@ -220,7 +220,7 @@ private:
share::schema::ObSimpleTableSchemaV2 *&table_schema); share::schema::ObSimpleTableSchemaV2 *&table_schema);
int record_need_move_table_lock_tablet_( int record_need_move_table_lock_tablet_(
share::schema::ObSimpleTableSchemaV2 &table_schema, share::schema::ObSimpleTableSchemaV2 &table_schema,
const ObTabletID &tablet_id, const common::ObTabletID &tablet_id,
share::ObDisplayTabletList &table_lock_tablet_list); share::ObDisplayTabletList &table_lock_tablet_list);
int set_transaction_timeout_(common::ObTimeoutCtx &ctx); int set_transaction_timeout_(common::ObTimeoutCtx &ctx);
int update_comment_for_expected_errors_( int update_comment_for_expected_errors_(

View File

@ -45,7 +45,7 @@ enum ObReplicaStatus
const char *ob_replica_status_str(const ObReplicaStatus status); const char *ob_replica_status_str(const ObReplicaStatus status);
int get_replica_status(const char* str, ObReplicaStatus &status); int get_replica_status(const char* str, ObReplicaStatus &status);
int get_replica_status(const ObString &status_str, ObReplicaStatus &status); int get_replica_status(const common::ObString &status_str, ObReplicaStatus &status);
// [class_full_name] SimpleMember // [class_full_name] SimpleMember
// [class_functions] Use this class to build a member_list consists of this simple SimpleMember // [class_functions] Use this class to build a member_list consists of this simple SimpleMember

View File

@ -1277,11 +1277,8 @@ int ObTabletMergeFinishTask::add_sstable_for_merge(ObTabletMergeCtx &ctx)
if (ctx.param_.tablet_id_.is_special_merge_tablet()) { if (ctx.param_.tablet_id_.is_special_merge_tablet()) {
param.multi_version_start_ = 1; param.multi_version_start_ = 1;
} }
// for mini merge, read all msd from frozen memtable
if (is_mini_merge(merge_type) && OB_FAIL(read_msd_from_memtable(ctx, param, allocator))) { if (OB_FAIL(ctx.ls_handle_.get_ls()->update_tablet_table_store(ctx.param_.tablet_id_, param, new_tablet_handle))) {
LOG_WARN("failed to read msd from memtable", K(ret), K(ctx));
} else if (OB_FAIL(ctx.ls_handle_.get_ls()->update_tablet_table_store(
ctx.param_.tablet_id_, param, new_tablet_handle))) {
LOG_WARN("failed to update tablet table store", K(ret), K(param)); LOG_WARN("failed to update tablet table store", K(ret), K(param));
} else if (FALSE_IT(ctx.time_guard_.click(ObCompactionTimeGuard::UPDATE_TABLET))) { } else if (FALSE_IT(ctx.time_guard_.click(ObCompactionTimeGuard::UPDATE_TABLET))) {
} else if (is_mini_merge(merge_type)) { } else if (is_mini_merge(merge_type)) {
@ -1344,73 +1341,6 @@ int ObTabletMergeFinishTask::try_report_tablet_stat_after_mini(ObTabletMergeCtx
return ret; return ret;
} }
int ObTabletMergeFinishTask::read_msd_from_memtable(
ObTabletMergeCtx &ctx,
ObUpdateTableStoreParam &param,
ObArenaAllocator &allocator)
{
int ret = OB_SUCCESS;
if (OB_FAIL(traverse_all_memtables(ctx, &param.tx_data_, MultiSourceDataUnitType::TABLET_TX_DATA, allocator))) {
LOG_WARN("failed to read tx data from memtables", K(ret));
} else if (OB_FAIL(traverse_all_memtables(ctx, &param.binding_info_, MultiSourceDataUnitType::TABLET_BINDING_INFO, allocator))) {
LOG_WARN("failed to read tx data from memtables", K(ret));
} else if (OB_FAIL(traverse_all_memtables(ctx, &param.autoinc_seq_, MultiSourceDataUnitType::TABLET_SEQ, allocator))) {
LOG_WARN("failed to read tx data from memtables", K(ret));
} else {
LOG_INFO("succeeded to read msd from memtable", K(ret),
"ls_id", ctx.param_.ls_id_,
"tablet_id", ctx.param_.tablet_id_,
"tx_data", param.tx_data_,
"binding_info", param.binding_info_,
"autoinc_seq", param.autoinc_seq_);
}
return ret;
}
int ObTabletMergeFinishTask::traverse_all_memtables(
ObTabletMergeCtx &ctx,
ObIMultiSourceDataUnit *msd,
const MultiSourceDataUnitType &type,
ObArenaAllocator &allocator)
{
int ret = OB_SUCCESS;
ObITable *table = nullptr;
ObMemtable *memtable = nullptr;
if (OB_ISNULL(msd)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid args", K(ret));
}
for (int64_t i = ctx.tables_handle_.get_count() - 1; OB_SUCC(ret) && i >= 0; --i) {
if (OB_ISNULL(table = ctx.tables_handle_.get_table(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table is null", K(ret), K(ctx.tables_handle_), KP(table));
} else if (OB_UNLIKELY(!table->is_memtable())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table is not memtable", K(ret), K(ctx.tables_handle_), KPC(table));
} else if (OB_UNLIKELY(!table->is_frozen_memtable())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table is not frozen memtable", K(ret), K(ctx.tables_handle_), KPC(table));
} else if (table->is_data_memtable()) {
// TODO(lixia) delete this code
// memtable = static_cast<ObMemtable*>(table);
// if (memtable->has_multi_source_data_unit(type)) {
// if (OB_FAIL(memtable->get_multi_source_data_unit(msd, &allocator))) {
// LOG_WARN("failed to get msd from memtable", K(ret), K(type));
// } else {
// // succeeded to get msd, just break
// break;
// }
// }
}
}
return ret;
}
int ObTabletMergeFinishTask::try_schedule_compaction_after_mini( int ObTabletMergeFinishTask::try_schedule_compaction_after_mini(
ObTabletMergeCtx &ctx, ObTabletMergeCtx &ctx,
ObTabletHandle &tablet_handle) ObTabletHandle &tablet_handle)

View File

@ -32,12 +32,6 @@ class ObTabletHandle;
struct ObUpdateTableStoreParam; struct ObUpdateTableStoreParam;
} }
namespace memtable
{
enum class MultiSourceDataUnitType;
class ObIMultiSourceDataUnit;
}
namespace blocksstable namespace blocksstable
{ {
class ObSSTable; class ObSSTable;
@ -164,15 +158,6 @@ private:
int add_sstable_for_merge(ObTabletMergeCtx &ctx); int add_sstable_for_merge(ObTabletMergeCtx &ctx);
int try_schedule_compaction_after_mini(ObTabletMergeCtx &ctx, storage::ObTabletHandle &tablet_handle); int try_schedule_compaction_after_mini(ObTabletMergeCtx &ctx, storage::ObTabletHandle &tablet_handle);
int try_report_tablet_stat_after_mini(ObTabletMergeCtx &ctx); int try_report_tablet_stat_after_mini(ObTabletMergeCtx &ctx);
int read_msd_from_memtable(
ObTabletMergeCtx &ctx,
storage::ObUpdateTableStoreParam &param,
ObArenaAllocator &allocator);
int traverse_all_memtables(
ObTabletMergeCtx &ctx,
memtable::ObIMultiSourceDataUnit *msd,
const memtable::MultiSourceDataUnitType &type,
ObArenaAllocator &allocator);
private: private:
bool is_inited_; bool is_inited_;
ObBasicTabletMergeDag *merge_dag_; ObBasicTabletMergeDag *merge_dag_;

View File

@ -2406,12 +2406,12 @@ int ObStorageHATabletBuilderUtil::inner_update_tablet_table_store_with_major_(
true/*need_check_sstable*/, true/*need_check_sstable*/,
true/*allow_duplicate_sstable*/, true/*allow_duplicate_sstable*/,
ObMergeType::MEDIUM_MERGE/*merge_type*/); ObMergeType::MEDIUM_MERGE/*merge_type*/);
if (tablet_storage_schema->get_version() < storage_schema.get_version()) { if (tablet_storage_schema->get_schema_version() < storage_schema.get_schema_version()) {
SERVER_EVENT_ADD("storage_ha", "schema_change_need_merge_tablet_meta", SERVER_EVENT_ADD("storage_ha", "schema_change_need_merge_tablet_meta",
"tenant_id", MTL_ID(), "tenant_id", MTL_ID(),
"tablet_id", tablet_id.id(), "tablet_id", tablet_id.id(),
"old_schema_version", tablet_storage_schema->get_version(), "old_schema_version", tablet_storage_schema->get_schema_version(),
"new_schema_version", storage_schema.get_version()); "new_schema_version", storage_schema.get_schema_version());
} }
#ifdef ERRSIM #ifdef ERRSIM
SERVER_EVENT_ADD("storage_ha", "update_major_tablet_table_store", SERVER_EVENT_ADD("storage_ha", "update_major_tablet_table_store",

View File

@ -290,21 +290,6 @@ int ObStorageSchema::deep_copy_column_array(
return ret; return ret;
} }
int ObStorageSchema::deep_copy(const memtable::ObIMultiSourceDataUnit *src, ObIAllocator *allocator)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(nullptr == src || nullptr == allocator)) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "invalid src info or allocator", K(ret), K(src), K(allocator));
} else if (OB_UNLIKELY(src->type() != type())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "invalid type", K(ret), K(type()), KPC(src));
} else {
ret = init(*allocator, *static_cast<const ObStorageSchema *>(src));
}
return ret;
}
void ObStorageSchema::reset() void ObStorageSchema::reset()
{ {
storage_schema_version_ = 0; storage_schema_version_ = 0;

View File

@ -15,7 +15,6 @@
#include "lib/container/ob_fixed_array.h" #include "lib/container/ob_fixed_array.h"
#include "share/schema/ob_table_schema.h" #include "share/schema/ob_table_schema.h"
#include "storage/memtable/ob_multi_source_data.h"
namespace oceanbase namespace oceanbase
{ {
@ -93,7 +92,7 @@ public:
ObObj orig_default_value_; ObObj orig_default_value_;
}; };
class ObStorageSchema : public share::schema::ObMergeSchema, public memtable::ObIMultiSourceDataUnit class ObStorageSchema : public share::schema::ObMergeSchema
{ {
public: public:
ObStorageSchema(); ObStorageSchema();
@ -113,17 +112,9 @@ public:
common::ObIAllocator &allocator, common::ObIAllocator &allocator,
const ObStorageSchema &src_schema, const ObStorageSchema &src_schema,
const int64_t copy_array_cnt); const int64_t copy_array_cnt);
void reset();
bool is_valid() const;
// ObIMultiSourceDataUnit section
virtual int deep_copy(const ObIMultiSourceDataUnit *src, ObIAllocator *allocator) override;
virtual void reset() override;
virtual bool is_valid() const override;
virtual inline int64_t get_data_size() const override { return sizeof(ObStorageSchema); }
virtual inline memtable::MultiSourceDataUnitType type() const override
{
return memtable::MultiSourceDataUnitType::STORAGE_SCHEMA;
}
virtual int64_t get_version() const override { return get_schema_version(); }
// serialize & deserialize // serialize & deserialize
int serialize(char *buf, const int64_t buf_len, int64_t &pos) const; int serialize(char *buf, const int64_t buf_len, int64_t &pos) const;
int deserialize( int deserialize(
@ -195,7 +186,7 @@ public:
return store_column_cnt_ < input_schema.store_column_cnt_; return store_column_cnt_ < input_schema.store_column_cnt_;
} }
INHERIT_TO_STRING_KV("ObIMultiSourceDataUnit", ObIMultiSourceDataUnit, KP(this), K_(storage_schema_version), K_(version), VIRTUAL_TO_STRING_KV(KP(this), K_(storage_schema_version), K_(version),
K_(is_use_bloomfilter), K_(column_info_simplified), K_(compat_mode), K_(table_type), K_(index_type), K_(is_use_bloomfilter), K_(column_info_simplified), K_(compat_mode), K_(table_type), K_(index_type),
K_(index_status), K_(row_store_type), K_(schema_version), K_(index_status), K_(row_store_type), K_(schema_version),
K_(column_cnt), K_(store_column_cnt), K_(tablet_size), K_(pctfree), K_(block_size), K_(progressive_merge_round), K_(column_cnt), K_(store_column_cnt), K_(tablet_size), K_(pctfree), K_(block_size), K_(progressive_merge_round),

View File

@ -204,7 +204,6 @@ int ObStorageSchemaRecorder::try_update_storage_schema(
void ObStorageSchemaRecorder::sync_clog_failed_for_leader() void ObStorageSchemaRecorder::sync_clog_failed_for_leader()
{ {
dec_ref_on_memtable(false/*sync_finish*/);
} }
int ObStorageSchemaRecorder::sync_clog_succ_for_leader(const int64_t update_version) int ObStorageSchemaRecorder::sync_clog_succ_for_leader(const int64_t update_version)
@ -216,8 +215,6 @@ int ObStorageSchemaRecorder::sync_clog_succ_for_leader(const int64_t update_vers
} else if (OB_UNLIKELY(storage_schema_->get_schema_version() != update_version)) { } else if (OB_UNLIKELY(storage_schema_->get_schema_version() != update_version)) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_ERROR("schema version not match", K(storage_schema_), K(update_version)); LOG_ERROR("schema version not match", K(storage_schema_), K(update_version));
} else if (OB_FAIL(dec_ref_on_memtable(true/*sync_finish*/))) {
LOG_WARN("failed to save storage schema", K_(tablet_id), K(storage_schema_));
} else { } else {
LOG_INFO("success to update storage schema", K(ret), K_(ls_id), K_(tablet_id), K(storage_schema_), LOG_INFO("success to update storage schema", K(ret), K_(ls_id), K_(tablet_id), K(storage_schema_),
K(update_version), K_(clog_scn)); K(update_version), K_(clog_scn));
@ -225,25 +222,6 @@ int ObStorageSchemaRecorder::sync_clog_succ_for_leader(const int64_t update_vers
return ret; return ret;
} }
int ObStorageSchemaRecorder::dec_ref_on_memtable(const bool sync_finish)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(nullptr == storage_schema_
|| nullptr == tablet_handle_ptr_
|| !tablet_handle_ptr_->is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("storage schema or tablet handle is unexpected null", K(ret), K_(ls_id), K_(tablet_id),
KP_(storage_schema), K_(tablet_handle_ptr));
} else {
storage_schema_->set_sync_finish(sync_finish);
// if (OB_FAIL(tablet_handle_ptr_->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_));
// }
}
return ret;
}
int ObStorageSchemaRecorder::prepare_struct_in_lock( int ObStorageSchemaRecorder::prepare_struct_in_lock(
int64_t &update_version, int64_t &update_version,
ObIAllocator *allocator, ObIAllocator *allocator,
@ -366,18 +344,8 @@ int ObStorageSchemaRecorder::submit_log(
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("log handler or storage_schema is null", K(ret), KP(storage_schema_), LOG_WARN("log handler or storage_schema is null", K(ret), KP(storage_schema_),
KP(clog_buf), K(clog_len), K(tablet_handle_ptr_)); KP(clog_buf), K(clog_len), K(tablet_handle_ptr_));
} else if (FALSE_IT(storage_schema_->set_sync_finish(false))) {
//} else if (OB_FAIL(tablet_handle_ptr_->get_obj()->save_multi_source_data_unit(storage_schema_,
// 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_));
// }
} else if (OB_FAIL(write_clog(clog_buf, clog_len))) { } else if (OB_FAIL(write_clog(clog_buf, clog_len))) {
LOG_WARN("fail to submit log", K(ret), K_(tablet_id)); LOG_WARN("fail to submit log", K(ret), K_(ls_id), K_(tablet_id));
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(dec_ref_on_memtable(false))) {
LOG_ERROR("failed to dec ref on memtable", K(tmp_ret), K_(ls_id), K_(tablet_id));
}
} else { } else {
LOG_INFO("submit schema log succeed", K(ret), K_(ls_id), K_(tablet_id), K(clog_scn_), LOG_INFO("submit schema log succeed", K(ret), K_(ls_id), K_(tablet_id), K(clog_scn_),
"schema_version", storage_schema_->get_schema_version()); "schema_version", storage_schema_->get_schema_version());

View File

@ -99,7 +99,6 @@ private:
int64_t calc_schema_log_size() const; int64_t calc_schema_log_size() const;
void free_allocated_info(); void free_allocated_info();
int try_update_with_lock(const int64_t table_id, const int64_t table_version, const int64_t expire_ts); int try_update_with_lock(const int64_t table_id, const int64_t table_version, const int64_t expire_ts);
OB_INLINE int dec_ref_on_memtable(const bool sync_finish);
bool is_inited_; bool is_inited_;
bool ignore_storage_schema_; bool ignore_storage_schema_;

View File

@ -4711,8 +4711,8 @@ int ObTablet::get_storage_schema_for_transfer_in(
} }
if (OB_FAIL(ret)) { if (OB_FAIL(ret)) {
} else if (OB_FAIL(storage_schema.deep_copy(tablet_storage_schema, &allocator))) { } else if (OB_FAIL(storage_schema.init(allocator, *tablet_storage_schema))) {
LOG_WARN("failed to get tx data from tablet", K(ret), K(ls_id), K(tablet_id), KPC(tablet_storage_schema)); LOG_WARN("failed to init storage schema", K(ret), K(ls_id), K(tablet_id), KPC(tablet_storage_schema));
} else { } else {
int64_t old_column_cnt = storage_schema.get_column_count(); int64_t old_column_cnt = storage_schema.get_column_count();
int64_t old_schema_version = storage_schema.get_schema_version(); int64_t old_schema_version = storage_schema.get_schema_version();