fix storage_schema compat serialize & wrong multi_version_start & major rs checker hung
This commit is contained in:
@ -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;
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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));
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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());
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
|
||||||
|
Reference in New Issue
Block a user