fix storage_schema compat serialize & wrong multi_version_start & major rs checker hung

This commit is contained in:
obdev
2024-02-07 23:33:09 +00:00
committed by ob-robot
parent 734cd239aa
commit d63522ff95
9 changed files with 122 additions and 37 deletions

View File

@ -845,7 +845,9 @@ void MockTenantModuleEnv::destroy()
TG_WAIT(lib::TGDefIDs::MemDumpTimer); TG_WAIT(lib::TGDefIDs::MemDumpTimer);
TG_DESTROY(lib::TGDefIDs::MemDumpTimer); TG_DESTROY(lib::TGDefIDs::MemDumpTimer);
THE_IO_DEVICE->destroy(); if (OB_NOT_NULL(THE_IO_DEVICE)) {
THE_IO_DEVICE->destroy();
}
destroyed_ = true; destroyed_ = true;

View File

@ -157,6 +157,7 @@ int ObChecksumValidator::get_table_compaction_info(
ret = OB_SUCCESS; ret = OB_SUCCESS;
table_compaction_info.reset(); table_compaction_info.reset();
table_compaction_info.table_id_ = table_id; table_compaction_info.table_id_ = table_id;
LOG_TRACE("return init table compaction info", KR(ret));
} else { } else {
LOG_WARN("fail to get val from hashmap", KR(ret), K(table_id)); LOG_WARN("fail to get val from hashmap", KR(ret), K(table_id));
} }
@ -267,10 +268,6 @@ int ObChecksumValidator::validate_checksum(
LOG_WARN("failed to validate index checksum", K(ret)); LOG_WARN("failed to validate index checksum", K(ret));
} else if (OB_FAIL(validate_cross_cluster_checksum())) { } else if (OB_FAIL(validate_cross_cluster_checksum())) {
LOG_WARN("failed to validate cross cluster checksum", K(ret)); LOG_WARN("failed to validate cross cluster checksum", K(ret));
} else if (OB_FAIL(table_compaction_map_.set_refactored(table_id_, table_compaction_info_, true /*overwrite*/))) {
LOG_WARN("fail to set refactored", KR(ret), K_(table_id), K_(table_compaction_info));
} else {
LOG_TRACE("success to validate table", KR(ret), K_(table_id), K_(table_compaction_info));
} }
if (OB_FAIL(ret)) { if (OB_FAIL(ret)) {
} else if (table_compaction_info_.unfinish_index_cnt_ <= 0 } else if (table_compaction_info_.unfinish_index_cnt_ <= 0
@ -291,6 +288,11 @@ int ObChecksumValidator::validate_checksum(
} }
cur_tablet_ls_pair_array_.reuse(); cur_tablet_ls_pair_array_.reuse();
} }
if (FAILEDx(table_compaction_map_.set_refactored(table_id_, table_compaction_info_, true /*overwrite*/))) {
LOG_WARN("fail to set refactored", KR(ret), K_(table_id), K_(table_compaction_info));
} else {
LOG_TRACE("success to validate table", KR(ret), K_(table_id), K_(table_compaction_info));
}
// do no clear table_compaction_info_ until validate next table // do no clear table_compaction_info_ until validate next table
replica_ckm_items_.reuse(); replica_ckm_items_.reuse();
schema_guard_ = nullptr; schema_guard_ = nullptr;
@ -798,7 +800,7 @@ int ObChecksumValidator::handle_index_table(
LOG_WARN("failed to set", K(ret), K(data_compaction_info)); LOG_WARN("failed to set", K(ret), K(data_compaction_info));
} }
} }
LOG_TRACE("handle index table", KR(ret), K_(table_id), K_(table_compaction_info)); LOG_TRACE("handle index table", KR(ret), K_(table_id), K_(table_compaction_info), K(data_compaction_info));
return ret; return ret;
} }

View File

