cherry-pick from 3_1_x_release to 3.1_opensource_release
This commit is contained in:
parent
ad7be27692
commit
20c0cae3dd
@ -772,10 +772,10 @@ STAT_EVENT_ADD_DEF(OBSERVER_PARTITION_TABLE_UPDATER_PROCESS_TIME, "observer part
|
||||
STAT_EVENT_ADD_DEF(OBSERVER_PARTITION_TABLE_UPDATER_DROP_COUNT, "observer partition table updater drop task count",
|
||||
ObStatClassIds::OBSERVER, "observer partition table updater drop task count", 100004, false, true)
|
||||
STAT_EVENT_ADD_DEF(OBSERVER_PARTITION_TABLE_UPDATER_REPUT_COUNT,
|
||||
"observer partition table updater repurt to queue count", ObStatClassIds::OBSERVER,
|
||||
"observer partition table updater reput to queue count", ObStatClassIds::OBSERVER,
|
||||
"observer partition table updater reput to queue count", 100005, false, true)
|
||||
STAT_EVENT_ADD_DEF(OBSERVER_PARTITION_TABLE_UPDATER_FAIL_TIMES, "observer partition table updater execute fail times",
|
||||
ObStatClassIds::OBSERVER, "observer partition table updater fail times", 10006, false, true)
|
||||
ObStatClassIds::OBSERVER, "observer partition table updater fail times", 100006, false, true)
|
||||
STAT_EVENT_ADD_DEF(OBSERVER_PARTITION_TABLE_UPDATER_FINISH_COUNT,
|
||||
"observer partition table updater success execute count", ObStatClassIds::OBSERVER,
|
||||
"observer partition table updater execute success count", 100007, false, true)
|
||||
|
@ -461,23 +461,32 @@ int ObPartitionTableUpdater::process_barrier(const ObPTUpdateRoleTask& task, boo
|
||||
return OB_NOT_SUPPORTED;
|
||||
}
|
||||
|
||||
int ObPartitionTableUpdater::do_batch_execute(
|
||||
const common::ObIArray<ObPartitionReplica>& tasks, const common::ObRole new_role)
|
||||
int ObPartitionTableUpdater::do_batch_execute(const int64_t start_time,
|
||||
const common::ObIArray<ObPTUpdateRoleTask>& tasks, const common::ObIArray<ObPartitionReplica>& replicas,
|
||||
const common::ObRole new_role)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(tasks.count() <= 0)) {
|
||||
if (OB_UNLIKELY(tasks.count() <= 0 || replicas.count() <= 0)) {
|
||||
// empty task
|
||||
} else if (OB_UNLIKELY(nullptr == GCTX.pt_operator_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("pt operator is null", KR(ret));
|
||||
} else {
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(GCTX.pt_operator_->batch_report_partition_role(tasks, new_role))) {
|
||||
LOG_WARN("fail to batch report partition role", KR(ret), K(tasks));
|
||||
if (OB_FAIL(GCTX.pt_operator_->batch_report_partition_role(replicas, new_role))) {
|
||||
LOG_WARN("fail to batch report partition role", KR(ret), K(replicas));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("batch execute report role failed", KR(ret), "cnt", tasks.count());
|
||||
LOG_WARN("batch execute report role failed", KR(ret), "cnt", replicas.count());
|
||||
bool is_sys = is_sys_table(tasks.at(0).pkey_.get_table_id());
|
||||
(void)throttle(is_sys, ret, ObTimeUtility::current_time() - start_time, stopped_);
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
if (OB_SUCCESS != (tmp_ret = reput_to_queue(tasks))) {
|
||||
LOG_WARN("update info fail to reput to queue", KR(tmp_ret), K(tasks));
|
||||
} else {
|
||||
LOG_INFO("batch update partition table failed, reput to queue", K(tasks));
|
||||
}
|
||||
} else if (tasks.count() > 0) {
|
||||
LOG_INFO("batch execute report role success", KR(ret), "cnt", tasks.count());
|
||||
}
|
||||
@ -534,7 +543,7 @@ int ObPartitionTableUpdater::batch_process_tasks(const ObIArray<ObPTUpdateRoleTa
|
||||
// try to check if tenant has been dropped before execute, ignore ret code
|
||||
(void)check_if_tenant_has_been_dropped(tenant_id, tenant_dropped);
|
||||
}
|
||||
|
||||
const int64_t start_time = ObTimeUtility::current_time();
|
||||
if (OB_SUCC(ret) && !tenant_dropped && tasks.count() > 0) {
|
||||
ObArray<ObPTUpdateRoleTask> leader_tasks;
|
||||
ObArray<ObPTUpdateRoleTask> standby_leader_tasks;
|
||||
@ -583,37 +592,23 @@ int ObPartitionTableUpdater::batch_process_tasks(const ObIArray<ObPTUpdateRoleTa
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
int tmp_ret = do_batch_execute(leader_r_array, LEADER);
|
||||
if (OB_SUCCESS == tmp_ret) {
|
||||
// good
|
||||
} else if (OB_SUCCESS != (tmp_ret = reput_to_queue(leader_tasks))) {
|
||||
LOG_WARN("update info fail to reput to queue", KR(tmp_ret), K(leader_tasks));
|
||||
} else {
|
||||
LOG_INFO("batch update partition table failed, reput to queue", K(leader_tasks));
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
if (OB_SUCCESS != (tmp_ret = do_batch_execute(start_time, leader_tasks, leader_r_array, LEADER))) {
|
||||
LOG_WARN("batch report leader failed", KR(ret), K(start_time), K(leader_r_array));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
int tmp_ret = do_batch_execute(standby_leader_r_array, STANDBY_LEADER);
|
||||
if (OB_SUCCESS == tmp_ret) {
|
||||
// good
|
||||
} else if (OB_SUCCESS != (tmp_ret = reput_to_queue(standby_leader_tasks))) {
|
||||
LOG_WARN("update info fail to reput to queue", KR(tmp_ret), K(standby_leader_tasks));
|
||||
} else {
|
||||
LOG_INFO("batch update partition table failed, reput to queue", K(standby_leader_tasks));
|
||||
if (OB_SUCCESS !=
|
||||
(tmp_ret = do_batch_execute(start_time, standby_leader_tasks, standby_leader_r_array, STANDBY_LEADER))) {
|
||||
LOG_WARN("batch report standby leader failed", KR(ret), K(start_time), K(standby_leader_r_array));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
int tmp_ret = do_batch_execute(restore_leader_r_array, RESTORE_LEADER);
|
||||
if (OB_SUCCESS == tmp_ret) {
|
||||
// good
|
||||
} else if (OB_SUCCESS != (tmp_ret = reput_to_queue(restore_leader_tasks))) {
|
||||
LOG_WARN("update info fail to reput to queue", KR(tmp_ret), K(restore_leader_tasks));
|
||||
} else {
|
||||
LOG_INFO("batch update partition table failed, reput to queue", K(restore_leader_tasks));
|
||||
if (OB_SUCCESS !=
|
||||
(tmp_ret = do_batch_execute(start_time, restore_leader_tasks, restore_leader_r_array, RESTORE_LEADER))) {
|
||||
LOG_WARN("batch report restore leader failed", KR(ret), K(start_time), K(restore_leader_r_array));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
const bool is_sys = tasks.count() > 0 ? is_sys_table(tasks.at(0).pkey_.get_table_id()) : false;
|
||||
(void)throttle(is_sys, ret, ObTimeUtility::current_time() - start_time, stopped_);
|
||||
if (OB_SUCCESS != (tmp_ret = reput_to_queue(batch_tasks))) {
|
||||
LOG_WARN("update info failed to reput to queue", KR(tmp_ret), K(batch_tasks));
|
||||
} else {
|
||||
@ -637,6 +632,7 @@ int ObPartitionTableUpdater::batch_process_tasks(const ObIArray<ObPTUpdateTask>&
|
||||
DEBUG_SYNC(BEFORE_BATCH_PROCESS_TASK);
|
||||
ObSEArray<ObPTUpdateTask, UNIQ_TASK_QUEUE_BATCH_EXECUTE_NUM> tasks;
|
||||
bool skip_to_reput_tasks = false;
|
||||
const int64_t start_time = ObTimeUtility::current_time();
|
||||
ObCurTraceId::init(GCONF.self_addr_);
|
||||
if (OB_ISNULL(GCTX.pt_operator_) || OB_ISNULL(GCTX.ob_service_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -737,12 +733,12 @@ int ObPartitionTableUpdater::batch_process_tasks(const ObIArray<ObPTUpdateTask>&
|
||||
skip_to_reput_tasks = true;
|
||||
// execute leader or removed replica report tasks
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
if (OB_SUCCESS != (tmp_ret = do_batch_execute(leader_tasks, leader_replicas, true /*with role*/))) {
|
||||
if (OB_SUCCESS != (tmp_ret = do_batch_execute(start_time, leader_tasks, leader_replicas, true /*with role*/))) {
|
||||
LOG_WARN("batch execute leader replicas failed", K(ret), "cnt", leader_tasks.count());
|
||||
}
|
||||
// execute follower replica report tasks with role
|
||||
if (OB_SUCCESS !=
|
||||
(tmp_ret = do_batch_execute(with_role_report_tasks, with_role_report_replicas, true /*with role*/))) {
|
||||
if (OB_SUCCESS != (tmp_ret = do_batch_execute(
|
||||
start_time, with_role_report_tasks, with_role_report_replicas, true /*with role*/))) {
|
||||
LOG_WARN("batch execute with role replicas failed",
|
||||
K(ret),
|
||||
"cnt",
|
||||
@ -751,8 +747,9 @@ int ObPartitionTableUpdater::batch_process_tasks(const ObIArray<ObPTUpdateTask>&
|
||||
true);
|
||||
}
|
||||
// execute follower replica report tasks without role
|
||||
if (OB_SUCCESS != (tmp_ret = do_batch_execute(
|
||||
without_role_report_tasks, without_role_report_replicas, false /*without role*/))) {
|
||||
if (OB_SUCCESS !=
|
||||
(tmp_ret = do_batch_execute(
|
||||
start_time, without_role_report_tasks, without_role_report_replicas, false /*without role*/))) {
|
||||
LOG_WARN("batch execute without role replicas failed",
|
||||
K(ret),
|
||||
"cnt",
|
||||
@ -763,6 +760,8 @@ int ObPartitionTableUpdater::batch_process_tasks(const ObIArray<ObPTUpdateTask>&
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret) && !skip_to_reput_tasks) {
|
||||
const bool is_sys = tasks.count() > 0 ? is_sys_table(tasks.at(0).part_key_.get_table_id()) : false;
|
||||
(void)throttle(is_sys, ret, ObTimeUtility::current_time() - start_time, stopped_);
|
||||
int tmp_ret = reput_to_queue(batch_tasks);
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
LOG_ERROR("update info fail to reput to queue", K(ret), K(batch_tasks.count()));
|
||||
@ -797,8 +796,8 @@ int ObPartitionTableUpdater::check_if_tenant_has_been_dropped(const uint64_t ten
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPartitionTableUpdater::do_batch_execute(
|
||||
const ObIArray<ObPTUpdateTask>& tasks, const ObIArray<ObPartitionReplica>& replicas, const bool with_role)
|
||||
int ObPartitionTableUpdater::do_batch_execute(const int64_t start_time, const ObIArray<ObPTUpdateTask>& tasks,
|
||||
const ObIArray<ObPartitionReplica>& replicas, const bool with_role)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool skip_to_reput_tasks = false;
|
||||
@ -812,9 +811,8 @@ int ObPartitionTableUpdater::do_batch_execute(
|
||||
} else {
|
||||
DEBUG_SYNC(BEFORE_ASYNC_PT_UPDATE_TASK_EXECUTE);
|
||||
// need to push back to task queue if failed
|
||||
const bool is_sys = is_sys_table(tasks.at(0).part_key_.get_table_id());
|
||||
const uint64_t tenant_id = tasks.at(0).part_key_.get_tenant_id();
|
||||
const int64_t start_time = ObTimeUtility::current_time();
|
||||
const bool is_sys = is_sys_table(tasks.at(0).part_key_.get_table_id());
|
||||
if ((replicas.at(0).is_leader_like() && (replicas.at(0).need_force_full_report() || with_role)) ||
|
||||
replicas.at(0).is_remove_) {
|
||||
ObSEArray<ObPartitionReplica, 1> tmp_replicas;
|
||||
@ -868,10 +866,7 @@ int ObPartitionTableUpdater::do_batch_execute(
|
||||
ObTimeUtility::current_time() - start_time);
|
||||
EVENT_ADD(OBSERVER_PARTITION_TABLE_UPDATER_FINISH_COUNT, replicas.count());
|
||||
}
|
||||
int tmp_ret = throttle(is_sys, ret, ObTimeUtility::current_time() - start_time, stopped_);
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
LOG_WARN("throttle failed", K(tmp_ret));
|
||||
}
|
||||
(void)throttle(is_sys, ret, ObTimeUtility::current_time() - start_time, stopped_);
|
||||
DEBUG_SYNC(AFTER_ASYNC_PT_UPDATE_TASK_EXECUTE);
|
||||
}
|
||||
if (OB_FAIL(ret) && !skip_to_reput_tasks) {
|
||||
|
@ -228,9 +228,10 @@ private:
|
||||
int reput_to_queue(const common::ObIArray<ObPTUpdateRoleTask>& tasks);
|
||||
|
||||
int check_if_tenant_has_been_dropped(const uint64_t tenant_id, bool& has_dropped);
|
||||
int do_batch_execute(const common::ObIArray<ObPTUpdateTask>& tasks,
|
||||
int do_batch_execute(const int64_t start_time, const common::ObIArray<ObPTUpdateTask>& tasks,
|
||||
const common::ObIArray<share::ObPartitionReplica>& replicas, const bool with_role);
|
||||
int do_batch_execute(const common::ObIArray<share::ObPartitionReplica>& tasks, const common::ObRole new_role);
|
||||
int do_batch_execute(const int64_t start_time, const common::ObIArray<ObPTUpdateRoleTask>& tasks,
|
||||
const common::ObIArray<share::ObPartitionReplica>& replicas, const common::ObRole new_role);
|
||||
|
||||
private:
|
||||
bool inited_;
|
||||
|
@ -98,65 +98,42 @@ int ObPGPartitionMTUpdater::init()
|
||||
max_partition_cnt,
|
||||
"PGPTMTUp"))) {
|
||||
LOG_WARN("fail to init task queue", K(ret));
|
||||
} else if (is_running_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ObPGPartitionMTUpdater already running", K(ret));
|
||||
} else {
|
||||
is_inited_ = true;
|
||||
is_running_ = true;
|
||||
stopped_ = false;
|
||||
LOG_INFO("ObPGPartitionMTUpdater init success", K(lbt()));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObPGPartitionMTUpdater::reset()
|
||||
{
|
||||
is_inited_ = false;
|
||||
is_running_ = false;
|
||||
}
|
||||
|
||||
void ObPGPartitionMTUpdater::stop()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
LOG_WARN("ObPGPartitionMTUpdater has not been inited");
|
||||
} else if (!is_running_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ObPGPartitionMTUpdater already not running", K(ret));
|
||||
} else {
|
||||
(void)task_queue_.stop();
|
||||
is_running_ = false;
|
||||
stopped_ = true;
|
||||
LOG_INFO("ObPGPartitionMTUpdater stop success");
|
||||
}
|
||||
UNUSED(ret);
|
||||
}
|
||||
|
||||
void ObPGPartitionMTUpdater::wait()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
LOG_WARN("ObPGPartitionMTUpdater has not been inited");
|
||||
} else if (!is_running_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ObPGPartitionMTUpdater already not running", K(ret));
|
||||
} else {
|
||||
(void)task_queue_.wait();
|
||||
LOG_INFO("ObPGPartitionMTUpdater wait success");
|
||||
}
|
||||
UNUSED(ret);
|
||||
}
|
||||
|
||||
void ObPGPartitionMTUpdater::destroy()
|
||||
{
|
||||
if (is_inited_) {
|
||||
if (is_running_) {
|
||||
(void)task_queue_.stop();
|
||||
(void)task_queue_.wait();
|
||||
}
|
||||
is_inited_ = false;
|
||||
is_running_ = false;
|
||||
LOG_INFO("ObPGPartitionMTUpdater wait success");
|
||||
}
|
||||
stop();
|
||||
wait();
|
||||
is_inited_ = false;
|
||||
stopped_ = true;
|
||||
LOG_INFO("ObPGPartitionMTUpdater destroy success");
|
||||
}
|
||||
|
||||
ObPGPartitionMTUpdater& ObPGPartitionMTUpdater::get_instance()
|
||||
@ -355,11 +332,8 @@ int ObPGPartitionMTUpdater::batch_process_tasks(const ObIArray<ObPGPartitionMTUp
|
||||
}
|
||||
}
|
||||
|
||||
int tmp_ret = ObPartitionTableUpdater::throttle(
|
||||
false /*is system table*/, ret, ObTimeUtility::current_time() - start_time, is_running_);
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
LOG_WARN("throttle failed", K(tmp_ret));
|
||||
}
|
||||
(void)ObPartitionTableUpdater::throttle(
|
||||
false /*is system table*/, ret, ObTimeUtility::current_time() - start_time, stopped_);
|
||||
|
||||
// ignore retcode, reput into task queue
|
||||
if (OB_FAIL(ret) && !skip_to_reput_tasks) {
|
||||
|
@ -71,16 +71,13 @@ typedef ObUniqTaskQueue<ObPGPartitionMTUpdateTask, ObPGPartitionMTUpdater> Parti
|
||||
|
||||
class ObPGPartitionMTUpdater {
|
||||
public:
|
||||
ObPGPartitionMTUpdater()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
ObPGPartitionMTUpdater() : is_inited_(false), stopped_(false), task_queue_()
|
||||
{}
|
||||
virtual ~ObPGPartitionMTUpdater()
|
||||
{
|
||||
destroy();
|
||||
}
|
||||
int init();
|
||||
void reset();
|
||||
void stop();
|
||||
void wait();
|
||||
void destroy();
|
||||
@ -103,7 +100,7 @@ private:
|
||||
|
||||
private:
|
||||
bool is_inited_;
|
||||
bool is_running_;
|
||||
bool stopped_;
|
||||
PartitionMetaTableTaskQueue task_queue_;
|
||||
};
|
||||
|
||||
|
@ -139,7 +139,7 @@ int ObTenantShowTables::inner_get_next_row()
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SERVER_LOG(WARN, "table schema is NULL", K(ret), K(table_schema_idx_), K(tenant_id_), K(database_id_));
|
||||
} else if (table_schema->is_dropped_schema()) {
|
||||
// skip
|
||||
is_allow = false;
|
||||
} else {
|
||||
uint64_t cell_idx = 0;
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < col_count; ++j) {
|
||||
|
@ -1090,11 +1090,26 @@ int ObBootstrap::create_all_schema(ObDDLService& ddl_service, ObIArray<ObTableSc
|
||||
} else {
|
||||
int64_t begin = 0;
|
||||
int64_t batch_count = BATCH_INSERT_SCHEMA_CNT;
|
||||
const int64_t MAX_RETRY_TIMES = 3;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < table_schemas.count(); ++i) {
|
||||
if (table_schemas.count() == (i + 1) || (i + 1 - begin) >= batch_count) {
|
||||
if (OB_FAIL(batch_create_schema(ddl_service, table_schemas, begin, i + 1))) {
|
||||
LOG_WARN("batch create schema failed", K(ret), "table count", i + 1 - begin);
|
||||
} else {
|
||||
int64_t retry_times = 1;
|
||||
while (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(batch_create_schema(ddl_service, table_schemas, begin, i + 1))) {
|
||||
LOG_WARN("batch create schema failed", K(ret), "table count", i + 1 - begin);
|
||||
// bugfix:https://work.aone.alibaba-inc.com/issue/34030283
|
||||
if ((OB_SCHEMA_EAGAIN == ret || OB_ERR_WAIT_REMOTE_SCHEMA_REFRESH == ret) &&
|
||||
retry_times <= MAX_RETRY_TIMES) {
|
||||
retry_times++;
|
||||
ret = OB_SUCCESS;
|
||||
LOG_INFO("schema error while create table, need retry", KR(ret), K(retry_times));
|
||||
usleep(1 * 1000 * 1000L); // 1s
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
begin = i + 1;
|
||||
}
|
||||
}
|
||||
|
@ -2348,8 +2348,8 @@ int ObDDLOperator::alter_table_column(const ObTableSchema& origin_table_schema,
|
||||
new_table_schema.get_charset_type(),
|
||||
new_table_schema.get_collation_type()))) {
|
||||
RS_LOG(WARN, "failed to fill column charset info");
|
||||
} else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(*alter_column_schema,
|
||||
origin_table_schema.get_table_id()))) {
|
||||
} else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(
|
||||
*alter_column_schema, origin_table_schema.get_table_id()))) {
|
||||
RS_LOG(WARN, "failed to check text or blob column length");
|
||||
}
|
||||
} else if (ObEnumSetTC == col_tc) {
|
||||
@ -2569,8 +2569,8 @@ int ObDDLOperator::alter_table_column(const ObTableSchema& origin_table_schema,
|
||||
new_table_schema.get_charset_type(),
|
||||
new_table_schema.get_collation_type()))) {
|
||||
RS_LOG(WARN, "failed to fill column charset info");
|
||||
} else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(*alter_column_schema,
|
||||
origin_table_schema.get_table_id()))) {
|
||||
} else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(
|
||||
*alter_column_schema, origin_table_schema.get_table_id()))) {
|
||||
RS_LOG(WARN, "failed to check text or blob column length");
|
||||
}
|
||||
}
|
||||
@ -2777,8 +2777,8 @@ int ObDDLOperator::alter_table_column(const ObTableSchema& origin_table_schema,
|
||||
new_table_schema.get_charset_type(),
|
||||
new_table_schema.get_collation_type()))) {
|
||||
RS_LOG(WARN, "failed to fill column charset info");
|
||||
} else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(*alter_column_schema,
|
||||
origin_table_schema.get_table_id()))) {
|
||||
} else if (OB_FAIL(ObDDLResolver::check_text_column_length_and_promote(
|
||||
*alter_column_schema, origin_table_schema.get_table_id()))) {
|
||||
RS_LOG(WARN, "failed to check text or blob column length");
|
||||
}
|
||||
}
|
||||
@ -4020,7 +4020,8 @@ int ObDDLOperator::drop_table_for_inspection(const ObTableSchema& orig_table_sch
|
||||
} else if (OB_FAIL(schema_service->get_table_sql_service().drop_table_for_inspection(
|
||||
trans, orig_table_schema, new_schema_version))) {
|
||||
LOG_WARN("drop table for inspection failed", K(ret));
|
||||
} else if (orig_table_schema.is_in_recyclebin()) {
|
||||
} else if (OB_RECYCLEBIN_SCHEMA_ID == extract_pure_id(orig_table_schema.get_database_id())) {
|
||||
// FIXME: remove [!is_dropped_schema()] from ObSimpleTableSchemaV2::is_in_recyclebin()
|
||||
ObRecycleObject::RecycleObjType recycle_type = ObRecycleObject::get_type_by_table_schema(orig_table_schema);
|
||||
const ObRecycleObject* recycle_obj = NULL;
|
||||
ObArray<ObRecycleObject> recycle_objs;
|
||||
@ -5163,9 +5164,10 @@ int ObDDLOperator::check_is_delay_delete(const int64_t tenant_id, bool& is_delay
|
||||
if (OB_SUCC(ret)) {
|
||||
int64_t reserved_schema_version = -1;
|
||||
ObBackupInfoMgr& bk_info = ObBackupInfoMgr::get_instance();
|
||||
|
||||
if (OB_FAIL(bk_info.get_delay_delete_schema_version(
|
||||
tenant_id, schema_service_, is_delay_delete, reserved_schema_version))) {
|
||||
if (OB_SYS_TENANT_ID == tenant_id) {
|
||||
// sys tenant won't backup or restore
|
||||
} else if (OB_FAIL(bk_info.get_delay_delete_schema_version(
|
||||
tenant_id, schema_service_, is_delay_delete, reserved_schema_version))) {
|
||||
LOG_WARN("get delay delete snaptshot version failed", KR(ret));
|
||||
}
|
||||
}
|
||||
|
@ -283,6 +283,7 @@ int ObDDLService::get_tenant_schema_guard_with_version_in_inner_table(
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_standby = false;
|
||||
bool is_restore = false;
|
||||
int64_t version_in_inner_table = OB_INVALID_VERSION;
|
||||
ObRefreshSchemaStatus schema_status;
|
||||
bool use_local = false;
|
||||
@ -294,7 +295,9 @@ int ObDDLService::get_tenant_schema_guard_with_version_in_inner_table(
|
||||
} else if (OB_ISNULL(schema_service_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("schema_service is null", K(ret));
|
||||
} else if (is_standby && OB_SYS_TENANT_ID != tenant_id) {
|
||||
} else if (OB_FAIL(schema_service_->check_tenant_is_restore(NULL, tenant_id, is_restore))) {
|
||||
LOG_WARN("fail to check tenant is restore", KR(ret), K(tenant_id));
|
||||
} else if ((is_standby && OB_SYS_TENANT_ID != tenant_id) || is_restore) {
|
||||
ObSchemaStatusProxy* schema_status_proxy = GCTX.schema_status_proxy_;
|
||||
if (OB_ISNULL(schema_status_proxy)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -302,8 +305,11 @@ int ObDDLService::get_tenant_schema_guard_with_version_in_inner_table(
|
||||
} else if (OB_FAIL(schema_status_proxy->get_refresh_schema_status(tenant_id, schema_status))) {
|
||||
LOG_WARN("failed to get tenant refresh schema status", KR(ret), K(tenant_id));
|
||||
} else if (OB_INVALID_VERSION == schema_status.readable_schema_version_) {
|
||||
// Although it is a standalone cluster, the schema status has been reset, and the internal table can be refreshed.
|
||||
// At this time, the standby database already has a leader
|
||||
// 1. For standby cluster: schema_status is reset, we can refresh schema now.
|
||||
// 2. For restore tenant: sys replicas are restored.
|
||||
} else if (is_restore) {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_WARN("Can't refresh schema when sys replicas are not restored yet", KR(ret), K(tenant_id));
|
||||
} else if (OB_FAIL(schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
|
||||
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
|
||||
} else {
|
||||
@ -14049,6 +14055,28 @@ int ObDDLService::drop_tenant(const ObDropTenantArg& arg)
|
||||
LOG_WARN("ddl_operator drop_tenant failed", K(tenant_id), KR(ret));
|
||||
} else if (OB_FAIL(ddl_operator.drop_restore_point(tenant_id, trans))) {
|
||||
LOG_WARN("fail to drop restore point", K(ret), K(tenant_id));
|
||||
} else if (tenant_schema->is_in_recyclebin()) {
|
||||
// try recycle record from __all_recyclebin
|
||||
ObArray<ObRecycleObject> recycle_objs;
|
||||
ObSchemaService* schema_service_impl = NULL;
|
||||
if (OB_ISNULL(schema_service_) || OB_ISNULL(schema_service_->get_schema_service())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("schema service is null", KR(ret), KP_(schema_service));
|
||||
} else if (FALSE_IT(schema_service_impl = schema_service_->get_schema_service())) {
|
||||
} else if (OB_FAIL(schema_service_impl->fetch_recycle_object(OB_SYS_TENANT_ID,
|
||||
tenant_schema->get_tenant_name_str(),
|
||||
ObRecycleObject::TENANT,
|
||||
trans,
|
||||
recycle_objs))) {
|
||||
LOG_WARN("get_recycle_object failed", KR(ret), KPC(tenant_schema));
|
||||
} else if (0 == recycle_objs.size()) {
|
||||
// skip
|
||||
} else if (1 < recycle_objs.size()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("records should not be more than 1", KR(ret), KPC(tenant_schema), K(recycle_objs));
|
||||
} else if (OB_FAIL(schema_service_impl->delete_recycle_object(OB_SYS_TENANT_ID, recycle_objs.at(0), trans))) {
|
||||
LOG_WARN("delete_recycle_object failed", KR(ret), KPC(tenant_schema));
|
||||
}
|
||||
}
|
||||
} else { // put tenant into recyclebin
|
||||
ObTenantSchema new_tenant_schema = *tenant_schema;
|
||||
@ -18810,12 +18838,14 @@ int ObDDLService::drop_outline(const obrpc::ObDropOutlineArg& arg)
|
||||
if (database_name == OB_OUTLINE_DEFAULT_DATABASE_NAME) {
|
||||
database_id = OB_OUTLINE_DEFAULT_DATABASE_ID;
|
||||
database_exist = true;
|
||||
} else if (OB_FAIL(schema_service_->check_database_exist(tenant_id,
|
||||
database_name,
|
||||
database_id,
|
||||
database_exist))) {
|
||||
LOG_WARN("failed to check database exist!", K(tenant_id), K(database_name),
|
||||
K(database_id), K(database_exist), K(ret));
|
||||
} else if (OB_FAIL(
|
||||
schema_service_->check_database_exist(tenant_id, database_name, database_id, database_exist))) {
|
||||
LOG_WARN("failed to check database exist!",
|
||||
K(tenant_id),
|
||||
K(database_name),
|
||||
K(database_id),
|
||||
K(database_exist),
|
||||
K(ret));
|
||||
} else if (!database_exist) {
|
||||
ret = OB_ERR_BAD_DATABASE;
|
||||
LOG_USER_ERROR(OB_ERR_BAD_DATABASE, database_name.length(), database_name.ptr());
|
||||
|
@ -984,6 +984,7 @@ int ObForceDropSchemaChecker::force_drop_schema(
|
||||
const uint64_t tenant_id, const int64_t recycle_schema_version, int64_t& task_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
DEBUG_SYNC(BEFORE_FORCE_DROP_SCHEMA);
|
||||
// drop schema according to the dependency of schema:
|
||||
// 1. data table schema is dropped defore index schema.
|
||||
// 2. table schema in tablegroup is dropped before tablegroup schema.
|
||||
|
@ -10158,7 +10158,22 @@ int ObRootService::get_tenant_schema_versions(
|
||||
ObRefreshSchemaStatus schema_status;
|
||||
schema_status.tenant_id_ = GCTX.is_schema_splited() ? tenant_id : OB_INVALID_TENANT_ID;
|
||||
int64_t version_in_inner_table = OB_INVALID_VERSION;
|
||||
if (OB_FAIL(schema_service_->get_schema_version_in_inner_table(
|
||||
bool is_restore = false;
|
||||
if (OB_FAIL(schema_service_->check_tenant_is_restore(&schema_guard, tenant_id, is_restore))) {
|
||||
LOG_WARN("fail to check tenant is restore", KR(ret), K(tenant_id));
|
||||
} else if (is_restore) {
|
||||
ObSchemaStatusProxy* schema_status_proxy = GCTX.schema_status_proxy_;
|
||||
if (OB_ISNULL(schema_status_proxy)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("schema_status_proxy is null", KR(ret));
|
||||
} else if (OB_FAIL(schema_status_proxy->get_refresh_schema_status(tenant_id, schema_status))) {
|
||||
LOG_WARN("failed to get tenant refresh schema status", KR(ret), K(tenant_id));
|
||||
} else if (OB_INVALID_VERSION != schema_status.readable_schema_version_) {
|
||||
ret = OB_EAGAIN;
|
||||
LOG_WARN("tenant's sys replicas are not restored yet, try later", KR(ret), K(tenant_id));
|
||||
}
|
||||
}
|
||||
if (FAILEDx(schema_service_->get_schema_version_in_inner_table(
|
||||
sql_proxy_, schema_status, version_in_inner_table))) {
|
||||
// failed tenant creation, inner table is empty, return OB_CORE_SCHEMA_VERSION
|
||||
if (OB_EMPTY_RESULT == ret) {
|
||||
|
@ -71,7 +71,6 @@ static const char* job_type_str_array[JOB_TYPE_MAX] = {NULL,
|
||||
"STOP_SCHEMA_SPLIT",
|
||||
"STOP_UPGRADE_STORAGE_FORMAT_VERSION",
|
||||
"MIGRATE_PARTITION_META_TABLE",
|
||||
"STOP_MIGRATE_PARTITION_META_TABLE"
|
||||
"STOP_MIGRATE_PARTITION_META_TABLE",
|
||||
"STATISTIC_PRIMARY_ZONE_ENTITY_COUNT",
|
||||
"CREATE_HA_GTS_UTIL",
|
||||
|
@ -268,6 +268,7 @@ class ObString;
|
||||
ACT(BEFORE_DO_MINOR_FREEZE, ) \
|
||||
ACT(BEFORE_UPDATE_RESTORE_FLAG_RESTORE_LOG, ) \
|
||||
ACT(SLOW_TXN_DURING_2PC_COMMIT_PHASE_FOR_PHYSICAL_BACKUP_1055, ) \
|
||||
ACT(BEFORE_FORCE_DROP_SCHEMA, ) \
|
||||
ACT(MAX_DEBUG_SYNC_POINT, )
|
||||
|
||||
DECLARE_ENUM(ObDebugSyncPoint, debug_sync_point, OB_DEBUG_SYNC_POINT_DEF);
|
||||
|
@ -179,7 +179,7 @@ int ObSSTableDataChecksumOperator::need_verify_checksum(const ObTableSchema& tab
|
||||
} else {
|
||||
// when each partition has at lease one replica which finished compaction with version global_snapshot_version,
|
||||
// we will validate checksum
|
||||
bool check_dropped_schema = true;
|
||||
bool check_dropped_schema = false;
|
||||
ObTablePartitionKeyIter part_iter(table_schema, check_dropped_schema);
|
||||
const int64_t part_num = part_iter.get_partition_num();
|
||||
ObPartitionKey pkey;
|
||||
|
@ -820,6 +820,12 @@ DEF_TIME(location_cache_refresh_min_interval, OB_CLUSTER_PARAMETER, "100ms", "[0
|
||||
"the time interval in which no request for location cache renewal will be executed. "
|
||||
"The default value is 100 milliseconds. [0s, +∞)",
|
||||
ObParameterAttr(Section::LOCATION_CACHE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
DEF_TIME(location_cache_refresh_rpc_timeout, OB_CLUSTER_PARAMETER, "500ms", "[1ms,)",
|
||||
"The timeout used for refreshing location cache by RPC. Range: [1ms, +∞)",
|
||||
ObParameterAttr(Section::LOCATION_CACHE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
DEF_TIME(location_cache_refresh_sql_timeout, OB_CLUSTER_PARAMETER, "1s", "[1ms,)",
|
||||
"The timeout used for refreshing location cache by SQL. Range: [1ms, +∞)",
|
||||
ObParameterAttr(Section::LOCATION_CACHE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
DEF_STR(all_server_list, OB_CLUSTER_PARAMETER, "", "all server addr in cluster",
|
||||
ObParameterAttr(Section::LOCATION_CACHE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
|
||||
@ -1435,6 +1441,5 @@ DEF_BOOL(_auto_drop_tenant_if_restore_failed, OB_CLUSTER_PARAMETER, "True",
|
||||
"auto drop restoring tenant if physical restore fails",
|
||||
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
DEF_BOOL(ob_proxy_readonly_transaction_routing_policy, OB_TENANT_PARAMETER, "true",
|
||||
"Proxy route policy for readonly sql",
|
||||
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
|
||||
"Proxy route policy for readonly sql",
|
||||
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
|
||||
|
@ -637,7 +637,7 @@ int ObLocationFetcher::fetch_vtable_location(const uint64_t table_id, ObSArray<O
|
||||
LOG_WARN("table is not virtual table", KT(table_id), K(ret));
|
||||
} else {
|
||||
LOG_INFO("fetch virtual table location with tenant", "tenant", THIS_WORKER.get_rpc_tenant(), KT(table_id));
|
||||
int64_t timeout_us = OB_FETCH_LOCATION_TIMEOUT;
|
||||
int64_t timeout_us = GCONF.location_cache_refresh_sql_timeout;
|
||||
if (ObTimeoutCtx::get_ctx().is_timeout_set()) {
|
||||
timeout_us = std::min(timeout_us, ObTimeoutCtx::get_ctx().get_timeout());
|
||||
}
|
||||
@ -718,7 +718,7 @@ int ObLocationFetcher::batch_renew_sys_table_location_by_rpc(const ObPartitionLo
|
||||
} else {
|
||||
results.reset();
|
||||
obrpc::ObBatchGetRoleArg arg;
|
||||
const int64_t timeout_ts = OB_FETCH_MEMBER_LIST_AND_LEADER_TIMEOUT;
|
||||
const int64_t timeout_ts = GCONF.location_cache_refresh_rpc_timeout;
|
||||
rootserver::ObBatchGetRoleProxy proxy(*srv_rpc_proxy_, &obrpc::ObSrvRpcProxy::batch_get_role);
|
||||
const ObIArray<ObReplicaLocation>& location_array = core_table_location.get_replica_locations();
|
||||
/*
|
||||
@ -805,7 +805,7 @@ int ObLocationFetcher::renew_location_with_rpc_v2(
|
||||
is_new_location_valid = true;
|
||||
common::ObPartitionKey pkey;
|
||||
const uint64_t tenant_id = get_rpc_tenant_id(cached_location.get_table_id());
|
||||
const int64_t default_timeout = OB_FETCH_MEMBER_LIST_AND_LEADER_TIMEOUT;
|
||||
const int64_t default_timeout = GCONF.location_cache_refresh_rpc_timeout;
|
||||
int64_t timeout_us = default_timeout;
|
||||
const ObIArray<ObReplicaLocation>& location_array = cached_location.get_replica_locations();
|
||||
ObMemberListAndLeaderArg member_info;
|
||||
@ -1305,7 +1305,7 @@ int ObLocationFetcher::batch_renew_location_with_rpc(const common::ObIArray<cons
|
||||
rootserver::ObBatchRpcRenewLocProxy proxy_batch(
|
||||
*srv_rpc_proxy_, &obrpc::ObSrvRpcProxy::batch_get_member_list_and_leader);
|
||||
// gather rpc results
|
||||
const int64_t timeout_ts = OB_FETCH_MEMBER_LIST_AND_LEADER_TIMEOUT;
|
||||
const int64_t timeout_ts = GCONF.location_cache_refresh_rpc_timeout;
|
||||
for (int64_t i = 0; i < infos.count(); i++) { // ignore failure
|
||||
ObLocationRpcRenewInfo& info = infos.at(i);
|
||||
if (!info.is_valid()) {
|
||||
@ -4137,7 +4137,8 @@ int ObPartitionLocationCache::set_timeout_ctx(common::ObTimeoutCtx& ctx)
|
||||
int64_t abs_timeout_us = ctx.get_abs_timeout();
|
||||
|
||||
if (abs_timeout_us < 0) {
|
||||
abs_timeout_us = ObTimeUtility::current_time() + DEFAULT_FETCH_LOCATION_TIMEOUT_US;
|
||||
abs_timeout_us = ObTimeUtility::current_time() + GCONF.location_cache_refresh_rpc_timeout +
|
||||
GCONF.location_cache_refresh_sql_timeout;
|
||||
}
|
||||
if (THIS_WORKER.get_timeout_ts() > 0 && THIS_WORKER.get_timeout_ts() < abs_timeout_us) {
|
||||
abs_timeout_us = THIS_WORKER.get_timeout_ts();
|
||||
@ -4325,7 +4326,8 @@ int ObPartitionLocationCache::batch_renew_location(const common::ObIArray<ObLoca
|
||||
ObWaitEventIds::PT_LOCATION_CACHE_LOCK_WAIT, wait_event_time_out, unused, unused, unused);
|
||||
|
||||
// ignore acquire fail
|
||||
abs_timeout_us = ObTimeUtility::current_time() + DEFAULT_FETCH_LOCATION_TIMEOUT_US;
|
||||
abs_timeout_us = ObTimeUtility::current_time() + GCONF.location_cache_refresh_rpc_timeout +
|
||||
GCONF.location_cache_refresh_sql_timeout;
|
||||
int tmp_ret = sem_.acquire(abs_timeout_us);
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
LOG_WARN("acquire failed", K(tmp_ret));
|
||||
@ -4636,9 +4638,9 @@ int ObPartitionLocationCache::set_batch_timeout_ctx(
|
||||
UNUSED(task_cnt);
|
||||
if (abs_timeout_us < 0) {
|
||||
if (ObLocationAsyncUpdateTask::MODE_SQL_ONLY == type) {
|
||||
abs_timeout_us = ObTimeUtility::current_time() + 2 * DEFAULT_FETCH_LOCATION_TIMEOUT_US;
|
||||
abs_timeout_us = ObTimeUtility::current_time() + 2 * GCONF.location_cache_refresh_sql_timeout;
|
||||
} else {
|
||||
abs_timeout_us = ObTimeUtility::current_time() + 2 * ObLocationFetcher::OB_FETCH_MEMBER_LIST_AND_LEADER_TIMEOUT;
|
||||
abs_timeout_us = ObTimeUtility::current_time() + 2 * GCONF.location_cache_refresh_rpc_timeout;
|
||||
}
|
||||
}
|
||||
if (THIS_WORKER.get_timeout_ts() > 0 && THIS_WORKER.get_timeout_ts() < abs_timeout_us) {
|
||||
@ -4676,7 +4678,7 @@ int ObPartitionLocationCache::batch_renew_sys_table_location_by_rpc(common::ObIA
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTimeoutCtx ctx;
|
||||
ctx.set_timeout(ObLocationFetcher::OB_FETCH_MEMBER_LIST_AND_LEADER_TIMEOUT * 2);
|
||||
ctx.set_timeout(GCONF.location_cache_refresh_rpc_timeout);
|
||||
ObPartitionLocation core_table_location;
|
||||
bool is_nonblock = false;
|
||||
bool filter_not_readable_replica = true;
|
||||
|
@ -482,8 +482,6 @@ protected:
|
||||
// used by observer
|
||||
class ObLocationFetcher : public ObILocationFetcher {
|
||||
public:
|
||||
static const int64_t OB_FETCH_LOCATION_TIMEOUT = 1 * 1000 * 1000; // 1s
|
||||
static const int64_t OB_FETCH_MEMBER_LIST_AND_LEADER_TIMEOUT = 500 * 1000; // 500ms
|
||||
ObLocationFetcher();
|
||||
virtual ~ObLocationFetcher();
|
||||
int init(common::ObServerConfig& config, share::ObPartitionTableOperator& pt,
|
||||
@ -601,7 +599,7 @@ public:
|
||||
};
|
||||
|
||||
public:
|
||||
ObLocationLeaderCache() : allocator_(), buffer_()
|
||||
ObLocationLeaderCache() : local_allocator_(), allocator_(local_allocator_), buffer_()
|
||||
{}
|
||||
virtual ~ObLocationLeaderCache()
|
||||
{}
|
||||
@ -610,7 +608,8 @@ public:
|
||||
|
||||
private:
|
||||
static const int64_t CACHE_NUM = 10000;
|
||||
common::ObArenaAllocator allocator_;
|
||||
common::ObArenaAllocator local_allocator_;
|
||||
common::ObSafeArenaAllocator allocator_;
|
||||
ObLocationLeaderInfo buffer_[CACHE_NUM];
|
||||
};
|
||||
|
||||
@ -789,8 +788,6 @@ private:
|
||||
int set_batch_timeout_ctx(const int64_t task_cnt, ObLocationAsyncUpdateTask::Type type, common::ObTimeoutCtx& ctx);
|
||||
/*-----batch async renew location end -----*/
|
||||
private:
|
||||
const static int64_t DEFAULT_FETCH_LOCATION_TIMEOUT_US =
|
||||
ObLocationFetcher::OB_FETCH_LOCATION_TIMEOUT + ObLocationFetcher::OB_FETCH_MEMBER_LIST_AND_LEADER_TIMEOUT; // 4s
|
||||
static const int64_t OB_SYS_LOCATION_CACHE_BUCKET_NUM = 512;
|
||||
typedef common::hash::ObHashMap<ObLocationCacheKey, ObPartitionLocation> NoSwapCache;
|
||||
typedef common::hash::ObHashMap<ObLocationCacheKey, LocationInfo> NoSwapLeaderCache;
|
||||
|
@ -57,10 +57,11 @@ int ObSchemaRetrieveUtils::retrieve_table_schema(const uint64_t tenant_id, const
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
uint64_t prev_table_id = common::OB_INVALID_ID;
|
||||
ObTableSchema table_schema;
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) {
|
||||
table_schema.reset();
|
||||
bool is_deleted = false;
|
||||
ObTableSchema* allocated_table_schema = NULL;
|
||||
ObTableSchema table_schema;
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(fill_table_schema(tenant_id, check_deleted, result, table_schema, is_deleted))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to fill table schema. ", K(check_deleted), K(ret));
|
||||
@ -84,14 +85,25 @@ int ObSchemaRetrieveUtils::retrieve_table_schema(const uint64_t tenant_id, const
|
||||
allocator.free(allocated_table_schema);
|
||||
allocated_table_schema = NULL;
|
||||
}
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve table schema", K(table_schema), K(is_deleted), K(ret));
|
||||
if (OB_SUCC(ret)) {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve table schema", K(table_schema), K(is_deleted), KR(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(WARN,
|
||||
"retrieve table schema failed",
|
||||
"table_id",
|
||||
table_schema.get_table_id(),
|
||||
"schema_version",
|
||||
table_schema.get_schema_version(),
|
||||
K(prev_table_id),
|
||||
K(is_deleted),
|
||||
KR(ret));
|
||||
}
|
||||
prev_table_id = table_schema.get_table_id();
|
||||
table_schema.reset();
|
||||
}
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get all table schema. iter quit. ", K(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve table schema");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve table schema", K(tenant_id));
|
||||
ret = common::OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
@ -214,7 +226,7 @@ int ObSchemaRetrieveUtils::retrieve_table_schema(const uint64_t tenant_id, const
|
||||
int ret = common::OB_SUCCESS;
|
||||
|
||||
table_schema = NULL;
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve table schema");
|
||||
SHARE_SCHEMA_LOG(DEBUG, "retrieve table schema");
|
||||
if (OB_FAIL(result.next())) {
|
||||
if (ret == common::OB_ITER_END) { // no record
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
@ -250,7 +262,7 @@ int ObSchemaRetrieveUtils::retrieve_tablegroup_schema(
|
||||
int ret = common::OB_SUCCESS;
|
||||
|
||||
tablegroup_schema = NULL;
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve tablegroup schema");
|
||||
SHARE_SCHEMA_LOG(DEBUG, "retrieve tablegroup schema");
|
||||
if (OB_FAIL(result.next())) {
|
||||
if (ret == common::OB_ITER_END) { // no record
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
@ -757,12 +769,12 @@ int ObSchemaRetrieveUtils::retrieve_recycle_object(
|
||||
int ret = common::OB_SUCCESS;
|
||||
ObRecycleObject recycle_obj;
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) {
|
||||
recycle_obj.reset();
|
||||
if (OB_FAIL(fill_recycle_object(tenant_id, result, recycle_obj))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to fill recycle object. ", K(ret));
|
||||
} else if (OB_FAIL(recycle_objs.push_back(recycle_obj))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "failed to push back", K(ret));
|
||||
}
|
||||
recycle_obj.reset();
|
||||
}
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get all recycle objects. iter quit. ", K(ret));
|
||||
@ -892,7 +904,18 @@ int ObSchemaRetrieveUtils::fill_tenant_schema(T& result, ObTenantSchema& tenant_
|
||||
}
|
||||
}
|
||||
}
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve tenant schema", K(tenant_schema), K(is_deleted), K(ret));
|
||||
if (OB_SUCC(ret)) {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve tenant schema", K(tenant_schema), K(is_deleted), KR(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(WARN,
|
||||
"retrieve tenant schema failed",
|
||||
"tenant_id",
|
||||
tenant_schema.get_tenant_id(),
|
||||
"schema_version",
|
||||
tenant_schema.get_schema_version(),
|
||||
K(is_deleted),
|
||||
KR(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -971,7 +994,17 @@ int ObSchemaRetrieveUtils::fill_temp_table_schema(const uint64_t tenant_id, T& r
|
||||
} else {
|
||||
table_schema.set_create_host(create_host);
|
||||
}
|
||||
SHARE_SCHEMA_LOG(INFO, "Get create_host ", K(create_host), K(table_schema));
|
||||
if (OB_SUCC(ret)) {
|
||||
SHARE_SCHEMA_LOG(INFO, "Get create_host ", K(create_host), K(table_schema));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(WARN,
|
||||
"Get create_host failed",
|
||||
KR(ret),
|
||||
"table_id",
|
||||
table_schema.get_table_id(),
|
||||
"schema_version",
|
||||
table_schema.get_schema_version());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -2154,9 +2187,7 @@ int ObSchemaRetrieveUtils::fill_schema_operation(const uint64_t tenant_id, T& re
|
||||
schema_operation.profile_id_ = schema_operation.table_id_; // table_id_ reused to store profile_id_
|
||||
schema_operation.grantee_id_ = schema_operation.user_id_; // table_id_ reused to store grantee_id_
|
||||
schema_operation.grantor_id_ = schema_operation.database_id_; // database_id_ reused to store grantor_id_
|
||||
|
||||
SHARE_SCHEMA_LOG(DEBUG, "fill schema operation", K(schema_operation));
|
||||
schema_operation.dblink_id_ = schema_operation.table_id_; // table_id_ reused to store dblink_id
|
||||
schema_operation.dblink_id_ = schema_operation.table_id_; // table_id_ reused to store dblink_id
|
||||
SHARE_SCHEMA_LOG(INFO, "fill schema operation", K(schema_operation));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to fill schema operation", KR(ret));
|
||||
@ -2180,6 +2211,7 @@ int ObSchemaRetrieveUtils::retrieve_tenant_schema(ObISQLClient& client, T& resul
|
||||
S tenant_schema;
|
||||
ObArenaAllocator allocator(ObModIds::OB_TEMP_VARIABLES);
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) {
|
||||
tenant_schema.reset();
|
||||
bool is_deleted = false;
|
||||
if (OB_FAIL(fill_tenant_schema(result, tenant_schema, is_deleted))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to fill tenant schema", K(ret));
|
||||
@ -2190,7 +2222,6 @@ int ObSchemaRetrieveUtils::retrieve_tenant_schema(ObISQLClient& client, T& resul
|
||||
SHARE_SCHEMA_LOG(WARN, "failed to push back", K(ret));
|
||||
}
|
||||
prev_tenant_id = tenant_schema.get_tenant_id();
|
||||
tenant_schema.reset();
|
||||
}
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get all tenant schema. iter quit. ", K(ret));
|
||||
@ -2342,6 +2373,7 @@ int ObSchemaRetrieveUtils::fill_sysvar_schema(
|
||||
S schema; \
|
||||
int64_t count = 0; \
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) { \
|
||||
schema.reset(); \
|
||||
bool is_deleted = false; \
|
||||
count++; \
|
||||
if (OB_FAIL(fill_##SCHEMA##_schema(tenant_id, result, schema, is_deleted))) { \
|
||||
@ -2357,7 +2389,6 @@ int ObSchemaRetrieveUtils::fill_sysvar_schema(
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve " #SCHEMA " schema succeed", K(schema)); \
|
||||
} \
|
||||
prev_id = schema.get_##SCHEMA##_id(); \
|
||||
schema.reset(); \
|
||||
} \
|
||||
if (ret != common::OB_ITER_END) { \
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get all " #SCHEMA " schema. iter quit. ", K(ret)); \
|
||||
@ -2374,7 +2405,6 @@ RETRIEVE_SCHEMA_FUNC_DEFINE(tablegroup);
|
||||
RETRIEVE_SCHEMA_FUNC_DEFINE(table);
|
||||
RETRIEVE_SCHEMA_FUNC_DEFINE(outline);
|
||||
RETRIEVE_SCHEMA_FUNC_DEFINE(sequence);
|
||||
|
||||
RETRIEVE_SCHEMA_FUNC_DEFINE(synonym);
|
||||
|
||||
template <typename T, typename S>
|
||||
@ -2384,6 +2414,7 @@ int ObSchemaRetrieveUtils::retrieve_udf_schema(const uint64_t tenant_id, T& resu
|
||||
common::ObString udf_name;
|
||||
S schema;
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) {
|
||||
schema.reset();
|
||||
bool is_deleted = false;
|
||||
if (OB_FAIL(fill_udf_schema(tenant_id, result, schema, is_deleted))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to fill udf schema", K(ret));
|
||||
@ -2397,13 +2428,12 @@ int ObSchemaRetrieveUtils::retrieve_udf_schema(const uint64_t tenant_id, T& resu
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve udf schema succeed", K(schema));
|
||||
}
|
||||
udf_name = schema.get_udf_name_str();
|
||||
schema.reset();
|
||||
}
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get all udf schema. iter quit. ", K(ret));
|
||||
} else {
|
||||
ret = common::OB_SUCCESS;
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve udf schemas succeed");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve udf schemas succeed", K(tenant_id));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -2419,7 +2449,7 @@ int ObSchemaRetrieveUtils::retrieve_link_table_schema(
|
||||
TST tmp_table_schema;
|
||||
|
||||
table_schema = NULL;
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve link table schema");
|
||||
SHARE_SCHEMA_LOG(DEBUG, "retrieve link table schema");
|
||||
if (OB_FAIL(result.next())) {
|
||||
if (ret == common::OB_ITER_END) { // no record
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
@ -2451,7 +2481,7 @@ int ObSchemaRetrieveUtils::retrieve_link_column_schema(const uint64_t tenant_id,
|
||||
bool is_deleted = false;
|
||||
ObColumnSchemaV2 column_schema;
|
||||
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve link column schema");
|
||||
SHARE_SCHEMA_LOG(DEBUG, "retrieve link column schema");
|
||||
while (OB_SUCC(ret) && OB_SUCC(result.next())) {
|
||||
column_schema.reset();
|
||||
if (OB_FAIL(fill_link_column_schema(tenant_id, result, column_schema, is_deleted))) {
|
||||
@ -2463,7 +2493,7 @@ int ObSchemaRetrieveUtils::retrieve_link_column_schema(const uint64_t tenant_id,
|
||||
}
|
||||
if (common::OB_ITER_END == ret) {
|
||||
ret = common::OB_SUCCESS;
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve link column schema succeed");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve link column schema succeed", K(table_schema));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -2475,6 +2505,7 @@ int ObSchemaRetrieveUtils::retrieve_db_priv_schema(const uint64_t tenant_id, T&
|
||||
S db_priv;
|
||||
S prev_priv;
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) {
|
||||
db_priv.reset();
|
||||
bool is_deleted = false;
|
||||
if (OB_FAIL(fill_db_priv_schema(tenant_id, result, db_priv, is_deleted))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "Fail to fill database privileges", K(ret));
|
||||
@ -2486,8 +2517,9 @@ int ObSchemaRetrieveUtils::retrieve_db_priv_schema(const uint64_t tenant_id, T&
|
||||
} else if (OB_FAIL(db_priv_array.push_back(db_priv))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "Failed to push back", K(ret));
|
||||
}
|
||||
prev_priv = db_priv;
|
||||
db_priv.reset();
|
||||
if (OB_SUCC(ret)) {
|
||||
prev_priv = db_priv;
|
||||
}
|
||||
}
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "Fail to get database privileges. iter quit", K(ret));
|
||||
@ -2673,6 +2705,7 @@ int ObSchemaRetrieveUtils::retrieve_table_priv_schema(
|
||||
S table_priv;
|
||||
S prev_table_priv;
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) {
|
||||
table_priv.reset();
|
||||
bool is_deleted = false;
|
||||
if (OB_FAIL(fill_table_priv_schema(tenant_id, result, table_priv, is_deleted))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "Fail to fill table_priv", K(ret));
|
||||
@ -2684,8 +2717,9 @@ int ObSchemaRetrieveUtils::retrieve_table_priv_schema(
|
||||
} else if (OB_FAIL(table_priv_array.push_back(table_priv))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "Failed to push back", K(ret));
|
||||
}
|
||||
prev_table_priv = table_priv;
|
||||
table_priv.reset();
|
||||
if (OB_SUCC(ret)) {
|
||||
prev_table_priv = table_priv;
|
||||
}
|
||||
}
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "Fail to get table privileges. iter quit", K(ret));
|
||||
@ -2860,7 +2894,18 @@ int ObSchemaRetrieveUtils::fill_tenant_schema(T& result, ObSimpleTenantSchema& t
|
||||
}
|
||||
}
|
||||
}
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve tenant schema", K(tenant_schema), K(is_deleted), K(ret));
|
||||
if (OB_SUCC(ret)) {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve tenant schema", K(tenant_schema), K(is_deleted), KR(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(WARN,
|
||||
"retrieve tenant schema failed",
|
||||
"tenant_id",
|
||||
tenant_schema.get_tenant_id(),
|
||||
"schema_version",
|
||||
tenant_schema.get_schema_version(),
|
||||
K(is_deleted),
|
||||
KR(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -3446,7 +3491,7 @@ int ObSchemaRetrieveUtils::retrieve_aux_tables(
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get aux table. iter quit. ", K(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve aux table finish");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve aux table finish", K(tenant_id));
|
||||
ret = common::OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
@ -3474,7 +3519,7 @@ int ObSchemaRetrieveUtils::retrieve_schema_version(T& result, VersionHisVal& ver
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get all schema version. iter quit. ", K(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve schema version");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve schema version", K(version_his_val));
|
||||
ret = common::OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
@ -3494,7 +3539,12 @@ int ObSchemaRetrieveUtils::retrieve_foreign_key_info(const uint64_t tenant_id, T
|
||||
} else if (foreign_key_info.foreign_key_id_ == prev_foreign_key_id) {
|
||||
ret = common::OB_SUCCESS;
|
||||
} else if (is_deleted) {
|
||||
SHARE_SCHEMA_LOG(INFO, "foreign key is is_deleted");
|
||||
SHARE_SCHEMA_LOG(INFO,
|
||||
"foreign key is is_deleted",
|
||||
"table_id",
|
||||
table_schema.get_table_id(),
|
||||
"foreign_key_id",
|
||||
foreign_key_info.foreign_key_id_);
|
||||
} else if (OB_FAIL(table_schema.add_foreign_key_info(foreign_key_info))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to add foreign key info", K(ret), K(foreign_key_info));
|
||||
}
|
||||
@ -3503,7 +3553,7 @@ int ObSchemaRetrieveUtils::retrieve_foreign_key_info(const uint64_t tenant_id, T
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get foreign key schema. iter quit. ", K(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve foreign key schema");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve foreign key schema", "table_id", table_schema.get_table_id());
|
||||
ret = common::OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
@ -3539,7 +3589,7 @@ int ObSchemaRetrieveUtils::retrieve_foreign_key_column_info(
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get foreing key. iter quit.", K(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve foreign key");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve foreign key", K(foreign_key_info));
|
||||
ret = common::OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
@ -3641,7 +3691,7 @@ int ObSchemaRetrieveUtils::retrieve_simple_foreign_key_info(
|
||||
} else if (fk_id == prev_foreign_key_id) {
|
||||
ret = OB_SUCCESS;
|
||||
} else if (is_deleted) {
|
||||
SHARE_SCHEMA_LOG(INFO, "foreign key is deleted");
|
||||
SHARE_SCHEMA_LOG(INFO, "foreign key is deleted", K(table_id), K(fk_id));
|
||||
} else if (table_id == common::OB_INVALID_ID ||
|
||||
OB_ISNULL(table_schema_ptr = ObSchemaRetrieveUtils::find_table_schema(table_id, table_schema_array))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to find table schema by table id", K(ret), K(table_id));
|
||||
@ -3657,7 +3707,7 @@ int ObSchemaRetrieveUtils::retrieve_simple_foreign_key_info(
|
||||
if (ret != OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get simple foreign key info. iter quit. ", K(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve simple foreign key info");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve simple foreign key info", K(tenant_id));
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
|
||||
@ -3719,7 +3769,7 @@ int ObSchemaRetrieveUtils::retrieve_simple_constraint_info(
|
||||
} else if (table_id == prev_table_id && cst_id == prev_constraint_id) {
|
||||
ret = OB_SUCCESS;
|
||||
} else if (is_deleted) {
|
||||
SHARE_SCHEMA_LOG(INFO, "constraint is deleted");
|
||||
SHARE_SCHEMA_LOG(INFO, "constraint is deleted", K(table_id), K(cst_id));
|
||||
} else if (table_id == common::OB_INVALID_ID ||
|
||||
OB_ISNULL(table_schema_ptr = ObSchemaRetrieveUtils::find_table_schema(table_id, table_schema_array))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to find table schema by table id", K(ret), K(table_id));
|
||||
@ -3743,7 +3793,7 @@ int ObSchemaRetrieveUtils::retrieve_simple_constraint_info(
|
||||
if (ret != OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get simple constraint info. iter quit. ", K(ret));
|
||||
} else {
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve simple constraint info");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve simple constraint info", K(tenant_id));
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
|
||||
@ -3804,6 +3854,7 @@ int ObSchemaRetrieveUtils::retrieve_profile_schema(const uint64_t tenant_id, T&
|
||||
uint64_t prev_id = common::OB_INVALID_ID;
|
||||
S schema;
|
||||
while (OB_SUCCESS == ret && common::OB_SUCCESS == (ret = result.next())) {
|
||||
schema.reset();
|
||||
bool is_deleted = false;
|
||||
if (OB_FAIL(fill_profile_schema(tenant_id, result, schema, is_deleted))) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to fill profile schema ", K(ret));
|
||||
@ -3818,13 +3869,12 @@ int ObSchemaRetrieveUtils::retrieve_profile_schema(const uint64_t tenant_id, T&
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve profile schema succeed", K(schema));
|
||||
}
|
||||
prev_id = schema.get_profile_id();
|
||||
schema.reset();
|
||||
}
|
||||
if (ret != common::OB_ITER_END) {
|
||||
SHARE_SCHEMA_LOG(WARN, "fail to get all profile schema. iter quit. ", K(ret));
|
||||
} else {
|
||||
ret = common::OB_SUCCESS;
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve profile schemas succeed");
|
||||
SHARE_SCHEMA_LOG(INFO, "retrieve profile schemas succeed", K(tenant_id));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -1688,8 +1688,19 @@ int ObTableSqlService::add_single_column(
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("affected_rows unexpected to be one", K(affected_rows), K(ret));
|
||||
} else {
|
||||
bool is_all_table = combine_id(OB_SYS_TENANT_ID, OB_ALL_TABLE_TID) == table_id ||
|
||||
combine_id(OB_SYS_TENANT_ID, OB_ALL_TABLE_V2_TID) == table_id;
|
||||
/*
|
||||
* We try to modify __all_table's content in __all_core_table when __all_table/__all_table_v2 adds new columns.
|
||||
* For compatibility, we will add new columns to __all_table/__all_table_v2 at the same time, which means we
|
||||
* modify
|
||||
* __all_table's content in __all_core_table twice in such situaction.
|
||||
*
|
||||
* When we add string-like columns to __all_table/__all_table_v2, it may cause -4016 error because the second
|
||||
* modification will do nothing and affected_rows won't change. To fix that, we skip the modification caused by
|
||||
* adding columns to __all_table_v2.
|
||||
*
|
||||
*/
|
||||
bool is_all_table = combine_id(OB_SYS_TENANT_ID, OB_ALL_TABLE_TID) == table_id;
|
||||
//|| combine_id(OB_SYS_TENANT_ID, OB_ALL_TABLE_V2_TID) == table_id;
|
||||
bool is_all_column = combine_id(OB_SYS_TENANT_ID, OB_ALL_COLUMN_TID) == table_id;
|
||||
if (is_all_table || is_all_column) {
|
||||
if (OB_FAIL(supplement_for_core_table(sql_client, is_all_table, column))) {
|
||||
|
@ -689,7 +689,7 @@ bool ObSQLSessionInfo::has_user_process_privilege() const
|
||||
int ObSQLSessionInfo::check_global_read_only_privilege(const bool read_only, const ObSqlTraits& sql_traits)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!has_user_super_privilege() && read_only) {
|
||||
if (!has_user_super_privilege() && !is_tenant_changed() && read_only) {
|
||||
/** session1 session2
|
||||
* insert into xxx;
|
||||
* set @@global.read_only = 1;
|
||||
|
@ -161,7 +161,7 @@ int ObBuildIndexBaseTask::check_partition_need_build_index(const ObPartitionKey&
|
||||
const ObTableSchema* new_index_schema = NULL;
|
||||
ObSchemaGetterGuard schema_guard;
|
||||
bool is_partition_exist = false;
|
||||
bool check_dropped_partition = true;
|
||||
bool check_dropped_partition = false;
|
||||
bool is_split_finished = true;
|
||||
need_build = false;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
|
Loading…
x
Reference in New Issue
Block a user