delete useless code & remove lambda

This commit is contained in:
yangqise7en
2023-11-17 07:41:07 +00:00
committed by ob-robot
parent ec834c2946
commit fb6afc1f23
25 changed files with 1342 additions and 1110 deletions

View File

@ -37,34 +37,6 @@ using namespace share;
using namespace schema;
using namespace compaction;
template<>
bool ObArrayWithIdx<share::ObTabletLSPair>::exist(const ObTabletID &tablet_id) const
{
bool exist = false;
if (last_idx_ < array_.count()
&& array_.at(last_idx_).get_tablet_id() == tablet_id) {
exist = true;
}
return exist;
}
template <typename T>
int64_t ObArrayWithIdx<T>::to_string(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
if (OB_ISNULL(buf) || buf_len <= 0) {
} else {
J_OBJ_START();
J_KV("array_cnt", array_.count(), K_(last_idx));
if (last_idx_ < array_.count()) {
J_COMMA();
J_KV("last_item", array_.at(last_idx_));
}
J_OBJ_END();
}
return pos;
}
///////////////////////////////////////////////////////////////////////////////
int ObChecksumValidator::init(
@ -113,6 +85,7 @@ int ObChecksumValidator::deal_with_special_table_at_last(bool &finish_validate)
finish_validate = false;
ObSchemaGetterGuard schema_guard(ObSchemaMgrItem::MOD_RS_MAJOR_CHECK);
cur_tablet_ls_pair_array_.reuse();
ObSEArray<ObTabletID, 1> tmp_tablet_ids;
if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
tenant_id_, schema_guard, OB_INVALID_VERSION, OB_INVALID_VERSION,
ObMultiVersionSchemaService::RefreshSchemaMode::FORCE_LAZY))) {
@ -122,25 +95,27 @@ int ObChecksumValidator::deal_with_special_table_at_last(bool &finish_validate)
} else if (FALSE_IT(table_id_ = ObChecksumValidator::SPECIAL_TABLE_ID)) {
} else if (OB_FAIL(get_table_compaction_info(table_id_, table_compaction_info_))) {
LOG_WARN("failed to get table compaction info", K(ret));
} else if (OB_FAIL(schema_guard_->get_table_schema(tenant_id_, table_id_, table_schema_))) {
} else if (OB_FAIL(schema_guard_->get_simple_table_schema(tenant_id_, table_id_, simple_schema_))) {
LOG_WARN("fail to get table schema", KR(ret), K_(tenant_id), K_(table_id));
} else if (OB_ISNULL(table_schema_)) {
} else if (OB_ISNULL(simple_schema_)) {
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("table schema is null", KR(ret), K_(tenant_id), K_(table_id));
} else if (OB_FAIL(ObTabletReplicaChecksumOperator::get_tablet_ls_pairs(
tenant_id_, *table_schema_, *sql_proxy_, cur_tablet_ls_pair_array_))) {
LOG_WARN("fail to get tablet_ls pairs", KR(ret), K_(tenant_id), K_(table_id));
} else if (OB_FAIL(simple_schema_->get_tablet_ids(tmp_tablet_ids))) {
LOG_WARN("fail to get tablet_ids from simple table schema", KR(ret), KPC_(simple_schema));
} else if (OB_FAIL(tablet_ls_pair_cache_.get_tablet_ls_pairs(
tenant_id_, tmp_tablet_ids, cur_tablet_ls_pair_array_))) {
LOG_WARN("failed to get tablet ls pairs from cache", KR(ret));
} else if (OB_UNLIKELY(cur_tablet_ls_pair_array_.empty())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get tablet_ls pairs of current table schema", KR(ret), K_(tenant_id), K_(table_id),
K_(tablet_ls_pair_array));
K_(cur_tablet_ls_pair_array));
} else if (OB_FAIL(get_tablet_replica_checksum_and_validate(true /*include_larger_than*/))) {
if (OB_ITEM_NOT_MATCH == ret) {
LOG_TRACE("mismatch checksum cnt when deal with special table", KR(ret), K_(cur_tablet_ls_pair_array));
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to validate tablet replica checksum", KR(ret), K_(compaction_scn), K_(table_id),
KPC(table_schema_), K_(cur_tablet_ls_pair_array));
KPC(simple_schema_), K_(cur_tablet_ls_pair_array));
}
} else if (FALSE_IT(table_compaction_info_.set_index_ckm_verified())) {
} else if (OB_FAIL(validate_cross_cluster_checksum())) {
@ -209,9 +184,8 @@ void ObChecksumValidator::clear_cached_info()
cross_cluster_ckm_sync_finish_ = false;
compaction_scn_.reset();
major_merge_start_us_ = 0;
tablet_ls_pair_array_.clear();
schema_guard_ = nullptr;
table_schema_ = nullptr;
simple_schema_ = nullptr;
table_compaction_info_.reset();
cur_tablet_ls_pair_array_.reuse();
finish_tablet_ls_pair_array_.reuse();
@ -219,43 +193,28 @@ void ObChecksumValidator::clear_cached_info()
last_table_ckm_items_.clear();
}
int ObChecksumValidator::init_tablet_ls_array_idx(
const share::schema::ObTableSchema &table_schema)
int ObChecksumValidator::get_tablet_ls_pairs(
const share::schema::ObSimpleTableSchemaV2 &simple_schema)
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
SMART_VAR(ObArray<ObTabletID>, tablet_ids) {
if (OB_UNLIKELY(!table_schema.has_tablet())) {
if (OB_UNLIKELY(!simple_schema.has_tablet())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet schema should have tablet", K(ret), K(table_schema));
} else if (OB_FAIL(table_schema.get_tablet_ids(tablet_ids))) {
LOG_WARN("fail to get tablet_ids from table schema", KR(ret), K(table_schema));
LOG_WARN("tablet schema should have tablet", K(ret), K(simple_schema));
} else if (OB_FAIL(simple_schema.get_tablet_ids(tablet_ids))) {
LOG_WARN("fail to get tablet_ids from simple schema", KR(ret), K(simple_schema));
} else if (OB_UNLIKELY(tablet_ids.empty())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get tablet_ids of current table schema", KR(ret), K(table_schema));
LOG_WARN("fail to get tablet_ids of current table schema", KR(ret), K(simple_schema));
} else if (OB_FAIL(cur_tablet_ls_pair_array_.reserve(tablet_ids.count()))) {
LOG_WARN("failed to reserve tablet array", KR(ret), K(tablet_ids.count()));
} else if (OB_FAIL(finish_tablet_ls_pair_array_.reserve(tablet_ids.count()))) {
LOG_WARN("failed to reserve finish tablet array", KR(ret));
} else if (OB_FAIL(tablet_ls_pair_cache_.get_tablet_ls_pairs(table_id_, tablet_ids, cur_tablet_ls_pair_array_))) {
LOG_WARN("failed to tablet ls pair", KR(ret), K(tablet_ids));
} else {
// compare tablet_ids in table_schema & tablet_ls_pair in array
for (int64_t idx = 0; OB_SUCC(ret) && idx < tablet_ids.count(); ++idx) {
if (!tablet_ls_pair_array_.exist(tablet_ids.at(idx))) {
LOG_TRACE("meet next tablet compaction status", KR(ret), K(idx),
K(idx), "tablet_id", tablet_ids.at(idx), K_(tablet_ls_pair_array));
table_compaction_info_.set_can_skip_verifying();
if (OB_TMP_FAIL(table_compaction_map_.set_refactored(table_id_, table_compaction_info_, true /*overwrite*/))) {
LOG_WARN("fail to set refactored", KR(tmp_ret), K_(table_id), K_(table_compaction_info));
} else {
ret = OB_ITEM_NOT_MATCH;
}
} else if (OB_FAIL(cur_tablet_ls_pair_array_.push_back(tablet_ls_pair_array_.next()))) {
LOG_WARN("failed to push back pair", KR(ret));
}
} // end of for
if (OB_SUCC(ret)) {
LOG_TRACE("success to get tablet ls array idx", KR(ret), K_(tablet_ls_pair_array), K_(cur_tablet_ls_pair_array));
}
LOG_TRACE("success to get tablet ls pairs", KR(ret), K_(cur_tablet_ls_pair_array));
}
}
return ret;
@ -280,23 +239,26 @@ int ObChecksumValidator::validate_checksum(
// do nothing
} else if (tablet_status_map_.empty()) {
table_compaction_info_.set_uncompacted();
} else if (OB_FAIL(schema_guard_->get_table_schema(tenant_id_, table_id_, table_schema_))) {
} else if (OB_FAIL(schema_guard_->get_simple_table_schema(tenant_id_, table_id_, simple_schema_))) {
LOG_WARN("fail to get table schema", KR(ret), K_(tenant_id), K(table_id), K_(table_compaction_info));
} else if (OB_UNLIKELY(nullptr == table_schema_ // table deleted
|| !table_schema_->has_tablet())) {
} else if (OB_UNLIKELY(nullptr == simple_schema_ // table deleted
|| !simple_schema_->has_tablet())) {
// like VIEW, it does not have tablet, treat it as compaction finished and can skip verifying
table_compaction_info_.set_can_skip_verifying();
} else if (OB_FAIL(init_tablet_ls_array_idx(*table_schema_))) {
if (OB_ITEM_NOT_MATCH != ret) {
LOG_WARN("failed to get table pairs", K(ret), KPC_(table_schema));
} else if (OB_FAIL(get_tablet_ls_pairs(*simple_schema_))) {
if (OB_ITEM_NOT_MATCH == ret) {
ret = OB_SUCCESS;
table_compaction_info_.set_can_skip_verifying();
} else {
LOG_WARN("failed to get table pairs", K(ret), KPC_(simple_schema));
}
} else if (OB_UNLIKELY(cur_tablet_ls_pair_array_.empty())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet ls pair array is unexpected empty", KR(ret), KPC_(table_schema), K_(cur_tablet_ls_pair_array));
LOG_WARN("tablet ls pair array is unexpected empty", KR(ret), KPC_(simple_schema), K_(cur_tablet_ls_pair_array));
} else {
if (OB_FAIL(validate_tablet_replica_checksum())) {
LOG_WARN("fail to validate tablet replica checksum", KR(ret), K_(compaction_scn), K_(table_id),
KPC(table_schema_), K_(cur_tablet_ls_pair_array));
KPC(simple_schema_), K_(cur_tablet_ls_pair_array));
} else if (OB_FAIL(validate_index_checksum())) {
LOG_WARN("failed to validate index checksum", K(ret));
} else if (OB_FAIL(validate_cross_cluster_checksum())) {
@ -314,7 +276,7 @@ int ObChecksumValidator::validate_checksum(
} else if (replica_ckm_items_.count() > 0) {
int tmp_ret = OB_SUCCESS;
last_table_ckm_items_.clear();
if (OB_TMP_FAIL(last_table_ckm_items_.build(*table_schema_, cur_tablet_ls_pair_array_, replica_ckm_items_.array_))) {
if (OB_TMP_FAIL(last_table_ckm_items_.build(*schema_guard_, *simple_schema_, cur_tablet_ls_pair_array_, replica_ckm_items_.array_))) {
LOG_WARN("failed to build table ckm items", KR(tmp_ret), K_(table_id), K_(cur_tablet_ls_pair_array),
K_(replica_ckm_items));
} else {
@ -328,7 +290,7 @@ int ObChecksumValidator::validate_checksum(
// do no clear table_compaction_info_ until validate next table
replica_ckm_items_.reuse();
schema_guard_ = nullptr;
table_schema_ = nullptr;
simple_schema_ = nullptr;
return ret;
}
@ -336,9 +298,9 @@ int ObChecksumValidator::validate_tablet_replica_checksum()
{
int ret = OB_SUCCESS;
if (table_compaction_info_.is_uncompacted()) {
if (OB_UNLIKELY(nullptr == table_schema_ || !table_schema_->has_tablet())) {
if (OB_UNLIKELY(nullptr == simple_schema_ || !simple_schema_->has_tablet())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet schema should have tablet", K(ret), KPC_(table_schema));
LOG_WARN("tablet schema should have tablet", K(ret), KPC_(simple_schema));
} else {
if (OB_FAIL(update_table_compaction_info_by_tablet())) {
LOG_WARN("failed to check table compaction finish", K(ret));
@ -374,6 +336,16 @@ int ObChecksumValidator::update_table_compaction_info_by_tablet()
table_compaction_info_.set_uncompacted();
LOG_TRACE("tablet not exist in tablet status map", KR(ret), K(tablet_id),
K_(cur_tablet_ls_pair_array), K_(table_compaction_info));
#ifdef ERRSIM
ret = OB_E(EventTable::EN_SKIP_INDEX_MAJOR) ret;
if (OB_FAIL(ret)) {
ret = OB_SUCCESS;
if (tablet_id.id() > ObTabletID::MIN_USER_TABLET_ID) {
LOG_INFO("ERRSIM EN_SKIP_INDEX_MAJOR", K(ret), K(tablet_id));
table_compaction_info_.set_can_skip_verifying();
}
}
#endif
break;
} else {
LOG_WARN("fail to get tablet compaction status from map", KR(ret), K(idx));
@ -412,7 +384,7 @@ int ObChecksumValidator::get_tablet_replica_checksum_and_validate(const bool inc
} else if (OB_FAIL(verify_tablet_replica_checksum())) {
if (OB_CHECKSUM_ERROR == ret) {
LOG_DBA_ERROR(OB_CHECKSUM_ERROR, "msg", "ERROR! ERROR! ERROR! checksum error in major "
"tablet_replica_checksum", KR(ret), K_(tenant_id), K_(compaction_scn), K_(tablet_ls_pair_array));
"tablet_replica_checksum", KR(ret), K_(tenant_id), K_(compaction_scn), K_(cur_tablet_ls_pair_array));
} else {
LOG_WARN("fail to check major tablet_replica checksum", KR(ret), K_(tenant_id));
}
@ -465,44 +437,88 @@ int ObChecksumValidator::validate_cross_cluster_checksum()
K_(compaction_scn), K_(expected_epoch), K_(table_id));
}
} else { // no need to validate cross-cluster checksum, write checksum to inner_table
if (OB_FAIL(write_tablet_checksum_at_table_level())) {
LOG_WARN("fail to wrote checksum", KR(ret), K_(tenant_id), K_(compaction_scn), KPC_(table_schema));
} else {
++statistics_.write_ckm_sql_cnt_;
if (OB_FAIL(try_update_tablet_checksum_items())) {
LOG_WARN("fail to wrote checksum", KR(ret), K_(tenant_id), K_(compaction_scn), KPC_(simple_schema));
}
}
ret = OB_ITEM_NOT_MATCH == ret ? OB_SUCCESS : ret; // clear errno
if (OB_SUCC(ret)) {
if (OB_FAIL(finish_tablet_ls_pair_array_.push_back(cur_tablet_ls_pair_array_))) {
if (OB_FAIL(push_finish_tablet_ls_pairs_with_update(cur_tablet_ls_pair_array_))) {
LOG_WARN("failed to push back tablet_ls_pair", KR(ret));
} else {
table_compaction_info_.set_verified();
LOG_TRACE("after cross cluster validate table checksum", K(ret), K_(table_compaction_info));
}
}
#ifdef ERRSIM
if (OB_SUCC(ret)) {
#else
if (finish_tablet_ls_pair_array_.count() >= MAX_BATCH_INSERT_COUNT || check_special_table) {
#endif
int64_t tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(ObTabletMetaTableCompactionOperator::batch_update_report_scn(
tenant_id_,
compaction_scn_.get_val_for_tx(),
} else { // no need to validate cross-cluster checksum
// do nothing. index validator should already wrote ckm and updated report_scn
}
return ret;
}
int ObChecksumValidator::batch_write_tablet_ckm()
{
int ret = OB_SUCCESS;
bool is_match = false;
if (finish_tablet_ckm_array_.empty()) {
} else if (!is_primary_service_) {
// only primary major_freeze_service need to write tablet checksum
} else if (OB_FAIL(ObServiceEpochProxy::check_service_epoch(*sql_proxy_, tenant_id_,
ObServiceEpochProxy::FREEZE_SERVICE_EPOCH, expected_epoch_, is_match))) {
LOG_WARN("fail to check freeze service epoch", KR(ret), K_(tenant_id), K_(expected_epoch));
// Can not select freeze_service_epoch for update and update tablet_checksum_items in one same
// transaction, since __all_service_epoch and __all_tablet_checksum are not in one same tenant.
// Therefore, just get and check freeze_service_epoch here. However, this does not impact the
// correctness of updating tablet_checksum_items.
} else if (!is_match) {
ret = OB_FREEZE_SERVICE_EPOCH_MISMATCH;
LOG_WARN("no need to update tablet checksum items, cuz freeze_service_epoch mismatch",
K_(tenant_id), K_(expected_epoch));
} else {
const int64_t IMMEDIATE_RETRY_CNT = 5;
int64_t fail_count = 0;
int64_t sleep_time_us = 200 * 1000; // 200 ms
while (OB_SUCC(ret) && !stop_
&& (fail_count < IMMEDIATE_RETRY_CNT)) {
if (OB_SUCC(ObTabletChecksumOperator::update_tablet_checksum_items(
*sql_proxy_, tenant_id_, finish_tablet_ckm_array_))) {
++statistics_.write_ckm_sql_cnt_;
break;
} else if (OB_FREEZE_SERVICE_EPOCH_MISMATCH == ret) {
LOG_WARN("freeze_service_epoch mismatch, no need to write tablet checksum items", KR(ret), K_(tenant_id));
break;
} else if (OB_ITEM_NOT_MATCH == ret) {
LOG_INFO("tablet replica checksum item is empty, no need to write tablet checksum items", KR(ret), K_(tenant_id));
break;
} else {
++fail_count;
LOG_WARN("fail to write tablet checksum items", KR(ret), K_(tenant_id), K(fail_count), K(sleep_time_us));
USLEEP(sleep_time_us);
sleep_time_us *= 2;
}
} // end of while
finish_tablet_ckm_array_.reuse();
}
return ret;
}
int ObChecksumValidator::batch_update_report_scn()
{
int ret = OB_SUCCESS;
if (finish_tablet_ls_pair_array_.empty()) {
} else if (OB_FAIL(ObTabletMetaTableCompactionOperator::batch_update_report_scn(
tenant_id_, compaction_scn_.get_val_for_tx(),
finish_tablet_ls_pair_array_,
ObTabletReplica::ScnStatus::SCN_STATUS_ERROR /*except_status*/,
expected_epoch_))) {
LOG_WARN("fail to batch update report_scn", KR(tmp_ret), K_(tenant_id),
K_(tablet_ls_pair_array));
LOG_WARN("fail to batch update report_scn", KR(ret), K_(tenant_id),
K_(finish_tablet_ls_pair_array));
} else {
++statistics_.update_report_scn_sql_cnt_;
LOG_INFO("success to batch update report_scn", KR(tmp_ret), K_(tenant_id), "table_cnt", finish_tablet_ls_pair_array_.count());
LOG_INFO("success to batch update report_scn", KR(ret), K_(tenant_id),
"table_cnt", finish_tablet_ls_pair_array_.count());
finish_tablet_ls_pair_array_.reuse();
}
}
} else { // no need to validate cross-cluster checksum
// do nothing. index validator should already wrote ckm and updated report_scn
}
return ret;
}
@ -604,55 +620,9 @@ bool ObChecksumValidator::check_waiting_tablet_checksum_timeout() const
return (total_wait_time_us > MAX_TABLET_CHECKSUM_WAIT_TIME_US);
}
int ObChecksumValidator::write_tablet_checksum_at_table_level()
{
int ret = OB_SUCCESS;
bool is_exist = false;
FREEZE_TIME_GUARD;
if (stop_) {
ret = OB_CANCELED;
LOG_WARN("already stop", KR(ret), K_(tenant_id));
} else if (is_primary_service_) { // only primary major_freeze_service need to write tablet checksum
const int64_t IMMEDIATE_RETRY_CNT = 5;
int64_t fail_count = 0;
int64_t sleep_time_us = 200 * 1000; // 200 ms
while (!stop_
&& (fail_count < IMMEDIATE_RETRY_CNT)
&& OB_FAIL(try_update_tablet_checksum_items())) {
if (OB_FREEZE_SERVICE_EPOCH_MISMATCH == ret) {
LOG_WARN("freeze_service_epoch mismatch, no need to write tablet checksum items", KR(ret), K_(tenant_id));
break;
} else if (OB_ITEM_NOT_MATCH == ret) {
LOG_INFO("tablet replica checksum item is empty, no need to write tablet checksum items", KR(ret), K_(tenant_id));
break;
} else {
++fail_count;
LOG_WARN("fail to write tablet checksum items", KR(ret), K_(tenant_id), K(fail_count), K(sleep_time_us));
USLEEP(sleep_time_us);
sleep_time_us *= 2;
}
} // end of while
}
return ret;
}
int ObChecksumValidator::try_update_tablet_checksum_items()
{
int ret = OB_SUCCESS;
bool is_match = false;
// Can not select freeze_service_epoch for update and update tablet_checksum_items in one same
// transaction, since __all_service_epoch and __all_tablet_checksum are not in one same tenant.
// Therefore, just get and check freeze_service_epoch here. However, this does not impact the
// correctness of updating tablet_checksum_items.
if (OB_FAIL(ObServiceEpochProxy::check_service_epoch(*sql_proxy_, tenant_id_,
ObServiceEpochProxy::FREEZE_SERVICE_EPOCH, expected_epoch_, is_match))) {
LOG_WARN("fail to check freeze service epoch", KR(ret), K_(tenant_id), K_(expected_epoch));
} else if (!is_match) {
ret = OB_FREEZE_SERVICE_EPOCH_MISMATCH;
LOG_WARN("no need to update tablet checksum items, cuz freeze_service_epoch mismatch",
K_(tenant_id), K_(expected_epoch));
} else {
FREEZE_TIME_GUARD;
const bool include_lager_than = (table_id_ == SPECIAL_TABLE_ID ? true : false);
if (replica_ckm_items_.empty() && OB_FAIL(get_replica_ckm(include_lager_than))) {
LOG_WARN("fail to batch get tablet replica checksum items", KR(ret), K_(tenant_id), K_(compaction_scn));
@ -660,48 +630,62 @@ int ObChecksumValidator::try_update_tablet_checksum_items()
ret = OB_ITEM_NOT_MATCH;
LOG_WARN("fail to get tablet replica checksum items", KR(ret), K_(tenant_id), K_(compaction_scn),
K_(cur_tablet_ls_pair_array), K(replica_ckm_items_));
} else if (OB_FAIL(push_tablet_ckm_items_with_update(replica_ckm_items_.array_))) {
LOG_WARN("fail to push tablet checksum items", KR(ret), K_(tenant_id));
}
return ret;
}
int ObChecksumValidator::push_finish_tablet_ls_pairs_with_update(
const common::ObIArray<share::ObTabletLSPair> &tablet_ls_pairs)
{
int ret = OB_SUCCESS;
if (OB_FAIL(finish_tablet_ls_pair_array_.push_back(tablet_ls_pairs))) {
LOG_WARN("failed to push back tablet_ls_pairs", KR(ret), K(tablet_ls_pairs));
} else {
bool need_update_report_scn = (finish_tablet_ls_pair_array_.count() >= MAX_BATCH_INSERT_COUNT)
|| table_id_ == SPECIAL_TABLE_ID;
#ifdef ERRSIM
need_update_report_scn = true;
#endif
if (need_update_report_scn) {
int64_t tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(batch_update_report_scn())) {
LOG_WARN("failed to batch update report scn", KR(tmp_ret));
}
}
}
return ret;
}
int ObChecksumValidator::push_tablet_ckm_items_with_update(
const ObIArray<ObTabletReplicaChecksumItem> &replica_ckm_items)
{
int ret = OB_SUCCESS;
const ObTabletReplicaChecksumItem *prev_replica_item = nullptr;
ObTabletChecksumItem tmp_checksum_item;
// mark_end_item is the 'first_ls & first_tablet' checksum item. If we get this checksum item,
// we need to insert it into __all_tablet_checksum table at last. In this case, if we get this
// tablet's checksum item in table, we can ensure all checksum items have already been inserted.
ObTabletChecksumItem mark_end_item;
SMART_VAR(ObArray<ObTabletChecksumItem>, tablet_checksum_items) {
for (int64_t i = 0; !stop_ && OB_SUCC(ret) && (i < replica_ckm_items_.count()); ++i) {
const ObTabletReplicaChecksumItem &curr_replica_item = replica_ckm_items_.at(i);
for (int64_t i = 0; !stop_ && OB_SUCC(ret) && (i < replica_ckm_items.count()); ++i) {
const ObTabletReplicaChecksumItem &curr_replica_item = replica_ckm_items.at(i);
if (OB_UNLIKELY(!curr_replica_item.is_key_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet replica checksum is not valid", KR(ret), K(curr_replica_item));
LOG_WARN("tablet replica checksum is not valid", KR(ret),
K(curr_replica_item));
} else {
if (nullptr != prev_replica_item
&& curr_replica_item.is_same_tablet(*prev_replica_item)) { // write one checksum_item per tablet
} else if (OB_FAIL(tmp_checksum_item.assign(curr_replica_item))) { // ObTabletReplicaChecksumItem->ObTabletChecksumItem
LOG_WARN("fail to assign tablet replica checksum item", KR(ret), K(curr_replica_item));
} else if (OB_FAIL(tablet_checksum_items.push_back(tmp_checksum_item))) {
LOG_WARN("fail to push back tablet checksum item", KR(ret), K(curr_replica_item), K(tmp_checksum_item));
if (nullptr != prev_replica_item && curr_replica_item.is_same_tablet( *prev_replica_item)) { // write one checksum_item per tablet
} else if (OB_FAIL(tmp_checksum_item.assign(curr_replica_item))) {
// ObTabletReplicaChecksumItem->ObTabletChecksumItem
LOG_WARN("fail to assign tablet replica checksum item", KR(ret),
K(curr_replica_item));
} else if (OB_FAIL(finish_tablet_ckm_array_.push_back(tmp_checksum_item))) {
LOG_WARN("fail to push back tablet checksum item", KR(ret),
K(curr_replica_item), K(tmp_checksum_item));
}
prev_replica_item = &curr_replica_item;
if (tablet_checksum_items.count() >= MAX_BATCH_INSERT_COUNT) { // write tablet checksum in batch
FREEZE_TIME_GUARD;
if (FAILEDx(ObTabletChecksumOperator::update_tablet_checksum_items(*sql_proxy_,
tenant_id_, tablet_checksum_items))) {
LOG_WARN("fail to try update tablet checksum items", KR(ret), K_(tenant_id));
} else {
tablet_checksum_items.reuse();
}
}
}
} // end of for
if (OB_SUCC(ret) && tablet_checksum_items.count() > 0) { // already iterate all tablet checksum items
FREEZE_TIME_GUARD;
if (OB_FAIL(ObTabletChecksumOperator::update_tablet_checksum_items(*sql_proxy_,
tenant_id_, tablet_checksum_items))) {
LOG_WARN("fail to try update tablet checksum items", KR(ret), K_(tenant_id));
}
}
} // SMART_VAR
}
if (OB_SUCC(ret)
&& (finish_tablet_ckm_array_.count() >= MAX_BATCH_INSERT_COUNT || table_id_ == SPECIAL_TABLE_ID)) {
(void) batch_write_tablet_ckm();
}
return ret;
}
@ -713,14 +697,14 @@ int ObChecksumValidator::validate_index_checksum() {
if (stop_) {
ret = OB_CANCELED;
LOG_WARN("already stop", KR(ret), K_(tenant_id));
} else if (OB_ISNULL(table_schema_)) {
} else if (OB_ISNULL(simple_schema_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table schema is unexpected null", K(ret), KPC_(table_schema));
LOG_WARN("table schema is unexpected null", K(ret), KPC_(simple_schema));
} else if (!need_validate_index_ckm_) { // no need to validate data-index checksum
table_compaction_info_.set_index_ckm_verified();
} else if (table_schema_->is_index_table()) { // for index table, do not check status
if (OB_FAIL(handle_index_table(*table_schema_))) {
LOG_WARN("fail to handle index table", KR(ret), KPC_(table_schema));
} else if (simple_schema_->is_index_table()) { // for index table, do not check status
if (OB_FAIL(handle_index_table(*simple_schema_))) {
LOG_WARN("fail to handle index table", KR(ret), KPC_(simple_schema));
}
} else if (table_compaction_info_.is_compacted()) { // for data table, check status
if (0 == table_compaction_info_.unfinish_index_cnt_) { // no unfinish index
@ -732,24 +716,16 @@ int ObChecksumValidator::validate_index_checksum() {
}
int ObChecksumValidator::handle_index_table(
const share::schema::ObTableSchema &index_schema)
const share::schema::ObSimpleTableSchemaV2 &index_simple_schema)
{
int ret = OB_SUCCESS;
const uint64_t index_table_id = index_schema.get_table_id();
const uint64_t data_table_id = index_schema.get_data_table_id();
const ObTableSchema *data_schema = nullptr;
if (OB_FAIL(schema_guard_->get_table_schema(tenant_id_, data_table_id, data_schema))) {
LOG_WARN("fail to get table schema", KR(ret), K_(tenant_id), K(data_table_id));
} else if (OB_ISNULL(data_schema)) {
ret = OB_TABLE_NOT_EXIST;
table_compaction_info_.set_can_skip_verifying();
LOG_WARN("fail to get data table schema", KR(ret), K_(tenant_id), K(index_table_id), K(data_table_id));
} else {
const uint64_t index_table_id = index_simple_schema.get_table_id();
const uint64_t data_table_id = index_simple_schema.get_data_table_id();
ObTableCompactionInfo &index_compaction_info = table_compaction_info_; // cur table is index
ObTableCompactionInfo data_compaction_info;
if (OB_FAIL(get_table_compaction_info(data_table_id, data_compaction_info))) {
LOG_WARN("fail to get table compaction info", KR(ret), KPC(data_schema));
} else if (!index_schema.can_read_index()) {
LOG_WARN("fail to get table compaction info", KR(ret), K(data_table_id));
} else if (!index_simple_schema.can_read_index()) {
// for index table can not read, directly mark it as VERIFIED
// do not check compaction_scn and validate checksum of can not read
// index's tablets. although update_all_tablets_report_scn will update
@ -785,14 +761,14 @@ int ObChecksumValidator::handle_index_table(
// set it to false, if succ to handle_table_can_not_verify
// both tables' all tablets finished compaction, validate column
// checksum if need_validate()
if (OB_UNLIKELY(index_schema.should_not_validate_data_index_ckm())) {
if (OB_UNLIKELY(index_simple_schema.should_not_validate_data_index_ckm())) {
// do nothing
// spatial index column is different from data table column
index_compaction_info.set_index_ckm_verified();
} else if (1 == data_compaction_info.unfinish_index_cnt_ || last_table_ckm_items_.is_inited()) {
// only one index
if (OB_FAIL(verify_table_index(*data_schema, index_schema, data_compaction_info, index_compaction_info))) {
LOG_WARN("failed to verify table index checksum", K(ret), KPC(data_schema), K(index_schema));
if (OB_FAIL(verify_table_index(index_simple_schema, data_compaction_info, index_compaction_info))) {
LOG_WARN("failed to verify table index checksum", K(ret), K(index_simple_schema));
}
} else if (OB_FAIL(idx_ckm_validate_array_.push_back(ObIndexCkmValidatePair(data_table_id, index_table_id)))) {
LOG_WARN("failed to push back table validate info", K(ret), K(data_table_id), K(index_table_id));
@ -816,33 +792,26 @@ int ObChecksumValidator::handle_index_table(
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));
return ret;
}
int ObChecksumValidator::verify_table_index(
const share::schema::ObTableSchema &data_schema,
const share::schema::ObTableSchema &index_schema,
const share::schema::ObSimpleTableSchemaV2 &index_simple_schema,
compaction::ObTableCompactionInfo &data_compaction_info,
compaction::ObTableCompactionInfo &index_compaction_info)
{
int ret = OB_SUCCESS;
FREEZE_TIME_GUARD;
const uint64_t index_table_id = index_schema.get_table_id();
const uint64_t data_table_id = index_schema.get_data_table_id();
if (OB_UNLIKELY(data_table_id != data_schema.get_table_id())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("current table and index should not validate checksum", K(ret), K(data_schema),
K(index_schema));
} else if (replica_ckm_items_.empty() && OB_FAIL(get_replica_ckm())) {
const uint64_t index_table_id = index_simple_schema.get_table_id();
const uint64_t data_table_id = index_simple_schema.get_data_table_id();
if (replica_ckm_items_.empty() && OB_FAIL(get_replica_ckm())) {
LOG_WARN("fail to batch get tablet replica checksum items", KR(ret), K_(tenant_id), K_(compaction_scn));
} else if (replica_ckm_items_.tablet_cnt_ < cur_tablet_ls_pair_array_.count()) {
ret = OB_ITEM_NOT_MATCH;
LOG_WARN("fail to get tablet replica checksum items", KR(ret), K_(tenant_id), K_(compaction_scn),
K_(tablet_ls_pair_array), K(replica_ckm_items_));
K_(cur_tablet_ls_pair_array), K(replica_ckm_items_));
} else {
const ObTableSchema *data_table_schema = &data_schema;
ObTableCkmItems data_table_ckm(tenant_id_);
ObTableCkmItems *data_table_ckm_ptr = nullptr;
ObTableCkmItems index_table_ckm(tenant_id_);
@ -859,26 +828,23 @@ int ObChecksumValidator::verify_table_index(
} else if (FALSE_IT(data_table_ckm_ptr = &data_table_ckm)) {
} else if (OB_FAIL(data_table_ckm.build(data_table_id, compaction_scn_,
*sql_proxy_, *schema_guard_,
data_table_schema,
false /*get_table_schema_flag*/))) {
tablet_ls_pair_cache_))) {
LOG_WARN("failed to get checksum items", K(ret), K(data_table_id), K_(compaction_scn));
} else {
++statistics_.query_ckm_sql_cnt_;
}
if (FAILEDx(index_table_ckm.build(index_schema, cur_tablet_ls_pair_array_,
if (FAILEDx(index_table_ckm.build(*schema_guard_, index_simple_schema, cur_tablet_ls_pair_array_,
replica_ckm_items_.array_))) {
LOG_WARN("failed to assgin checksum items", K(ret), K(replica_ckm_items_));
} else {
const bool is_global_index = index_schema.is_global_index_table();
const bool is_global_index = index_simple_schema.is_global_index_table();
if (OB_FAIL(ObTableCkmItems::validate_ckm_func[is_global_index](
compaction_scn_,
data_schema,
index_schema,
*sql_proxy_,
*data_table_ckm_ptr,
index_table_ckm))) {
LOG_WARN("failed to validate checksum", K(ret), K(data_table_id),
K(index_table_id), KPC(data_table_ckm_ptr), K(index_table_ckm), K_(tablet_ls_pair_array),
K(index_table_id), KPC(data_table_ckm_ptr), K(index_table_ckm),
K_(replica_ckm_items), K_(cur_tablet_ls_pair_array), K_(last_table_ckm_items));
}
}

View File

@ -32,26 +32,6 @@ namespace rootserver
class ObZoneMergeManager;
class ObServerManager;
template <typename T>
struct ObArrayWithIdx {
ObArrayWithIdx(const common::ObArray<T> &array)
: last_idx_(0),
array_(array)
{}
bool exist(const ObTabletID &tablet_id) const;
const T &next()
{
return array_.at(last_idx_++);
}
void clear()
{
last_idx_ = 0;
}
int64_t to_string(char* buf, const int64_t buf_len) const;
int64_t last_idx_;
const common::ObArray<T> &array_;
};
struct ObReplicaCkmItems
{
ObReplicaCkmItems()
@ -77,11 +57,13 @@ public:
ObChecksumValidator(
const uint64_t tenant_id,
volatile bool &stop,
const compaction::ObTabletLSPairCache &tablet_ls_pair_cache,
const compaction::ObTabletStatusMap &tablet_status_map,
const compaction::ObTabletLSPairArray &tablet_to_ls_array,
compaction::ObTableCompactionInfoMap &table_compaction_map,
compaction::ObIndexCkmValidatePairArray &idx_ckm_validate_array,
compaction::ObCkmValidatorStatistics &statistics)
compaction::ObCkmValidatorStatistics &statistics,
ObArray<share::ObTabletLSPair> &finish_tablet_ls_pair_array,
ObArray<share::ObTabletChecksumItem> &finish_tablet_ckm_array)
: is_inited_(false),
is_primary_service_(false),
need_validate_index_ckm_(false),
@ -95,12 +77,14 @@ public:
major_merge_start_us_(0),
statistics_(statistics),
sql_proxy_(nullptr),
tablet_ls_pair_cache_(tablet_ls_pair_cache),
tablet_status_map_(tablet_status_map),
tablet_ls_pair_array_(tablet_to_ls_array),
table_compaction_map_(table_compaction_map),
idx_ckm_validate_array_(idx_ckm_validate_array),
finish_tablet_ls_pair_array_(finish_tablet_ls_pair_array),
finish_tablet_ckm_array_(finish_tablet_ckm_array),
schema_guard_(nullptr),
table_schema_(nullptr),
simple_schema_(nullptr),
table_compaction_info_(),
replica_ckm_items_(),
last_table_ckm_items_(tenant_id)
@ -124,16 +108,21 @@ public:
void clear_cached_info();
void clear_array_index()
{
tablet_ls_pair_array_.last_idx_ = 0;
last_table_ckm_items_.clear();
}
int push_tablet_ckm_items_with_update(
const ObIArray<share::ObTabletReplicaChecksumItem> &replica_ckm_items);
int push_finish_tablet_ls_pairs_with_update(
const common::ObIArray<share::ObTabletLSPair> &tablet_ls_pairs);
int batch_write_tablet_ckm();
int batch_update_report_scn();
static const int64_t SPECIAL_TABLE_ID = 1;
TO_STRING_KV(K_(tenant_id), K_(is_primary_service), K_(table_id), K_(compaction_scn));
private:
int check_inner_status();
int get_table_compaction_info(const uint64_t table_id, compaction::ObTableCompactionInfo &table_compaction_info);
int set_need_validate();
int init_tablet_ls_array_idx(const share::schema::ObTableSchema &table_schema);
int get_tablet_ls_pairs(const share::schema::ObSimpleTableSchemaV2 &simple_schema);
int get_replica_ckm(const bool include_larger_than = false);
/* Tablet Replica Checksum Section */
int validate_tablet_replica_checksum();
@ -144,10 +133,9 @@ private:
/* Index Checksum Section */
int validate_index_checksum();
int handle_index_table(const share::schema::ObTableSchema &index_schema);
int handle_index_table(const share::schema::ObSimpleTableSchemaV2 &index_simple_schema);
int verify_table_index(
const share::schema::ObTableSchema &data_schema,
const share::schema::ObTableSchema &index_schema,
const share::schema::ObSimpleTableSchemaV2 &index_simple_schema,
compaction::ObTableCompactionInfo &data_compaction_info,
compaction::ObTableCompactionInfo &index_compaction_info);
@ -158,7 +146,6 @@ private:
int check_column_checksum(
const ObArray<share::ObTabletReplicaChecksumItem> &tablet_replica_checksum_items,
const ObArray<share::ObTabletChecksumItem> &tablet_checksum_items);
int write_tablet_checksum_at_table_level();
bool check_waiting_tablet_checksum_timeout() const;
int try_update_tablet_checksum_items();
static const int64_t PRINT_CROSS_CLUSTER_LOG_INVERVAL = 10 * 60 * 1000 * 1000; // 10 mins
@ -178,17 +165,18 @@ private:
compaction::ObCkmValidatorStatistics &statistics_;
common::ObMySQLProxy *sql_proxy_;
/* reference to obj in PorgressChecker */
const compaction::ObTabletLSPairCache &tablet_ls_pair_cache_;
const compaction::ObTabletStatusMap &tablet_status_map_;
ObArrayWithIdx<share::ObTabletLSPair> tablet_ls_pair_array_;
compaction::ObTableCompactionInfoMap &table_compaction_map_;
compaction::ObIndexCkmValidatePairArray &idx_ckm_validate_array_;
ObArray<share::ObTabletLSPair> &finish_tablet_ls_pair_array_;
ObArray<share::ObTabletChecksumItem> &finish_tablet_ckm_array_;
/* different for every table */
share::schema::ObSchemaGetterGuard *schema_guard_;
const share::schema::ObTableSchema *table_schema_;
const share::schema::ObSimpleTableSchemaV2 *simple_schema_;
compaction::ObTableCompactionInfo table_compaction_info_;
ObArray<share::ObTabletLSPair> cur_tablet_ls_pair_array_;
ObArray<share::ObTabletLSPair> finish_tablet_ls_pair_array_;
ObReplicaCkmItems replica_ckm_items_;
compaction::ObTableCkmItems last_table_ckm_items_; // only cached last data table with index
};

View File

@ -51,12 +51,12 @@ ObMajorMergeProgressChecker::ObMajorMergeProgressChecker(
compaction_scn_(), expected_epoch_(OB_INVALID_ID), sql_proxy_(nullptr),
schema_service_(nullptr), server_trace_(nullptr), progress_(),
tablet_status_map_(), table_compaction_map_(),
ckm_validator_(tenant_id, stop_, tablet_status_map_,
tablet_ls_pair_array_, table_compaction_map_,
idx_ckm_validate_array_, validator_statistics_),
ckm_validator_(tenant_id, stop_, tablet_ls_pair_cache_, tablet_status_map_,
table_compaction_map_, idx_ckm_validate_array_, validator_statistics_,
finish_tablet_ls_pair_array_, finish_tablet_ckm_array_),
uncompacted_tablets_(),
diagnose_rw_lock_(ObLatchIds::MAJOR_FREEZE_DIAGNOSE_LOCK),
ls_locality_cache_(), total_time_guard_(), validator_statistics_() {}
ls_locality_cache_(), total_time_guard_(), validator_statistics_(), batch_size_mgr_() {}
int ObMajorMergeProgressChecker::init(
const bool is_primary_service,
@ -69,7 +69,7 @@ int ObMajorMergeProgressChecker::init(
if (IS_INIT) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", KR(ret));
} else if (OB_FAIL(tablet_status_map_.create(TABLET_ID_BATCH_CHECK_SIZE, "RSStCompactMap", "RSStCompactMap", tenant_id_))) {
} else if (OB_FAIL(tablet_status_map_.create(TABLET_ID_BATCH_CHECK_SIZE, "RSCompStMap", "RSCompStMap", tenant_id_))) {
LOG_WARN("fail to create tablet compaction status map", KR(ret), K_(tenant_id));
} else if (OB_FAIL(table_compaction_map_.create(TABLE_MAP_BUCKET_CNT, "RSCompactMap", "RSCompactMap", tenant_id_))) {
LOG_WARN("fail to create table compaction info map", KR(ret), K_(tenant_id), K(TABLE_MAP_BUCKET_CNT));
@ -78,8 +78,10 @@ int ObMajorMergeProgressChecker::init(
} else if (OB_FAIL(ls_locality_cache_.init(tenant_id_, &merge_info_mgr))) {
LOG_WARN("failed to init ls locality cache", K(ret));
} else {
(void) tablet_ls_pair_cache_.set_tenant_id(tenant_id_);
idx_ckm_validate_array_.set_attr(ObMemAttr(tenant_id_, "RSCompCkmPair"));
tablet_ls_pair_array_.set_attr(ObMemAttr(tenant_id_, "RSCompLsPair"));
finish_tablet_ls_pair_array_.set_attr(ObMemAttr(tenant_id_, "RSCompTabPair"));
finish_tablet_ckm_array_.set_attr(ObMemAttr(tenant_id_, "RSCompCkmArray"));
sql_proxy_ = &sql_proxy;
schema_service_ = &schema_service;
server_trace_ = &server_trace;
@ -89,6 +91,38 @@ int ObMajorMergeProgressChecker::init(
return ret;
}
int ObMajorMergeProgressChecker::rebuild_map_by_tablet_cnt()
{
int ret = OB_SUCCESS;
int64_t recommend_map_bucked_cnt = 0;
if (first_loop_in_cur_round_) {
const bool need_rebuild_table_map = ObScheduleBatchSizeMgr::need_rebuild_map(
TABLE_MAP_BUCKET_CNT, table_ids_.count(), table_compaction_map_.bucket_count(), recommend_map_bucked_cnt);
if (need_rebuild_table_map) {
table_compaction_map_.destroy();
if (OB_FAIL(table_compaction_map_.create(recommend_map_bucked_cnt, "RSCompactMap", "RSCompactMap", tenant_id_))) {
LOG_WARN("fail to create table compaction info map", KR(ret), K_(tenant_id), K(recommend_map_bucked_cnt));
} else {
LOG_INFO("success to rebuild table compaction info map", KR(ret), K(recommend_map_bucked_cnt));
}
}
}
if (OB_SUCC(ret)) {
int64_t tablet_cnt = MAX(table_ids_.count(), tablet_status_map_.size());
const bool need_rebuild_tablet_map = ObScheduleBatchSizeMgr::need_rebuild_map(
TABLET_ID_BATCH_CHECK_SIZE, tablet_cnt, tablet_status_map_.bucket_count(), recommend_map_bucked_cnt);
if (need_rebuild_tablet_map) {
tablet_status_map_.destroy();
if (OB_FAIL(tablet_status_map_.create(recommend_map_bucked_cnt, "RSCompStMap", "RSCompStMap", tenant_id_))) {
LOG_WARN("fail to create table compaction info map", KR(ret), K_(tenant_id), K(recommend_map_bucked_cnt));
} else {
LOG_INFO("success to rebuild tablet status map", KR(ret), K(recommend_map_bucked_cnt));
}
}
}
return ret;
}
int ObMajorMergeProgressChecker::set_basic_info(
SCN global_broadcast_scn,
const int64_t expected_epoch)
@ -122,12 +156,14 @@ int ObMajorMergeProgressChecker::clear_cached_info()
expected_epoch_ = OB_INVALID_ID;
first_loop_in_cur_round_ = true;
table_ids_.reset();
tablet_ls_pair_array_.reset();
idx_ckm_validate_array_.reset();
finish_tablet_ls_pair_array_.reset();
finish_tablet_ckm_array_.reset();
progress_.reset();
ckm_validator_.clear_cached_info();
uncompacted_tablets_.reset();
loop_cnt_ = 0;
tablet_ls_pair_cache_.reuse();
}
return ret;
}
@ -156,21 +192,31 @@ int ObMajorMergeProgressChecker::check_verification(
int tmp_ret = OB_SUCCESS;
const ObTableCompactionInfo *table_compaction_info_ptr = &ckm_validator_.get_table_compaction_info();
ckm_validator_.clear_array_index();
for (int64_t idx = table_ids_.batch_start_idx_; !stop_ && idx < table_ids_.batch_end_idx_; ++idx) {
int64_t table_batch_size = 0;
(void) batch_size_mgr_.get_rs_check_batch_size(table_ids_.count(), table_batch_size);
int64_t index_cnt = 0;
int64_t table_cnt = 0;
int64_t idx = table_ids_.batch_start_idx_;
for ( ; OB_SUCC(ret) && !stop_ && idx < table_ids_.count(); ++idx) {
const uint64_t table_id = table_ids_.at(idx);
LOG_TRACE("verify table id", KR(ret), K_(tenant_id), K(table_id));
if (OB_TMP_FAIL(ckm_validator_.validate_checksum(table_id, schema_guard))) {
if (OB_CHECKSUM_ERROR == tmp_ret) {
ret = OB_CHECKSUM_ERROR;
if (can_not_ignore_warning(tmp_ret)) {
ret = tmp_ret;
if (OB_CHECKSUM_ERROR == ret) {
LOG_ERROR("checksum error", KR(ret), K(table_id));
} else if (OB_FREEZE_SERVICE_EPOCH_MISMATCH == tmp_ret) {
ret = OB_FREEZE_SERVICE_EPOCH_MISMATCH;
} else if (OB_FREEZE_SERVICE_EPOCH_MISMATCH == ret) {
LOG_INFO("freeze service epoch mismatch", KR(tmp_ret));
break;
}
} else {
LOG_WARN("failed to verify table", KR(tmp_ret), K(idx), K(table_id), KPC(table_compaction_info_ptr));
}
}
if (table_compaction_info_ptr->is_index_table()) {
--index_cnt;
} else {
index_cnt = table_compaction_info_ptr->unfinish_index_cnt_;
}
// ignore errno, need update progress & unfinish table id array
LOG_TRACE("check verification", KR(tmp_ret), KPC(table_compaction_info_ptr), K_(progress));
(void) progress_.update_table_cnt(table_compaction_info_ptr->status_);
@ -179,7 +225,16 @@ int ObMajorMergeProgressChecker::check_verification(
LOG_WARN("failed to push table_id into finish_array", KR(tmp_ret), KPC(table_compaction_info_ptr));
}
}
if (0 >= index_cnt // data & index should be in same batch
&& (++table_cnt >= table_batch_size)) {
break;
}
} // end of for
if (OB_SUCC(ret)) {
table_ids_.batch_start_idx_ = idx + 1;
} else {
(void) reuse_rest_table(idx, unfinish_table_id_array);
}
return ret;
}
@ -344,45 +399,26 @@ int ObMajorMergeProgressChecker::get_table_and_index_schema(
return ret;
}
void ObMajorMergeProgressChecker::reuse_batch_table(
ObIArray<uint64_t> &unfinish_table_id_array,
const bool reuse_rest_table)
void ObMajorMergeProgressChecker::reuse_rest_table(
const int64_t start_idx,
ObIArray<uint64_t> &unfinish_table_id_array)
{
int tmp_ret = OB_SUCCESS;
int64_t start_idx = 0;
int64_t end_idx = 0;
if (reuse_rest_table) {
start_idx = table_ids_.batch_end_idx_;
end_idx = table_ids_.count();
} else {
start_idx = table_ids_.batch_start_idx_;
end_idx = table_ids_.batch_end_idx_;
}
for (int64_t idx = start_idx; idx < end_idx; ++idx) {
for (int64_t idx = start_idx; idx < table_ids_.count(); ++idx) {
const uint64_t table_id = table_ids_.at(idx);
if (OB_TMP_FAIL(unfinish_table_id_array.push_back(table_id))) {
LOG_WARN_RET(tmp_ret, "failed to push table_id into finish_array", KR(tmp_ret));
LOG_WARN_RET(tmp_ret, "failed to push table_id into finish_array");
}
}
}
void ObMajorMergeProgressChecker::get_check_batch_size(
int64_t &tablet_id_batch_size,
int64_t &table_id_batch_size) const
{
tablet_id_batch_size = TABLET_ID_BATCH_CHECK_SIZE;
table_id_batch_size = TABLE_ID_BATCH_CHECK_SIZE;
if (table_ids_.count() > TOTAL_TABLE_CNT_THREASHOLD) {
int64_t factor = (table_ids_.count() / TOTAL_TABLE_CNT_THREASHOLD) * 2;
tablet_id_batch_size *= factor;
table_id_batch_size *= factor;
}
}
int ObMajorMergeProgressChecker::prepare_check_progress()
int ObMajorMergeProgressChecker::prepare_check_progress(
ObRSCompactionTimeGuard &tmp_time_guard,
bool &exist_uncompacted_table)
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
exist_uncompacted_table = true;
if (OB_TMP_FAIL(ls_locality_cache_.refresh_ls_locality(first_loop_in_cur_round_ /*force_refresh*/))) {
LOG_WARN("failed to refresh ls locality", K(tmp_ret));
}
@ -394,13 +430,28 @@ int ObMajorMergeProgressChecker::prepare_check_progress()
} else {
total_time_guard_.click(ObRSCompactionTimeGuard::PREPARE_UNFINISH_TABLE_IDS);
progress_.total_table_cnt_ = table_ids_.count() + 1/*SPECIAL_TABLE_ID*/;
}
}
if (FAILEDx(rebuild_map_by_tablet_cnt())) {
LOG_WARN("failed to rebuild map by tablet cnt", K(ret));
} else if (OB_FAIL(tablet_ls_pair_cache_.try_refresh(first_loop_in_cur_round_ /*force_refresh*/))) {
LOG_WARN("failed to refresh tablet ls pair", K(ret));
} else {
tmp_time_guard.click(ObRSCompactionTimeGuard::GET_TABLET_LS_PAIRS);
first_loop_in_cur_round_ = false;
}
}
if (OB_SUCC(ret)) {
table_ids_.start_looping();
exist_uncompacted_table = progress_.exist_uncompacted_table();
progress_.clear_before_each_loop();
reset_uncompacted_tablets();
if (is_extra_check_round()) {
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id_));
if (OB_UNLIKELY(!tenant_config.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tenant config is not valid", KR(ret), K_(tenant_id));
} else {
batch_size_mgr_.set_tablet_batch_size(tenant_config->compaction_schedule_tablet_batch_cnt);
}
}
}
return ret;
}
@ -434,6 +485,8 @@ int ObMajorMergeProgressChecker::check_index_and_rest_table()
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cnt in progress is not equal to table_ids", KR(ret), K(table_ids_), K(progress_));
}
(void) ckm_validator_.batch_update_report_scn();
(void) ckm_validator_.batch_write_tablet_ckm();
return ret;
}
@ -443,6 +496,8 @@ int ObMajorMergeProgressChecker::check_progress(
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
const int64_t start_time = ObTimeUtility::fast_current_time();
ObRSCompactionTimeGuard tmp_time_guard;
bool exist_uncompacted_table = true;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret), K_(tenant_id));
@ -452,24 +507,22 @@ int ObMajorMergeProgressChecker::check_progress(
} else if (OB_UNLIKELY(expected_epoch_ < 0 || !compaction_scn_.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cached info may be cleared", KR(ret), K_(expected_epoch), K_(compaction_scn));
} else if (OB_FAIL(prepare_check_progress())) {
} else if (OB_FAIL(prepare_check_progress(tmp_time_guard, exist_uncompacted_table))) {
LOG_WARN("failed to prepare check progress", KR(ret));
} else {
SMART_VARS_2((ObArray<uint64_t>, unfinish_table_id_array),
(ObCompactionTabletMetaIterator, iter)) {
SMART_VAR(ObArray<uint64_t>, unfinish_table_id_array) {
int64_t tenant_schema_version = 0;
unfinish_table_id_array.set_attr(ObMemAttr(tenant_id_, "RSCompTableIds"));
if (OB_FAIL(unfinish_table_id_array.reserve(DEFAULT_ARRAY_CNT))) {
LOG_WARN("failed to reserve unfinish table id array", KR(ret), "array_cnt", DEFAULT_ARRAY_CNT);
} else if (exist_uncompacted_table && OB_FAIL(generate_tablet_status_map())) {
LOG_WARN("failed to generate tablet status map", KR(ret));
} else {
tmp_time_guard.click(ObRSCompactionTimeGuard::GET_TABLET_META_TABLE);
}
ObSchemaGetterGuard schema_guard(ObSchemaMgrItem::MOD_RS_MAJOR_CHECK);
int64_t last_epoch_check_us = 0;
ObRSCompactionTimeGuard tmp_time_guard;
while (OB_SUCC(ret) && !table_ids_.loop_finish() && !stop_) { // split batch table_ids
tmp_time_guard.reuse();
tablet_status_map_.reuse();
tablet_ls_pair_array_.reuse();
iter.reset();
if (OB_FAIL(ObMajorFreezeUtil::check_epoch_periodically(
*sql_proxy_, tenant_id_, expected_epoch_, last_epoch_check_us))) {
LOG_WARN("fail to check freeze service epoch", KR(ret), K_(tenant_id), K_(expected_epoch), K_(compaction_scn));
@ -479,18 +532,6 @@ int ObMajorMergeProgressChecker::check_progress(
tenant_schema_version, OB_INVALID_VERSION,
ObMultiVersionSchemaService::RefreshSchemaMode::FORCE_LAZY))) {
LOG_WARN("fail to get schema guard", KR(ret), K_(tenant_id));
} else if (OB_FAIL(get_tablet_ls_pairs_by_tables(schema_guard, table_ids_, tablet_ls_pair_array_))) {
// table_id -> tablet_ids -> tablet_ls_pairs
LOG_WARN("failed to get tablet_ls_pairs by tables", KR(ret), K_(table_ids));
} else if (FALSE_IT(tmp_time_guard.click(ObRSCompactionTimeGuard::GET_TABLET_LS_PAIRS))) {
} else if (tablet_ls_pair_array_.empty()) {
// empty tablet_ls info, do nothing
} else if (OB_FAIL(iter.init(*sql_proxy_, tenant_id_, *server_trace_, tablet_ls_pair_array_))) {
LOG_WARN("failed to init iter", KR(ret), K(tablet_ls_pair_array_.count()));
} else if (OB_FAIL(generate_tablet_map_by_iter(iter))) {
LOG_WARN("failed to generate tablet status map", KR(ret));
} else {
tmp_time_guard.click(ObRSCompactionTimeGuard::GET_TABLET_META_TABLE);
}
#ifdef ERRSIM
if (OB_SUCC(ret)) {
@ -500,10 +541,7 @@ int ObMajorMergeProgressChecker::check_progress(
}
}
#endif
if (OB_FAIL(ret)) {
// for any failure, should reuse cur table id array
reuse_batch_table(unfinish_table_id_array, false/*reuse_rest_table*/);
} else if (OB_FAIL(check_verification(schema_guard, unfinish_table_id_array))) {
if (FAILEDx(check_verification(schema_guard, unfinish_table_id_array))) {
// check tablet_replica_checksum & table_index_checksum & cross_cluter_checksum
LOG_WARN("failed to check verification", KR(ret), K_(compaction_scn), K_(expected_epoch));
// only record OB_CHECKSUM_ERROR, and thus avoid confusing DBA
@ -514,7 +552,6 @@ int ObMajorMergeProgressChecker::check_progress(
"service_addr", GCONF.self_addr_);
}
}
tmp_time_guard.click(ObRSCompactionTimeGuard::CKM_VERIFICATION);
if (OB_FAIL(ret) && last_errno_ == ret) {
if (OB_TMP_FAIL(compaction::ADD_COMMON_SUSPECT_INFO(compaction::MAJOR_MERGE, ObDiagnoseTabletType::TYPE_RS_MAJOR_MERGE,
ObSuspectInfoType::SUSPECT_RS_SCHEDULE_ERROR,
@ -528,12 +565,10 @@ int ObMajorMergeProgressChecker::check_progress(
if (!can_not_ignore_warning(ret)) {
// do not ignore ret, therefore not continue to check next table_schema
ret = OB_SUCCESS;
} else {
reuse_batch_table(unfinish_table_id_array, true/*reuse_rest_table*/);
}
table_ids_.finish_cur_batch(); // finish cur batch
total_time_guard_.add_time_guard(tmp_time_guard);
} // end of while
tmp_time_guard.click(ObRSCompactionTimeGuard::CKM_VERIFICATION);
total_time_guard_.add_time_guard(tmp_time_guard);
// deal with finish_table_id_array after loop table_ids_
if (OB_TMP_FAIL(table_ids_.assign(unfinish_table_id_array))) {
LOG_WARN("failed to assign", KR(tmp_ret), K(unfinish_table_id_array));
@ -576,16 +611,18 @@ void ObMajorMergeProgressChecker::print_unfinish_info(const int64_t cost_us)
}
}
}
// table in table_ids_ may finish verified in deal_with_rest_data_table()
// need next loop to delete from array
ADD_RS_COMPACTION_EVENT(
compaction_scn_.get_val_for_tx(),
ObServerCompactionEvent::RS_FINISH_CUR_LOOP,
common::ObTimeUtility::fast_current_time(),
K(cost_us), K_(progress), "unfinish_table_id_count", table_ids_.count(),
"unfinish_table_ids", tmp_table_id_array,
K(cost_us), K_(progress), "remain_table_id_count", table_ids_.count(),
"remain_table_ids", tmp_table_id_array,
K_(total_time_guard), K_(validator_statistics));
LOG_INFO("succ to check merge progress", K_(tenant_id), K_(loop_cnt), K_(compaction_scn), K(cost_us),
K_(progress), "unfinish_table_id_count", table_ids_.count(),
"unfinish_table_ids", tmp_table_id_array,
K_(progress), "remain_table_id_count", table_ids_.count(),
"remain_table_ids", tmp_table_id_array,
"uncompacted_tablets", tmp_replica_array,
K_(total_time_guard), K_(validator_statistics));
}
@ -621,10 +658,7 @@ int ObMajorMergeProgressChecker::deal_with_rest_data_table()
if (OB_FAIL(table_compaction_map_.get_refactored(table_id, table_compaction_info))) {
LOG_WARN("failed to get table compaction info", KR(ret), K_(tenant_id), K(table_id));
} else if (table_compaction_info.is_compacted()) {
if (OB_TMP_FAIL(update_table_compaction_info(table_id,
[](ObTableCompactionInfo &table_compaction_info) {
table_compaction_info.set_index_ckm_verified();
}))) {
if (OB_TMP_FAIL(set_table_compaction_info_status(table_id, ObTableCompactionInfo::INDEX_CKM_VERIFIED))) {
LOG_WARN("failed to update table compaction info", KR(tmp_ret), K(idx), K(table_id));
} else {
LOG_TRACE("deal with data table", KR(tmp_ret), K(idx), K(table_id));
@ -636,41 +670,86 @@ int ObMajorMergeProgressChecker::deal_with_rest_data_table()
return ret;
}
int ObMajorMergeProgressChecker::set_table_compaction_info_status(
const uint64_t table_id,
const ObTableCompactionInfo::Status status)
{
int ret = OB_SUCCESS;
ObTableCompactionInfo table_compaction_info;
if (OB_FAIL(table_compaction_map_.get_refactored(table_id, table_compaction_info))) {
if (OB_HASH_NOT_EXIST == ret) { // first initialization
ret = OB_SUCCESS;
table_compaction_info.reset();
table_compaction_info.table_id_ = table_id;
} else {
LOG_WARN("fail to get val from hashmap", KR(ret), K(table_id));
}
}
if (OB_FAIL(ret)) {
} else if (FALSE_IT(table_compaction_info.set_status(status))) {
} 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));
}
return ret;
}
int ObMajorMergeProgressChecker::validate_index_ckm()
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
if (idx_ckm_validate_array_.count() < 50
&& progress_.get_wait_index_ckm_table_cnt() > 100
&& !is_extra_check_round()) {
// do nothing
} else if (idx_ckm_validate_array_.count() > 0) {
ObSEArray<uint64_t, DEFAULT_ARRAY_CNT> finish_validate_table_ids;
if (OB_FAIL(loop_index_ckm_validate_array(finish_validate_table_ids))) {
if (OB_FAIL(loop_index_ckm_validate_array())) {
LOG_WARN("failed to loop index ckm validate array", KR(ret), K_(tenant_id));
}
for (int64_t idx = 0; idx < finish_validate_table_ids.count(); ++idx) {
if (OB_TMP_FAIL(update_table_compaction_info(finish_validate_table_ids.at(idx),
[](ObTableCompactionInfo &table_compaction_info) {
table_compaction_info.set_index_ckm_verified();
}))) {
LOG_WARN("failed to update table compaction info", KR(tmp_ret), K(idx),
"table_id", finish_validate_table_ids.at(idx));
}
} // end of for
}
idx_ckm_validate_array_.reset();
}
return ret;
}
int ObMajorMergeProgressChecker::loop_index_ckm_validate_array(
ObIArray<uint64_t> &finish_validate_table_ids)
int ObMajorMergeProgressChecker::deal_with_validated_table(
const uint64_t table_id,
const int64_t finish_index_cnt,
const ObTableCkmItems &table_ckm)
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
bool validate_finish = (finish_index_cnt == 0);
if (OB_UNLIKELY(OB_INVALID_ID == table_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(table_id));
} else if (!validate_finish
&& OB_TMP_FAIL(update_finish_index_cnt_for_data_table(
table_id, finish_index_cnt, validate_finish))) {
// set table_compaction_info::unfinish_index_cnt
LOG_WARN("failed to update finish index", KR(tmp_ret), K(table_id), K(finish_index_cnt));
}
if (OB_SUCC(ret) && validate_finish) {
if (table_ckm.is_inited() ) {
if (OB_TMP_FAIL(ckm_validator_.push_finish_tablet_ls_pairs_with_update(
table_ckm.get_tablet_ls_pairs()))) {
LOG_WARN("failed to push back tablet_ls_pair", KR(tmp_ret));
}
if (OB_TMP_FAIL(ckm_validator_.push_tablet_ckm_items_with_update(
table_ckm.get_ckm_items()))) {
LOG_WARN("failed to push checksum items", KR(tmp_ret), K(table_id));
}
}
if (OB_TMP_FAIL(set_table_compaction_info_status(table_id, ObTableCompactionInfo::VERIFIED))) {
LOG_WARN("failed to update table compaction info", KR(tmp_ret), K(table_id));
}
}
return ret;
}
int ObMajorMergeProgressChecker::loop_index_ckm_validate_array()
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
uint64_t finish_index_cnt = 0;
uint64_t prev_data_table_id = OB_INVALID_ID;
const ObTableSchema *table_schema = nullptr;
ObSchemaGetterGuard schema_guard(ObSchemaMgrItem::MOD_RS_MAJOR_CHECK);
ObTableCkmItems data_table_ckm(tenant_id_);
if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id_, schema_guard))) {
@ -680,96 +759,84 @@ int ObMajorMergeProgressChecker::loop_index_ckm_validate_array(
const uint64_t data_table_id = idx_ckm_validate_array_.at(idx).data_table_id_;
const uint64_t index_table_id = idx_ckm_validate_array_.at(idx).index_table_id_;
if (prev_data_table_id != data_table_id) { // not same table
// set table_compaction_info::unfinish_index_cnt
if (OB_INVALID_ID != prev_data_table_id
&& OB_TMP_FAIL(update_finish_index_cnt_for_data_table(prev_data_table_id, finish_index_cnt))) {
LOG_WARN("failed to push table_id", KR(tmp_ret), K(prev_data_table_id), K(finish_index_cnt));
if (OB_TMP_FAIL(finish_validate_table_ids.push_back(prev_data_table_id))) {
LOG_WARN("failed to push table_id", KR(tmp_ret), K(prev_data_table_id));
}
if (OB_INVALID_ID != prev_data_table_id) {
(void) deal_with_validated_table(prev_data_table_id, finish_index_cnt, data_table_ckm);
}
finish_index_cnt = 0;
data_table_ckm.clear();
prev_data_table_id = data_table_id;
if (OB_FAIL(data_table_ckm.build(data_table_id, compaction_scn_,
*sql_proxy_, schema_guard, table_schema))) {
*sql_proxy_, schema_guard, get_tablet_ls_pair_cache()))) {
LOG_WARN("fail to prepare schema checksum items", KR(ret), K_(tenant_id), K(data_table_id));
} else {
++validator_statistics_.query_ckm_sql_cnt_;
LOG_TRACE("success to get data table ckm", KR(ret), K(data_table_id), K(data_table_ckm));
}
}
if (FAILEDx(get_idx_ckm_and_validate(index_table_id, schema_guard, data_table_ckm))) {
if (OB_ITEM_NOT_MATCH == ret || OB_TABLE_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
++validator_statistics_.use_cached_ckm_cnt_;
LOG_WARN("failed to get index ckm and validate", KR(ret), K(data_table_id), K(index_table_id));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(get_idx_ckm_and_validate(*table_schema, index_table_id, schema_guard, data_table_ckm))) {
LOG_WARN("failed to get index ckm and validate", KR(ret), K(data_table_id), K(index_table_id));
} else {
++finish_index_cnt;
}
}
if (OB_SUCC(ret) || OB_TABLE_NOT_EXIST == ret) {
ret = OB_SUCCESS; // clear errno
if (OB_TMP_FAIL(finish_validate_table_ids.push_back(index_table_id))) {
LOG_WARN("failed to push table_id", KR(tmp_ret), K(index_table_id));
}
++validator_statistics_.checker_validate_idx_cnt_;
}
} // end of for
if (OB_SUCC(ret) && finish_index_cnt > 0 && OB_INVALID_ID != prev_data_table_id) {
// deal with last validate data table id
if (OB_TMP_FAIL(update_finish_index_cnt_for_data_table(prev_data_table_id, finish_index_cnt))) {
LOG_WARN("failed to push table_id", KR(tmp_ret), K(prev_data_table_id), K(finish_index_cnt));
if (OB_TMP_FAIL(finish_validate_table_ids.push_back(prev_data_table_id))) {
LOG_WARN("failed to push table_id", KR(tmp_ret), K(prev_data_table_id));
}
}
(void) deal_with_validated_table(prev_data_table_id, finish_index_cnt, data_table_ckm);
}
return ret;
}
int ObMajorMergeProgressChecker::get_idx_ckm_and_validate(
const ObTableSchema &table_schema,
const uint64_t index_table_id,
ObSchemaGetterGuard &schema_guard,
ObTableCkmItems &data_table_ckm)
{
int ret = OB_SUCCESS;
const ObTableSchema *index_table_schema = nullptr;
ObTableCkmItems index_table_ckm(tenant_id_);
if (OB_FAIL(index_table_ckm.build(index_table_id, compaction_scn_,
*sql_proxy_, schema_guard,
index_table_schema))) {
get_tablet_ls_pair_cache()))) {
LOG_WARN("failed to get checksum items", KR(ret), K(index_table_id), K_(compaction_scn));
} else if (OB_UNLIKELY(index_table_schema->should_not_validate_data_index_ckm())) {
} else if (OB_UNLIKELY(index_table_ckm.get_table_schema()->should_not_validate_data_index_ckm())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("should not validate spatial index and data table", KR(ret), K(index_table_id), KPC(index_table_schema));
LOG_WARN("should not validate spatial index and data table", KR(ret), K(index_table_id), K(index_table_ckm));
} else {
const bool is_global_index = index_table_schema->is_global_index_table();
const bool is_global_index = index_table_ckm.get_table_schema()->is_global_index_table();
if (OB_FAIL(ObTableCkmItems::validate_ckm_func[is_global_index](
compaction_scn_,
table_schema,
*index_table_schema,
*sql_proxy_,
data_table_ckm,
index_table_ckm))) {
LOG_WARN("failed to validate checksum", KR(ret), "data_table_id", table_schema.get_table_id(),
K(index_table_id), K(data_table_ckm), K(index_table_ckm), K(table_schema), KPC(index_table_schema));
LOG_WARN("failed to validate checksum", KR(ret), "data_table_id", data_table_ckm.get_table_id(),
K(index_table_id), K(data_table_ckm), K(index_table_ckm));
if (OB_ITEM_NOT_MATCH == ret) {
ret = OB_SUCCESS;
}
}
}
if (OB_SUCC(ret) || OB_TABLE_NOT_EXIST == ret) {
(void) deal_with_validated_table(index_table_id, 0 /*finish_index_cnt*/, index_table_ckm);
}
return ret;
}
int ObMajorMergeProgressChecker::update_finish_index_cnt_for_data_table(
const uint64_t data_table_id,
const uint64_t finish_index_cnt)
const uint64_t finish_index_cnt,
bool &idx_validate_finish)
{
int ret = OB_SUCCESS;
if (OB_FAIL(update_table_compaction_info(data_table_id,
[finish_index_cnt](ObTableCompactionInfo &table_compaction_info) {
idx_validate_finish = false;
ObTableCompactionInfo table_compaction_info;
if (OB_FAIL(table_compaction_map_.get_refactored(data_table_id, table_compaction_info))) {
LOG_WARN("failed to get table compaction info", KR(ret), K_(tenant_id), K(data_table_id));
} else {
if (table_compaction_info.unfinish_index_cnt_ < finish_index_cnt) {
LOG_WARN_RET(OB_ERR_UNEXPECTED, "finish index cnt is unexpected", K(table_compaction_info), K(finish_index_cnt));
table_compaction_info.unfinish_index_cnt_ = 0;
@ -778,22 +845,29 @@ int ObMajorMergeProgressChecker::update_finish_index_cnt_for_data_table(
}
if (0 == table_compaction_info.unfinish_index_cnt_) {
table_compaction_info.set_index_ckm_verified();
idx_validate_finish = true;
}
LOG_TRACE("success to update finish index cnt", K(finish_index_cnt), K(table_compaction_info));
} ))) {
LOG_WARN("failed to update table compaction info", KR(ret), K_(tenant_id), K(data_table_id));
}
if (FAILEDx(table_compaction_map_.set_refactored(data_table_id, table_compaction_info, true /*overwrite*/))) {
LOG_WARN("fail to set refactored", KR(ret), K(data_table_id), K(table_compaction_info));
} else {
(void) progress_.update_table_cnt(table_compaction_info.status_);
}
LOG_TRACE("success to update finish index cnt", K(finish_index_cnt), K(table_compaction_info), K(idx_validate_finish));
return ret;
}
int ObMajorMergeProgressChecker::generate_tablet_map_by_iter(
ObCompactionTabletMetaIterator &iter)
int ObMajorMergeProgressChecker::generate_tablet_status_map()
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
ObTabletInfo tablet_info;
int64_t idx = 0;
bool filter = false;
ObCompactionTabletMetaIterator iter(!is_extra_check_round(), compaction_scn_.get_val_for_tx());
if (OB_FAIL(iter.init(tenant_id_, batch_size_mgr_.get_inner_table_scan_batch_size(), *server_trace_))) {
LOG_WARN("failed to init iter", KR(ret));
}
while (OB_SUCC(ret) && !stop_) {
if (OB_FAIL(iter.next(tablet_info))) {
if (OB_ITER_END != ret) {
@ -853,16 +927,6 @@ int ObMajorMergeProgressChecker::generate_tablet_map_by_iter(
ret = OB_CHECKSUM_ERROR;
LOG_ERROR("ERROR! ERROR! ERROR! find error status tablet replica", KR(ret), K(tablet_info));
} else if (replica_snapshot_scn < compaction_scn_) {
#ifdef ERRSIM
ret = OB_E(EventTable::EN_SKIP_INDEX_MAJOR) ret;
if (OB_FAIL(ret)
&& replica->get_tablet_id().id() > ObTabletID::MIN_USER_TABLET_ID) {
ret = OB_SUCCESS;
LOG_INFO("ERRSIM EN_SKIP_INDEX_MAJOR", K(ret), KPC(replica));
status = ObTabletCompactionStatus::CAN_SKIP_VERIFYING;
break;
}
#endif
status = ObTabletCompactionStatus::INITIAL;
if (progress_.unmerged_tablet_cnt_++ < DEBUG_INFO_CNT) { // add into uncompacted tablets array to show in diagnose
SpinWLockGuard w_guard(diagnose_rw_lock_);
@ -882,7 +946,7 @@ int ObMajorMergeProgressChecker::generate_tablet_map_by_iter(
} // end of FOREACH
if (OB_SUCC(ret) && ObTabletCompactionStatus::INITIAL != status) {
++progress_.merged_tablet_cnt_;
if (OB_FAIL(tablet_status_map_.set_refactored(tablet_info.get_tablet_id(), status))) {
if (OB_FAIL(tablet_status_map_.set_refactored(tablet_info.get_tablet_id(), status, 1/*overwrite*/))) {
LOG_WARN("failed to push back status", KR(ret), K(tablet_info), K(status));
} else {
LOG_TRACE("success to add tablet status", KR(ret), K(tablet_info), K(status));
@ -893,125 +957,5 @@ int ObMajorMergeProgressChecker::generate_tablet_map_by_iter(
return ret;
}
int ObMajorMergeProgressChecker::get_tablet_ls_pairs_by_tables(
ObSchemaGetterGuard &schema_guard,
ObUnfinishTableIds &table_ids,
ObArray<ObTabletLSPair> &tablet_ls_pair_array)
{
int ret = OB_SUCCESS;
int tmp_ret = OB_SUCCESS;
if (OB_UNLIKELY(table_ids_.loop_finish())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(table_ids_));
} else {
const ObSimpleTableSchemaV2 *simple_schema = nullptr;
uint64_t table_id = 0;
uint64_t table_cnt = 0;
SMART_VARS_2((ObArray<ObTabletLSPair>, tmp_tablet_ls_pairs),
(ObArray<ObTabletID>, tmp_tablet_ids)) {
if (OB_FAIL(tmp_tablet_ls_pairs.reserve(DEFAULT_ARRAY_CNT))) {
LOG_WARN("failed to reserve array", K(ret), "array_cnt", DEFAULT_ARRAY_CNT);
} else if (OB_FAIL(tmp_tablet_ids.reserve(DEFAULT_ARRAY_CNT))) {
LOG_WARN("failed to reserve array", K(ret), "array_cnt", DEFAULT_ARRAY_CNT);
}
int64_t idx = table_ids_.batch_start_idx_;
int64_t idx_cnt = 0;
ObTableCompactionInfo table_compaction_info;
int64_t tablet_batch_size = 0;
int64_t table_batch_size = 0;
(void) get_check_batch_size(tablet_batch_size, table_batch_size);
for ( ; OB_SUCC(ret) && idx < table_ids_.count(); ++idx) {
tmp_tablet_ls_pairs.reuse();
tmp_tablet_ids.reuse();
table_id = table_ids_.at(idx);
LOG_TRACE("loop table id", KR(ret), K_(tenant_id), K(table_id));
if (OB_FAIL(schema_guard.get_simple_table_schema(tenant_id_, table_id, simple_schema))) {
LOG_WARN("fail to get table schema", KR(ret), K_(tenant_id), K(table_id));
} else if (should_ignore_cur_table(simple_schema)) {
// skip table
if (OB_TMP_FAIL(update_table_compaction_info(table_id,
[](ObTableCompactionInfo &table_compaction_info) {
table_compaction_info.set_can_skip_verifying();
}, false/*need_update_progress*/))) {
LOG_WARN("failed to update table compaction info", KR(tmp_ret), K(idx), K(table_id));
}
continue;
} else if (!simple_schema->is_index_table()) {
if (OB_FAIL(table_compaction_map_.get_refactored(table_id, table_compaction_info))) {
LOG_WARN("failed to get refactor", KR(ret), K(table_id));
} else {
idx_cnt = table_compaction_info.unfinish_index_cnt_;
}
} else {
--idx_cnt;
}
if (FAILEDx(simple_schema->get_tablet_ids(tmp_tablet_ids))) {
LOG_WARN("fail to get tablet_ids from table schema", KR(ret), K(simple_schema));
} else if (OB_UNLIKELY(tmp_tablet_ids.empty())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get tablet_ids of current table schema", KR(ret), K_(tenant_id), K(simple_schema));
} else if (OB_FAIL(ObTabletReplicaChecksumOperator::get_tablet_ls_pairs(
tenant_id_, table_id, *sql_proxy_, tmp_tablet_ids, tmp_tablet_ls_pairs))) {
if (OB_ITEM_NOT_MATCH == ret) {
LOG_WARN("OB_ITEM_NOT_MATCH", KR(ret), K(idx), K(table_id));
ret = OB_SUCCESS; // clear errno
if (OB_TMP_FAIL(update_table_compaction_info(table_id,
[](ObTableCompactionInfo &table_compaction_info) {
table_compaction_info.set_can_skip_verifying();
}, false/*need_update_progress*/))) {
LOG_WARN("failed to update table compaction info", KR(tmp_ret), K(idx), K(table_id));
}
} else {
LOG_WARN("fail to get tablet_ls pairs", KR(ret), K_(tenant_id), K(table_id));
}
} else if (OB_FAIL(tablet_ls_pair_array.push_back(tmp_tablet_ls_pairs))) {
LOG_WARN("failed to push_back tablet_to_ls pair", K(ret), K(tmp_tablet_ls_pairs));
}
if (OB_SUCC(ret) && 0 >= idx_cnt // data & index should be in same batch
&& (tablet_ls_pair_array.count() >= tablet_batch_size
|| ++table_cnt >= table_batch_size)) {
++idx;
break;
}
} // end of for
if (OB_SUCC(ret)) { // [batch_start_idx, batch_end_idx)
table_ids_.batch_end_idx_ = idx;
if (REACH_TENANT_TIME_INTERVAL(PRINT_LOG_INTERVAL)) {
LOG_INFO("success to set batch end idx", KR(ret), K(table_ids_.batch_start_idx_),
K(table_ids_.batch_end_idx_), K(tablet_ls_pair_array.count()));
}
}
} // SMART_VARS
}
return ret;
}
int ObMajorMergeProgressChecker::update_table_compaction_info(
const uint64_t table_id,
const ObFunction<void(ObTableCompactionInfo&)> &info_op,
const bool need_update_progress/*true*/)
{
int ret = OB_SUCCESS;
ObTableCompactionInfo table_compaction_info;
if (OB_FAIL(table_compaction_map_.get_refactored(table_id, table_compaction_info))) {
if (OB_HASH_NOT_EXIST != ret) {
LOG_WARN("failed to get table compaction info", KR(ret), K_(tenant_id), K(table_id));
} else {
ret = OB_SUCCESS;
table_compaction_info.reset();
table_compaction_info.table_id_ = table_id;
}
}
if (OB_FAIL(ret)) {
} else if (FALSE_IT(info_op(table_compaction_info))) { // execute operation on ObTableCompactionInfo
} 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 if (need_update_progress) {
(void) progress_.update_table_cnt(table_compaction_info.status_);
}
return ret;
}
} // namespace rootserver
} // namespace oceanbase

View File

@ -21,6 +21,7 @@
#include "common/ob_tablet_id.h"
#include "rootserver/freeze/ob_major_freeze_util.h"
#include "rootserver/freeze/ob_major_merge_progress_util.h"
#include "share/compaction/ob_schedule_batch_size_mgr.h"
namespace oceanbase
{
@ -67,47 +68,44 @@ public:
int get_uncompacted_tablets(common::ObArray<share::ObTabletReplica> &uncompacted_tablets) const;
void reset_uncompacted_tablets();
int check_progress(compaction::ObMergeProgress &progress);
const compaction::ObTabletLSPairCache &get_tablet_ls_pair_cache() const { return tablet_ls_pair_cache_; }
private:
int update_table_compaction_info(
const uint64_t table_id,
const common::ObFunction<void(compaction::ObTableCompactionInfo&)> &info_op,
const bool need_update_progress = true);
int set_table_compaction_info_status(const uint64_t table_id, const compaction::ObTableCompactionInfo::Status status);
void reuse_batch_table(ObIArray<uint64_t> &unfinish_table_id_array, const bool reuse_rest_table);
void reuse_rest_table(const int64_t start_idx, ObIArray<uint64_t> &unfinish_table_id_array);
bool can_not_ignore_warning(int ret)
{
return OB_FREEZE_SERVICE_EPOCH_MISMATCH == ret || OB_CHECKSUM_ERROR == ret;
}
void get_check_batch_size(int64_t &tablet_id_batch_size, int64_t &table_id_batch_size) const;
const static int64_t TABLET_ID_BATCH_CHECK_SIZE = 3000;
const static int64_t TABLET_ID_BATCH_CHECK_SIZE = 10000;
const static int64_t TABLE_ID_BATCH_CHECK_SIZE = 200;
const static int64_t TOTAL_TABLE_CNT_THREASHOLD = 100 * 1000; // 10w
const static int64_t TABLE_MAP_BUCKET_CNT = 10000;
const static int64_t DEFAULT_ARRAY_CNT = 200;
int get_tablet_ls_pairs_by_tables(
ObSchemaGetterGuard &schema_guard,
compaction::ObUnfinishTableIds &table_ids,
ObArray<share::ObTabletLSPair> &tablet_ls_pair_array);
int generate_tablet_map_by_iter(
share::ObCompactionTabletMetaIterator &iter);
int generate_tablet_status_map();
int check_verification(
ObSchemaGetterGuard &schema_guard,
ObIArray<uint64_t> &unfinish_table_id_array);
int prepare_unfinish_table_ids();
int check_schema_version();
int prepare_check_progress();
int prepare_check_progress(
compaction::ObRSCompactionTimeGuard &tmp_time_guard,
bool &exist_uncompacted_table);
int check_index_and_rest_table();
int validate_index_ckm();
int get_idx_ckm_and_validate(
const ObTableSchema &table_schema,
const uint64_t index_table_id,
ObSchemaGetterGuard &schema_guard,
compaction::ObTableCkmItems &data_table_ckm);
int loop_index_ckm_validate_array(
ObIArray<uint64_t> &finish_validate_table_ids);
int loop_index_ckm_validate_array();
int update_finish_index_cnt_for_data_table(
const uint64_t data_table_id,
const uint64_t finish_index_cnt);
const uint64_t finish_index_cnt,
bool &idx_validate_finish);
int deal_with_validated_table(
const uint64_t data_table_id,
const int64_t finish_index_cnt,
const compaction::ObTableCkmItems &data_table_ckm);
int deal_with_validated_data_table(const uint64_t data_table_id);
bool should_ignore_cur_table(const ObSimpleTableSchemaV2 *simple_schema);
int deal_with_rest_data_table();
bool is_extra_check_round() const { return 0 == (loop_cnt_ % 8); } // check every 8 rounds
@ -117,6 +115,7 @@ private:
const uint64_t table_id,
bool &is_table_valid,
ObIArray<const ObSimpleTableSchemaV2 *> &index_schemas);
int rebuild_map_by_tablet_cnt();
private:
static const int64_t ADD_RS_EVENT_INTERVAL = 10L * 60 * 1000 * 1000; // 10m
static const int64_t PRINT_LOG_INTERVAL = 2 * 60 * 1000 * 1000; // 2m
@ -141,7 +140,6 @@ private:
compaction::ObUnfinishTableIds table_ids_; // record unfinish table_id
// record tablet whose status is COMPACTED/CAN_SKIP_VERIFYING
compaction::ObTabletStatusMap tablet_status_map_;
compaction::ObTabletLSPairArray tablet_ls_pair_array_;
// record each table compaction/verify status
compaction::ObTableCompactionInfoMap table_compaction_map_; // <table_id, compaction_info>
ObChecksumValidator ckm_validator_;
@ -152,6 +150,10 @@ private:
// statistics section
compaction::ObRSCompactionTimeGuard total_time_guard_;
compaction::ObCkmValidatorStatistics validator_statistics_;
compaction::ObTabletLSPairCache tablet_ls_pair_cache_;
compaction::ObScheduleBatchSizeMgr batch_size_mgr_;
ObArray<share::ObTabletLSPair> finish_tablet_ls_pair_array_;
ObArray<share::ObTabletChecksumItem> finish_tablet_ckm_array_;
DISALLOW_COPY_AND_ASSIGN(ObMajorMergeProgressChecker);
};

View File

@ -7,10 +7,18 @@
// EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
// MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
// See the Mulan PubL v2 for more details.
#define USING_LOG_PREFIX RS_COMPACTION
#include "rootserver/freeze/ob_major_merge_progress_util.h"
#include "share/tablet/ob_tablet_info.h"
#include "share/tablet/ob_tablet_to_ls_operator.h"
#include "observer/ob_server_struct.h"
#include "share/transfer/ob_transfer_task_operator.h"
#include "share/compaction/ob_schedule_batch_size_mgr.h"
namespace oceanbase
{
using namespace share;
using namespace common;
namespace compaction
{
@ -104,5 +112,188 @@ int64_t ObRSCompactionTimeGuard::to_string(char *buf, const int64_t buf_len) con
return pos;
}
/**
* -------------------------------------------------------------------ObTabletLSPairCache-------------------------------------------------------------------
*/
const int64_t ObTabletLSPairCache::TABLET_LS_MAP_BUCKET_CNT;
const int64_t ObTabletLSPairCache::RANGE_SIZE;
ObTabletLSPairCache::ObTabletLSPairCache()
: tenant_id_(0),
last_refresh_ts_(0),
max_task_id_()
{
}
ObTabletLSPairCache::~ObTabletLSPairCache()
{
destroy();
}
void ObTabletLSPairCache::reuse()
{
last_refresh_ts_ = 0;
max_task_id_.reset();
map_.reuse();
}
void ObTabletLSPairCache::destroy()
{
tenant_id_ = 0;
last_refresh_ts_ = 0;
max_task_id_.reset();
if (map_.created()) {
map_.destroy();
}
}
int ObTabletLSPairCache::refresh()
{
int ret = OB_SUCCESS;
ObTabletID start_tablet_id;
int64_t cost_ts = ObTimeUtility::fast_current_time();
if (OB_ISNULL(GCTX.sql_proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql proxy is unexpected null", KR(ret));
} else {
map_.reuse();
}
SMART_VAR(ObArray<ObTabletLSPair>, tablet_ls_pair_array) {
tablet_ls_pair_array.set_attr(ObMemAttr(tenant_id_, "RSCompPairCache"));
if (OB_FAIL(tablet_ls_pair_array.reserve(RANGE_SIZE))) {
LOG_WARN("failed to reserve array", KR(ret));
}
while (OB_SUCC(ret)) {
tablet_ls_pair_array.reuse();
if (OB_FAIL(ObTabletToLSTableOperator::range_get_tablet(
*GCTX.sql_proxy_,
tenant_id_,
start_tablet_id,
RANGE_SIZE,
tablet_ls_pair_array))) {
LOG_WARN("fail to get a range of tablet through tablet_to_ls_table_operator",
KR(ret), K_(tenant_id), K(start_tablet_id), K(RANGE_SIZE),
K(tablet_ls_pair_array));
} else if (tablet_ls_pair_array.empty()) {
break;
} else {
for (int64_t idx = 0; OB_SUCC(ret) && idx < tablet_ls_pair_array.count(); ++idx) {
ObTabletLSPair &pair = tablet_ls_pair_array.at(idx);
if (OB_FAIL(map_.set_refactored(pair.get_tablet_id(), pair.get_ls_id()))) {
LOG_WARN("fail set tablet ls map", KR(ret));
}
} // end of for
if (OB_SUCC(ret)) {
start_tablet_id = tablet_ls_pair_array.at(tablet_ls_pair_array.count() - 1).get_tablet_id();
}
}
} // end of while
}
if (OB_SUCC(ret)) {
last_refresh_ts_ = ObTimeUtility::fast_current_time();
cost_ts = last_refresh_ts_ - cost_ts;
LOG_INFO("success to refresh tablet ls pair cache", KR(ret), K(cost_ts), "map_item_cnt", map_.size(), K(map_.bucket_count()));
}
return ret;
}
int ObTabletLSPairCache::rebuild_map_by_tablet_cnt()
{
int ret = OB_SUCCESS;
int64_t tablet_cnt = 0;
if (map_.empty()) {
if (OB_FAIL(ObTabletToLSTableOperator::get_tablet_ls_pairs_cnt(*GCTX.sql_proxy_, tenant_id_, tablet_cnt))) {
LOG_WARN("failed to get tablet_ls pair cnt", KR(ret));
}
} else {
tablet_cnt = map_.size();
}
if (OB_FAIL(ret) || tablet_cnt == 0) {
} else {
int64_t recommend_map_bucked_cnt = 0;
const int64_t cur_bucket_cnt = map_.created() ? map_.bucket_count() : 0;
bool rebuild_map_flag = ObScheduleBatchSizeMgr::need_rebuild_map(
TABLET_LS_MAP_BUCKET_CNT, tablet_cnt, cur_bucket_cnt, recommend_map_bucked_cnt);
if (rebuild_map_flag) {
if (map_.created()) {
map_.destroy();
}
if (OB_FAIL(map_.create(recommend_map_bucked_cnt, "RSCompPairCache", "RSCompPairCache", tenant_id_))) {
LOG_WARN("fail to create tablet ls pair map", KR(ret), K_(tenant_id), K(recommend_map_bucked_cnt));
} else {
LOG_INFO("success to rebuild or create map", KR(ret), K(tablet_cnt), K(map_.bucket_count()));
}
}
}
return ret;
}
int ObTabletLSPairCache::check_exist_new_transfer_task(
bool &exist,
share::ObTransferTaskID &tmp_max_task_id)
{
int ret = OB_SUCCESS;
exist = false;
if (OB_FAIL(ObTransferTaskOperator::get_max_task_id_from_history(
*GCTX.sql_proxy_,
tenant_id_,
tmp_max_task_id))) {
LOG_WARN("get max transfer task id from history failed", KR(ret), K_(tenant_id),
K(tmp_max_task_id));
} else if (tmp_max_task_id.is_valid() && tmp_max_task_id > max_task_id_) {
exist = true;
}
return ret;
}
int ObTabletLSPairCache::try_refresh(const bool force_refresh/* = false*/)
{
int ret = OB_SUCCESS;
bool exist = false;
share::ObTransferTaskID tmp_max_task_id;
if (OB_FAIL(check_exist_new_transfer_task(exist, tmp_max_task_id))) {
LOG_WARN("failed to check transfer task", KR(ret));
} else if (force_refresh && OB_FAIL(rebuild_map_by_tablet_cnt())) {
LOG_WARN("failed to rebuild map by tablet cnt", KR(ret), K(force_refresh));
} else if (force_refresh
|| (exist && (ObTimeUtility::fast_current_time() - last_refresh_ts_ >= REFRESH_CACHE_TIME_INTERVAL))) {
if (OB_FAIL(refresh())) {
LOG_WARN("failed to refresh", KR(ret));
} else {
max_task_id_ = tmp_max_task_id;
}
}
return ret;
}
int ObTabletLSPairCache::get_tablet_ls_pairs(
const uint64_t table_id,
const ObIArray<ObTabletID> &tablet_ids,
ObIArray<share::ObTabletLSPair> &pairs) const
{
int ret = OB_SUCCESS;
if (is_sys_tenant(tenant_id_) || is_sys_table(table_id)) {
ObLSID tmp_ls_id(ObLSID::SYS_LS_ID);
for (int64_t idx = 0; OB_SUCC(ret) && idx < tablet_ids.count(); ++idx) {
if (OB_FAIL(pairs.push_back(ObTabletLSPair(tablet_ids.at(idx), tmp_ls_id)))) {
LOG_WARN("fail to push back pair", KR(ret), K(table_id));
}
} // end of for
} else {
ObLSID tmp_ls_id;
for (int64_t idx = 0; OB_SUCC(ret) && idx < tablet_ids.count(); ++idx) {
if (OB_FAIL(map_.get_refactored(tablet_ids.at(idx), tmp_ls_id))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_ITEM_NOT_MATCH;
} else {
LOG_WARN("failed to get ls id", KR(ret), K(idx), K(tablet_ids));
}
} else if (OB_FAIL(pairs.push_back(ObTabletLSPair(tablet_ids.at(idx), tmp_ls_id)))) {
LOG_WARN("fail to push back pair", KR(ret), K(table_id));
}
} // end of for
}
return ret;
}
} // namespace compaction
} // namespace oceanbase

View File

@ -12,6 +12,7 @@
#include "share/compaction/ob_compaction_time_guard.h"
#include "share/ob_delegate.h"
#include "share/ob_ls_id.h"
#include "share/ob_balance_define.h"
namespace oceanbase
{
namespace compaction
@ -58,6 +59,7 @@ public:
}
ObTableCompactionInfo &operator=(const ObTableCompactionInfo &other);
void set_status(const Status status) { status_ = status;}
void set_uncompacted() { status_ = Status::INITIAL; }
void set_compacted() { status_ = Status::COMPACTED; }
bool is_compacted() const { return Status::COMPACTED == status_; }
@ -106,7 +108,10 @@ public:
return total_table_cnt_ > 0
&& (total_table_cnt_ == get_finish_verified_table_cnt());
}
bool exist_uncompacted_table() const
{
return table_cnt_[ObTableCompactionInfo::INITIAL] > 0;
}
bool is_merge_abnomal() const
{
return total_table_cnt_ > 0
@ -134,6 +139,7 @@ public:
{
// clear info that will change in cur loop
unmerged_tablet_cnt_ = 0;
merged_tablet_cnt_ = 0;
table_cnt_[ObTableCompactionInfo::INITIAL] = 0;
table_cnt_[ObTableCompactionInfo::COMPACTED] = 0;
table_cnt_[ObTableCompactionInfo::INDEX_CKM_VERIFIED] = 0;
@ -156,7 +162,6 @@ struct ObUnfinishTableIds
{
ObUnfinishTableIds()
: batch_start_idx_(0),
batch_end_idx_(0),
array_()
{
array_.set_label("RSCompTableIds");
@ -165,7 +170,6 @@ struct ObUnfinishTableIds
void reset()
{
batch_start_idx_ = 0;
batch_end_idx_ = 0;
array_.reset();
}
CONST_DELEGATE_WITH_RET(array_, empty, bool);
@ -179,20 +183,14 @@ struct ObUnfinishTableIds
}
bool loop_finish() const
{
return batch_end_idx_ >= array_.count();
}
void finish_cur_batch()
{
batch_start_idx_ = batch_end_idx_;
return batch_start_idx_ >= array_.count();
}
void start_looping()
{
batch_start_idx_ = 0;
batch_end_idx_ = 0;
}
TO_STRING_KV(K_(batch_start_idx), K_(batch_end_idx), "count", array_.count());
TO_STRING_KV(K_(batch_start_idx), "count", array_.count());
int64_t batch_start_idx_;
int64_t batch_end_idx_;
// record the table_ids in the schema_guard obtained in check_merge_progress
common::ObArray<uint64_t> array_;
};
@ -231,14 +229,44 @@ struct ObCkmValidatorStatistics
use_cached_ckm_cnt_ = 0;
write_ckm_sql_cnt_ = 0;
update_report_scn_sql_cnt_ = 0;
checker_validate_idx_cnt_ = 0;
}
TO_STRING_KV(K_(query_ckm_sql_cnt), K_(use_cached_ckm_cnt), K_(write_ckm_sql_cnt), K_(update_report_scn_sql_cnt));
TO_STRING_KV(K_(query_ckm_sql_cnt), K_(use_cached_ckm_cnt), K_(write_ckm_sql_cnt), K_(update_report_scn_sql_cnt), K_(checker_validate_idx_cnt));
int64_t query_ckm_sql_cnt_;
int64_t use_cached_ckm_cnt_;
int64_t write_ckm_sql_cnt_;
int64_t update_report_scn_sql_cnt_;
int64_t checker_validate_idx_cnt_;
};
// single thread operation
struct ObTabletLSPairCache
{
public:
ObTabletLSPairCache();
~ObTabletLSPairCache();
void set_tenant_id(const uint64_t tenant_id) { tenant_id_ = tenant_id; }
void reuse();
void destroy();
int try_refresh(const bool force_refresh = false);
int get_tablet_ls_pairs(
const uint64_t table_id,
const ObIArray<ObTabletID> &tablet_ids,
ObIArray<share::ObTabletLSPair> &pairs) const;
TO_STRING_KV(K_(tenant_id), K_(last_refresh_ts), K_(max_task_id), "map_cnt", map_.size());
private:
int refresh();
int rebuild_map_by_tablet_cnt();
int check_exist_new_transfer_task(bool &exist, share::ObTransferTaskID &max_task_id);
const static int64_t RANGE_SIZE = 1000;
const static int64_t REFRESH_CACHE_TIME_INTERVAL = 60 * 1000 * 1000; // 1m
const static int64_t TABLET_LS_MAP_BUCKET_CNT = 3000;
const static int64_t TABLET_LS_MAP_BUCKET_MAX_CNT = 300000;
uint64_t tenant_id_;
int64_t last_refresh_ts_;
share::ObTransferTaskID max_task_id_;
hash::ObHashMap<common::ObTabletID, share::ObLSID> map_;
};
} // namespace compaction
} // namespace oceanbase

View File

@ -71,6 +71,7 @@ ob_set_subtarget(ob_share compaction
compaction/ob_compaction_locality_cache.cpp
compaction/ob_table_ckm_items.cpp
compaction/ob_compaction_time_guard.cpp
compaction/ob_schedule_batch_size_mgr.cpp
)
ob_set_subtarget(ob_share common

View File

@ -0,0 +1,60 @@
//Copyright (c) 2021 OceanBase
// OceanBase is licensed under Mulan PubL v2.
// You can use this software according to the terms and conditions of the Mulan PubL v2.
// You may obtain a copy of Mulan PubL v2 at:
// http://license.coscl.org.cn/MulanPubL-2.0
// THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
// EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
// MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
// See the Mulan PubL v2 for more details.
#define USING_LOG_PREFIX STORAGE_COMPACTION
#include "share/compaction/ob_schedule_batch_size_mgr.h"
#include "lib/oblog/ob_log_module.h"
namespace oceanbase
{
namespace compaction
{
void ObScheduleBatchSizeMgr::set_tablet_batch_size(const int64_t tablet_batch_size)
{
if (tablet_batch_size != tablet_batch_size_) {
LOG_INFO("succeeded to reload new merge schedule tablet batch cnt", K(tablet_batch_size));
tablet_batch_size_ = tablet_batch_size;
}
}
void ObScheduleBatchSizeMgr::get_rs_check_batch_size(
const int64_t table_cnt,
int64_t &table_id_batch_size) const
{
table_id_batch_size = TABLE_ID_BATCH_CHECK_SIZE;
if (table_cnt > TOTAL_TABLE_CNT_THREASHOLD) {
int64_t factor = (table_cnt / TOTAL_TABLE_CNT_THREASHOLD) * 2;
table_id_batch_size *= factor;
}
}
int64_t ObScheduleBatchSizeMgr::get_inner_table_scan_batch_size() const
{
return MAX(1, (tablet_batch_size_ / DEFAULT_TABLET_BATCH_CNT)) * DEFAULT_INNER_TABLE_SCAN_BATCH_SIZE;
}
bool ObScheduleBatchSizeMgr::need_rebuild_map(
const int64_t default_map_bucket_cnt,
const int64_t item_cnt,
const int64_t cur_bucket_cnt,
int64_t &recommend_map_bucked_cnt)
{
bool rebuild_map_flag = false;
int64_t map_cnt = MAX(item_cnt / 3, default_map_bucket_cnt);
recommend_map_bucked_cnt = MIN(map_cnt, default_map_bucket_cnt * 30);
if ((cur_bucket_cnt == 0)
|| (recommend_map_bucked_cnt < map_cnt / 2)
|| (recommend_map_bucked_cnt > map_cnt * 3)) {
rebuild_map_flag = true;
}
return rebuild_map_flag;
}
} // namespace compaction
} // namespace oceanbase

View File

@ -0,0 +1,47 @@
//Copyright (c) 2021 OceanBase
// OceanBase is licensed under Mulan PubL v2.
// You can use this software according to the terms and conditions of the Mulan PubL v2.
// You may obtain a copy of Mulan PubL v2 at:
// http://license.coscl.org.cn/MulanPubL-2.0
// THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
// EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
// MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
// See the Mulan PubL v2 for more details.
#ifndef OB_SHARE_COMPACTION_SCHEDULE_BATCH_SIZE_MGR_H_
#define OB_SHARE_COMPACTION_SCHEDULE_BATCH_SIZE_MGR_H_
#include "/usr/include/stdint.h"
namespace oceanbase
{
namespace compaction
{
struct ObScheduleBatchSizeMgr
{
static const int64_t DEFAULT_TABLET_BATCH_CNT = 50 * 1000L; // 5w
ObScheduleBatchSizeMgr()
: tablet_batch_size_(DEFAULT_TABLET_BATCH_CNT)
{}
~ObScheduleBatchSizeMgr() {}
void set_tablet_batch_size(const int64_t tablet_batch_size);
int64_t get_schedule_batch_size() const { return tablet_batch_size_; }
void get_rs_check_batch_size(
const int64_t table_cnt,
int64_t &table_id_batch_size) const;
int64_t get_inner_table_scan_batch_size() const;
static bool need_rebuild_map(
const int64_t default_map_bucket_cnt,
const int64_t item_cnt,
const int64_t cur_bucket_cnt,
int64_t &recommend_map_bucked_cnt);
private:
const static int64_t TABLET_ID_BATCH_CHECK_SIZE = 3000;
const static int64_t TABLE_ID_BATCH_CHECK_SIZE = 200;
const static int64_t TOTAL_TABLE_CNT_THREASHOLD = 100 * 1000; // 10w
const static int64_t DEFAULT_INNER_TABLE_SCAN_BATCH_SIZE = 500;
int64_t tablet_batch_size_;
};
} // namespace compaction
} // namespace oceanbase
#endif // OB_SHARE_COMPACTION_SCHEDULE_BATCH_SIZE_MGR_H_

View File

@ -10,6 +10,7 @@
#define USING_LOG_PREFIX RS_COMPACTION
#include "share/compaction/ob_table_ckm_items.h"
#include "lib/utility/ob_tracepoint.h"
#include "rootserver/freeze/ob_major_merge_progress_util.h"
namespace oceanbase
{
using namespace oceanbase::common;
@ -134,11 +135,12 @@ ObTableCkmItems::ObTableCkmItems(const uint64_t tenant_id)
: is_inited_(false),
tenant_id_(tenant_id),
table_id_(0),
row_count_(0),
table_schema_(nullptr),
tablet_pairs_(),
ckm_items_(),
sort_col_id_array_(),
ckm_sum_array_(),
row_count_(0)
ckm_sum_array_()
{
ckm_items_.set_attr(ObMemAttr(tenant_id, "TableCkmItems"));
ckm_sum_array_.set_attr(ObMemAttr(tenant_id, "TableCkmItems"));
@ -150,11 +152,13 @@ ObTableCkmItems::~ObTableCkmItems()
}
int ObTableCkmItems::build(
const schema::ObTableSchema &table_schema,
share::schema::ObSchemaGetterGuard &schema_guard,
const schema::ObSimpleTableSchemaV2 &simple_schema,
const ObArray<share::ObTabletLSPair> &input_tablet_pairs,
const ObArray<share::ObTabletReplicaChecksumItem> &input_ckm_items)
{
int ret = OB_SUCCESS;
const int64_t table_id = simple_schema.get_table_id();
if (IS_INIT) {
ret = OB_INIT_TWICE;
LOG_WARN("is inited before", KR(ret), KPC(this));
@ -162,10 +166,12 @@ int ObTableCkmItems::build(
LOG_WARN("failed to assgin tablet ls pair array", KR(ret), K(input_tablet_pairs));
} else if (OB_FAIL(ckm_items_.assign(input_ckm_items))) {
LOG_WARN("failed to assgin tablet replica ckm array", KR(ret), K(input_ckm_items));
} else if (!table_schema.is_index_table() && OB_FAIL(sort_col_id_array_.build(tenant_id_, table_schema))) {
LOG_WARN("failed to build column id array for data table", KR(ret), K(table_schema));
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, table_id, table_schema_))) {
LOG_WARN("fail to get table schema", KR(ret), K_(tenant_id), K(table_id));
} else if (!simple_schema.is_index_table() && OB_FAIL(sort_col_id_array_.build(tenant_id_, *table_schema_))) {
LOG_WARN("failed to build column id array for data table", KR(ret), KPC_(table_schema));
} else {
table_id_ = table_schema.get_table_id();
table_id_ = simple_schema.get_table_id();
is_inited_ = true;
}
if (OB_FAIL(ret)) {
@ -179,31 +185,31 @@ int ObTableCkmItems::build(
const share::SCN &compaction_scn,
common::ObMySQLProxy &sql_proxy,
schema::ObSchemaGetterGuard &schema_guard,
const schema::ObTableSchema *&table_schema,
const bool get_table_schema_flag)
const compaction::ObTabletLSPairCache &tablet_ls_pair_cache)
{
int ret = OB_SUCCESS;
ObSEArray<ObTabletID, 64> tablet_id_array;
if (IS_INIT) {
ret = OB_INIT_TWICE;
LOG_WARN("is inited before", KR(ret), KPC(this));
} else if (OB_UNLIKELY(!get_table_schema_flag && nullptr == table_schema)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(get_table_schema_flag), KP(table_schema));
} else if (get_table_schema_flag
&& OB_FAIL(schema_guard.get_table_schema(tenant_id_, table_id, table_schema))) {
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, table_id, table_schema_))) {
LOG_WARN("fail to get table schema", KR(ret), K_(tenant_id), K(table_id));
} else if (OB_ISNULL(table_schema)) {
} else if (OB_ISNULL(table_schema_)) {
// table schemas are changed, and index_table or data_table does not exist
// in new table schemas. no need to check index column checksum.
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("table schema is null", KR(ret), K(table_id), KP(table_schema));
LOG_WARN("table schema is null", KR(ret), K(table_id), KP_(table_schema));
} else if (OB_FAIL(table_schema_->get_tablet_ids(tablet_id_array))) {
LOG_WARN("fail to get tablet_ids from table schema", KR(ret), KPC(table_schema_));
} else if (OB_FAIL(tablet_ls_pair_cache.get_tablet_ls_pairs(table_id, tablet_id_array, tablet_pairs_))) {
LOG_WARN("failed to get tablet ls pairs", KR(ret), K_(tenant_id), K(table_id), K(tablet_id_array));
} else if (OB_FAIL(ObTabletReplicaChecksumOperator::get_tablet_replica_checksum_items(
tenant_id_, sql_proxy, *table_schema,
tenant_id_, sql_proxy,
compaction_scn, tablet_pairs_,
ckm_items_))) {
LOG_WARN("failed to get table column checksum items", KR(ret));
} else if (!table_schema->is_index_table() && OB_FAIL(sort_col_id_array_.build(tenant_id_, *table_schema))) {
LOG_WARN("failed to build column id array for data table", KR(ret), KPC(table_schema));
} else if (!table_schema_->is_index_table() && OB_FAIL(sort_col_id_array_.build(tenant_id_, *table_schema_))) {
LOG_WARN("failed to build column id array for data table", KR(ret), KPC(table_schema_));
} else {
table_id_ = table_id;
is_inited_ = true;
@ -270,8 +276,8 @@ int ObTableCkmItems::build_column_ckm_sum_array(
#define RECORD_CKM_ERROR_INFO(tablet_array_idx) \
ckm_error_info.tenant_id_ = data_ckm.tenant_id_; \
ckm_error_info.frozen_scn_ = compaction_scn; \
ckm_error_info.data_table_id_ = data_table_schema.get_table_id(); \
ckm_error_info.index_table_id_ = index_table_schema.get_table_id(); \
ckm_error_info.data_table_id_ = data_table_schema->get_table_id(); \
ckm_error_info.index_table_id_ = index_table_schema->get_table_id(); \
if (tablet_array_idx >= 0 && tablet_array_idx < data_ckm.tablet_pairs_.count()) { \
ckm_error_info.data_tablet_id_ = data_ckm.tablet_pairs_.at(tablet_array_idx).get_tablet_id(); \
ckm_error_info.index_tablet_id_ = index_ckm.tablet_pairs_.at(tablet_array_idx).get_tablet_id(); \
@ -279,8 +285,6 @@ int ObTableCkmItems::build_column_ckm_sum_array(
int ObTableCkmItems::validate_column_ckm_sum(
const share::SCN &compaction_scn,
const schema::ObTableSchema &data_table_schema,
const schema::ObTableSchema &index_table_schema,
common::ObMySQLProxy &sql_proxy,
ObTableCkmItems &data_ckm,
ObTableCkmItems &index_ckm)
@ -291,21 +295,24 @@ int ObTableCkmItems::validate_column_ckm_sum(
ObColumnChecksumErrorInfo ckm_error_info;
int64_t data_row_cnt = 0;
int64_t index_row_cnt = 0;
if (OB_UNLIKELY(data_table_schema.get_table_id() != index_table_schema.get_data_table_id())) {
const schema::ObTableSchema *data_table_schema = data_ckm.table_schema_;
const schema::ObTableSchema *index_table_schema = index_ckm.table_schema_;
if (OB_UNLIKELY(nullptr == data_table_schema || nullptr == index_table_schema
|| data_table_schema->get_table_id() != index_table_schema->get_data_table_id())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("data table and index table should not validate column checksum", KR(ret), K(data_table_schema),
K(index_table_schema));
} else if (OB_FAIL(data_ckm.build_column_ckm_sum_array(compaction_scn, data_table_schema, data_row_cnt))) {
LOG_WARN("data table and index table should not validate column checksum", KR(ret), KPC(data_table_schema),
KPC(index_table_schema));
} else if (OB_FAIL(data_ckm.build_column_ckm_sum_array(compaction_scn, *data_table_schema, data_row_cnt))) {
LOG_WARN("failed to build column ckm sum map for data table", KR(ret));
} else if (OB_FAIL(index_ckm.build_column_ckm_sum_array(compaction_scn, index_table_schema, index_row_cnt))) {
} else if (OB_FAIL(index_ckm.build_column_ckm_sum_array(compaction_scn, *index_table_schema, index_row_cnt))) {
LOG_WARN("failed to build column ckm sum map for index table", KR(ret));
} else if (OB_UNLIKELY(data_row_cnt != index_row_cnt)) {
ret = OB_CHECKSUM_ERROR;
LOG_ERROR("sum row count in data & global index is not equal", KR(ret), K(data_row_cnt), K(index_row_cnt));
} else if (OB_FAIL(compare_ckm_by_column_ids(
data_ckm,
data_table_schema,
index_table_schema,
*data_table_schema,
*index_table_schema,
data_ckm.ckm_sum_array_,
index_ckm.ckm_sum_array_,
ckm_error_info))) {
@ -343,8 +350,6 @@ int64_t ObTableCkmItems::get_replica_checksum_idx(
int ObTableCkmItems::validate_tablet_column_ckm(
const share::SCN &compaction_scn,
const schema::ObTableSchema &data_table_schema,
const schema::ObTableSchema &index_table_schema,
common::ObMySQLProxy &sql_proxy,
ObTableCkmItems &data_ckm,
ObTableCkmItems &index_ckm)
@ -353,10 +358,13 @@ int ObTableCkmItems::validate_tablet_column_ckm(
int tmp_ret = OB_SUCCESS;
int64_t check_cnt = 0;
ObColumnChecksumErrorInfo ckm_error_info;
if (OB_UNLIKELY(data_table_schema.get_table_id() != index_table_schema.get_data_table_id())) {
const schema::ObTableSchema *data_table_schema = data_ckm.table_schema_;
const schema::ObTableSchema *index_table_schema = index_ckm.table_schema_;
if (OB_UNLIKELY(nullptr == data_table_schema || nullptr == index_table_schema
|| data_table_schema->get_table_id() != index_table_schema->get_data_table_id())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("data table and index table should not validate column checksum", KR(ret), K(data_table_schema),
K(index_table_schema));
LOG_WARN("data table and index table should not validate column checksum", KR(ret), KPC(data_table_schema),
KPC(index_table_schema));
} else if (OB_UNLIKELY(index_ckm.tablet_pairs_.count() != data_ckm.tablet_pairs_.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet ls pairs count are not equal", KR(ret), K(data_ckm), K(index_ckm),
@ -395,8 +403,8 @@ int ObTableCkmItems::validate_tablet_column_ckm(
"index_row_cnt", index_replica_ckm.row_count_);
} else if (OB_FAIL(compare_ckm_by_column_ids(
data_ckm,
data_table_schema,
index_table_schema,
*data_table_schema,
*index_table_schema,
data_replica_ckm.column_meta_.column_checksums_,
index_replica_ckm.column_meta_.column_checksums_,
ckm_error_info))) {

View File

@ -17,6 +17,7 @@ namespace oceanbase
{
namespace compaction
{
struct ObTabletLSPairCache;
struct ObIndexCkmValidatePair
{
public:
@ -103,8 +104,12 @@ public:
void clear();
void reset();
int64_t get_table_id() const { return table_id_; }
const share::schema::ObTableSchema * get_table_schema() const { return table_schema_; }
const common::ObIArray<share::ObTabletReplicaChecksumItem> &get_ckm_items() const { return ckm_items_; }
const common::ObIArray<share::ObTabletLSPair> &get_tablet_ls_pairs() const { return tablet_pairs_; }
int build(
const share::schema::ObTableSchema &table_schema,
share::schema::ObSchemaGetterGuard &schema_guard,
const share::schema::ObSimpleTableSchemaV2 &simple_schema,
const ObArray<share::ObTabletLSPair> &input_tablet_pairs,
const ObArray<share::ObTabletReplicaChecksumItem> &input_ckm_items);
int build(
@ -112,17 +117,13 @@ public:
const share::SCN &compaction_scn,
common::ObMySQLProxy &sql_proxy,
share::schema::ObSchemaGetterGuard &schema_guard,
const share::schema::ObTableSchema *&table_schema,
const bool get_table_schema_flag = true);
const compaction::ObTabletLSPairCache &tablet_ls_pair_cache);
int build_column_ckm_sum_array(
const share::SCN &compaction_scn,
const share::schema::ObTableSchema &table_schema,
int64_t &row_cnt);
typedef int (*VALIDATE_CKM_FUNC)(
const share::SCN &compaction_scn,
const share::schema::ObTableSchema &data_table_schema,
const share::schema::ObTableSchema &index_table_schema,
common::ObMySQLProxy &sql_proxy,
ObTableCkmItems &data_ckm,
ObTableCkmItems &index_ckm);
@ -135,15 +136,11 @@ public:
private:
static int validate_column_ckm_sum(
const share::SCN &compaction_scn,
const share::schema::ObTableSchema &data_table_schema,
const share::schema::ObTableSchema &index_table_schema,
common::ObMySQLProxy &sql_proxy,
ObTableCkmItems &data_ckm,
ObTableCkmItems &index_ckm);
static int validate_tablet_column_ckm(
const share::SCN &compaction_scn,
const share::schema::ObTableSchema &data_table_schema,
const share::schema::ObTableSchema &index_table_schema,
common::ObMySQLProxy &sql_proxy,
ObTableCkmItems &data_ckm,
ObTableCkmItems &index_ckm);
@ -163,11 +160,12 @@ private:
bool is_inited_;
uint64_t tenant_id_;
uint64_t table_id_;
int64_t row_count_;
const share::schema::ObTableSchema *table_schema_;
common::ObSEArray<share::ObTabletLSPair, DEFAULT_TABLET_CNT> tablet_pairs_;
common::ObArray<share::ObTabletReplicaChecksumItem> ckm_items_; // order by TableSchema::tablet_ids
ObSortColumnIdArray sort_col_id_array_; // column_id -> array_idx
common::ObSEArray<int64_t, DEFAULT_COLUMN_CNT> ckm_sum_array_; // order by TableSchema::tablet_ids
int64_t row_count_;
};
typedef common::ObArray<ObIndexCkmValidatePair> ObIndexCkmValidatePairArray;

View File

@ -21,6 +21,7 @@
#include "share/ob_service_epoch_proxy.h"
#include "share/scn.h"
#include "observer/ob_server_struct.h"
#include "share/tablet/ob_tablet_table_operator.h"
namespace oceanbase
{
@ -209,8 +210,10 @@ int ObTabletMetaTableCompactionOperator::batch_update_unequal_report_scn_tablet(
} else if (OB_ISNULL(result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get mysql result", KR(ret), "sql", sql.ptr());
} else if (OB_FAIL(construct_unequal_tablet_id_array(*result.get_result(), unequal_tablet_id_array))) {
} else if (OB_FAIL(construct_tablet_id_array(*result.get_result(), unequal_tablet_id_array))) {
LOG_WARN("fail to construct tablet id array", KR(ret), "sql", sql.ptr());
} else if (unequal_tablet_id_array.count() > 0) {
LOG_TRACE("success to get uneuqal tablet_id array", K(ret), K(unequal_tablet_id_array));
}
}
if (OB_FAIL(ret) || unequal_tablet_id_array.empty()) {
@ -285,9 +288,9 @@ int ObTabletMetaTableCompactionOperator::get_min_compaction_scn(
return ret;
}
int ObTabletMetaTableCompactionOperator::construct_unequal_tablet_id_array(
int ObTabletMetaTableCompactionOperator::construct_tablet_id_array(
sqlclient::ObMySQLResult &result,
common::ObIArray<ObTabletID> &unequal_tablet_id_array)
common::ObIArray<ObTabletID> &tablet_id_array)
{
int ret = OB_SUCCESS;
int64_t tablet_id = 0;
@ -301,13 +304,10 @@ int ObTabletMetaTableCompactionOperator::construct_unequal_tablet_id_array(
break;
} else if (OB_FAIL(result.get_int("tablet_id", tablet_id))) {
LOG_WARN("fail to get uint", KR(ret));
} else if (OB_FAIL(unequal_tablet_id_array.push_back(ObTabletID(tablet_id)))) {
} else if (OB_FAIL(tablet_id_array.push_back(ObTabletID(tablet_id)))) {
LOG_WARN("failed to push back tablet id", K(ret), K(tablet_id));
}
}
if (OB_SUCC(ret) && unequal_tablet_id_array.count() > 0) {
LOG_TRACE("success to get uneuqal tablet_id array", K(ret), K(unequal_tablet_id_array));
}
return ret;
}
@ -412,7 +412,7 @@ int ObTabletMetaTableCompactionOperator::batch_update_report_scn(
LOG_INFO("start to batch update report scn", KR(ret), K(tenant_id), K(global_broadcast_scn_val), K(expected_epoch));
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
bool update_done = false;
SMART_VAR(ObArray<uint64_t>, tablet_ids) {
SMART_VAR(ObArray<ObTabletID>, tablet_ids) {
while (OB_SUCC(ret) && !update_done && !stop) {
bool is_match = true;
ObMySQLTransaction trans;
@ -470,7 +470,7 @@ int ObTabletMetaTableCompactionOperator::batch_update_status(
LOG_INFO("start to batch update status", KR(ret), K(tenant_id), K(expected_epoch));
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
bool update_done = false;
SMART_VAR(ObArray<uint64_t>, tablet_ids) {
SMART_VAR(ObArray<ObTabletID>, tablet_ids) {
while (OB_SUCC(ret) && !update_done) {
bool is_match = true;
ObMySQLTransaction trans;
@ -508,9 +508,8 @@ int ObTabletMetaTableCompactionOperator::batch_update_status(
int ObTabletMetaTableCompactionOperator::batch_get_tablet_ids(
const uint64_t tenant_id,
const uint64_t start_tablet_id,
const int64_t limit_cnt,
ObIArray<uint64_t> &tablet_ids)
const ObSqlString &sql,
ObIArray<ObTabletID> &tablet_ids)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
@ -519,35 +518,15 @@ int ObTabletMetaTableCompactionOperator::batch_get_tablet_ids(
} else {
tablet_ids.reuse();
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
ObSqlString sql;
SMART_VAR(ObISQLClient::ReadResult, res) {
ObMySQLResult *result = nullptr;
if (OB_FAIL(sql.append_fmt("SELECT DISTINCT tablet_id from %s WHERE tenant_id = '%ld' "
"AND tablet_id > '%ld' ORDER BY tenant_id, tablet_id ASC LIMIT %ld",
OB_ALL_TABLET_META_TABLE_TNAME, tenant_id, start_tablet_id, limit_cnt))) {
LOG_WARN("failed to append fmt", K(ret), K(tenant_id), K(start_tablet_id), K(limit_cnt));
} else if (OB_FAIL(GCTX.sql_proxy_->read(res, meta_tenant_id, sql.ptr()))) {
if (OB_FAIL(GCTX.sql_proxy_->read(res, meta_tenant_id, sql.ptr()))) {
LOG_WARN("fail to execute sql", KR(ret), K(tenant_id), K(meta_tenant_id), K(sql));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get mysql result", KR(ret), K(tenant_id), K(sql));
} else {
while (OB_SUCC(ret)) {
if (OB_FAIL(result->next())) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to get next result", KR(ret));
}
} else {
int64_t tmp_tablet_id = 0;
EXTRACT_INT_FIELD_MYSQL(*result, "tablet_id", tmp_tablet_id, int64_t);
if (FAILEDx(tablet_ids.push_back(tmp_tablet_id))) {
LOG_WARN("fail to push_back tablet_id", KR(ret), K(tmp_tablet_id));
}
}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
} else if (OB_FAIL(construct_tablet_id_array(*result, tablet_ids))) {
LOG_WARN("fail to push_back tablet_id", KR(ret));
}
}
LOG_INFO("finish to batch get tablet_ids", KR(ret), K(tenant_id), K(sql));
@ -559,7 +538,7 @@ int ObTabletMetaTableCompactionOperator::construct_batch_update_report_scn_sql_s
const uint64_t tenant_id,
const uint64_t global_braodcast_scn_val,
const ObTabletReplica::ScnStatus &except_status,
const ObIArray<uint64_t> &tablet_ids,
const ObIArray<ObTabletID> &tablet_ids,
ObSqlString &sql)
{
int ret = OB_SUCCESS;
@ -570,7 +549,7 @@ int ObTabletMetaTableCompactionOperator::construct_batch_update_report_scn_sql_s
} else if (OB_FAIL(sql.assign_fmt("UPDATE %s SET report_scn = '%lu' WHERE tenant_id = '%ld' AND"
" tablet_id >= '%lu' AND tablet_id <= '%lu' AND compaction_scn >= '%lu' AND report_scn"
" < '%lu' AND status != '%ld'", OB_ALL_TABLET_META_TABLE_TNAME, global_braodcast_scn_val,
tenant_id, tablet_ids.at(0), tablet_ids.at(tablet_ids_cnt - 1), global_braodcast_scn_val,
tenant_id, tablet_ids.at(0).id(), tablet_ids.at(tablet_ids_cnt - 1).id(), global_braodcast_scn_val,
global_braodcast_scn_val, (int64_t)except_status))) {
LOG_WARN("fail to assign sql", KR(ret), K(tenant_id), K(global_braodcast_scn_val), K(except_status),
"start_tablet_id", tablet_ids.at(0), "end_tablet_id", tablet_ids.at(tablet_ids_cnt - 1));
@ -580,7 +559,7 @@ int ObTabletMetaTableCompactionOperator::construct_batch_update_report_scn_sql_s
int ObTabletMetaTableCompactionOperator::construct_batch_update_status_sql_str_(
const uint64_t tenant_id,
const ObIArray<uint64_t> &tablet_ids,
const ObIArray<ObTabletID> &tablet_ids,
ObSqlString &sql)
{
int ret = OB_SUCCESS;
@ -591,7 +570,7 @@ int ObTabletMetaTableCompactionOperator::construct_batch_update_status_sql_str_(
} else if (OB_FAIL(sql.assign_fmt("UPDATE %s SET status = '%ld' WHERE tenant_id = '%ld' AND"
" tablet_id >= '%lu' AND tablet_id <= '%lu' AND status = '%ld'",
OB_ALL_TABLET_META_TABLE_TNAME, (int64_t)ObTabletReplica::ScnStatus::SCN_STATUS_IDLE,
tenant_id, tablet_ids.at(0), tablet_ids.at(tablet_ids_cnt - 1),
tenant_id, tablet_ids.at(0).id(), tablet_ids.at(tablet_ids_cnt - 1).id(),
(int64_t)ObTabletReplica::ScnStatus::SCN_STATUS_ERROR))) {
LOG_WARN("fail to assign sql", KR(ret), K(tenant_id), "start_tablet_id", tablet_ids.at(0),
"end_tablet_id", tablet_ids.at(tablet_ids_cnt - 1));
@ -728,91 +707,157 @@ int ObTabletMetaTableCompactionOperator::batch_update_report_scn(
return ret;
}
int ObTabletMetaTableCompactionOperator::get_unique_status(
const uint64_t tenant_id,
ObIArray<ObTabletLSPair> &pairs,
ObIArray<ObTabletReplica::ScnStatus> &status_arr)
{
int ret = OB_SUCCESS;
const int64_t pair_cnt = pairs.count();
if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || pair_cnt < 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(pair_cnt));
} else {
ObSqlString sql;
SMART_VAR(ObISQLClient::ReadResult, res) {
ObMySQLResult *result = nullptr;
if (OB_FAIL(sql.assign_fmt("SELECT distinct status FROM %s WHERE tenant_id = '%lu' AND (ls_id, tablet_id) "
"IN (", OB_ALL_TABLET_META_TABLE_TNAME, tenant_id))) {
LOG_WARN("fail to assign sql", KR(ret), K(tenant_id));
} else {
for (int64_t i = 0; (i < pair_cnt) && OB_SUCC(ret); ++i) {
const ObTabletLSPair &pair = pairs.at(i);
if (OB_UNLIKELY(!pair.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(i), K(pair));
} else if (OB_FAIL(sql.append_fmt("(%ld, %ld)%s", pair.get_ls_id().id(),
pair.get_tablet_id().id(), ((i == pair_cnt - 1) ? ")" : ", ")))) {
LOG_WARN("fail to assign sql", KR(ret), K(i), K(tenant_id), K(pair));
}
}
}
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
if (FAILEDx(GCTX.sql_proxy_->read(res, meta_tenant_id, sql.ptr()))) {
LOG_WARN("fail to execute sql", KR(ret), K(tenant_id), K(meta_tenant_id), K(sql));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get mysql result", KR(ret), K(tenant_id), K(sql));
} else {
while (OB_SUCC(ret)) {
int64_t tmp_status = 0;
if (OB_FAIL(result->next())) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to get next result", KR(ret), K(tenant_id), K(sql));
}
} else if (OB_FAIL(result->get_int("status", tmp_status))) {
LOG_WARN("failed to get int", KR(ret), K(tmp_status));
} else if (OB_UNLIKELY(!ObTabletReplica::is_status_valid((ObTabletReplica::ScnStatus)tmp_status))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("status is invalid", KR(ret), K(tenant_id), K(tmp_status));
} else if (OB_FAIL(status_arr.push_back(ObTabletReplica::ScnStatus(tmp_status)))) {
LOG_WARN("fail to push back status", KR(ret), K(tenant_id), K(tmp_status));
}
} // end while loop
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
}
}
}
return ret;
}
int ObTabletMetaTableCompactionOperator::get_next_batch_tablet_ids(
const uint64_t tenant_id,
const int64_t batch_update_cnt,
ObIArray<uint64_t> &tablet_ids)
ObIArray<ObTabletID> &tablet_ids)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || batch_update_cnt < 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(batch_update_cnt));
} else {
uint64_t start_tablet_id = ObTabletID::INVALID_TABLET_ID;
ObSqlString sql;
ObTabletID start_tablet_id = ObTabletID(ObTabletID::INVALID_TABLET_ID);
if (tablet_ids.count() > 0) {
start_tablet_id = tablet_ids.at(tablet_ids.count() - 1);
}
tablet_ids.reuse();
if (OB_FAIL(ObTabletMetaTableCompactionOperator::batch_get_tablet_ids(tenant_id,
start_tablet_id, batch_update_cnt, tablet_ids))) {
if (OB_FAIL(sql.append_fmt("SELECT DISTINCT tablet_id from %s WHERE tenant_id = '%ld' "
"AND tablet_id > '%ld' ORDER BY tenant_id, tablet_id ASC LIMIT %ld",
OB_ALL_TABLET_META_TABLE_TNAME, tenant_id, start_tablet_id.id(), batch_update_cnt))) {
LOG_WARN("failed to append fmt", K(ret), K(tenant_id), K(start_tablet_id), K(batch_update_cnt));
} else if (OB_FAIL(batch_get_tablet_ids(tenant_id, sql, tablet_ids))) {
LOG_WARN("fail to batch get tablet_ids", KR(ret), K(tenant_id), K(start_tablet_id), K(batch_update_cnt));
}
}
return ret;
}
int ObTabletMetaTableCompactionOperator::range_scan_for_compaction(
const uint64_t tenant_id,
const int64_t compaction_scn,
const common::ObTabletID &start_tablet_id,
const int64_t batch_size,
const bool add_report_scn_filter,
common::ObTabletID &end_tablet_id,
ObIArray<ObTabletInfo> &tablet_infos)
{
int ret = OB_SUCCESS;
tablet_infos.reset();
if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id) || batch_size <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(start_tablet_id), K(batch_size));
} else if (start_tablet_id.id() == INT64_MAX) {
ret = OB_ITER_END;
} else {
ObTabletID tmp_start_tablet_id = start_tablet_id;
ObTabletID tmp_end_tablet_id;
while (OB_SUCC(ret) && tmp_start_tablet_id.id() < INT64_MAX) {
if (OB_SUCC(inner_range_scan_for_compaction(
tenant_id, compaction_scn, tmp_start_tablet_id, batch_size,
add_report_scn_filter, tmp_end_tablet_id, tablet_infos))) {
if (tablet_infos.empty()) {
tmp_start_tablet_id = tmp_end_tablet_id;
tmp_end_tablet_id.reset();
} else {
break;
}
}
} // end of while
if (OB_SUCC(ret)) {
end_tablet_id = tmp_end_tablet_id;
if (tablet_infos.empty()) {
ret = OB_ITER_END;
}
}
}
return ret;
}
int ObTabletMetaTableCompactionOperator::inner_range_scan_for_compaction(
const uint64_t tenant_id,
const int64_t compaction_scn,
const common::ObTabletID &start_tablet_id,
const int64_t batch_size,
const bool add_report_scn_filter,
common::ObTabletID &end_tablet_id,
ObIArray<ObTabletInfo> &tablet_infos)
{
int ret = OB_SUCCESS;
ObTabletID max_tablet_id;
if (OB_ISNULL(GCTX.sql_proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql proxy in ctx is null", KR(ret), K(GCTX.sql_proxy_));
} else if (OB_FAIL(inner_get_max_tablet_id_in_range(tenant_id, start_tablet_id, batch_size, max_tablet_id))) {
if (OB_ITER_END != ret) {
LOG_WARN("failed to get max tablet id in range", KR(ret), K(start_tablet_id));
} else {
ret = OB_SUCCESS;
max_tablet_id = ObTabletID(INT64_MAX);
}
}
if (OB_SUCC(ret)) {
SMART_VAR(ObISQLClient::ReadResult, result) {
const uint64_t sql_tenant_id = gen_meta_tenant_id(tenant_id);
ObSqlString sql;
ObSqlString report_scn_sql;
if (add_report_scn_filter && OB_FAIL(report_scn_sql.append_fmt("AND report_scn < %ld", compaction_scn))) {
LOG_WARN("fail to assign sql", KR(ret), K(report_scn_sql));
} else if (OB_FAIL(sql.append_fmt(
"SELECT * from %s where tenant_id=%lu AND tablet_id > %ld AND tablet_id <= %ld %s",
OB_ALL_TABLET_META_TABLE_TNAME, tenant_id, start_tablet_id.id(), max_tablet_id.id(), add_report_scn_filter ? report_scn_sql.ptr() : ""))) {
LOG_WARN("fail to assign sql", KR(ret), K(sql));
} else if (OB_FAIL(GCTX.sql_proxy_->read(result, sql_tenant_id, sql.ptr()))) {
LOG_WARN("execute sql failed", KR(ret), K(tenant_id), K(sql_tenant_id), K(sql));
} else if (OB_ISNULL(result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get mysql result failed", KR(ret), K(sql));
} else if (OB_FAIL(ObTabletTableOperator::construct_tablet_infos(*result.get_result(), tablet_infos))) {
LOG_WARN("construct tablet info failed", KR(ret), K(sql), K(tablet_infos));
} else {
end_tablet_id = max_tablet_id;
LOG_INFO("success to get tablet info", KR(ret), K(batch_size), K(tablet_infos), K(end_tablet_id), K(add_report_scn_filter));
}
}
}
return ret;
}
int ObTabletMetaTableCompactionOperator::inner_get_max_tablet_id_in_range(
const uint64_t tenant_id,
const common::ObTabletID &start_tablet_id,
const int64_t batch_size,
common:: ObTabletID &end_tablet_id)
{
int ret = OB_SUCCESS;
ObSqlString sql;
sqlclient::ObMySQLResult *result_ptr = nullptr;
const uint64_t sql_tenant_id = gen_meta_tenant_id(tenant_id);
int64_t tablet_id = 0;
SMART_VAR(ObISQLClient::ReadResult, result) {
if (OB_FAIL(sql.append_fmt("SELECT tablet_id FROM %s "
"WHERE tenant_id = %lu AND tablet_id > %ld ORDER BY tablet_id asc LIMIT 1 OFFSET %ld",
OB_ALL_TABLET_META_TABLE_TNAME, tenant_id, start_tablet_id.id(), batch_size))) {
LOG_WARN("fail to assign sql", KR(ret), K(sql));
} else if (OB_FAIL(GCTX.sql_proxy_->read(result, sql_tenant_id, sql.ptr()))) {
LOG_WARN("execute sql failed", KR(ret), K(tenant_id), K(sql_tenant_id), K(sql));
} else if (OB_ISNULL(result_ptr = result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get mysql result failed", KR(ret), K(sql));
} else if (OB_FAIL(result_ptr->next())) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to get next result", KR(ret));
}
} else if (OB_FAIL(result_ptr->get_int("tablet_id", tablet_id))) {
LOG_WARN("fail to get uint", KR(ret));
} else {
end_tablet_id = tablet_id;
}
}
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -113,10 +113,6 @@ public:
static int batch_update_status(
const uint64_t tenant_id,
const int64_t expected_epoch);
static int get_unique_status(
const uint64_t tenant_id,
common::ObIArray<ObTabletLSPair> &pairs,
common::ObIArray<ObTabletReplica::ScnStatus> &status_arr);
static int batch_update_unequal_report_scn_tablet(
const uint64_t tenant_id,
const share::ObLSID &ls_id,
@ -125,7 +121,28 @@ public:
static int get_min_compaction_scn(
const uint64_t tenant_id,
SCN &min_compaction_scn);
static int range_scan_for_compaction(
const uint64_t tenant_id,
const int64_t compaction_scn,
const common::ObTabletID &start_tablet_id,
const int64_t batch_size,
const bool add_report_scn_filter,
common::ObTabletID &end_tablet_id,
ObIArray<ObTabletInfo> &tablet_infos);
private:
static int inner_range_scan_for_compaction(
const uint64_t tenant_id,
const int64_t compaction_scn,
const common::ObTabletID &start_tablet_id,
const int64_t batch_size,
const bool add_report_scn_filter,
common::ObTabletID &end_tablet_id,
ObIArray<ObTabletInfo> &tablet_infos);
static int inner_get_max_tablet_id_in_range(
const uint64_t tenant_id,
const common::ObTabletID &start_tablet_id,
const int64_t batch_size,
common::ObTabletID &max_tablet_id);
static int inner_batch_set_info_status_(
const uint64_t tenant_id,
const ObIArray<ObTabletLSPair> &tablet_ls_pairs,
@ -164,31 +181,30 @@ private:
const int64_t start_idx,
const int64_t end_idx,
ObSqlString &sql);
static int construct_unequal_tablet_id_array(
static int construct_tablet_id_array(
sqlclient::ObMySQLResult &result,
common::ObIArray<ObTabletID> &unequal_tablet_id_array);
common::ObIArray<ObTabletID> &tablet_id_array);
static int get_estimated_timeout_us(const uint64_t tenant_id, int64_t &estimated_timeout_us);
static int get_tablet_replica_cnt(const uint64_t tenant_id, int64_t &tablet_replica_cnt);
// get tablet_ids larger than @start_tablet_id, and get up to @limit_cnt records
static int batch_get_tablet_ids(
const uint64_t tenant_id,
const uint64_t start_tablet_id,
const int64_t limit_cnt,
common::ObIArray<uint64_t> &tablet_ids);
const ObSqlString &sql,
common::ObIArray<ObTabletID> &tablet_ids);
static int construct_batch_update_report_scn_sql_str_(
const uint64_t tenant_id,
const uint64_t global_braodcast_scn_val,
const ObTabletReplica::ScnStatus &except_status,
const common::ObIArray<uint64_t> &tablet_ids,
const common::ObIArray<ObTabletID> &tablet_ids,
ObSqlString &sql);
static int construct_batch_update_status_sql_str_(
const uint64_t tenant_id,
const common::ObIArray<uint64_t> &tablet_ids,
const common::ObIArray<ObTabletID> &tablet_ids,
ObSqlString &sql);
static int get_next_batch_tablet_ids(
const uint64_t tenant_id,
const int64_t batch_update_cnt,
common::ObIArray<uint64_t> &tablet_ids);
common::ObIArray<ObTabletID> &tablet_ids);
private:
const static int64_t MAX_BATCH_COUNT = 500;
};

View File

@ -777,102 +777,20 @@ int ObTabletReplicaChecksumOperator::inner_batch_insert_or_update_by_sql_(
return ret;
}
int ObTabletReplicaChecksumOperator::get_tablet_ls_pairs(
const uint64_t tenant_id,
const ObSimpleTableSchemaV2 &simple_schema,
ObMySQLProxy &sql_proxy,
ObIArray<ObTabletLSPair> &pairs)
{
int ret = OB_SUCCESS;
if ((!is_valid_tenant_id(tenant_id)) || (!simple_schema.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id));
} else {
SMART_VAR(ObArray<ObTabletID>, tablet_ids) {
if (OB_FAIL(get_table_all_tablet_ids_(simple_schema, tablet_ids))) {
LOG_WARN("fail to get table all tablet ids", KR(ret), K(simple_schema));
} else if (tablet_ids.count() > 0) {
const uint64_t table_id = simple_schema.get_table_id();
if (OB_FAIL(get_tablet_ls_pairs(tenant_id, table_id, sql_proxy, tablet_ids, pairs))) {
LOG_WARN("fail to get tablet_ls_pairs", KR(ret), K(tenant_id), K(table_id));
}
}
}
}
return ret;
}
int ObTabletReplicaChecksumOperator::get_tablet_ls_pairs(
const uint64_t tenant_id,
const uint64_t table_id,
ObMySQLProxy &sql_proxy,
const ObIArray<ObTabletID> &tablet_ids,
ObIArray<ObTabletLSPair> &pairs)
{
int ret = OB_SUCCESS;
if (!is_valid_tenant_id(tenant_id) || (tablet_ids.count() < 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id), K(tablet_ids.count()));
} else {
SMART_VAR(ObArray<ObLSID>, ls_ids) {
// sys_table's tablet->ls relation won't be written into __all_tablet_to_ls
if (is_sys_tenant(tenant_id) || is_sys_table(table_id)) {
for (int64_t i = 0; (i < tablet_ids.count()) && OB_SUCC(ret); ++i) {
ObLSID tmp_ls_id(ObLSID::SYS_LS_ID);
if (OB_FAIL(ls_ids.push_back(tmp_ls_id))) {
LOG_WARN("fail to push back ls_id", KR(ret), K(tenant_id), K(table_id));
}
}
} else if (OB_FAIL(ObTabletToLSTableOperator::batch_get_ls(sql_proxy, tenant_id, tablet_ids, ls_ids))) {
LOG_WARN("fail to batch get ls", KR(ret), K(tenant_id), K(tablet_ids));
}
const int64_t ls_id_cnt = ls_ids.count();
if (OB_SUCC(ret) && (ls_id_cnt != tablet_ids.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("count mismatch", KR(ret), K(ls_id_cnt), K(tablet_ids.count()));
}
for (int64_t i = 0; (i < ls_id_cnt) && OB_SUCC(ret); ++i) {
ObTabletLSPair cur_pair;
const ObTabletID &cur_tablet_id = tablet_ids.at(i);
const ObLSID &cur_ls_id = ls_ids.at(i);
if (OB_FAIL(cur_pair.init(cur_tablet_id, cur_ls_id))) {
LOG_WARN("fail to init tablet_ls_pair", KR(ret), K(i), K(cur_tablet_id), K(cur_ls_id));
} else if (OB_FAIL(pairs.push_back(cur_pair))) {
LOG_WARN("fail to push back pair", KR(ret), K(cur_pair));
}
}
}
if (OB_FAIL(ret)){
} else if (OB_UNLIKELY(pairs.count() != tablet_ids.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("some unexpected err about tablet_ls_pair count", KR(ret), "tablet_id_cnt",
tablet_ids.count(), "pair_cnt", pairs.count());
}
}
return ret;
}
int ObTabletReplicaChecksumOperator::get_tablet_replica_checksum_items(
const uint64_t tenant_id,
ObMySQLProxy &sql_proxy,
const ObSimpleTableSchemaV2 &simple_schema,
const SCN &compaction_scn,
ObIArray<ObTabletLSPair> &tablet_pairs,
const ObIArray<ObTabletLSPair> &tablet_pairs,
ObIArray<ObTabletReplicaChecksumItem> &items)
{
int ret = OB_SUCCESS;
if ((!is_valid_tenant_id(tenant_id)) || (!simple_schema.is_valid())) {
if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id));
} else {
int64_t tablet_items_cnt = 0;
const uint64_t table_id = simple_schema.get_table_id();
if (OB_FAIL(get_tablet_ls_pairs(tenant_id, simple_schema, sql_proxy, tablet_pairs))) {
LOG_WARN("fail to get tablet_ls_pairs", KR(ret), K(tenant_id), K(table_id));
} else if (OB_FAIL(ObTabletReplicaChecksumOperator::batch_get(tenant_id, tablet_pairs, compaction_scn,
if (OB_FAIL(batch_get(tenant_id, tablet_pairs, compaction_scn,
sql_proxy, items, tablet_items_cnt, false/*include_larger_than*/))) {
LOG_WARN("fail to batch get tablet checksum item", KR(ret), K(tenant_id), K(compaction_scn),
"pairs_count", tablet_pairs.count());
@ -885,24 +803,6 @@ int ObTabletReplicaChecksumOperator::get_tablet_replica_checksum_items(
return ret;
}
int ObTabletReplicaChecksumOperator::get_table_all_tablet_ids_(
const ObSimpleTableSchemaV2 &simple_schema,
ObIArray<ObTabletID> &schema_tablet_ids)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!simple_schema.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arguments", KR(ret), K(simple_schema));
} else {
if (simple_schema.has_tablet()) {
if (OB_FAIL(simple_schema.get_tablet_ids(schema_tablet_ids))) {
LOG_WARN("fail to get tablet_ids from table schema", KR(ret), K(simple_schema));
}
}
}
return ret;
}
int ObTabletReplicaChecksumOperator::set_column_meta_with_hex_str(
const common::ObString &hex_str,
ObTabletReplicaReportColumnMeta &column_meta)

View File

@ -139,20 +139,6 @@ public:
const uint64_t tenant_id,
const ObIArray<compaction::ObTabletCheckInfo> &pairs,
ObIArray<ObTabletReplicaChecksumItem> &tablet_replica_checksum_items);
static int get_tablet_ls_pairs(
const uint64_t tenant_id,
const schema::ObSimpleTableSchemaV2 &simple_schema,
common::ObMySQLProxy &sql_proxy,
common::ObIArray<ObTabletLSPair> &tablet_ls_pairs);
static int get_tablet_ls_pairs(
const uint64_t tenant_id,
const uint64_t table_id,
common::ObMySQLProxy &sql_proxy,
const common::ObIArray<common::ObTabletID> &tablet_ids,
common::ObIArray<ObTabletLSPair> &tablet_ls_pairs);
static int set_column_meta_with_hex_str(
const ObString &hex_str,
ObTabletReplicaReportColumnMeta &column_meta);
@ -228,14 +214,10 @@ public:
static int get_tablet_replica_checksum_items(
const uint64_t tenant_id,
common::ObMySQLProxy &mysql_proxy,
const schema::ObSimpleTableSchemaV2 &simple_schema,
const SCN &compaction_scn,
common::ObIArray<ObTabletLSPair> &tablet_pairs,
const common::ObIArray<ObTabletLSPair> &tablet_pairs,
common::ObIArray<ObTabletReplicaChecksumItem> &items);
private:
static int get_table_all_tablet_ids_(
const schema::ObSimpleTableSchemaV2 &simple_schema,
common::ObIArray<common::ObTabletID> &schema_tablet_ids);
const static int64_t MAX_BATCH_COUNT = 120;
const static int64_t PRINT_LOG_INVERVAL = 2 * 60 * 1000 * 1000L; // 2m
};

View File

@ -18,6 +18,7 @@
#include "share/schema/ob_part_mgr_util.h"
#include "share/schema/ob_multi_version_schema_service.h"
#include "observer/ob_server_struct.h"
#include "share/ob_tablet_meta_table_compaction_operator.h"
namespace oceanbase
{
@ -26,167 +27,43 @@ namespace share
using namespace common;
using namespace schema;
ObCompactionTabletMetaIterator::ObCompactionTabletMetaIterator()
ObTabletMetaIterator::ObTabletMetaIterator()
: is_inited_(false),
first_prefetch_(true),
sql_proxy_(nullptr),
tablet_table_operator_(),
tenant_id_(OB_INVALID_TENANT_ID),
tablet_ls_pairs_(),
prefetch_tablet_idx_(0)
prefetch_tablet_idx_(0),
tenant_id_(OB_INVALID_TENANT_ID)
{}
int ObCompactionTabletMetaIterator::init(
common::ObISQLClient &sql_proxy,
const uint64_t tenant_id,
share::ObIServerTrace &server_trace,
ObIArray<share::ObTabletLSPair> &tablet_ls_pairs)
void ObTabletMetaIterator::reset()
{
is_inited_ = false;
tenant_id_ = OB_INVALID_TENANT_ID;
prefetch_tablet_idx_ = -1;
prefetched_tablets_.reset();
}
int ObTabletMetaIterator::inner_init(
const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
if (IS_INIT) {
ret = OB_INIT_TWICE;
LOG_WARN("ObLSTabletMetaIterator init twice", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id || tablet_ls_pairs.empty())) {
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tenant_id invalid", KR(ret), K(tenant_id), K(tablet_ls_pairs));
} else if (OB_FAIL(tablet_table_operator_.init(sql_proxy))) {
LOG_WARN("fail to init tablet table operator", KR(ret), K(tenant_id));
// Keep set_filter_not_exist_server before setting all the other filters,
// otherwise the other filters may return OB_ENTRY_NOT_EXIST error code.
} else if (OB_FAIL(filters_.set_filter_not_exist_server(server_trace))) {
LOG_WARN("fail to set not exist server filter", KR(ret), K(tenant_id));
} else if (OB_FAIL(filters_.set_filter_permanent_offline(server_trace))) {
LOG_WARN("fail to set filter", KR(ret), K(tenant_id));
LOG_WARN("tenant_id invalid", KR(ret), K(tenant_id));
} else {
sql_proxy_ = &sql_proxy;
tenant_id_ = tenant_id;
tablet_ls_pairs_ = &tablet_ls_pairs;
if (OB_FAIL(prefetch())) { // need to prefetch a batch of tablet_info
if (OB_ITER_END != ret) {
LOG_WARN("fail to prefetch", KR(ret), K_(tenant_id), K_(prefetch_tablet_idx));
}
} else {
is_inited_ = true;
}
}
return ret;
}
void ObCompactionTabletMetaIterator::reset()
{
is_inited_ = false;
sql_proxy_ = nullptr;
tenant_id_ = OB_INVALID_TENANT_ID;
tablet_ls_pairs_ = nullptr;
prefetch_tablet_idx_ = -1;
prefetched_tablets_.reset();
tablet_table_operator_.reset();
}
int ObCompactionTabletMetaIterator::next(ObTabletInfo &tablet_info)
int ObTabletMetaIterator::next(ObTabletInfo &tablet_info)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else {
bool find = false;
while (OB_SUCC(ret) && !find) {
if (!iter_prefetch_info_finish()) {
// directly get from prefetched tablet_info
tablet_info.reset();
if (OB_FAIL(tablet_info.assign(prefetched_tablets_.at(prefetch_tablet_idx_)))) {
LOG_WARN("fail to assign tablet_info", KR(ret), K_(prefetch_tablet_idx));
} else if (tablet_info.replica_count() > 0) {
//
if (OB_FAIL(tablet_info.filter(filters_))) {
LOG_WARN("fail to filter tablet_info", KR(ret), K(tablet_info));
} else {
find = true;
}
}
++prefetch_tablet_idx_;
} else {
ret = OB_ITER_END;
}
}
}
return ret;
}
int ObCompactionTabletMetaIterator::prefetch()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(tablet_ls_pairs_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet ls pairs is unexpected null", K(ret), KP(tablet_ls_pairs_));
} else {
prefetch_tablet_idx_ = 0;
prefetched_tablets_.reuse();
if (OB_FAIL(tablet_table_operator_.batch_get(
tenant_id_, *tablet_ls_pairs_, prefetched_tablets_))) {
LOG_WARN("fail to do batch_get through tablet_table_operator", KR(ret),
K_(tenant_id), K_(prefetched_tablets));
} else {
LOG_INFO("finish batch get", K(ret), K(prefetched_tablets_.count()));
}
}
return ret;
}
////////////////////////////////////////////////////////////////////////
ObTenantTabletMetaIterator::ObTenantTabletMetaIterator()
: is_inited_(false),
sql_proxy_(nullptr),
tablet_table_operator_(),
tenant_id_(OB_INVALID_TENANT_ID),
first_prefetch_(true),
prefetched_tablets_(),
valid_tablet_ls_pairs_(),
valid_tablet_ls_pairs_idx_(0),
prefetch_tablet_idx_(0),
filters_()
{
}
ObTenantTabletMetaIterator::~ObTenantTabletMetaIterator()
{
}
int ObTenantTabletMetaIterator::init(
common::ObISQLClient &sql_proxy,
const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(is_inited())) {
ret = OB_INIT_TWICE;
LOG_WARN("ObTenantTabletMetaIterator init twice", KR(ret));
} else if (OB_INVALID_TENANT_ID == tenant_id) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tenant_id invalid", KR(ret), K(tenant_id));
} else if (OB_FAIL(tablet_table_operator_.init(sql_proxy))) {
LOG_WARN("fail to init tablet table operator", KR(ret), K(tenant_id));
} else {
sql_proxy_ = &sql_proxy;
tenant_id_ = tenant_id;
first_prefetch_ = true;
prefetched_tablets_.reuse();
valid_tablet_ls_pairs_.reuse();
valid_tablet_ls_pairs_idx_ = 0;
prefetch_tablet_idx_ = 0;
is_inited_ = true;
}
return ret;
}
int ObTenantTabletMetaIterator::next(ObTabletInfo &tablet_info)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited())) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else if (prefetch_tablet_idx_ == -1) {
ret = OB_ITER_END;
} else {
@ -219,13 +96,132 @@ int ObTenantTabletMetaIterator::next(ObTabletInfo &tablet_info)
return ret;
}
/**
* -------------------------------------------------------------------ObCompactionTabletMetaIterator-------------------------------------------------------------------
*/
ObCompactionTabletMetaIterator::ObCompactionTabletMetaIterator(
const bool first_check, const int64_t compaction_scn)
: ObTabletMetaIterator(),
first_check_(first_check),
compaction_scn_(compaction_scn),
batch_size_(TABLET_META_TABLE_RANGE_GET_SIZE),
end_tablet_id_()
{}
void ObCompactionTabletMetaIterator::reset()
{
ObTabletMetaIterator::reset();
first_check_ = false;
compaction_scn_ = 0;
end_tablet_id_.reset();
batch_size_ = 0;
}
int ObCompactionTabletMetaIterator::init(
const uint64_t tenant_id,
const int64_t batch_size,
share::ObIServerTrace &server_trace)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(batch_size <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(batch_size));
} else if (OB_FAIL(ObTabletMetaIterator::inner_init(tenant_id))) {
LOG_WARN("failed to init", KR(ret), K(tenant_id));
// Keep set_filter_not_exist_server before setting all the other filters,
// otherwise the other filters may return OB_ENTRY_NOT_EXIST error code.
} else if (OB_FAIL(filters_.set_filter_not_exist_server(server_trace))) {
LOG_WARN("fail to set not exist server filter", KR(ret), K(tenant_id));
} else if (OB_FAIL(filters_.set_filter_permanent_offline(server_trace))) {
LOG_WARN("fail to set filter", KR(ret), K(tenant_id));
} else {
batch_size_ = batch_size;
is_inited_ = true;
}
return ret;
}
int ObCompactionTabletMetaIterator::prefetch()
{
int ret = OB_SUCCESS;
if (prefetch_tablet_idx_ >= prefetched_tablets_.count()) {
ObTabletID tmp_last_tablet_id;
if (OB_FAIL(ObTabletMetaTableCompactionOperator::range_scan_for_compaction(
tenant_id_,
compaction_scn_,
end_tablet_id_,
batch_size_,
!first_check_/*add_report_scn_filter*/,
tmp_last_tablet_id,
prefetched_tablets_))) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to range get by operator", KR(ret),
K_(tenant_id), K_(end_tablet_id), K_(batch_size), K_(prefetched_tablets));
} else {
prefetch_tablet_idx_ = -1;
}
} else if (prefetched_tablets_.count() <= 0) {
prefetch_tablet_idx_ = -1;
ret = OB_ITER_END;
} else {
end_tablet_id_ = tmp_last_tablet_id;
prefetch_tablet_idx_ = 0;
}
}
return ret;
}
////////////////////////////////////////////////////////////////////////
ObTenantTabletMetaIterator::ObTenantTabletMetaIterator()
: ObTabletMetaIterator(),
first_prefetch_(true),
sql_proxy_(nullptr),
valid_tablet_ls_pairs_idx_(0)
{
}
ObTenantTabletMetaIterator::~ObTenantTabletMetaIterator()
{
}
int ObTenantTabletMetaIterator::init(
common::ObISQLClient &sql_proxy,
const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObTabletMetaIterator::inner_init(tenant_id))) {
LOG_WARN("fail to init", KR(ret), K(tenant_id));
} else if (OB_FAIL(tablet_table_operator_.init(sql_proxy))) {
LOG_WARN("fail to init tablet table operator", KR(ret), K(tenant_id));
} else {
sql_proxy_ = &sql_proxy;
valid_tablet_ls_pairs_.reuse();
valid_tablet_ls_pairs_idx_ = 0;
if (OB_FAIL(prefetch())) { // need to prefetch a batch of tablet_info
if (OB_ITER_END != ret) {
LOG_WARN("fail to prefetch", KR(ret), K_(tenant_id), K_(prefetch_tablet_idx));
}
} else {
is_inited_ = true;
}
}
return ret;
}
void ObTenantTabletMetaIterator::reset()
{
ObTabletMetaIterator::reset();
first_prefetch_ = true;
sql_proxy_ = nullptr;
valid_tablet_ls_pairs_idx_ = -1;
valid_tablet_ls_pairs_.reset();
tablet_table_operator_.reset();
}
int ObTenantTabletMetaIterator::prefetch()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited())) {
ret = OB_NOT_INIT;
LOG_WARN("ObTenantTabletMetaIterator is not inited", KR(ret));
} else if (OB_FAIL(prefetch_valid_tablet_ids())) {
if (OB_FAIL(prefetch_valid_tablet_ids())) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to prefetch valid tablet ids", KR(ret), K_(tenant_id),
K_(valid_tablet_ls_pairs_idx), K_(valid_tablet_ls_pairs));
@ -240,7 +236,7 @@ int ObTenantTabletMetaIterator::prefetch()
int ObTenantTabletMetaIterator::prefetch_valid_tablet_ids()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited())) {
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("ObTenantTabletMetaIterator is not inited", KR(ret));
} else if (OB_UNLIKELY(prefetch_tablet_idx_ != prefetched_tablets_.count())) {
@ -269,7 +265,7 @@ int ObTenantTabletMetaIterator::prefetch_valid_tablet_ids()
int ObTenantTabletMetaIterator::prefetch_tablets()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited())) {
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("ObTenantTabletMetaIterator is not inited", KR(ret));
} else if (OB_UNLIKELY(prefetch_tablet_idx_ != prefetched_tablets_.count())) {

View File

@ -28,72 +28,71 @@ class ObTabletToLSTableOperator;
class ObTabletTableOperator;
class ObTabletInfo;
class ObCompactionTabletMetaIterator
class ObTabletMetaIterator
{
public:
ObCompactionTabletMetaIterator();
~ObCompactionTabletMetaIterator() { reset(); }
int init(
common::ObISQLClient &sql_proxy,
const uint64_t tenant_id,
share::ObIServerTrace &server_trace,
ObIArray<share::ObTabletLSPair> &tablet_ls_pairs);
void reset();
ObTabletMetaIterator();
~ObTabletMetaIterator() { reset(); }
virtual void reset();
int next(ObTabletInfo &tablet_info);
void set_batch_size(int64_t batch_size) {tablet_table_operator_.set_batch_size(batch_size);}
private:
int prefetch();
int iter_prefetch_info_finish() const
{
return prefetch_tablet_idx_ >= prefetched_tablets_.count();
}
const static int64_t TABLET_META_TABLE_RANGE_GET_SIZE = 150;
private:
protected:
int inner_init(
const uint64_t tenant_id);
virtual int prefetch() = 0;
protected:
bool is_inited_;
bool first_prefetch_;
common::ObISQLClient *sql_proxy_;
ObTabletTableOperator tablet_table_operator_;
uint64_t tenant_id_;
ObIArray<share::ObTabletLSPair> *tablet_ls_pairs_;
int64_t prefetch_tablet_idx_;
uint64_t tenant_id_;
common::ObArray<ObTabletInfo> prefetched_tablets_;
ObTabletReplicaFilterHolder filters_;
};
//TOOD @lixia merge ObTenantTabletMetaIterator and ObCompactionTabletMetaIterator
class ObTenantTabletMetaIterator
class ObCompactionTabletMetaIterator : public ObTabletMetaIterator
{
public:
ObCompactionTabletMetaIterator(
const bool first_check,
const int64_t compaction_scn);
~ObCompactionTabletMetaIterator() { reset(); }
int init(
const uint64_t tenant_id,
const int64_t batch_size,
share::ObIServerTrace &server_trace);
virtual void reset() override;
private:
virtual int prefetch() override;
const static int64_t TABLET_META_TABLE_RANGE_GET_SIZE = 1500;
bool first_check_;
int64_t compaction_scn_;
int64_t batch_size_;
ObTabletID end_tablet_id_;
};
class ObTenantTabletMetaIterator : public ObTabletMetaIterator
{
public:
ObTenantTabletMetaIterator();
virtual ~ObTenantTabletMetaIterator();
int init(common::ObISQLClient &sql_proxy,
const uint64_t tenant_id);
bool is_inited() const { return is_inited_; }
int next(ObTabletInfo &tablet_info);
ObTabletReplicaFilterHolder &get_filters() { return filters_; }
virtual void reset() override;
void set_batch_size(int64_t batch_size) {tablet_table_operator_.set_batch_size(batch_size);}
private:
int prefetch();
virtual int prefetch() override;
int prefetch_valid_tablet_ids();
int prefetch_sys_table_tablet_ids();
int prefetch_user_table_tablet_ids();
int prefetch_tablets();
private:
bool is_inited_;
common::ObISQLClient *sql_proxy_;
ObTabletTableOperator tablet_table_operator_;
uint64_t tenant_id_;
bool first_prefetch_;
common::ObArray<ObTabletInfo> prefetched_tablets_;
common::ObArray<ObTabletLSPair> valid_tablet_ls_pairs_;
common::ObISQLClient *sql_proxy_;
int64_t valid_tablet_ls_pairs_idx_;
int64_t prefetch_tablet_idx_;
ObTabletReplicaFilterHolder filters_;
common::ObArray<ObTabletLSPair> valid_tablet_ls_pairs_;
ObTabletTableOperator tablet_table_operator_;
private:
DISALLOW_COPY_AND_ASSIGN(ObTenantTabletMetaIterator);
};

View File

@ -210,7 +210,7 @@ int ObTabletTableOperator::inner_batch_get_tablet_by_sql_(
} else if (OB_ISNULL(result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get mysql result failed", KR(ret), "sql", sql.ptr());
} else if (OB_FAIL(construct_tablet_infos_(*result.get_result(), tablet_infos))) {
} else if (OB_FAIL(construct_tablet_infos(*result.get_result(), tablet_infos))) {
LOG_WARN("construct tablet info failed", KR(ret), K(tablet_infos));
}
}
@ -366,7 +366,7 @@ int ObTabletTableOperator::inner_batch_get_by_sql_(
} else if (OB_ISNULL(result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get mysql result failed", KR(ret), "sql", sql.ptr());
} else if (OB_FAIL(construct_tablet_infos_(*result.get_result(), tablet_infos))) {
} else if (OB_FAIL(construct_tablet_infos(*result.get_result(), tablet_infos))) {
LOG_WARN("construct tablet info failed", KR(ret), K(tablet_infos));
}
}
@ -374,7 +374,7 @@ int ObTabletTableOperator::inner_batch_get_by_sql_(
return ret;
}
int ObTabletTableOperator::construct_tablet_infos_(
int ObTabletTableOperator::construct_tablet_infos(
sqlclient::ObMySQLResult &res,
ObIArray<ObTabletInfo> &tablet_infos)
{
@ -655,7 +655,7 @@ int ObTabletTableOperator::range_get(
} else if (OB_ISNULL(result.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get mysql result failed", KR(ret), K(sql));
} else if (OB_FAIL(construct_tablet_infos_(*result.get_result(), tablet_infos))) {
} else if (OB_FAIL(construct_tablet_infos(*result.get_result(), tablet_infos))) {
LOG_WARN("construct tablet info failed", KR(ret), K(sql), K(tablet_infos));
} else if (OB_UNLIKELY(tablet_infos.count() > range_size)) {
ret = OB_ERR_UNEXPECTED;

View File

@ -136,6 +136,9 @@ public:
const ObAddr &server,
const int64_t limit,
int64_t &affected_rows);
static int construct_tablet_infos(
common::sqlclient::ObMySQLResult &res,
ObIArray<ObTabletInfo> &tablet_infos);
public:
static int batch_get_tablet_info(
common::ObISQLClient *sql_proxy,
@ -163,9 +166,6 @@ private:
const int64_t start_idx,
const int64_t end_idx,
common::ObISQLClient &sql_client);
static int construct_tablet_infos_(
common::sqlclient::ObMySQLResult &res,
ObIArray<ObTabletInfo> &tablet_infos);
static int construct_tablet_replica_(
common::sqlclient::ObMySQLResult &res,
ObTabletReplica &replica);

View File

@ -713,5 +713,39 @@ int ObTabletToLSTableOperator::inner_batch_get_(
return ret;
}
int ObTabletToLSTableOperator::get_tablet_ls_pairs_cnt(
common::ObISQLClient &sql_proxy,
const uint64_t tenant_id,
int64_t &input_cnt)
{
int ret = OB_SUCCESS;
ObSqlString sql;
input_cnt = 0;
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(tenant_id));
} else if (OB_FAIL(sql.append_fmt(
"select count(*) as cnt from %s",
OB_ALL_TABLET_TO_LS_TNAME))) {
LOG_WARN("failed to append fmt", K(ret), K(tenant_id));
} else {
common::sqlclient::ObMySQLResult *result = nullptr;
int64_t cnt = 0;
SMART_VAR(ObISQLClient::ReadResult, res) {
if (OB_FAIL(sql_proxy.read(res, tenant_id, sql.ptr()))) {
LOG_WARN("fail to do read", KR(ret), K(tenant_id), K(sql));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get result", KR(ret), K(tenant_id), K(sql));
} else if (OB_FAIL(result->get_int("cnt", cnt))) {
LOG_WARN("failed to get int", KR(ret), K(cnt));
} else {
input_cnt = cnt;
}
}
}
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -190,7 +190,10 @@ public:
const uint64_t tenant_id,
const ObIArray<common::ObTabletID> &tablet_ids,
ObIArray<ObTabletLSPair> &tablet_ls_pairs);
static int get_tablet_ls_pairs_cnt(
common::ObISQLClient &sql_proxy,
const uint64_t tenant_id,
int64_t &input_cnt);
const static int64_t MAX_BATCH_COUNT = 200;
private:
static int inner_batch_get_(

View File

@ -32,6 +32,16 @@ namespace schema
class ObMergeSchema;
}
}
/*
ObStaticDataStoreDesc : record static info
ObColDataStoreDesc : record column related info
ObDataStoreDesc : ObStaticDataStoreDesc & ObColDataStoreDesc ptr
ObWholeDataStoreDesc : ObStaticDataStoreDesc & ObColDataStoreDesc object
for compaction, ObDataStoreDesc will record a common ObStaticDataStoreDesc ptr and a special ObColDataStoreDesc object for each table/cg
for other situation, use ObWholeDataStoreDesc instead of ObDataStoreDesc
*/
namespace blocksstable {
class ObSSTableIndexBuilder;
struct ObSSTableBasicMeta;

View File

@ -25,6 +25,7 @@
#include "storage/ls/ob_ls.h"
#include "storage/tx_storage/ob_tenant_freezer.h"
#include "common/ob_tablet_id.h"
#include "storage/compaction/ob_tenant_tablet_scheduler.h"
namespace oceanbase
{
@ -819,9 +820,21 @@ int ObFreezer::tablet_freeze_with_rewrite_meta(const ObTabletID &tablet_id)
LOG_WARN("exist running mini compaction dag, try later", K(ret), K(ls_id), K(tablet_id));
} else if (OB_FAIL(get_ls_tablet_svr()->update_tablet_snapshot_version(tablet_id, freeze_snapshot_version.get_val_for_tx()))) {
LOG_WARN("failed to update tablet snapshot version", K(ret), K(ls_id), K(tablet_id), K(freeze_snapshot_version));
} else if (memtable_mgr->get_medium_info_recorder().get_max_saved_version() >= freeze_snapshot_version.get_val_for_tx()) {
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(compaction::ObTenantTabletScheduler::schedule_merge_dag(
ls_id,
*tablet,
MEDIUM_MERGE,
freeze_snapshot_version.get_val_for_tx()))) {
if (OB_SIZE_OVERFLOW != tmp_ret && OB_EAGAIN != tmp_ret) {
ret = tmp_ret;
LOG_WARN("failed to schedule medium merge dag", K(ret), K(ls_id), K(tablet_id));
}
} else {
TRANS_LOG(INFO, "[Freezer] memtable_mgr doesn't have memtable", K(ret), K(ls_id), K(tablet_id));
}
}
} else {
TRANS_LOG(INFO, "[Freezer] memtable_mgr has memtable", K(ret), K(ls_id), K(tablet_id));
}

View File

@ -5703,7 +5703,7 @@ int ObTablet::read_mds_table(common::ObIAllocator &allocator,
LOG_WARN("failed to traverse mds table", K(ret), K(ls_id), K(tablet_id));
} else if (!op.dumped()) {
ret = OB_EMPTY_RESULT;
LOG_INFO("read nothing from mds table", K(ret), K(ls_id), K(tablet_id));
LOG_DEBUG("read nothing from mds table", K(ret), K(ls_id), K(tablet_id));
}
}

View File

@ -92,6 +92,7 @@ void TestCompactionIter::test_iter(
{
LOG_INFO("test_iter", K(ls_cnt), K(max_batch_tablet_cnt), K(tablet_cnt_per_ls), K(error_tablet_idx), K(input_errno));
MockObCompactionScheduleIterator iter(max_batch_tablet_cnt);
iter.max_batch_tablet_cnt_ = max_batch_tablet_cnt;
iter.prepare_ls_id_array(ls_cnt);
iter.mock_tablet_id_cnt_ = tablet_cnt_per_ls;
iter.error_tablet_idx_ = error_tablet_idx;