@ -202,7 +202,7 @@ int ObMajorMergeProgressChecker::check_verification(
if (OB_CHECKSUM_ERROR == ret) { if (OB_CHECKSUM_ERROR == ret) {
LOG_ERROR("checksum error", KR(ret), K(table_id)); LOG_ERROR("checksum error", KR(ret), K(table_id));
} else if (OB_FREEZE_SERVICE_EPOCH_MISMATCH == ret) { } else if (OB_FREEZE_SERVICE_EPOCH_MISMATCH == ret) {
LOG_INFO("freeze service epoch mismatch", KR(tmp_ret)); LOG_INFO("freeze service epoch mismatch", KR(ret));
} }
} else { } else {
LOG_WARN("failed to verify table", KR(tmp_ret), K(idx), K(table_id), KPC(table_compaction_info_ptr)); LOG_WARN("failed to verify table", KR(tmp_ret), K(idx), K(table_id), KPC(table_compaction_info_ptr));
@ -227,7 +227,7 @@ int ObMajorMergeProgressChecker::check_verification(
break; break;
} }
} // end of for } // end of for
if (OB_SUCC(ret)) { if (OB_SUCC(ret)) { // record next untouched tablet
table_ids_.batch_start_idx_ = idx + 1; table_ids_.batch_start_idx_ = idx + 1;
} else { } else {
// record first failed table, need check in next loop // record first failed table, need check in next loop
@ -632,7 +632,7 @@ int ObMajorMergeProgressChecker::deal_with_rest_data_table()
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
bool exist_index_table = false; bool exist_index_table = false;
bool exist_finish_data_table = false; bool exist_data_table = false;
if ((is_extra_check_round() && table_ids_.count() > 0 && table_ids_.count() < DEAL_REST_TABLE_CNT_THRESHOLD) if ((is_extra_check_round() && table_ids_.count() > 0 && table_ids_.count() < DEAL_REST_TABLE_CNT_THRESHOLD)
|| REACH_TENANT_TIME_INTERVAL(DEAL_REST_TABLE_INTERVAL)) { || REACH_TENANT_TIME_INTERVAL(DEAL_REST_TABLE_INTERVAL)) {
ObTableCompactionInfo table_compaction_info; ObTableCompactionInfo table_compaction_info;
@ -646,12 +646,12 @@ int ObMajorMergeProgressChecker::deal_with_rest_data_table()
} else if (table_compaction_info.is_index_table()) { } else if (table_compaction_info.is_index_table()) {
LOG_TRACE("exist index table", K(ret), K(table_compaction_info)); LOG_TRACE("exist index table", K(ret), K(table_compaction_info));
exist_index_table = true; exist_index_table = true;
break; } else {
} else if (table_compaction_info.is_compacted()) { LOG_TRACE("exist data table", K(ret), K(table_compaction_info));
exist_finish_data_table = true; exist_data_table = true;
} }
} // end of for } // end of for
if (OB_SUCC(ret) && !exist_index_table && exist_finish_data_table) { // rest table are data table if (OB_SUCC(ret) && (exist_index_table != exist_data_table)) { // rest table are data table/ index table
int tmp_ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS;
LOG_INFO("start to deal with rest data table", K(ret), K_(table_ids)); LOG_INFO("start to deal with rest data table", K(ret), K_(table_ids));
for (int64_t idx = 0; idx < table_ids_.count(); ++idx) { for (int64_t idx = 0; idx < table_ids_.count(); ++idx) {
@ -932,7 +932,7 @@ int ObMajorMergeProgressChecker::generate_tablet_status_map()
} else if (replica_snapshot_scn < compaction_scn_) { } else if (replica_snapshot_scn < compaction_scn_) {
status = ObTabletCompactionStatus::INITIAL; status = ObTabletCompactionStatus::INITIAL;
(void) uncompact_info_.add_tablet(*replica); (void) uncompact_info_.add_tablet(*replica);
LOG_TRACE("unfinish tablet", KR(ret), K(replica_snapshot_scn), K_(compaction_scn)); LOG_TRACE("unfinish tablet", KR(ret), KPC(replica), K(replica_snapshot_scn), K_(compaction_scn));
break; break;
} else if (OB_FAIL(report_scn.convert_for_tx(replica->get_report_scn()))) { // check report_scn } else if (OB_FAIL(report_scn.convert_for_tx(replica->get_report_scn()))) { // check report_scn
LOG_WARN("fail to convert val to SCN", KR(ret), KPC(replica)); LOG_WARN("fail to convert val to SCN", KR(ret), KPC(replica));

View File

@ -78,7 +78,7 @@ int ObMediumCompactionScheduleFunc::choose_medium_snapshot(
|| medium_info.medium_snapshot_ > tablet.get_snapshot_version()) { || medium_info.medium_snapshot_ > tablet.get_snapshot_version()) {
// chosen medium snapshot is far too old // chosen medium snapshot is far too old
if (OB_FAIL(choose_new_medium_snapshot(max_reserved_snapshot, medium_info, result, schema_version))) { if (OB_FAIL(choose_new_medium_snapshot(max_reserved_snapshot, medium_info, result, schema_version))) {
LOG_WARN("failed to choose new medium snapshot", KR(ret), K(medium_info)); LOG_WARN("failed to choose new medium snapshot", KR(ret), K(medium_info), K(max_reserved_snapshot));
} }
} else if (OB_FAIL(tablet.get_schema_version_from_storage_schema(schema_version))) { } else if (OB_FAIL(tablet.get_schema_version_from_storage_schema(schema_version))) {
LOG_WARN("failed to get schema version from tablet", KR(ret), K(tablet)); LOG_WARN("failed to get schema version from tablet", KR(ret), K(tablet));
@ -458,7 +458,7 @@ int ObMediumCompactionScheduleFunc::check_frequency(
LOG_WARN("major sstable should not be empty", K(ret), K(last_major_snapshot_version)); LOG_WARN("major sstable should not be empty", K(ret), K(last_major_snapshot_version));
} else if (last_major_snapshot_version + time_interval > medium_snapshot) { } else if (last_major_snapshot_version + time_interval > medium_snapshot) {
ret = OB_NO_NEED_MERGE; ret = OB_NO_NEED_MERGE;
LOG_DEBUG("schedule medium frequently", K(ret), K(last_major_snapshot_version), K(medium_snapshot), LOG_TRACE("schedule medium frequently", K(ret), K(last_major_snapshot_version), K(medium_snapshot),
K(time_interval)); K(time_interval));
} }
} }

View File

@ -670,8 +670,10 @@ int ObProhibitScheduleMediumMap::add_flag(const ObTabletID &tablet_id, const Pro
} else { } else {
obsys::ObWLockGuard lock_guard(lock_); obsys::ObWLockGuard lock_guard(lock_);
if (OB_FAIL(tablet_id_map_.get_refactored(tablet_id, tmp_flag))) { if (OB_FAIL(tablet_id_map_.get_refactored(tablet_id, tmp_flag))) {
if (OB_HASH_NOT_EXIST == ret && OB_FAIL(tablet_id_map_.set_refactored(tablet_id, input_flag))) { if (OB_HASH_NOT_EXIST == ret) {
LOG_WARN("failed to stop tablet schedule medium", K(ret), K(tablet_id), K(input_flag)); if (OB_FAIL(tablet_id_map_.set_refactored(tablet_id, input_flag))) {
LOG_WARN("failed to stop tablet schedule medium", K(ret), K(tablet_id), K(input_flag));
}
} else { } else {
LOG_WARN("failed to get map", K(ret), K(tablet_id), K(tmp_flag)); LOG_WARN("failed to get map", K(ret), K(tablet_id), K(tmp_flag));
} }

View File

@ -149,6 +149,18 @@ int ObStorageColumnSchema::legacy_serialize(char *buf, const int64_t buf_len, in
return ret; return ret;
} }
int64_t ObStorageColumnSchema::legacy_serialize_len() const
{
// For schema version before 4_2_0_0
int64_t len = 0;
LST_DO_CODE(OB_UNIS_ADD_LEN,
info_,
default_checksum_,
meta_type_,
orig_default_value_);
return len;
}
/* /*
* ObStorageColumnGroupSchema * ObStorageColumnGroupSchema
*/ */
@ -869,6 +881,21 @@ int ObStorageSchema::deserialize_rowkey_column_array(
return ret; return ret;
} }
int64_t ObStorageSchema::get_column_array_serialize_length(
const common::ObIArray<ObStorageColumnSchema> &array) const
{
int64_t len = 0;
len += serialization::encoded_length_vi64(array.count());
for (int64_t i = 0; i < array.count(); ++i) {
if (STORAGE_SCHEMA_VERSION_V3 > storage_schema_version_) {
len += array.at(i).legacy_serialize_len();
} else {
len += array.at(i).get_serialize_size();
}
}
return len;
}
int ObStorageSchema::deserialize_column_array( int ObStorageSchema::deserialize_column_array(
ObIAllocator &allocator, ObIAllocator &allocator,
const char *buf, const char *buf,
@ -1078,7 +1105,7 @@ int64_t ObStorageSchema::get_serialize_size() const
compressor_type_, compressor_type_,
encryption_, encryption_,
encrypt_key_); encrypt_key_);
len += get_column_array_serialize_length(rowkey_array_); len += get_array_serialize_length(rowkey_array_);
//get columms size //get columms size
if (!column_info_simplified_) { if (!column_info_simplified_) {
len += get_column_array_serialize_length(column_array_); len += get_column_array_serialize_length(column_array_);
@ -1087,8 +1114,8 @@ int64_t ObStorageSchema::get_serialize_size() const
len += serialization::encoded_length_i64(store_column_cnt_); len += serialization::encoded_length_i64(store_column_cnt_);
} }
if (storage_schema_version_ >= STORAGE_SCHEMA_VERSION_V3) { if (storage_schema_version_ >= STORAGE_SCHEMA_VERSION_V3) {
len += get_column_array_serialize_length(column_group_array_); len += get_array_serialize_length(column_group_array_);
len += get_column_array_serialize_length(skip_idx_attr_array_); len += get_array_serialize_length(skip_idx_attr_array_);
} }
return len; return len;
} }

View File

@ -72,6 +72,7 @@ public:
int legacy_deserialize(const char *buf, const int64_t data_len, int64_t &pos); int legacy_deserialize(const char *buf, const int64_t data_len, int64_t &pos);
int legacy_serialize(char *buf, const int64_t buf_len, int64_t &pos) const; int legacy_serialize(char *buf, const int64_t buf_len, int64_t &pos) const;
int64_t legacy_serialize_len() const;
TO_STRING_KV(K_(meta_type), K_(is_column_stored_in_sstable), K_(is_rowkey_column), TO_STRING_KV(K_(meta_type), K_(is_column_stored_in_sstable), K_(is_rowkey_column),
K_(is_generated_column), K_(orig_default_value)); K_(is_generated_column), K_(orig_default_value));
@ -291,10 +292,11 @@ private:
int deserialize_rowkey_column_array(const char *buf, const int64_t data_len, int64_t &pos); int deserialize_rowkey_column_array(const char *buf, const int64_t data_len, int64_t &pos);
int deserialize_column_array(ObIAllocator &allocator, const char *buf, const int64_t data_len, int64_t &pos); int deserialize_column_array(ObIAllocator &allocator, const char *buf, const int64_t data_len, int64_t &pos);
int deserialize_column_group_array(ObIAllocator &allocator, const char *buf, const int64_t data_len, int64_t &pos); int deserialize_column_group_array(ObIAllocator &allocator, const char *buf, const int64_t data_len, int64_t &pos);
int64_t get_column_array_serialize_length(const common::ObIArray<ObStorageColumnSchema> &array) const;
int deserialize_skip_idx_attr_array(const char *buf, const int64_t data_len, int64_t &pos); int deserialize_skip_idx_attr_array(const char *buf, const int64_t data_len, int64_t &pos);
int generate_all_column_group_schema(ObStorageColumnGroupSchema &column_group, const ObRowStoreType row_store_type); int generate_all_column_group_schema(ObStorageColumnGroupSchema &column_group, const ObRowStoreType row_store_type);
template <typename T> template <typename T>
int64_t get_column_array_serialize_length(const common::ObIArray<T> &array) const; int64_t get_array_serialize_length(const common::ObIArray<T> &array) const;
template <typename T> template <typename T>
bool check_column_array_valid(const common::ObIArray<T> &array) const; bool check_column_array_valid(const common::ObIArray<T> &array) const;
@ -427,7 +429,7 @@ int ObStorageSchema::serialize_schema_array(
} }
template <typename T> template <typename T>
int64_t ObStorageSchema::get_column_array_serialize_length(const common::ObIArray<T> &array) const int64_t ObStorageSchema::get_array_serialize_length(const common::ObIArray<T> &array) const
{ {
int64_t len = 0; int64_t len = 0;
len += serialization::encoded_length_vi64(array.count()); len += serialization::encoded_length_vi64(array.count());

View File

@ -4876,9 +4876,8 @@ int ObTablet::get_kept_snapshot_info(
int tmp_ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS;
snapshot_info.reset(); snapshot_info.reset();
int64_t max_merged_snapshot = 0; int64_t max_merged_snapshot = 0;
int64_t min_reserved_snapshot = 0;
int64_t min_medium_snapshot = INT64_MAX; int64_t min_medium_snapshot = INT64_MAX;
int64_t ls_min_reserved_snapshot = INT64_MAX; int64_t old_min_reserved_snapshot = 0;
const ObTabletID &tablet_id = get_tablet_meta().tablet_id_; const ObTabletID &tablet_id = get_tablet_meta().tablet_id_;
if (0 < get_major_table_count()) { if (0 < get_major_table_count()) {
@ -4901,9 +4900,9 @@ int ObTablet::get_kept_snapshot_info(
if (OB_SUCC(ret)) { if (OB_SUCC(ret)) {
bool use_multi_version_start_on_tablet = false; bool use_multi_version_start_on_tablet = false;
const int64_t old_min_reserved_snapshot = min_reserved_snapshot; old_min_reserved_snapshot = snapshot_info.snapshot_;
if (min_reserved_snapshot_on_ls > 0) { if (min_reserved_snapshot_on_ls > 0) {
snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_LS_RESERVED, ls_min_reserved_snapshot); snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_LS_RESERVED, min_reserved_snapshot_on_ls);
snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_MIN_MEDIUM, min_medium_snapshot); snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_MIN_MEDIUM, min_medium_snapshot);
if (snapshot_info.snapshot_ < get_multi_version_start()) { if (snapshot_info.snapshot_ < get_multi_version_start()) {
use_multi_version_start_on_tablet = true; use_multi_version_start_on_tablet = true;
@ -4925,7 +4924,7 @@ int ObTablet::get_kept_snapshot_info(
LOG_INFO("tablet multi version start not advance for a long time", K(ret), LOG_INFO("tablet multi version start not advance for a long time", K(ret),
"ls_id", get_tablet_meta().ls_id_, K(tablet_id), "ls_id", get_tablet_meta().ls_id_, K(tablet_id),
K(snapshot_info), K(old_min_reserved_snapshot), K(min_medium_snapshot), K(snapshot_info), K(old_min_reserved_snapshot), K(min_medium_snapshot),
"ls_min_reserved_snapshot", min_reserved_snapshot_on_ls); K(min_reserved_snapshot_on_ls));
} }
} }
} }
@ -4934,7 +4933,7 @@ int ObTablet::get_kept_snapshot_info(
LOG_WARN("snapshot info is invalid", KR(ret), K(snapshot_info)); LOG_WARN("snapshot info is invalid", KR(ret), K(snapshot_info));
} }
LOG_TRACE("get multi version start", "ls_id", get_tablet_meta().ls_id_, K(tablet_id), LOG_TRACE("get multi version start", "ls_id", get_tablet_meta().ls_id_, K(tablet_id),
K(snapshot_info), K(min_reserved_snapshot), K(get_tablet_meta()), K(snapshot_info), K(old_min_reserved_snapshot), K(get_tablet_meta()),
K(min_medium_snapshot), K(min_reserved_snapshot_on_ls), K(max_merged_snapshot)); K(min_medium_snapshot), K(min_reserved_snapshot_on_ls), K(max_merged_snapshot));
return ret; return ret;
} }

