[SCN] fix farm failure
This commit is contained in:
@ -259,6 +259,8 @@ int ObDDLTask::convert_to_record(
|
||||
task_record.task_id_ = get_task_id();
|
||||
task_record.parent_task_id_ = get_parent_task_id();
|
||||
task_record.task_version_ = get_task_version();
|
||||
task_record.execution_id_ = get_execution_id();
|
||||
task_record.ret_code_ = get_ret_code();
|
||||
const ObString &ddl_stmt_str = get_ddl_stmt_str();
|
||||
if (serialize_param_size > 0) {
|
||||
char *buf = nullptr;
|
||||
@ -312,7 +314,8 @@ int ObDDLTask::switch_status(ObDDLTaskStatus new_status, const int ret_code)
|
||||
LOG_WARN("start transaction failed", K(ret));
|
||||
} else {
|
||||
int64_t table_task_status = 0;
|
||||
if (OB_FAIL(ObDDLTaskRecordOperator::select_for_update(trans, tenant_id_, task_id_, table_task_status))) {
|
||||
int64_t execution_id = 0;
|
||||
if (OB_FAIL(ObDDLTaskRecordOperator::select_for_update(trans, tenant_id_, task_id_, table_task_status, execution_id))) {
|
||||
LOG_WARN("select for update failed", K(ret), K(task_id_));
|
||||
} else if (old_status != task_status_) {
|
||||
ret = OB_EAGAIN;
|
||||
@ -328,6 +331,8 @@ int ObDDLTask::switch_status(ObDDLTaskStatus new_status, const int ret_code)
|
||||
} else if (OB_FAIL(ObDDLTaskRecordOperator::update_task_status(
|
||||
trans, tenant_id_, task_id_, static_cast<int64_t>(real_new_status)))) {
|
||||
LOG_WARN("update task status failed", K(ret), K(task_id_), K(new_status));
|
||||
} else if (OB_FAIL(ObDDLTaskRecordOperator::update_ret_code(trans, tenant_id_, task_id_, ret_code_))) {
|
||||
LOG_WARN("failed to update ret code", K(ret));
|
||||
}
|
||||
|
||||
bool commit = (OB_SUCCESS == ret);
|
||||
@ -336,8 +341,8 @@ int ObDDLTask::switch_status(ObDDLTaskStatus new_status, const int ret_code)
|
||||
ret = (OB_SUCCESS == ret) ? tmp_ret : ret;
|
||||
}
|
||||
if (OB_SUCC(ret) && old_status != real_new_status) {
|
||||
ROOTSERVICE_EVENT_ADD("ddl_scheduler", "switch_state", K_(tenant_id), K_(object_id), K_(target_object_id),
|
||||
"pre_state", old_status, "new_state", real_new_status, K_(snapshot_version));
|
||||
ROOTSERVICE_EVENT_ADD("ddl_scheduler", "switch_state", K_(tenant_id), K_(task_id), K_(object_id), K_(target_object_id),
|
||||
"new_state", real_new_status, K_(snapshot_version), ret_code_);
|
||||
task_status_ = real_new_status;
|
||||
}
|
||||
}
|
||||
@ -551,6 +556,136 @@ int ObDDLTask::batch_release_snapshot(
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDDLTask::check_is_latest_execution_id(const int64_t execution_id, bool &is_latest)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
is_latest = true;
|
||||
ObMySQLTransaction trans;
|
||||
ObRootService *root_service = nullptr;
|
||||
int64_t table_task_status = 0;
|
||||
int64_t table_execution_id = 0;
|
||||
if (OB_ISNULL(root_service = GCTX.root_service_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("error unexpected, root service must not be nullptr", K(ret));
|
||||
} else if (OB_FAIL(trans.start(&root_service->get_sql_proxy(), tenant_id_))) {
|
||||
LOG_WARN("start transaction failed", K(ret));
|
||||
} else {
|
||||
if (OB_FAIL(ObDDLTaskRecordOperator::select_for_update(trans, tenant_id_, task_id_, table_task_status, table_execution_id))) {
|
||||
LOG_WARN("select for update failed", K(ret), K(task_id_));
|
||||
} else if (table_execution_id > execution_id) {
|
||||
is_latest = false;
|
||||
}
|
||||
trans.end(false);// abort
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDDLTask::push_execution_id()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMySQLTransaction trans;
|
||||
ObRootService *root_service = nullptr;
|
||||
int64_t table_task_status = 0;
|
||||
int64_t table_execution_id = 0;
|
||||
if (OB_ISNULL(root_service = GCTX.root_service_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("error unexpected, root service must not be nullptr", K(ret));
|
||||
} else if (OB_FAIL(trans.start(&root_service->get_sql_proxy(), tenant_id_))) {
|
||||
LOG_WARN("start transaction failed", K(ret));
|
||||
} else {
|
||||
if (OB_FAIL(ObDDLTaskRecordOperator::select_for_update(trans, tenant_id_, task_id_, table_task_status, table_execution_id))) {
|
||||
LOG_WARN("select for update failed", K(ret), K(task_id_));
|
||||
} else if (OB_FAIL(ObDDLTaskRecordOperator::update_execution_id(trans, tenant_id_, task_id_, table_execution_id + 1))) {
|
||||
LOG_WARN("update task status failed", K(ret));
|
||||
} else {
|
||||
execution_id_ = table_execution_id + 1;
|
||||
}
|
||||
bool commit = (OB_SUCCESS == ret);
|
||||
int tmp_ret = trans.end(commit);
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
LOG_WARN("fail to end trans", K(tmp_ret));
|
||||
ret = (OB_SUCCESS == ret) ? tmp_ret : ret;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObDDLTask::calc_next_schedule_ts(int ret_code)
|
||||
{
|
||||
if (OB_TIMEOUT == ret_code) {
|
||||
const int64_t SEC = 1000000;
|
||||
delay_schedule_time_ = std::min(delay_schedule_time_ * 6/5 + SEC/10, 30*SEC);
|
||||
const int64_t max_dt = delay_schedule_time_;
|
||||
const int64_t min_dt = std::max(0L, max_dt - 3*SEC);
|
||||
next_schedule_ts_ = ObTimeUtility::current_time() + ObRandom::rand(min_dt, max_dt);
|
||||
} else {
|
||||
delay_schedule_time_ = 0;
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
// check if the current replica build task should be scheduled again.
|
||||
bool ObDDLTask::is_replica_build_need_retry(
|
||||
const int ret_code)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool need_retry = true;
|
||||
bool is_table_exist = false;
|
||||
ObSchemaGetterGuard schema_guard;
|
||||
if (ObIDDLTask::in_ddl_retry_white_list(ret_code)
|
||||
|| OB_REPLICA_NOT_READABLE == ret_code
|
||||
|| OB_ERR_INSUFFICIENT_PX_WORKER == ret_code) {
|
||||
// need retry.
|
||||
} else if (OB_TABLE_NOT_EXIST == ret_code) {
|
||||
// Sometimes, the tablet leader has not refreshed the latest schema.
|
||||
// Thus, check whether the table really does not exist.
|
||||
const ObTableSchema *table_schema = nullptr;
|
||||
if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(tenant_id_, schema_guard))) {
|
||||
LOG_WARN("get tenant schema guard failed", K(ret), K_(tenant_id));
|
||||
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, object_id_, table_schema))) {
|
||||
LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(object_id_));
|
||||
} else if (OB_ISNULL(table_schema)) {
|
||||
ret = OB_TABLE_NOT_EXIST;
|
||||
LOG_INFO("table schema not exist", K(ret), K(tenant_id_), K(object_id_));
|
||||
} else {
|
||||
if (ObDDLType::DDL_CHECK_CONSTRAINT == task_type_ || ObDDLType::DDL_ADD_NOT_NULL_COLUMN == task_type_) {
|
||||
// need retry.
|
||||
} else if (ObDDLType::DDL_FOREIGN_KEY_CONSTRAINT == task_type_) {
|
||||
// check whether the parent/child table does not exist.
|
||||
bool found = false;
|
||||
const ObIArray<ObForeignKeyInfo> &fk_infos = table_schema->get_foreign_key_infos();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !found && i < fk_infos.count(); ++i) {
|
||||
if (target_object_id_ != fk_infos.at(i).foreign_key_id_) {
|
||||
} else {
|
||||
found = true;
|
||||
if (OB_FAIL(schema_guard.check_table_exist(tenant_id_, fk_infos.at(i).parent_table_id_, is_table_exist))) {
|
||||
LOG_WARN("check schema exist failed", K(ret), K(tenant_id_), K(fk_infos.at(i)));
|
||||
} else if (!is_table_exist) {
|
||||
ret = OB_TABLE_NOT_EXIST;
|
||||
LOG_INFO("table schema not exist", K(ret), K(tenant_id_), K(object_id_), K(fk_infos.at(i)));
|
||||
} else if (OB_FAIL(schema_guard.check_table_exist(tenant_id_, fk_infos.at(i).child_table_id_, is_table_exist))) {
|
||||
LOG_WARN("check schema exist failed", K(ret), K(tenant_id_), K(fk_infos.at(i)));
|
||||
} else if (!is_table_exist) {
|
||||
ret = OB_TABLE_NOT_EXIST;
|
||||
LOG_INFO("table schema not exist", K(ret), K(tenant_id_), K(object_id_), K(fk_infos.at(i)));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(schema_guard.check_table_exist(tenant_id_, target_object_id_, is_table_exist))) {
|
||||
LOG_WARN("check table exist failed", K(ret), K(tenant_id_), K(target_object_id_));
|
||||
} else if (!is_table_exist) {
|
||||
ret = OB_TABLE_NOT_EXIST;
|
||||
LOG_WARN("not exist", K(ret), K(tenant_id_), K(target_object_id_));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// ret_code is not in some predefined error code list.
|
||||
need_retry = false;
|
||||
}
|
||||
need_retry = OB_TABLE_NOT_EXIST == ret ? false : need_retry;
|
||||
return need_retry;
|
||||
}
|
||||
|
||||
#ifdef ERRSIM
|
||||
int ObDDLTask::check_errsim_error()
|
||||
{
|
||||
@ -626,6 +761,160 @@ void ObDDLWaitTransEndCtx::reset()
|
||||
snapshot_array_.reset();
|
||||
}
|
||||
|
||||
struct SendItem final
|
||||
{
|
||||
public:
|
||||
bool operator < (const SendItem &other) const { return leader_addr_ < other.leader_addr_; }
|
||||
TO_STRING_KV(K_(leader_addr), K_(ls_id), K_(tablet_id), KP_(other_info));
|
||||
public:
|
||||
ObAddr leader_addr_;
|
||||
ObLSID ls_id_;
|
||||
ObTabletID tablet_id_;
|
||||
void *other_info_;
|
||||
};
|
||||
|
||||
int group_tablets_leader_addr(const uint64_t tenant_id, const ObIArray<ObTabletID> &tablet_ids, ObLocationService *location_service, ObArray<SendItem> &group_items)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
group_items.reuse();
|
||||
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || nullptr == location_service)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(tablet_ids.count()));
|
||||
} else {
|
||||
const int64_t rpc_timeout = max(GCONF.rpc_timeout, 1000L * 1000L * 9L);
|
||||
if (OB_FAIL(group_items.reserve(tablet_ids.count()))) {
|
||||
LOG_WARN("reserve send array failed", K(ret), K(tablet_ids.count()));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); ++i) {
|
||||
const ObTabletID &tablet_id = tablet_ids.at(i);
|
||||
SendItem item;
|
||||
if (OB_FAIL(ObDDLUtil::get_tablet_leader_addr(location_service,
|
||||
tenant_id,
|
||||
tablet_id,
|
||||
rpc_timeout,
|
||||
item.ls_id_,
|
||||
item.leader_addr_))) {
|
||||
LOG_WARN("get tablet leader addr failed", K(ret));
|
||||
} else if (FALSE_IT(item.tablet_id_ = tablet_id)) {
|
||||
} else if (OB_FAIL(group_items.push_back(item))) {
|
||||
LOG_WARN("push back send item failed", K(ret), K(item));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<typename Proxy, typename Arg, typename Res>
|
||||
int check_trans_end(const ObArray<SendItem> &send_array,
|
||||
Proxy &proxy,
|
||||
Arg &arg,
|
||||
Res *res,
|
||||
ObIArray<int> &ret_array,
|
||||
ObIArray<int64_t> &snapshot_array)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret_array.reuse();
|
||||
snapshot_array.reuse();
|
||||
hash::ObHashMap<obrpc::ObLSTabletPair, obrpc::ObCheckTransElapsedResult> result_map;
|
||||
ObArray<SendItem> tmp_send_array;
|
||||
if (OB_UNLIKELY(send_array.empty())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret));
|
||||
} else if (OB_FAIL(tmp_send_array.assign(send_array))) {
|
||||
LOG_WARN("copy send array failed", K(ret), K(send_array.count()));
|
||||
} else if (OB_FAIL(result_map.create(send_array.count(), "check_trans_map"))) {
|
||||
LOG_WARN("create return code map failed", K(ret));
|
||||
} else {
|
||||
// group by leader addr and send batch rpc
|
||||
std::sort(tmp_send_array.begin(), tmp_send_array.end());
|
||||
const int64_t rpc_timeout = max(GCONF.rpc_timeout, 1000L * 1000L * 9L);
|
||||
ObAddr last_addr;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tmp_send_array.count(); ++i) {
|
||||
const SendItem &send_item = tmp_send_array.at(i);
|
||||
if (send_item.leader_addr_ != last_addr) {
|
||||
if (arg.tablets_.count() > 0) {
|
||||
if (OB_FAIL(proxy.call(last_addr, rpc_timeout, arg.tenant_id_, arg))) {
|
||||
LOG_WARN("send rpc failed", K(ret), K(arg), K(last_addr), K(arg.tenant_id_));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
arg.tablets_.reuse();
|
||||
last_addr = send_item.leader_addr_;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ObLSTabletPair ls_tablet_pair;
|
||||
ls_tablet_pair.ls_id_ = send_item.ls_id_;
|
||||
ls_tablet_pair.tablet_id_ = send_item.tablet_id_;
|
||||
if (OB_FAIL(arg.tablets_.push_back(ls_tablet_pair))) {
|
||||
LOG_WARN("push back send item failed", K(ret), K(i), K(send_item));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && arg.tablets_.count() > 0) {
|
||||
if (OB_FAIL(proxy.call(last_addr, rpc_timeout, arg.tenant_id_, arg))) {
|
||||
LOG_WARN("send rpc failed", K(ret), K(arg), K(last_addr), K(arg.tenant_id_));
|
||||
}
|
||||
}
|
||||
|
||||
// collect result
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
common::ObArray<int> tmp_ret_array;
|
||||
if (OB_SUCCESS != (tmp_ret = proxy.wait_all(tmp_ret_array))) {
|
||||
LOG_WARN("rpc proxy wait failed", K(tmp_ret));
|
||||
ret = OB_SUCCESS == ret ? tmp_ret : ret;
|
||||
} else if (OB_SUCC(ret)) {
|
||||
const ObIArray<const Res *> &result_array = proxy.get_results();
|
||||
const ObIArray<Arg> &arg_array = proxy.get_args();
|
||||
const ObIArray<ObAddr> &dest_array = proxy.get_dests();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < result_array.count(); ++i) {
|
||||
const Res *cur_result = result_array.at(i);
|
||||
const Arg &cur_arg = arg_array.at(i);
|
||||
const ObAddr &cur_dest_addr = dest_array.at(i);
|
||||
if (OB_ISNULL(cur_result)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("result it null", K(ret), K(i), KP(cur_result));
|
||||
} else if (OB_FAIL(tmp_ret_array.at(i))) {
|
||||
LOG_WARN("check shema trans elapsed failed", K(ret), K(i), K(cur_dest_addr), K(cur_arg), KPC(cur_result));
|
||||
} else if (cur_arg.tablets_.count() != cur_result->results_.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("the result count does not match the argument", K(ret), K(cur_arg), KPC(cur_result));
|
||||
} else {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < cur_result->results_.count(); ++j) {
|
||||
const obrpc::ObLSTabletPair &send_item = cur_arg.tablets_.at(j);
|
||||
const obrpc::ObCheckTransElapsedResult &result_item = cur_result->results_.at(j);
|
||||
if (OB_FAIL(result_map.set_refactored(send_item, result_item))) {
|
||||
LOG_WARN("insert into result map failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ret_array.reserve(send_array.count()))) {
|
||||
LOG_WARN("reserve return code array failed", K(ret), K(send_array.count()));
|
||||
} else if (OB_FAIL(snapshot_array.reserve(send_array.count()))) {
|
||||
LOG_WARN("reserve snapshot array failed", K(ret), K(send_array.count()));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < send_array.count(); ++i) {
|
||||
const SendItem &send_item = send_array.at(i);
|
||||
ObLSTabletPair ls_tablet_pair;
|
||||
ls_tablet_pair.ls_id_ = send_item.ls_id_;
|
||||
ls_tablet_pair.tablet_id_ = send_item.tablet_id_;
|
||||
obrpc::ObCheckTransElapsedResult result_item;
|
||||
if (OB_FAIL(result_map.get_refactored(ls_tablet_pair, result_item))) {
|
||||
LOG_WARN("get result failed", K(ret), K(send_item));
|
||||
} else if (OB_FAIL(ret_array.push_back(result_item.ret_code_))) {
|
||||
LOG_WARN("push back return code failed", K(ret), K(send_item), K(result_item));
|
||||
} else if (OB_FAIL(snapshot_array.push_back(result_item.snapshot_))) {
|
||||
LOG_WARN("push back snapshot failed", K(ret), K(send_item), K(result_item));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDDLWaitTransEndCtx::check_schema_trans_end(
|
||||
const int64_t schema_version,
|
||||
const common::ObIArray<common::ObTabletID> &tablet_ids,
|
||||
@ -639,52 +928,22 @@ int ObDDLWaitTransEndCtx::check_schema_trans_end(
|
||||
int ret = OB_SUCCESS;
|
||||
ret_array.reset();
|
||||
snapshot_array.reset();
|
||||
ObArray<SendItem> send_array;
|
||||
if (OB_UNLIKELY(schema_version <= 0 || tablet_ids.count() <= 0 || OB_INVALID_ID == tenant_id
|
||||
|| nullptr == rpc_proxy || nullptr == location_service)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(schema_version), K(tablet_ids.count()), K(tenant_id), KP(rpc_proxy), KP(location_service));
|
||||
} else if (OB_FAIL(group_tablets_leader_addr(tenant_id, tablet_ids, location_service, send_array))) {
|
||||
LOG_WARN("group tablet by leader addr failed", K(ret), K(tenant_id), K(tablet_ids.count()));
|
||||
} else {
|
||||
ObCheckSchemaVersionElapsedProxy proxy(*rpc_proxy,
|
||||
&obrpc::ObSrvRpcProxy::check_schema_version_elapsed);
|
||||
ObCheckSchemaVersionElapsedProxy proxy(*rpc_proxy, &obrpc::ObSrvRpcProxy::check_schema_version_elapsed);
|
||||
obrpc::ObCheckSchemaVersionElapsedArg arg;
|
||||
const int64_t rpc_timeout = max(GCONF.rpc_timeout, 1000L * 1000L * 9L);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); ++i) {
|
||||
const ObTabletID &tablet_id = tablet_ids.at(i);
|
||||
arg.tenant_id_ = tenant_id;
|
||||
arg.data_tablet_id_ = tablet_id;
|
||||
arg.schema_version_ = schema_version;
|
||||
arg.need_wait_trans_end_ = need_wait_trans_end;
|
||||
ObAddr leader_addr;
|
||||
if (OB_FAIL(ObDDLUtil::get_tablet_leader_addr(location_service, tenant_id, tablet_id, rpc_timeout, arg.ls_id_, leader_addr))) {
|
||||
LOG_WARN("get tablet leader addr failed", K(ret));
|
||||
} else if (OB_FAIL(proxy.call(leader_addr, rpc_timeout, tenant_id, arg))) {
|
||||
LOG_WARN("send rpc failed", K(ret), K(arg), K(leader_addr), K(tenant_id));
|
||||
}
|
||||
}
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
common::ObArray<int> tmp_ret_array;
|
||||
if (OB_SUCCESS != (tmp_ret = proxy.wait_all(tmp_ret_array))) {
|
||||
LOG_WARN("rpc proxy wait failed", K(tmp_ret));
|
||||
ret = OB_SUCCESS == ret ? tmp_ret : ret;
|
||||
} else if (OB_SUCC(ret)) {
|
||||
const ObIArray<const obrpc::ObCheckSchemaVersionElapsedResult *> &result_array = proxy.get_results();
|
||||
if (tmp_ret_array.count() != tablet_ids.count() || result_array.count() != tablet_ids.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("result count not match", K(ret), K(tablet_ids.count()), K(tmp_ret_array.count()), K(result_array.count()));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < result_array.count(); ++i) {
|
||||
const ObTabletID &tablet_id = tablet_ids.at(i);
|
||||
const obrpc::ObCheckSchemaVersionElapsedResult *cur_result = result_array.at(i);
|
||||
if (OB_ISNULL(cur_result)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("result it null", K(ret), K(tablet_id), K(i), KP(cur_result));
|
||||
} else if (OB_FAIL(ret_array.push_back(tmp_ret_array.at(i)))) {
|
||||
LOG_WARN("push back ret code failed", K(ret), K(tablet_id), K(tmp_ret_array.at(i)));
|
||||
} else if (OB_FAIL(snapshot_array.push_back(cur_result->snapshot_))) {
|
||||
LOG_WARN("push back snapshot failed", K(ret), K(tablet_id), K(*cur_result));
|
||||
}
|
||||
}
|
||||
}
|
||||
obrpc::ObCheckSchemaVersionElapsedResult *res = nullptr;
|
||||
arg.tenant_id_ = tenant_id;
|
||||
arg.schema_version_ = schema_version;
|
||||
arg.need_wait_trans_end_ = need_wait_trans_end;
|
||||
if (OB_FAIL(check_trans_end(send_array, proxy, arg, res, ret_array, snapshot_array))) {
|
||||
LOG_WARN("check trans end failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -702,52 +961,22 @@ int ObDDLWaitTransEndCtx::check_sstable_trans_end(
|
||||
int ret = OB_SUCCESS;
|
||||
ret_array.reset();
|
||||
snapshot_array.reset();
|
||||
ObArray<SendItem> send_array;
|
||||
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || sstable_exist_ts <= 0 || tablet_ids.count() <= 0
|
||||
|| nullptr == rpc_proxy || nullptr == location_service)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(sstable_exist_ts), K(tablet_ids.count()),
|
||||
KP(rpc_proxy), KP(location_service));
|
||||
} else if (OB_FAIL(group_tablets_leader_addr(tenant_id, tablet_ids, location_service, send_array))) {
|
||||
LOG_WARN("group tablet by leader addr failed", K(ret), K(tenant_id), K(tablet_ids.count()));
|
||||
} else {
|
||||
ObCheckCtxCreateTimestampElapsedProxy proxy(*rpc_proxy,
|
||||
&obrpc::ObSrvRpcProxy::check_modify_time_elapsed);
|
||||
ObCheckCtxCreateTimestampElapsedProxy proxy(*rpc_proxy, &obrpc::ObSrvRpcProxy::check_modify_time_elapsed);
|
||||
obrpc::ObCheckModifyTimeElapsedArg arg;
|
||||
int64_t rpc_timeout = max(GCONF.rpc_timeout, 1000L * 1000L * 9L);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); ++i) {
|
||||
const ObTabletID &tablet_id = tablet_ids.at(i);
|
||||
arg.tenant_id_ = tenant_id;
|
||||
arg.tablet_id_ = tablet_id;
|
||||
arg.sstable_exist_ts_ = sstable_exist_ts;
|
||||
ObAddr leader_addr;
|
||||
if (OB_FAIL(ObDDLUtil::get_tablet_leader_addr(location_service, tenant_id, tablet_id, rpc_timeout, arg.ls_id_, leader_addr))) {
|
||||
LOG_WARN("get tablet leader addr failed", K(ret));
|
||||
} else if (OB_FAIL(proxy.call(leader_addr, rpc_timeout, tenant_id, arg))) {
|
||||
LOG_WARN("send rpc failed", K(ret), K(arg), K(leader_addr), K(tenant_id));
|
||||
}
|
||||
}
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
common::ObArray<int> tmp_ret_array;
|
||||
if (OB_SUCCESS != (tmp_ret = proxy.wait_all(tmp_ret_array))) {
|
||||
LOG_WARN("rpc proxy wait failed", K(tmp_ret));
|
||||
ret = OB_SUCCESS == ret ? tmp_ret : ret;
|
||||
} else if (OB_SUCC(ret)) {
|
||||
const auto &result_array = proxy.get_results();
|
||||
if (tmp_ret_array.count() != tablet_ids.count() || result_array.count() != tablet_ids.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("result count not match", K(ret), K(tablet_ids.count()), K(tmp_ret_array.count()), K(result_array.count()));
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < result_array.count(); ++i) {
|
||||
const ObTabletID &tablet_id = tablet_ids.at(i);
|
||||
const auto *cur_result = result_array.at(i);
|
||||
if (OB_ISNULL(cur_result)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("result it null", K(ret), K(tablet_id), K(i), KP(cur_result));
|
||||
} else if (OB_FAIL(ret_array.push_back(tmp_ret_array.at(i)))) {
|
||||
LOG_WARN("push back ret code failed", K(ret), K(tablet_id), K(tmp_ret_array.at(i)));
|
||||
} else if (OB_FAIL(snapshot_array.push_back(cur_result->snapshot_))) {
|
||||
LOG_WARN("push back snapshot failed", K(ret), K(tablet_id), K(*cur_result));
|
||||
}
|
||||
}
|
||||
}
|
||||
obrpc::ObCheckModifyTimeElapsedResult *res = nullptr;
|
||||
arg.tenant_id_ = tenant_id;
|
||||
arg.sstable_exist_ts_ = sstable_exist_ts;
|
||||
if (OB_FAIL(check_trans_end(send_array, proxy, arg, res, ret_array, snapshot_array))) {
|
||||
LOG_WARN("check trans end failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -768,6 +997,8 @@ int ObDDLWaitTransEndCtx::try_wait(bool &is_trans_end, int64_t &snapshot_version
|
||||
ObArray<int64_t> tablet_pos_indexes;
|
||||
if (OB_FAIL(get_snapshot_check_list(need_check_tablets, tablet_pos_indexes))) {
|
||||
LOG_WARN("get snapshot check list failed", K(ret));
|
||||
} else if (need_check_tablets.empty()) {
|
||||
is_trans_end_ = true;
|
||||
} else {
|
||||
const int64_t check_count = need_check_tablets.count();
|
||||
ObArray<int> ret_codes;
|
||||
@ -1125,6 +1356,48 @@ int ObDDLWaitColumnChecksumCtx::refresh_zombie_task()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int send_batch_calc_rpc(obrpc::ObSrvRpcProxy &rpc_proxy,
|
||||
const ObAddr &leader_addr,
|
||||
const ObCalcColumnChecksumRequestArg &arg,
|
||||
ObCalcColumnChecksumRequestRes &res,
|
||||
ObIArray<SendItem> &send_array,
|
||||
const int64_t group_start_idx,
|
||||
const int64_t group_end_idx,
|
||||
common::SpinRWLock &item_lock,
|
||||
int64_t &send_succ_count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const int64_t rpc_timeout = max(GCONF.rpc_timeout, 1000L * 1000L * 9L);
|
||||
if (OB_FAIL(rpc_proxy.to(leader_addr)
|
||||
.by(arg.tenant_id_)
|
||||
.timeout(rpc_timeout)
|
||||
.calc_column_checksum_request(arg, res))) {
|
||||
LOG_WARN("send rpc failed", K(ret), K(arg), K(leader_addr), K(arg.tenant_id_));
|
||||
} else if (res.ret_codes_.count() != arg.calc_items_.count() || res.ret_codes_.count() != (group_end_idx - group_start_idx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("return codes count not match the argument", K(ret), K(arg.calc_items_.count()),
|
||||
K(res.ret_codes_.count()), "group_count", group_end_idx - group_start_idx);
|
||||
} else {
|
||||
LOG_INFO("send checksum validation task", K(arg));
|
||||
SpinWLockGuard guard(item_lock);
|
||||
for (int64_t j = group_start_idx, k = 0; j < group_end_idx; ++j, ++k) { // ignore ret
|
||||
PartitionColChecksumStat *item = reinterpret_cast<PartitionColChecksumStat *>(send_array.at(j).other_info_);
|
||||
int ret_code = res.ret_codes_.at(k);
|
||||
if (OB_SUCCESS == ret_code) {
|
||||
item->snapshot_ = arg.snapshot_version_;
|
||||
item->col_checksum_stat_ = CCS_INVALID;
|
||||
++send_succ_count;
|
||||
} else if (OB_EAGAIN == ret_code || OB_HASH_EXIST == ret_code) { // ignore
|
||||
LOG_INFO("send checksum rpc not success", K(ret), KPC(item));
|
||||
} else {
|
||||
ret = OB_SUCCESS == ret ? ret_code : ret; // keep first error code
|
||||
LOG_WARN("fail to calc column checksum request", K(ret_code), K(arg), KPC(item));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDDLWaitColumnChecksumCtx::send_calc_rpc(int64_t &send_succ_count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -1143,6 +1416,7 @@ int ObDDLWaitColumnChecksumCtx::send_calc_rpc(int64_t &send_succ_count)
|
||||
} else {
|
||||
ObLSID ls_id;
|
||||
const int64_t rpc_timeout = max(GCONF.rpc_timeout, 1000L * 1000L * 9L);
|
||||
ObArray<SendItem> send_array;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stat_array_.count(); ++i) {
|
||||
PartitionColChecksumStat &item = stat_array_.at(i);
|
||||
ObAddr leader_addr;
|
||||
@ -1154,34 +1428,64 @@ int ObDDLWaitColumnChecksumCtx::send_calc_rpc(int64_t &send_succ_count)
|
||||
if (OB_FAIL(ObDDLUtil::get_tablet_leader_addr(location_service, tenant_id_, item.tablet_id_, rpc_timeout, ls_id, leader_addr))) {
|
||||
LOG_WARN("get tablet leader addr failed", K(ret));
|
||||
} else {
|
||||
ObCalcColumnChecksumRequestArg arg;
|
||||
arg.tenant_id_ = tenant_id_;
|
||||
arg.ls_id_ = ls_id;
|
||||
arg.tablet_id_ = item.tablet_id_;
|
||||
arg.target_table_id_ = target_table_id_;
|
||||
arg.schema_version_ = schema_version_;
|
||||
arg.execution_id_ = item.execution_id_;
|
||||
arg.snapshot_version_ = snapshot_version_;
|
||||
arg.source_table_id_ = source_table_id_;
|
||||
arg.calc_table_id_ = item.table_id_;
|
||||
arg.task_id_ = task_id_;
|
||||
if (OB_FAIL(root_service->get_rpc_proxy().to(leader_addr).by(tenant_id_).timeout(rpc_timeout).calc_column_checksum_request(arg))) {
|
||||
if (OB_EAGAIN == ret || OB_HASH_EXIST == ret) { // ignore
|
||||
LOG_INFO("send checksum rpc not success", K(ret), K(arg));
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to calc column checksum request", K(ret), K(arg));
|
||||
SendItem send_item;
|
||||
send_item.leader_addr_ = leader_addr;
|
||||
send_item.ls_id_ = ls_id;
|
||||
send_item.tablet_id_ = item.tablet_id_;
|
||||
send_item.other_info_ = reinterpret_cast<void *>(&item);
|
||||
if (OB_FAIL(send_array.push_back(send_item))) {
|
||||
LOG_WARN("push send array failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
// group by leader addr and send batch rpc
|
||||
std::sort(send_array.begin(), send_array.end());
|
||||
|
||||
ObAddr last_addr;
|
||||
int64_t group_start_idx = 0;
|
||||
ObCalcColumnChecksumRequestArg arg;
|
||||
ObCalcColumnChecksumRequestRes res;
|
||||
arg.tenant_id_ = tenant_id_;
|
||||
arg.task_id_ = task_id_;
|
||||
arg.source_table_id_ = source_table_id_;
|
||||
arg.target_table_id_ = target_table_id_;
|
||||
arg.schema_version_ = schema_version_;
|
||||
arg.execution_id_ = execution_id_;
|
||||
arg.snapshot_version_ = snapshot_version_;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < send_array.count(); ++i) {
|
||||
const SendItem &send_item = send_array.at(i);
|
||||
if (send_item.leader_addr_ != last_addr) {
|
||||
if (arg.calc_items_.count() > 0) {
|
||||
if (OB_FAIL(send_batch_calc_rpc(root_service->get_rpc_proxy(), last_addr,
|
||||
arg, res, send_array, group_start_idx, i, lock_, send_succ_count))) {
|
||||
LOG_WARN("send batch calc rpc failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
LOG_INFO("send checksum validation task", K(arg));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
SpinWLockGuard guard(lock_);
|
||||
item.snapshot_ = snapshot_version_;
|
||||
item.col_checksum_stat_ = CCS_INVALID;
|
||||
++send_succ_count;
|
||||
arg.calc_items_.reuse();
|
||||
res.ret_codes_.reuse();
|
||||
last_addr = send_item.leader_addr_;
|
||||
group_start_idx = i;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ObCalcColumnChecksumRequestArg::SingleItem calc_item;
|
||||
calc_item.ls_id_ = send_item.ls_id_;
|
||||
calc_item.tablet_id_ = send_item.tablet_id_;
|
||||
calc_item.calc_table_id_ = reinterpret_cast<PartitionColChecksumStat *>(send_item.other_info_)->table_id_;
|
||||
if (OB_FAIL(arg.calc_items_.push_back(calc_item))) {
|
||||
LOG_WARN("push back send item failed", K(ret), K(i), K(send_item));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && arg.calc_items_.count() > 0) {
|
||||
if (OB_FAIL(send_batch_calc_rpc(root_service->get_rpc_proxy(), last_addr,
|
||||
arg, res, send_array, group_start_idx, send_array.count(), lock_, send_succ_count))) {
|
||||
LOG_WARN("send batch calc rpc failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1199,7 +1503,9 @@ bool ObDDLTaskRecord::is_valid() const
|
||||
&& tenant_id_ > 0
|
||||
&& task_version_ > 0
|
||||
&& OB_INVALID_ID != object_id_
|
||||
&& schema_version_ > 0;
|
||||
&& schema_version_ > 0
|
||||
&& ret_code_ >= 0
|
||||
&& execution_id_ >= 0;
|
||||
return is_valid;
|
||||
}
|
||||
|
||||
@ -1217,6 +1523,8 @@ void ObDDLTaskRecord::reset()
|
||||
snapshot_version_ = 0;
|
||||
message_.reset();
|
||||
task_version_ = 0;
|
||||
ret_code_ = OB_SUCCESS;
|
||||
execution_id_ = 0;
|
||||
}
|
||||
|
||||
|
||||
@ -1269,26 +1577,75 @@ int ObDDLTaskRecordOperator::update_snapshot_version(
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDDLTaskRecordOperator::update_message(
|
||||
common::ObMySQLProxy &proxy,
|
||||
int ObDDLTaskRecordOperator::update_ret_code(
|
||||
common::ObISQLClient &sql_client,
|
||||
const uint64_t tenant_id,
|
||||
const int64_t task_id,
|
||||
const char *message)
|
||||
const int64_t ret_code)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlString sql_string;
|
||||
int64_t affected_rows = 0;
|
||||
if (OB_UNLIKELY(!proxy.is_inited() || nullptr == message
|
||||
|| 0 == strlen(message)
|
||||
|| strlen(message) >= ObDDLTaskRecord::MAX_MESSAGE_LENGTH
|
||||
if (OB_ISNULL(sql_client.get_pool()) || OB_UNLIKELY(task_id <= 0 || tenant_id <= 0)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arg", K(ret), K(tenant_id), K(task_id));
|
||||
} else if (OB_FAIL(sql_string.assign_fmt(" UPDATE %s SET ret_code=%lu WHERE task_id=%lu ",
|
||||
OB_ALL_DDL_TASK_STATUS_TNAME, ret_code, task_id))) {
|
||||
LOG_WARN("assign sql string failed", K(ret), K(ret_code), K(task_id));
|
||||
} else if (OB_FAIL(sql_client.write(tenant_id, sql_string.ptr(), affected_rows))) {
|
||||
LOG_WARN("update snapshot_version of ddl task record failed", K(ret), K(sql_string));
|
||||
} else if (OB_UNLIKELY(affected_rows < 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected affected_rows", K(ret), K(affected_rows));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDDLTaskRecordOperator::update_execution_id(
|
||||
common::ObISQLClient &sql_client,
|
||||
const uint64_t tenant_id,
|
||||
const int64_t task_id,
|
||||
const int64_t execution_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlString sql_string;
|
||||
int64_t affected_rows = 0;
|
||||
if (OB_ISNULL(sql_client.get_pool()) || OB_UNLIKELY(task_id <= 0 || tenant_id <= 0 || execution_id <= 0)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arg", K(ret), K(tenant_id), K(task_id));
|
||||
} else if (OB_FAIL(sql_string.assign_fmt(" UPDATE %s SET execution_id=%lu WHERE task_id=%lu ",
|
||||
OB_ALL_DDL_TASK_STATUS_TNAME, execution_id, task_id))) {
|
||||
LOG_WARN("assign sql string failed", K(ret), K(execution_id), K(task_id));
|
||||
} else if (OB_FAIL(sql_client.write(tenant_id, sql_string.ptr(), affected_rows))) {
|
||||
LOG_WARN("update snapshot_version of ddl task record failed", K(ret), K(sql_string));
|
||||
} else if (OB_UNLIKELY(affected_rows < 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected affected_rows", K(ret), K(affected_rows));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDDLTaskRecordOperator::update_message(
|
||||
common::ObISQLClient &proxy,
|
||||
const uint64_t tenant_id,
|
||||
const int64_t task_id,
|
||||
const ObString &message)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlString sql_string;
|
||||
ObSqlString message_string;
|
||||
int64_t affected_rows = 0;
|
||||
if (OB_UNLIKELY(message.empty()
|
||||
|| tenant_id <= 0 || task_id <= 0)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid argument", K(ret), K(proxy.is_inited()), K(tenant_id), K(task_id), K(message));
|
||||
} else if (OB_FAIL(sql_string.assign_fmt(" UPDATE %s SET message='%s' WHERE task_id=%lu",
|
||||
OB_ALL_DDL_TASK_STATUS_TNAME, message, task_id))) {
|
||||
LOG_WARN("assign sql string failed", K(ret), K(message));
|
||||
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(task_id), K(message));
|
||||
} else if (OB_FAIL(to_hex_str(message, message_string))) {
|
||||
LOG_WARN("append hex escaped string failed", K(ret));
|
||||
} else if (OB_FAIL(sql_string.assign_fmt(" UPDATE %s SET message=\"%.*s\" WHERE task_id=%lu",
|
||||
OB_ALL_DDL_TASK_STATUS_TNAME, static_cast<int>(message_string.length()), message_string.ptr(), task_id))) {
|
||||
LOG_WARN("assign sql string failed", K(ret), K(message_string));
|
||||
} else if (OB_FAIL(proxy.write(tenant_id, sql_string.ptr(), affected_rows))) {
|
||||
LOG_WARN("update message of ddl task record failed", K(ret), K(sql_string));
|
||||
LOG_WARN("update message of ddl task record failed", K(ret), K(sql_string), K(message_string));
|
||||
} else if (OB_UNLIKELY(affected_rows < 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected affected_rows", K(ret), K(affected_rows));
|
||||
@ -1332,7 +1689,7 @@ int ObDDLTaskRecordOperator::check_is_adding_constraint(
|
||||
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
||||
sqlclient::ObMySQLResult *result = NULL;
|
||||
if (OB_FAIL(sql_string.assign_fmt(" SELECT tenant_id, task_id, object_id, target_object_id, ddl_type, "
|
||||
"schema_version, parent_task_id, trace_id, status, snapshot_version, task_version,"
|
||||
"schema_version, parent_task_id, trace_id, status, snapshot_version, task_version, execution_id, "
|
||||
"UNHEX(ddl_stmt_str) as ddl_stmt_str_unhex, ret_code, UNHEX(message) as message_unhex FROM %s "
|
||||
"WHERE object_id = %" PRIu64 " && ddl_type IN (%d, %d, %d)", OB_ALL_VIRTUAL_DDL_TASK_STATUS_TNAME,
|
||||
object_id, DDL_CHECK_CONSTRAINT, DDL_FOREIGN_KEY_CONSTRAINT, DDL_ADD_NOT_NULL_COLUMN))) {
|
||||
@ -1424,7 +1781,7 @@ int ObDDLTaskRecordOperator::check_has_conflict_ddl(
|
||||
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
||||
sqlclient::ObMySQLResult *result = nullptr;
|
||||
if (OB_FAIL(sql_string.assign_fmt("SELECT tenant_id, task_id, object_id, target_object_id, ddl_type,"
|
||||
"schema_version, parent_task_id, trace_id, status, snapshot_version, task_version,"
|
||||
"schema_version, parent_task_id, trace_id, status, snapshot_version, task_version, execution_id,"
|
||||
"UNHEX(ddl_stmt_str) as ddl_stmt_str_unhex, ret_code, UNHEX(message) as message_unhex FROM %s "
|
||||
"WHERE tenant_id = %lu AND object_id = %lu", OB_ALL_VIRTUAL_DDL_TASK_STATUS_TNAME,
|
||||
tenant_id, table_id))) {
|
||||
@ -1480,7 +1837,7 @@ int ObDDLTaskRecordOperator::get_all_record(
|
||||
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
||||
sqlclient::ObMySQLResult *result = NULL;
|
||||
if (OB_FAIL(sql_string.assign_fmt(" SELECT tenant_id, task_id, object_id, target_object_id, ddl_type, "
|
||||
"schema_version, parent_task_id, trace_id, status, snapshot_version, task_version,"
|
||||
"schema_version, parent_task_id, trace_id, status, snapshot_version, task_version, execution_id, "
|
||||
"UNHEX(ddl_stmt_str) as ddl_stmt_str_unhex, ret_code, UNHEX(message) as message_unhex FROM %s ", OB_ALL_VIRTUAL_DDL_TASK_STATUS_TNAME))) {
|
||||
LOG_WARN("assign sql string failed", K(ret));
|
||||
} else if (OB_FAIL(proxy.read(res, sql_string.ptr()))) {
|
||||
@ -1549,11 +1906,11 @@ int ObDDLTaskRecordOperator::insert_record(
|
||||
} else if (OB_FAIL(to_hex_str(record.message_, message_string))) {
|
||||
LOG_WARN("append hex escaped string failed", K(ret));
|
||||
} else if (OB_FAIL(sql_string.assign_fmt(
|
||||
" INSERT INTO %s (task_id, parent_task_id, tenant_id, object_id, schema_version, target_object_id, ddl_type, trace_id, status, task_version, ddl_stmt_str, message) "
|
||||
" VALUES (%lu, %lu, %lu, %lu, %lu, %lu, %d, '%s', %ld, %lu, '%.*s', \"%.*s\") ",
|
||||
" INSERT INTO %s (task_id, parent_task_id, tenant_id, object_id, schema_version, target_object_id, ddl_type, trace_id, status, task_version, execution_id, ret_code, ddl_stmt_str, message) "
|
||||
" VALUES (%lu, %lu, %lu, %lu, %lu, %lu, %d, '%s', %ld, %lu, %lu, %lu, '%.*s', \"%.*s\") ",
|
||||
OB_ALL_DDL_TASK_STATUS_TNAME, record.task_id_, record.parent_task_id_,
|
||||
ObSchemaUtils::get_extract_tenant_id(record.tenant_id_, record.tenant_id_), record.object_id_, record.schema_version_,
|
||||
get_record_id(record.ddl_type_, record.target_object_id_), record.ddl_type_, trace_id_str, record.task_status_, record.task_version_,
|
||||
get_record_id(record.ddl_type_, record.target_object_id_), record.ddl_type_, trace_id_str, record.task_status_, record.task_version_, record.execution_id_, record.ret_code_,
|
||||
static_cast<int>(ddl_stmt_string.length()), ddl_stmt_string.ptr(), static_cast<int>(message_string.length()), message_string.ptr()))) {
|
||||
LOG_WARN("assign sql string failed", K(ret), K(record));
|
||||
} else if (OB_FAIL(proxy.write(record.tenant_id_, sql_string.ptr(), affected_rows))) {
|
||||
@ -1593,6 +1950,8 @@ int ObDDLTaskRecordOperator::fill_task_record(
|
||||
EXTRACT_INT_FIELD_MYSQL(*result_row, "status", task_record.task_status_, int64_t);
|
||||
EXTRACT_UINT_FIELD_MYSQL(*result_row, "snapshot_version", task_record.snapshot_version_, uint64_t);
|
||||
EXTRACT_INT_FIELD_MYSQL(*result_row, "task_version", task_record.task_version_, int64_t);
|
||||
EXTRACT_INT_FIELD_MYSQL(*result_row, "ret_code", task_record.ret_code_, int64_t);
|
||||
EXTRACT_INT_FIELD_MYSQL(*result_row, "execution_id", task_record.execution_id_, int64_t);
|
||||
EXTRACT_VARCHAR_FIELD_MYSQL(*result_row, "message_unhex", task_message);
|
||||
EXTRACT_VARCHAR_FIELD_MYSQL(*result_row, "ddl_stmt_str_unhex", ddl_stmt_str);
|
||||
if (OB_SUCC(ret)) {
|
||||
@ -1645,18 +2004,20 @@ int ObDDLTaskRecordOperator::select_for_update(
|
||||
common::ObMySQLTransaction &trans,
|
||||
const uint64_t tenant_id,
|
||||
const int64_t task_id,
|
||||
int64_t &task_status)
|
||||
int64_t &task_status,
|
||||
int64_t &execution_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlString sql_string;
|
||||
task_status = 0;
|
||||
execution_id = 0;
|
||||
if (OB_UNLIKELY(task_id <= 0 || tenant_id <= 0)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("not init", K(ret), K(tenant_id), K(task_id));
|
||||
} else {
|
||||
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
|
||||
sqlclient::ObMySQLResult *result = NULL;
|
||||
if (OB_FAIL(sql_string.assign_fmt("SELECT status FROM %s WHERE task_id = %lu FOR UPDATE",
|
||||
if (OB_FAIL(sql_string.assign_fmt("SELECT status, execution_id FROM %s WHERE task_id = %lu FOR UPDATE",
|
||||
OB_ALL_DDL_TASK_STATUS_TNAME, task_id))) {
|
||||
LOG_WARN("assign sql string failed", K(ret), K(task_id), K(tenant_id));
|
||||
} else if (OB_FAIL(trans.read(res, tenant_id, sql_string.ptr()))) {
|
||||
@ -1668,6 +2029,7 @@ int ObDDLTaskRecordOperator::select_for_update(
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
} else {
|
||||
EXTRACT_INT_FIELD_MYSQL(*result, "status", task_status, int64_t);
|
||||
EXTRACT_INT_FIELD_MYSQL(*result, "execution_id", execution_id, int64_t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user