[SCN] fix farm failure
This commit is contained in:
@ -163,8 +163,7 @@ ObIndexBuildTask::ObIndexBuildTask()
|
||||
: ObDDLTask(ObDDLType::DDL_CREATE_INDEX), index_table_id_(target_object_id_),
|
||||
is_unique_index_(false), is_global_index_(false), root_service_(nullptr), snapshot_held_(false),
|
||||
is_sstable_complete_task_submitted_(false), sstable_complete_request_time_(0), sstable_complete_ts_(0),
|
||||
check_unique_snapshot_(0), complete_sstable_job_ret_code_(INT64_MAX),
|
||||
redefinition_execution_id_(0), create_index_arg_()
|
||||
check_unique_snapshot_(0), complete_sstable_job_ret_code_(INT64_MAX), create_index_arg_()
|
||||
{
|
||||
|
||||
}
|
||||
@ -344,12 +343,14 @@ int ObIndexBuildTask::init(const ObDDLTaskRecord &task_record)
|
||||
index_table_id_ = index_table_id;
|
||||
schema_version_ = schema_version;
|
||||
snapshot_version_ = task_record.snapshot_version_;
|
||||
execution_id_ = task_record.execution_id_;
|
||||
task_status_ = static_cast<ObDDLTaskStatus>(task_record.task_status_);
|
||||
if (ObDDLTaskStatus::VALIDATE_CHECKSUM == task_status_) {
|
||||
sstable_complete_ts_ = ObTimeUtility::current_time();
|
||||
}
|
||||
task_id_ = task_record.task_id_;
|
||||
parent_task_id_ = task_record.parent_task_id_;
|
||||
ret_code_ = task_record.ret_code_;
|
||||
is_inited_ = true;
|
||||
}
|
||||
return ret;
|
||||
@ -536,10 +537,32 @@ int ObIndexBuildTask::hold_snapshot(const int64_t snapshot)
|
||||
} else {
|
||||
ObDDLService &ddl_service = root_service_->get_ddl_service();
|
||||
ObSEArray<ObTabletID, 2> tablet_ids;
|
||||
if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, object_id_, tablet_ids))) {
|
||||
ObSchemaGetterGuard schema_guard;
|
||||
const ObTableSchema *data_table_schema = nullptr;
|
||||
const ObTableSchema *index_table_schema = nullptr;
|
||||
ObMultiVersionSchemaService &schema_service = ObMultiVersionSchemaService::get_instance();
|
||||
bool need_acquire_lob = false;
|
||||
if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id_, schema_guard))) {
|
||||
LOG_WARN("get tenant schema guard failed", K(ret));
|
||||
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, object_id_, data_table_schema))) {
|
||||
LOG_WARN("get table schema failed", K(ret), K(object_id_));
|
||||
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, target_object_id_, index_table_schema))) {
|
||||
LOG_WARN("get table schema failed", K(ret), K(target_object_id_));
|
||||
} else if (OB_ISNULL(data_table_schema) || OB_ISNULL(index_table_schema)) {
|
||||
ret = OB_TABLE_NOT_EXIST;
|
||||
LOG_WARN("table not exist", K(ret), K(object_id_), K(target_object_id_), KP(data_table_schema), KP(index_table_schema));
|
||||
} else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, object_id_, tablet_ids))) {
|
||||
LOG_WARN("failed to get data table snapshot", K(ret));
|
||||
} else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, target_object_id_, tablet_ids))) {
|
||||
LOG_WARN("failed to get dest table snapshot", K(ret));
|
||||
} else if (OB_FAIL(check_need_acquire_lob_snapshot(data_table_schema, index_table_schema, need_acquire_lob))) {
|
||||
LOG_WARN("failed to check if need to acquire lob snapshot", K(ret));
|
||||
} else if (need_acquire_lob && data_table_schema->get_aux_lob_meta_tid() != OB_INVALID_ID &&
|
||||
OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, data_table_schema->get_aux_lob_meta_tid(), tablet_ids))) {
|
||||
LOG_WARN("failed to get data lob meta table snapshot", K(ret));
|
||||
} else if (need_acquire_lob && data_table_schema->get_aux_lob_piece_tid() != OB_INVALID_ID &&
|
||||
OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, data_table_schema->get_aux_lob_piece_tid(), tablet_ids))) {
|
||||
LOG_WARN("failed to get data lob piece table snapshot", K(ret));
|
||||
} else if (OB_FAIL(ddl_service.get_snapshot_mgr().batch_acquire_snapshot(
|
||||
ddl_service.get_sql_proxy(), SNAPSHOT_FOR_DDL, tenant_id_, schema_version_, snapshot_scn, nullptr, tablet_ids))) {
|
||||
LOG_WARN("batch acquire snapshot failed", K(ret), K(tablet_ids));
|
||||
@ -558,6 +581,9 @@ int ObIndexBuildTask::release_snapshot(const int64_t snapshot)
|
||||
} else {
|
||||
ObDDLService &ddl_service = root_service_->get_ddl_service();
|
||||
ObSEArray<ObTabletID, 2> tablet_ids;
|
||||
ObSchemaGetterGuard schema_guard;
|
||||
const ObTableSchema *data_table_schema = nullptr;
|
||||
ObMultiVersionSchemaService &schema_service = ObMultiVersionSchemaService::get_instance();
|
||||
if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, object_id_, tablet_ids))) {
|
||||
if (OB_TABLE_NOT_EXIST == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
@ -571,6 +597,29 @@ int ObIndexBuildTask::release_snapshot(const int64_t snapshot)
|
||||
LOG_WARN("failed to get dest table snapshot", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(schema_service.get_tenant_schema_guard(tenant_id_, schema_guard))) {
|
||||
LOG_WARN("get tenant schema guard failed", K(ret));
|
||||
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, object_id_, data_table_schema))) {
|
||||
LOG_WARN("get table schema failed", K(ret), K(object_id_));
|
||||
} else if (OB_ISNULL(data_table_schema)) {
|
||||
LOG_INFO("table not exist", K(ret), K(object_id_), K(target_object_id_), KP(data_table_schema));
|
||||
} else if (data_table_schema->get_aux_lob_meta_tid() != OB_INVALID_ID &&
|
||||
OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, data_table_schema->get_aux_lob_meta_tid(), tablet_ids))) {
|
||||
if (OB_TABLE_NOT_EXIST == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("failed to get data lob meta table snapshot", K(ret));
|
||||
}
|
||||
} else if ( data_table_schema->get_aux_lob_piece_tid() != OB_INVALID_ID &&
|
||||
OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, data_table_schema->get_aux_lob_piece_tid(), tablet_ids))) {
|
||||
if (OB_TABLE_NOT_EXIST == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("failed to get data lob piece table snapshot", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && tablet_ids.count() > 0 && OB_FAIL(batch_release_snapshot(snapshot, tablet_ids))) {
|
||||
LOG_WARN("batch relase snapshot failed", K(ret), K(tablet_ids));
|
||||
}
|
||||
@ -596,7 +645,7 @@ int ObIndexBuildTask::send_build_single_replica_request()
|
||||
target_object_id_,
|
||||
schema_version_,
|
||||
snapshot_version_,
|
||||
redefinition_execution_id_,
|
||||
execution_id_,
|
||||
trace_id_,
|
||||
parallelism_,
|
||||
root_service_);
|
||||
@ -626,7 +675,7 @@ int ObIndexBuildTask::check_build_single_replica(bool &is_end)
|
||||
} else if (OB_SUCCESS != complete_sstable_job_ret_code_) {
|
||||
ret = complete_sstable_job_ret_code_;
|
||||
LOG_WARN("sstable complete job has failed", K(ret), K(object_id_), K(index_table_id_));
|
||||
if (ObIDDLTask::in_ddl_retry_white_list(ret) || OB_REPLICA_NOT_READABLE == ret || OB_ERR_INSUFFICIENT_PX_WORKER == ret) {
|
||||
if (is_replica_build_need_retry(ret)) {
|
||||
// retry sql job by re-submit
|
||||
is_sstable_complete_task_submitted_ = false;
|
||||
complete_sstable_job_ret_code_ = INT64_MAX;
|
||||
@ -661,8 +710,9 @@ int ObIndexBuildTask::wait_data_complement()
|
||||
|
||||
// submit a job to complete sstable for the index table on snapshot_version
|
||||
if (OB_SUCC(ret) && !state_finished && !is_sstable_complete_task_submitted_) {
|
||||
redefinition_execution_id_ = ObTimeUtility::current_time();
|
||||
if (OB_FAIL(send_build_single_replica_request())) {
|
||||
if (OB_FAIL(push_execution_id())) {
|
||||
LOG_WARN("failed to push execution id", K(ret));
|
||||
} else if (OB_FAIL(send_build_single_replica_request())) {
|
||||
LOG_WARN("fail to send build single replica request", K(ret));
|
||||
}
|
||||
}
|
||||
@ -678,10 +728,9 @@ int ObIndexBuildTask::wait_data_complement()
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && state_finished) {
|
||||
uint64_t execution_id = OB_INVALID_ID;
|
||||
bool dummy_equal = false;
|
||||
if (OB_FAIL(ObDDLChecksumOperator::check_column_checksum(
|
||||
tenant_id_, redefinition_execution_id_, object_id_, index_table_id_, task_id_, dummy_equal, root_service_->get_sql_proxy()))) {
|
||||
tenant_id_, execution_id_, object_id_, index_table_id_, task_id_, dummy_equal, root_service_->get_sql_proxy()))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to check column checksum", K(ret), K(index_table_id_), K(object_id_), K(task_id_));
|
||||
state_finished = true;
|
||||
@ -743,6 +792,43 @@ int ObIndexBuildTask::check_need_verify_checksum(bool &need_verify)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObIndexBuildTask::check_need_acquire_lob_snapshot(const ObTableSchema *data_table_schema,
|
||||
const ObTableSchema *index_table_schema,
|
||||
bool &need_acquire)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
need_acquire = false;
|
||||
ObTableSchema::const_column_iterator iter = index_table_schema->column_begin();
|
||||
ObTableSchema::const_column_iterator iter_end = index_table_schema->column_end();
|
||||
for (; OB_SUCC(ret) && !need_acquire && iter != iter_end; iter++) {
|
||||
const ObColumnSchemaV2 *index_col = *iter;
|
||||
if (OB_ISNULL(index_col)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("column schema is null", K(ret));
|
||||
} else {
|
||||
const ObColumnSchemaV2 *col = data_table_schema->get_column_schema(index_col->get_column_id());
|
||||
if (OB_ISNULL(col)) {
|
||||
} else if (col->is_generated_column()) {
|
||||
ObSEArray<uint64_t, 8> ref_columns;
|
||||
if (OB_FAIL(col->get_cascaded_column_ids(ref_columns))) {
|
||||
STORAGE_LOG(WARN, "Failed to get cascaded column ids", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !need_acquire && i < ref_columns.count(); i++) {
|
||||
const ObColumnSchemaV2 *data_table_col = data_table_schema->get_column_schema(ref_columns.at(i));
|
||||
if (OB_ISNULL(data_table_col)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("column schema is null", K(ret));
|
||||
} else if (is_lob_v2(data_table_col->get_data_type())) {
|
||||
need_acquire = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
// verify column checksum between data table and index table
|
||||
int ObIndexBuildTask::verify_checksum()
|
||||
{
|
||||
@ -781,11 +867,11 @@ int ObIndexBuildTask::verify_checksum()
|
||||
|
||||
// send column checksum calculation request and wait finish, then verify column checksum
|
||||
if (OB_SUCC(ret) && !state_finished && check_unique_snapshot_ > 0) {
|
||||
static int64_t checksum_wait_timeout = max(OB_MAX_DDL_SINGLE_REPLICA_BUILD_TIMEOUT / 50, 3600L * 1000L * 1000L);
|
||||
static int64_t checksum_wait_timeout = 10 * 1000 * 1000L; // 10s
|
||||
bool is_column_checksum_ready = false;
|
||||
bool dummy_equal = false;
|
||||
if (!wait_column_checksum_ctx_.is_inited() && OB_FAIL(wait_column_checksum_ctx_.init(
|
||||
task_id_, tenant_id_, object_id_, index_table_id_, schema_version_, check_unique_snapshot_, redefinition_execution_id_, checksum_wait_timeout))) {
|
||||
task_id_, tenant_id_, object_id_, index_table_id_, schema_version_, check_unique_snapshot_, execution_id_, checksum_wait_timeout))) {
|
||||
LOG_WARN("init context of wait column checksum failed", K(ret), K(object_id_), K(index_table_id_));
|
||||
} else {
|
||||
if (OB_FAIL(wait_column_checksum_ctx_.try_wait(is_column_checksum_ready))) {
|
||||
@ -798,7 +884,7 @@ int ObIndexBuildTask::verify_checksum()
|
||||
// do nothing
|
||||
} else {
|
||||
if (OB_FAIL(ObDDLChecksumOperator::check_column_checksum(
|
||||
tenant_id_, redefinition_execution_id_, object_id_, index_table_id_, task_id_, dummy_equal, root_service_->get_sql_proxy()))) {
|
||||
tenant_id_, execution_id_, object_id_, index_table_id_, task_id_, dummy_equal, root_service_->get_sql_proxy()))) {
|
||||
if (OB_CHECKSUM_ERROR == ret && is_unique_index_) {
|
||||
ret = OB_ERR_DUPLICATED_UNIQUE_KEY;
|
||||
}
|
||||
@ -820,6 +906,7 @@ int ObIndexBuildTask::update_column_checksum_calc_status(
|
||||
const int ret_code)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_latest_execution_id = false;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("not init", K(ret));
|
||||
@ -844,6 +931,7 @@ int ObIndexBuildTask::update_complete_sstable_job_status(
|
||||
const int ret_code)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_latest_execution_id = false;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("not init", K(ret));
|
||||
@ -856,8 +944,10 @@ int ObIndexBuildTask::update_complete_sstable_job_status(
|
||||
} else if (snapshot_version != snapshot_version_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("snapshot version not match", K(ret), K(snapshot_version), K(snapshot_version_));
|
||||
} else if (execution_id != redefinition_execution_id_) {
|
||||
LOG_INFO("receive a mismatch execution result, ignore", K(execution_id), K(redefinition_execution_id_));
|
||||
} else if (OB_FAIL(check_is_latest_execution_id(execution_id, is_latest_execution_id))) {
|
||||
LOG_WARN("failed to check latest execution id", K(ret), K(execution_id));
|
||||
} else if (!is_latest_execution_id) {
|
||||
LOG_INFO("receive a mismatch execution result, ignore", K(execution_id), K(execution_id_));
|
||||
} else {
|
||||
complete_sstable_job_ret_code_ = ret_code;
|
||||
sstable_complete_ts_ = ObTimeUtility::current_time();
|
||||
|
||||
Reference in New Issue
Block a user