View File

@ -19,6 +19,7 @@
#include "storage/ob_storage_schema.h" #include "storage/ob_storage_schema.h"
#include "share/ob_encryption_util.h" #include "share/ob_encryption_util.h"
#include "storage/test_schema_prepare.h" #include "storage/test_schema_prepare.h"
#include "mittest/mtlenv/mock_tenant_module_env.h"
namespace oceanbase namespace oceanbase
{ {
@ -30,17 +31,41 @@ namespace unittest
class TestStorageSchema : public ::testing::Test class TestStorageSchema : public ::testing::Test
{ {
public: public:
TestStorageSchema() : allocator_(ObModIds::TEST) {} TestStorageSchema() : allocator_(ObModIds::TEST), tenant_base_(tenant_id) {}
virtual ~TestStorageSchema() {} virtual ~TestStorageSchema() {}
bool judge_storage_schema_equal(ObStorageSchema &schema1, ObStorageSchema &schema2); bool judge_storage_schema_equal(ObStorageSchema &schema1, ObStorageSchema &schema2);
virtual void SetUp() override;
virtual void TearDown() override;
static void SetUpTestCase();
static void TearDownTestCase();
static const int64_t tenant_id = 1;
common::ObArenaAllocator allocator_; common::ObArenaAllocator allocator_;
ObTenantBase tenant_base_;
}; };
void TestStorageSchema::SetUp()
{
ASSERT_EQ(OB_SUCCESS, tenant_base_.init());
}
void TestStorageSchema::TearDown()
{
ObTenantEnv::set_tenant(nullptr);
}
void TestStorageSchema::SetUpTestCase()
{
EXPECT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init());
}
void TestStorageSchema::TearDownTestCase()
{
MockTenantModuleEnv::get_instance().destroy();
}
bool TestStorageSchema::judge_storage_schema_equal(ObStorageSchema &schema1, ObStorageSchema &schema2) bool TestStorageSchema::judge_storage_schema_equal(ObStorageSchema &schema1, ObStorageSchema &schema2)
{ {
bool equal = false; bool equal = false;
equal = schema1.version_ == schema2.version_ equal = schema1.is_use_bloomfilter_ == schema2.is_use_bloomfilter_
&& schema1.is_use_bloomfilter_ == schema2.is_use_bloomfilter_
&& schema1.table_type_ == schema2.table_type_ && schema1.table_type_ == schema2.table_type_
&& schema1.table_mode_ == schema2.table_mode_ && schema1.table_mode_ == schema2.table_mode_
&& schema1.row_store_type_ == schema2.row_store_type_ && schema1.row_store_type_ == schema2.row_store_type_
@ -54,8 +79,7 @@ bool TestStorageSchema::judge_storage_schema_equal(ObStorageSchema &schema1, ObS
&& schema1.encryption_ == schema2.encryption_ && schema1.encryption_ == schema2.encryption_
&& schema1.encrypt_key_ == schema2.encrypt_key_ && schema1.encrypt_key_ == schema2.encrypt_key_
&& schema1.rowkey_array_.count() == schema2.rowkey_array_.count() && schema1.rowkey_array_.count() == schema2.rowkey_array_.count()
&& schema1.column_array_.count() == schema2.column_array_.count() && schema1.column_array_.count() == schema2.column_array_.count();
&& schema1.skip_idx_attr_array_.count() == schema2.skip_idx_attr_array_.count();
for (int64_t i = 0; equal && i < schema1.rowkey_array_.count(); ++i) { for (int64_t i = 0; equal && i < schema1.rowkey_array_.count(); ++i) {
equal = schema1.rowkey_array_[i].meta_type_ == schema1.rowkey_array_[i].meta_type_; equal = schema1.rowkey_array_[i].meta_type_ == schema1.rowkey_array_[i].meta_type_;
@ -65,9 +89,13 @@ bool TestStorageSchema::judge_storage_schema_equal(ObStorageSchema &schema1, ObS
equal = schema1.column_array_[i].meta_type_ == schema2.column_array_[i].meta_type_ equal = schema1.column_array_[i].meta_type_ == schema2.column_array_[i].meta_type_
&& schema1.column_array_[i].is_column_stored_in_sstable_ == schema2.column_array_[i].is_column_stored_in_sstable_; && schema1.column_array_[i].is_column_stored_in_sstable_ == schema2.column_array_[i].is_column_stored_in_sstable_;
} }
for (int i = 0; equal && i < schema1.skip_idx_attr_array_.count(); ++i) { if (equal && schema1.version_ >= ObStorageSchema::STORAGE_SCHEMA_VERSION_V3
equal = schema1.skip_idx_attr_array_[i].col_idx_ == schema2.skip_idx_attr_array_[i].col_idx_ && schema2.version_ >= ObStorageSchema::STORAGE_SCHEMA_VERSION_V3) {
&& schema1.skip_idx_attr_array_[i].skip_idx_attr_ == schema2.skip_idx_attr_array_[i].skip_idx_attr_; equal = schema1.skip_idx_attr_array_.count() == schema2.skip_idx_attr_array_.count();
for (int i = 0; equal && i < schema1.skip_idx_attr_array_.count(); ++i) {
equal = schema1.skip_idx_attr_array_[i].col_idx_ == schema2.skip_idx_attr_array_[i].col_idx_
&& schema1.skip_idx_attr_array_[i].skip_idx_attr_ == schema2.skip_idx_attr_array_[i].skip_idx_attr_;
}
} }
return equal; return equal;
@ -229,6 +257,29 @@ TEST_F(TestStorageSchema, deep_copy_str)
} }
TEST_F(TestStorageSchema, compat_serialize_and_deserialize)
{
share::schema::ObTableSchema table_schema;
ObStorageSchema storage_schema;
TestSchemaPrepare::prepare_schema(table_schema);
ASSERT_EQ(OB_SUCCESS, storage_schema.init(allocator_, table_schema, lib::Worker::CompatMode::MYSQL));
storage_schema.storage_schema_version_ = ObStorageSchema::STORAGE_SCHEMA_VERSION;
const int64_t buf_len = 1024 * 1024;
int64_t ser_pos = 0;
char buf[buf_len] = "\0";
ASSERT_EQ(OB_SUCCESS, storage_schema.serialize(buf, buf_len, ser_pos));
ASSERT_EQ(ser_pos, storage_schema.get_serialize_size());
ObStorageSchema des_storage_schema;
int64_t pos = 0;
ASSERT_EQ(OB_SUCCESS, des_storage_schema.deserialize(allocator_, buf, ser_pos, pos));
COMMON_LOG(INFO, "test", K(storage_schema), K(des_storage_schema));
ASSERT_EQ(true, judge_storage_schema_equal(storage_schema, des_storage_schema));
}
} // namespace unittest } // namespace unittest
} // namespace oceanbase } // namespace oceanbase