[FEAT MERGE] Support batch construction of local indexes to solve the problem of temporary space enlargement

This commit is contained in:
791065426@qq.com 2024-06-17 11:04:15 +00:00 committed by ob-robot
parent 188418cf13
commit 6dfa79db04
23 changed files with 2192 additions and 199 deletions

View File

@ -128,6 +128,7 @@ ob_set_subtarget(ob_rootserver ddl_task
ddl_task/ob_modify_autoinc_task.cpp
ddl_task/ob_table_redefinition_task.cpp
ddl_task/ob_recover_restore_table_task.cpp
ddl_task/ob_ddl_tablet_scheduler.cpp
)
ob_set_subtarget(ob_rootserver parallel_ddl

View File

@ -273,7 +273,7 @@ int ObColumnRedefinitionTask::copy_table_indexes()
} else {
create_index_arg.index_type_ = index_schema->get_index_type();
ObCreateDDLTaskParam param(tenant_id_,
ObDDLType::DDL_CREATE_INDEX,
((DATA_VERSION_4_2_2_0 <= data_format_version_ && data_format_version_ < DATA_VERSION_4_3_0_0) || data_format_version_ >= DATA_VERSION_4_3_2_0) && index_schema->is_storage_local_index_table() && index_schema->is_partitioned_table() ? ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX : ObDDLType::DDL_CREATE_INDEX,
table_schema,
index_schema,
0/*object_id*/,
@ -845,4 +845,4 @@ void ObColumnRedefinitionTask::flt_set_status_span_tag() const
break;
}
}
}
}

View File

@ -143,6 +143,7 @@ int ObDDLRedefinitionSSTableBuildTask::process()
LOG_WARN("fail to generate build mview replica sql", K(ret));
}
} else {
ObString partition_names;
if (OB_FAIL(ObDDLUtil::generate_build_replica_sql(tenant_id_,
data_table_id_,
dest_table_id_,
@ -154,6 +155,7 @@ int ObDDLRedefinitionSSTableBuildTask::process()
use_heap_table_ddl_plan_,
true/*use_schema_version_hint_for_src_table*/,
&col_name_map_,
partition_names,
sql_string))) {
LOG_WARN("fail to generate build replica sql", K(ret));
}

View File

@ -984,6 +984,7 @@ int ObDDLScheduler::create_ddl_task(const ObCreateDDLTaskParam &param,
switch (param.type_) {
case DDL_CREATE_INDEX:
case DDL_CREATE_MLOG:
case DDL_CREATE_PARTITIONED_LOCAL_INDEX:
create_index_arg = static_cast<const obrpc::ObCreateIndexArg *>(param.ddl_arg_);
if (OB_FAIL(create_build_index_task(proxy,
param.type_,
@ -994,6 +995,7 @@ int ObDDLScheduler::create_ddl_task(const ObCreateDDLTaskParam &param,
param.consumer_group_id_,
param.sub_task_trace_id_,
create_index_arg,
param.type_,
param.tenant_data_version_,
*param.allocator_,
task_record))) {
@ -1527,6 +1529,7 @@ int ObDDLScheduler::create_build_index_task(
const int64_t consumer_group_id,
const int32_t sub_task_trace_id,
const obrpc::ObCreateIndexArg *create_index_arg,
const share::ObDDLType task_type,
const uint64_t tenant_data_version,
ObIAllocator &allocator,
ObDDLTaskRecord &task_record)
@ -1553,6 +1556,7 @@ int ObDDLScheduler::create_build_index_task(
consumer_group_id,
sub_task_trace_id,
*create_index_arg,
task_type,
parent_task_id,
tenant_data_version))) {
LOG_WARN("init global index task failed", K(ret), K(data_table_schema), K(index_schema));
@ -2147,6 +2151,7 @@ int ObDDLScheduler::schedule_ddl_task(const ObDDLTaskRecord &record)
switch (record.ddl_type_) {
case ObDDLType::DDL_CREATE_INDEX:
case ObDDLType::DDL_CREATE_MLOG:
case ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX:
ret = schedule_build_index_task(record);
break;
case ObDDLType::DDL_DROP_INDEX:
@ -2699,7 +2704,7 @@ int ObDDLScheduler::on_column_checksum_calc_reply(
LOG_WARN("invalid argument", K(ret), K(task_key), K(tablet_id), K(ret_code));
} else if (OB_FAIL(task_queue_.modify_task(task_key, [&tablet_id, &ret_code](ObDDLTask &task) -> int {
int ret = OB_SUCCESS;
if (OB_UNLIKELY(ObDDLType::DDL_CREATE_INDEX != task.get_task_type())) {
if (OB_UNLIKELY(!is_create_index(task.get_task_type()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ddl task type not global index", K(ret), K(task));
} else if (OB_FAIL(DDL_SIM(task.get_tenant_id(), task.get_task_id(), ON_COLUMN_CHECKSUM_REPLY_FAILED))) {
@ -2753,6 +2758,7 @@ int ObDDLScheduler::on_sstable_complement_job_reply(
const int64_t task_type = task.get_task_type();
switch (task_type) {
case ObDDLType::DDL_CREATE_INDEX:
case ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX:
if (OB_FAIL(static_cast<ObIndexBuildTask *>(&task)->update_complete_sstable_job_status(tablet_id, snapshot_version, execution_id, ret_code, addition_info))) {
LOG_WARN("update complete sstable job status failed", K(ret));
}

View File

@ -363,6 +363,7 @@ private:
const int64_t consumer_group_id,
const int32_t sub_task_trace_id,
const obrpc::ObCreateIndexArg *create_index_arg,
const share::ObDDLType task_type,
const uint64_t tenant_data_version,
ObIAllocator &allocator,
ObDDLTaskRecord &task_record);

View File

@ -0,0 +1,731 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#define USING_LOG_PREFIX RS
#include "ob_ddl_tablet_scheduler.h"
#include "rootserver/ob_root_service.h"
#include "share/ob_ddl_checksum.h"
#include "share/ob_ddl_error_message_table_operator.h"
#include "share/ob_ddl_common.h"
#include "share/schema/ob_multi_version_schema_service.h"
#include "share/scn.h"
#include "share/tablet/ob_tablet_to_ls_operator.h"
#include "storage/ddl/ob_ddl_lock.h"
using namespace oceanbase::rootserver;
using namespace oceanbase::common;
using namespace oceanbase::common::sqlclient;
using namespace oceanbase::obrpc;
using namespace oceanbase::share;
using namespace oceanbase::share::schema;
using namespace oceanbase::sql;
ObDDLTabletScheduler::ObDDLTabletScheduler()
: is_inited_(false), allocator_("TabletScheduler")
{
}
ObDDLTabletScheduler::~ObDDLTabletScheduler()
{
}
int ObDDLTabletScheduler::init(const uint64_t tenant_id,
const uint64_t table_id,
const uint64_t ref_data_table_id,
const int64_t task_id,
const int64_t parallelism,
const int64_t snapshot_version,
const common::ObCurTraceId::TraceId &trace_id,
const ObIArray<ObTabletID> &tablets)
{
int ret = OB_SUCCESS;
common::ObAddr inner_sql_exec_addr;
common::ObArray<ObString> running_sql_info;
common::ObArray<ObLSID> ls_ids;
common::ObArray<ObTabletID> ref_data_table_tablets;
common::hash::ObHashMap<uint64_t, bool> tablet_checksum_status_map;
if (OB_UNLIKELY(is_inited_)) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", K(ret), K(is_inited_));
} else if (OB_ISNULL(root_service_ = GCTX.root_service_)) {
ret = OB_ERR_SYS;
LOG_WARN("root_service is null", K(ret), KP(root_service_));
} else if (!root_service_->in_service()) {
ret = OB_STATE_NOT_MATCH;
LOG_WARN("root service not in service", K(ret));
} else if (OB_UNLIKELY(
!(OB_INVALID_ID != tenant_id
&& OB_INVALID_ID != table_id
&& OB_INVALID_ID != ref_data_table_id
&& task_id > 0
&& parallelism > 0
&& snapshot_version > 0
&& trace_id.is_valid()
&& tablets.count() > 0))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(table_id), K(ref_data_table_id), K(task_id), K(parallelism), K(snapshot_version), K(trace_id), K(tablets.count()));
} else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id, ref_data_table_id, ref_data_table_tablets))) {
LOG_WARN("failed to get ref data table tablet ids", K(ret), K(tenant_id), K(ref_data_table_id), K(ref_data_table_tablets));
} else if (OB_UNLIKELY(tablets.count() != ref_data_table_tablets.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("index table tablets count is not equal to data table tablets count", K(ret), K(tablets.count()), K(ref_data_table_tablets.count()));
} else if (OB_ISNULL(GCTX.root_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("rootservice is null", K(ret));
} else if (OB_FAIL(ObTabletToLSTableOperator::batch_get_ls(GCTX.root_service_->get_sql_proxy(), tenant_id, tablets, ls_ids))) {
LOG_WARN("failed to batch get ls", K(ret), K(tenant_id), K(tablets), K(ls_ids));
} else if (OB_UNLIKELY(tablets.count() != ls_ids.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablets count is not equal to ls id count", K(ret), K(tablets.count()), K(ls_ids.count()));
} else if (OB_FAIL(all_ls_to_tablets_map_.create(tablets.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create lsid to tablet id map", K(ret), K(tablets.count()));
} else if (OB_FAIL(running_ls_to_tablets_map_.create(tablets.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create lsid to tablet id map", K(ret), K(tablets.count()));
} else if (OB_FAIL(ls_location_map_.create(tablets.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create lsid location map", K(ret), K(tablets.count()));
} else if (OB_FAIL(running_ls_to_execution_id_.create(tablets.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create lsid to execution id map", K(ret), K(tablets.count()));
} else if (OB_FAIL(tablet_checksum_status_map.create(tablets.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create column checksum map", K(ret), K(tablets.count()));
} else if (OB_FAIL(tablet_id_to_data_size_.create(ref_data_table_tablets.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create column checksum map", K(ret), K(ref_data_table_tablets.count()));
} else if (OB_FAIL(tablet_id_to_data_row_cnt_.create(ref_data_table_tablets.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create column checksum map", K(ret), K(ref_data_table_tablets.count()));
} else if (OB_FAIL(ObDDLChecksumOperator::get_tablet_checksum_record_without_execution_id(
tenant_id,
table_id,
task_id,
tablets,
GCTX.root_service_->get_sql_proxy(),
tablet_checksum_status_map))) {
LOG_WARN("fail to get tablet checksum status", K(ret), K(tenant_id), K(table_id), K(task_id), K(tablets));
} else if (OB_FAIL(ObDDLTaskRecordOperator::get_running_tasks_inner_sql(root_service_->get_sql_proxy(), trace_id, tenant_id, task_id, snapshot_version, inner_sql_exec_addr, allocator_, running_sql_info))) {
LOG_WARN("get running tasks inner sql fail", K(ret), K(trace_id), K(tenant_id), K(task_id), K(snapshot_version), K(inner_sql_exec_addr), K(running_sql_info));
} else {
bool is_running_status = false;
bool is_finished_status = false;
int64_t tablet_data_size = 0;
int64_t tablet_data_row_cnt = 0;
ObArray<ObTabletID> part_tablets;
ObArray<ObString> partition_names;
for (int64_t i = 0; i < tablets.count() && OB_SUCC(ret); i++) {
is_running_status = false;
is_finished_status = false;
tablet_data_size = 0;
tablet_data_row_cnt = 0;
part_tablets.reuse();
partition_names.reuse();
if (OB_FAIL(ObDDLUtil::get_tablet_data_size(tenant_id, ref_data_table_tablets.at(i), ls_ids.at(i), tablet_data_size))) {
LOG_WARN("fail to get tablet data size", K(ret), K(tenant_id), K(ref_data_table_tablets.at(i)), K(ls_ids.at(i)), K(tablet_data_size));
} else if (OB_FAIL(ObDDLUtil::get_tablet_data_row_cnt(tenant_id, ref_data_table_tablets.at(i), ls_ids.at(i), tablet_data_row_cnt))) {
LOG_WARN("fail to get tablet data size", K(ret), K(tenant_id), K(ref_data_table_tablets.at(i)), K(ls_ids.at(i)), K(tablet_data_row_cnt));
} else if (OB_FAIL(tablet_id_to_data_size_.set_refactored(ref_data_table_tablets.at(i).id(), tablet_data_size, true /* overwrite */))) {
LOG_WARN("table id to data size map set fail", K(ret), K(ref_data_table_tablets.at(i).id()), K(tablet_data_size));
} else if (OB_FAIL(tablet_id_to_data_row_cnt_.set_refactored(ref_data_table_tablets.at(i).id(), tablet_data_row_cnt, true /* overwrite */))) {
LOG_WARN("table id to data size map set fail", K(ret), K(ref_data_table_tablets.at(i).id()), K(tablet_data_row_cnt));
} else if (OB_FAIL(part_tablets.push_back(tablets.at(i)))) {
LOG_WARN("fail to push back", K(ret), K(tablets.at(i)));
} else if (OB_FAIL(ObDDLUtil::get_index_table_batch_partition_names(tenant_id, ref_data_table_id, table_id, part_tablets, allocator_, partition_names))) {
LOG_WARN("fail to get index table batch partition names", K(ret), K(tenant_id), K(ref_data_table_id), K(table_id), K(part_tablets), K(partition_names));
} else {
if (OB_FAIL(tablet_checksum_status_map.get_refactored(tablets.at(i).id(), is_finished_status))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
}
}
for (int64_t j = 0; j < running_sql_info.count() && OB_SUCC(ret); j++) {
is_running_status = false;
if (OB_FAIL(ObDDLUtil::check_target_partition_is_running(running_sql_info.at(j), partition_names.at(0), allocator_, is_running_status))) {
LOG_WARN("fail to check target partition is running", K(ret), K(running_sql_info.at(j)), K(partition_names.at(0)), K(is_running_status));
} else if (is_running_status) {
break;
}
}
if (OB_SUCC(ret)) {
if (!is_running_status && is_finished_status) {
LOG_INFO("tablet has complemented data", K(ret), K(tenant_id), K(table_id), K(ref_data_table_id), K(tablets.at(i)));
} else {
common::ObAddr leader_addr;
share::ObLocationService *location_service = nullptr;
int64_t rpc_timeout = ObDDLUtil::get_default_ddl_rpc_timeout();
const int64_t retry_interval_us = 200 * 1000; // 200ms
if (OB_ISNULL(location_service = GCTX.location_service_)) {
ret = OB_ERR_SYS;
LOG_WARN("location_cache is null", K(ret), KP(location_service));
} else if (OB_FAIL(location_service->get_leader_with_retry_until_timeout(GCONF.cluster_id,
tenant_id, ls_ids.at(i), leader_addr, rpc_timeout, retry_interval_us))) {
LOG_WARN("fail to get ls locaiton leader", K(ret), K(tenant_id), K(ls_ids.at(i)));
} else if (OB_FAIL(ls_location_map_.set_refactored(ls_ids.at(i), leader_addr, true /* overwrite */))) {
LOG_WARN("ls location map set fail", K(ret), K(ls_ids.at(i)), K(leader_addr));
} else if (is_running_status) {
if (OB_FAIL(ObDDLUtil::construct_ls_tablet_id_map(tenant_id, ls_ids.at(i), tablets.at(i), running_ls_to_tablets_map_))) {
LOG_WARN("fail to create running lsid to tablet id map", K(ret), K(tenant_id), K(ls_ids.at(i)), K(tablets.at(i)), K(table_id), K(ref_data_table_id), K(running_ls_to_tablets_map_.size()));
} else if (common::is_contain(running_task_ls_ids_before_, ls_ids.at(i))) {
} else if (OB_FAIL(running_task_ls_ids_before_.push_back(ls_ids.at(i)))) {
LOG_WARN("fail to push back", K(ret), K(tenant_id), K(table_id), K(ref_data_table_id), K(ls_ids.at(i)), K(is_finished_status));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(all_tablets_.push_back(tablets.at(i)))) {
LOG_WARN("fail to push back", K(ret), K(tablets.at(i)));
} else if (OB_FAIL(ObDDLUtil::construct_ls_tablet_id_map(tenant_id, ls_ids.at(i), tablets.at(i), all_ls_to_tablets_map_))) {
LOG_WARN("fail to create lsid to tablet id map", K(ret), K(tenant_id), K(ls_ids.at(i)), K(tablets.at(i)), K(table_id), K(ref_data_table_id), K(all_ls_to_tablets_map_.size()));
}
}
}
}
}
}
}
if (OB_SUCC(ret)) {
tenant_id_ = tenant_id;
table_id_ = table_id;
ref_data_table_id_ = ref_data_table_id;
task_id_ = task_id;
parallelism_ = parallelism;
snapshot_version_ = snapshot_version;
trace_id_ = trace_id;
is_inited_ = true;
LOG_INFO("success to init", K(ret), K(tenant_id), K(table_id), K(ref_data_table_id), K(task_id), K(parallelism), K(snapshot_version), K(trace_id), K(tablets), K(all_ls_to_tablets_map_.size()), K(running_ls_to_tablets_map_.size()), K(running_task_ls_ids_before_.count()));
} else {
LOG_INFO("fail to init", K(ret), K(tenant_id), K(table_id), K(ref_data_table_id), K(task_id), K(parallelism), K(snapshot_version), K(trace_id), K(tablets), K(all_ls_to_tablets_map_.size()), K(running_ls_to_tablets_map_.size()), K(running_task_ls_ids_before_.count()));
destroy();
}
return ret;
}
int ObDDLTabletScheduler::get_next_batch_tablets(int64_t &parallelism, int64_t &new_execution_id, share::ObLSID &ls_id, common::ObAddr &leader_addr, ObIArray<ObTabletID> &tablets)
{
int ret = OB_SUCCESS;
bool need_send_task = false;
parallelism = 0;
new_execution_id = 0;
uint64_t tenant_data_version = 0;
ls_id.reset();
leader_addr.reset();
tablets.reset();
share::ObDDLType task_type = share::DDL_CREATE_PARTITIONED_LOCAL_INDEX;
common::hash::ObHashMap<uint64_t, bool> tablet_checksum_status_map;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (is_all_tasks_finished()) {
ret = OB_ITER_END;
} else if (OB_FAIL(determine_if_need_to_send_new_task(need_send_task))) {
LOG_WARN("fail to get status of if need to send new task", K(ret), K(need_send_task));
} else if (!need_send_task) {
if (!is_running_tasks_before_finished()) {
ObArray<share::ObLSID> running_task_ls_ids_now;
ObArray<share::ObLSID> potential_finished_ls_ids;
if (OB_FAIL(get_session_running_lsid(running_task_ls_ids_now))) {
LOG_WARN("fail to get session running lsid", K(ret), K(running_task_ls_ids_now));
} else if (OB_FAIL(get_potential_finished_lsid(running_task_ls_ids_now, potential_finished_ls_ids))) {
LOG_WARN("fail to get potential finished lsid", K(ret), K(running_task_ls_ids_now), K(potential_finished_ls_ids));
} else {
for (int64_t i = 0; i < potential_finished_ls_ids.count() && OB_SUCC(ret); i++) {
if (OB_FAIL(check_target_ls_tasks_completion_status(potential_finished_ls_ids.at(i)))) {
LOG_WARN("fail to check target ls tasks completion status", K(ret), K(potential_finished_ls_ids.at(i)));
}
}
}
}
if (OB_SUCC(ret)) {
ret = OB_EAGAIN;
}
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id_, tenant_data_version))) {
LOG_WARN("get min data version failed", K(ret), K(tenant_id_));
} else if (OB_FAIL(ObDDLTask::push_execution_id(tenant_id_, task_id_, task_type, true/*is ddl retryable*/, tenant_data_version, new_execution_id))) {
LOG_WARN("failed to fetch new execution id", K(ret), K(tenant_id_), K(task_id_), K(new_execution_id));
} else if (OB_FAIL(get_next_parallelism(parallelism))) {
LOG_WARN("fail to get next parallelism", K(ret), K(parallelism));
} else if (OB_FAIL(get_unfinished_tablets(new_execution_id, ls_id, leader_addr, tablets))) {
LOG_WARN("failed to get unfinished tablets", K(ret), K(new_execution_id), K(tablets));
}
return ret;
}
int ObDDLTabletScheduler::confirm_batch_tablets_status(const int64_t execution_id, const bool finish_status, const ObLSID &ls_id, const ObIArray<ObTabletID> &tablets)
{
int ret = OB_SUCCESS;
int64_t ls_execution_id = 0;
TCWLockGuard guard(lock_);
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_UNLIKELY(tablets.count() < 1 || !ls_id.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tablets array is null", K(ret), K(tablets.count()), K(ls_id));
} else if (OB_UNLIKELY(all_ls_to_tablets_map_.size() == 0 || running_ls_to_tablets_map_.size() == 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls to tablets map is null", K(ret), K(all_ls_to_tablets_map_.size()), K(running_ls_to_tablets_map_.size()));
} else {
if (execution_id != -1) { //execution_id == -1 indicates the task before switching to rs is confirming
if (OB_FAIL(running_ls_to_execution_id_.get_refactored(ls_id, ls_execution_id))) {
LOG_WARN("fail to get execution id", K(ret), K(ls_id), K(ls_execution_id));
} else if (OB_UNLIKELY(execution_id != ls_execution_id)) {
ret = OB_TASK_EXPIRED;
LOG_WARN("receive a mismatch execution result", K(ret), K(execution_id), K(ls_execution_id), K(tablets), K(finish_status));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(running_ls_to_tablets_map_.erase_refactored(ls_id))) {
LOG_WARN("failed to erase ls id", K(ret), K(ls_id));
} else if (finish_status) {
ObTabletIdUpdater updater(tablets);
if (OB_FAIL(all_ls_to_tablets_map_.atomic_refactored(ls_id, updater))) {
LOG_WARN("fail to update tablet ids", K(ret), K(ls_id), K(tablets));
} else {
bool is_erased = false;
HashMapEraseIfNull functor;
if (OB_FAIL(all_ls_to_tablets_map_.erase_if(ls_id, functor, is_erased))) {
LOG_WARN("fail to erase lsid", K(ret), K(ls_id), K(tablets));
}
}
}
}
}
LOG_INFO("confirm batch tablets status", K(ret), K(execution_id), K(finish_status), K(ls_id), K(tablets));
return ret;
}
int ObDDLTabletScheduler::get_next_parallelism(int64_t &parallelism)
{
int ret = OB_SUCCESS;
parallelism = 0;
TCRLockGuard guard(lock_);
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret), K(is_inited_));
} else {
int64_t ls_num = all_ls_to_tablets_map_.size();
if (ls_num > 0) { // it is ensured that the value of ls_num is greater than 0
parallelism = (parallelism_ + ls_num - 1) / ls_num;
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the all ls to tablet map size is less than 1", K(ret), K(all_ls_to_tablets_map_.size()), K(parallelism));
}
}
return ret;
}
int ObDDLTabletScheduler::get_running_sql_parallelism(int64_t &parallelism)
{
int ret = OB_SUCCESS;
parallelism = 0;
common::ObAddr inner_sql_exec_addr;
common::ObArray<ObString> running_sql_info;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret), K(is_inited_));
} else if (OB_FAIL(ObDDLTaskRecordOperator::get_running_tasks_inner_sql(root_service_->get_sql_proxy(), trace_id_, tenant_id_, task_id_, snapshot_version_, inner_sql_exec_addr, allocator_, running_sql_info))) {
LOG_WARN("get running tasks inner sql fail", K(ret), K(tenant_id_), K(trace_id_), K(task_id_), K(snapshot_version_), K(inner_sql_exec_addr), K(running_sql_info));
} else {
for (int64_t i = 0; i < running_sql_info.count() && OB_SUCC(ret); i++) {
ObString parallel_flag = ObString::make_string("parallel(");
int64_t loc = ObCharset::instr(ObCollationType::CS_TYPE_UTF8MB4_BIN, running_sql_info.at(i).ptr(), running_sql_info.at(i).length(), parallel_flag.ptr(), parallel_flag.length());
if (OB_UNLIKELY(0 != loc)) {
uint64_t value;
int err = 0;
ObString parallel;
if (OB_FAIL(ob_sub_str(allocator_, running_sql_info.at(i), loc+parallel_flag.length()-1, running_sql_info.at(i).length() - 1, parallel))) {
LOG_WARN("failed to extract parallel info from running sql", K(ret), K(running_sql_info.at(i)), K(parallel_flag), K(parallel));
} else {
parallel = parallel.clip(parallel.find(')'));
if (parallel.is_numeric()) {
value = ObCharset::strntoull(parallel.ptr(), parallel.length(), 10, &err);
parallelism = parallelism + value;
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("parallel value is not int", K(ret), K(running_sql_info.at(i)), K(parallel_flag), K(parallel));
}
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("running sql is wrong", K(ret), K(running_sql_info.at(i)), K(parallel_flag), K(parallelism));
}
}
}
return ret;
}
int ObDDLTabletScheduler::get_unfinished_tablets(const int64_t execution_id, share::ObLSID &ls_id, common::ObAddr &leader_addr, ObIArray<ObTabletID> &tablets)
{
int ret = OB_SUCCESS;
tablets.reset();
ls_id.reset();
leader_addr.reset();
ObArray<ObTabletID> tablet_queue;
uint64_t left_space_size = 0;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret), K(is_inited_));
} else if (OB_FAIL(get_to_be_scheduled_tablets(ls_id, leader_addr, tablet_queue))) {
LOG_WARN("fail to get to be scheduled tablets", K(ret), K(tablet_queue));
} else if (OB_UNLIKELY(tablet_queue.count() < 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet queue is null", K(ret), K(tablet_queue.count()));
} else if (OB_UNLIKELY(!ls_id.is_valid() || !leader_addr.is_valid())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("got argument is error", K(ret), K(ls_id), K(leader_addr));
} else if (OB_FAIL(ObDDLUtil::get_ls_host_left_disk_space(tenant_id_, ls_id, leader_addr, left_space_size))) {
LOG_WARN("fail to get ls host left disk space", K(ret), K(tenant_id_), K(ls_id), K(leader_addr), K(left_space_size));
} else if (OB_FAIL(calculate_candidate_tablets(left_space_size, tablet_queue, tablets))) {
LOG_WARN("fail to use strategy to get tablets", K(ret), K(left_space_size), K(tablet_queue), K(tablets));
} else {
TCWLockGuard guard(lock_);
if (OB_FAIL(running_ls_to_execution_id_.set_refactored(ls_id, execution_id, true /* overwrite */))) {
LOG_WARN("running ls to execution id map set fail", K(ret), K(ls_id), K(execution_id));
} else {
ObArray<ObTabletID> running_tablet_queue;
if (OB_FAIL(running_tablet_queue.assign(tablets))) {
LOG_WARN("ObArray assign failed", K(ret), K(tablets));
} else if (OB_FAIL(running_ls_to_tablets_map_.set_refactored(ls_id, running_tablet_queue, true /* overwrite */))) {
LOG_WARN("ls tablets map set fail", K(ret), K(ls_id), K(running_tablet_queue));
}
}
}
return ret;
}
int ObDDLTabletScheduler::get_to_be_scheduled_tablets(share::ObLSID &ls_id, common::ObAddr &leader_addr, ObIArray<ObTabletID> &tablets)
{
int ret = OB_SUCCESS;
ls_id.reset();
leader_addr.reset();
tablets.reset();
TCWLockGuard guard(lock_);
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret), K(is_inited_));
} else if (OB_UNLIKELY(all_ls_to_tablets_map_.size() <= running_ls_to_tablets_map_.size())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("all ls tablets map is less than running ls tablets map", K(ret), K(all_ls_to_tablets_map_.size()), K(running_ls_to_tablets_map_.size()));
} else {
common::hash::ObHashMap<share::ObLSID, ObArray<ObTabletID>>::iterator iter;
for (iter = all_ls_to_tablets_map_.begin(); iter != all_ls_to_tablets_map_.end() && OB_SUCC(ret); ++iter) {
ls_id = iter->first;
ObArray<ObTabletID> &tablet_queue = iter->second;
ObArray<ObTabletID> running_tablet_queue;
bool is_running_ls = true;
if (OB_FAIL(running_ls_to_tablets_map_.get_refactored(ls_id, running_tablet_queue))) {
if (OB_UNLIKELY(OB_HASH_NOT_EXIST == ret)) {
is_running_ls = false;
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to get tablet queue from refactored", K(ret), K(ls_id));
}
}
if (OB_SUCC(ret) && OB_LIKELY(!is_running_ls && tablet_queue.count() > 0)) {
if (OB_FAIL(ls_location_map_.get_refactored(ls_id, leader_addr))) {
LOG_WARN("fail to get leader addr from ls location map", K(ret), K(ls_id), K(leader_addr));
} else if (OB_FAIL(tablets.assign(tablet_queue))) {
LOG_WARN("ObArray assign failed", K(ret), K(tablet_queue));
}
break;
}
}
}
return ret;
}
int ObDDLTabletScheduler::calculate_candidate_tablets(const uint64_t left_space_size, const ObIArray<ObTabletID> &in_tablets, ObIArray<ObTabletID> &out_tablets)
{
int ret = OB_SUCCESS;
ObSchemaGetterGuard schema_guard;
const ObTableSchema *data_table_schema = nullptr;
const ObTableSchema *index_schema = nullptr;
out_tablets.reset();
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(tenant_id_, schema_guard))) {
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id_));
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, ref_data_table_id_, data_table_schema))) {
LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(ref_data_table_id_));
} else if (OB_ISNULL(data_table_schema)) {
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("error unexpected, data table schema is null", K(ret), K(ref_data_table_id_));
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_, table_id_, index_schema))) {
LOG_WARN("get table schema failed", K(ret), K(tenant_id_), K(table_id_));
} else if (OB_ISNULL(index_schema)) {
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("error unexpected, index table schema is null", K(ret), K(table_id_));
} else {
ObPartition **data_partitions = data_table_schema->get_part_array();
const ObPartitionLevel part_level = data_table_schema->get_part_level();
if (OB_ISNULL(data_partitions)) {
ret = OB_PARTITION_NOT_EXIST;
LOG_WARN("data table part array is null", K(ret), KPC(this));
} else {
int64_t part_index = -1;
int64_t subpart_index = -1;
int64_t pre_data_size = 0;
int64_t pre_data_row_cnt = 0;
int64_t tablet_data_size = 0;
int64_t tablet_data_row_cnt = 0;
uint64_t task_max_data_size = 0;
const int64_t task_max_data_row_cnt = 50000000;
if (left_space_size > 0) {
task_max_data_size = left_space_size / 30; // according to the estimated maximum temporary space amplification factor 30, ensure that the current remaining disk space can complete index construction
} else {
task_max_data_size = 5368709120; // 5GB
}
for (int64_t i = 0; i < in_tablets.count() && OB_SUCC(ret); i++) {
tablet_data_size = 0;
tablet_data_row_cnt = 0;
if (OB_FAIL(index_schema->get_part_idx_by_tablet(in_tablets.at(i), part_index, subpart_index))) {
LOG_WARN("failed to get part idx by tablet", K(ret), K(in_tablets.at(i)), K(part_index), K(subpart_index));
} else {
if (PARTITION_LEVEL_ONE == part_level) {
if (OB_FAIL(tablet_id_to_data_size_.get_refactored(data_partitions[part_index]->get_tablet_id().id(), tablet_data_size))) {
LOG_WARN("fail to get tablet data size", K(ret), K(data_partitions[part_index]->get_tablet_id()), K(tablet_data_size));
} else if (OB_FAIL(tablet_id_to_data_row_cnt_.get_refactored(data_partitions[part_index]->get_tablet_id().id(), tablet_data_row_cnt))) {
LOG_WARN("fail to get tablet data size", K(ret), K(data_partitions[part_index]->get_tablet_id()), K(tablet_data_row_cnt));
}
} else if (PARTITION_LEVEL_TWO == part_level) {
ObSubPartition **data_subpart_array = data_partitions[part_index]->get_subpart_array();
if (OB_ISNULL(data_subpart_array)) {
ret = OB_PARTITION_NOT_EXIST;
LOG_WARN("part array is null", K(ret), KPC(this));
} else if (OB_FAIL(tablet_id_to_data_size_.get_refactored(data_subpart_array[subpart_index]->get_tablet_id().id(), tablet_data_size))) {
LOG_WARN("fail to get tablet data size", K(ret), K(data_subpart_array[subpart_index]->get_tablet_id()), K(tablet_data_size));
} else if (OB_FAIL(tablet_id_to_data_row_cnt_.get_refactored(data_subpart_array[subpart_index]->get_tablet_id().id(), tablet_data_row_cnt))) {
LOG_WARN("fail to get tablet data size", K(ret), K(data_subpart_array[subpart_index]->get_tablet_id()), K(tablet_data_row_cnt));
}
}
if (OB_SUCC(ret)) {
if (pre_data_size == 0 || ((tablet_data_row_cnt + pre_data_row_cnt) <= task_max_data_row_cnt && (tablet_data_size + pre_data_size) <= task_max_data_size)) {
if (OB_FAIL(out_tablets.push_back(in_tablets.at(i)))) {
LOG_WARN("fail to push back", K(ret), K(in_tablets.at(i)));
} else {
pre_data_size = pre_data_size + tablet_data_size;
pre_data_row_cnt = pre_data_row_cnt + tablet_data_row_cnt;
}
} else {
break;
}
}
}
}
}
}
return ret;
}
int ObDDLTabletScheduler::get_session_running_lsid(ObIArray<share::ObLSID> &running_ls_ids)
{
int ret = OB_SUCCESS;
common::ObAddr inner_sql_exec_addr;
common::ObArray<ObString> running_sql_info;
running_ls_ids.reset();
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(ObDDLTaskRecordOperator::get_running_tasks_inner_sql(root_service_->get_sql_proxy(), trace_id_, tenant_id_, task_id_, snapshot_version_, inner_sql_exec_addr, allocator_, running_sql_info))) {
LOG_WARN("get running tasks inner sql fail", K(ret), K(tenant_id_), K(trace_id_), K(task_id_), K(snapshot_version_), K(inner_sql_exec_addr), K(running_sql_info));
} else {
TCRLockGuard guard(lock_);
common::hash::ObHashMap<share::ObLSID, ObArray<ObTabletID>>::iterator iter;
for (iter = running_ls_to_tablets_map_.begin(); iter != running_ls_to_tablets_map_.end() && OB_SUCC(ret); ++iter) {
share::ObLSID &ls_id = iter->first;
ObArray<ObTabletID> &tablet_queue = iter->second;
ObArray<ObString> partition_names;
if (OB_FAIL(ObDDLUtil::get_index_table_batch_partition_names(tenant_id_, ref_data_table_id_, table_id_, tablet_queue, allocator_, partition_names))) {
LOG_WARN("fail to get index table batch partition names", K(ret), K(tenant_id_), K(ref_data_table_id_), K(table_id_), K(tablet_queue), K(partition_names));
} else {
bool is_running_status = false;
for (int64_t i = 0; i < partition_names.count() && OB_SUCC(ret); i++) {
is_running_status = false;
for (int64_t j = 0; j < running_sql_info.count() && OB_SUCC(ret); j++) {
if (OB_FAIL(ObDDLUtil::check_target_partition_is_running(running_sql_info.at(j), partition_names.at(i), allocator_, is_running_status))) {
LOG_WARN("fail to check target partition is running", K(ret), K(running_sql_info.at(j)), K(partition_names.at(i)), K(is_running_status));
} else if (is_running_status) {
break;
}
}
if (is_running_status) {
break;
}
}
if (is_running_status && OB_SUCC(ret)) {
if (OB_FAIL(running_ls_ids.push_back(ls_id))) {
LOG_WARN("ObArray assign failed", K(ret), K(ls_id));
}
}
}
}
}
return ret;
}
int ObDDLTabletScheduler::get_target_running_ls_tablets(const share::ObLSID &ls_id, ObIArray<ObTabletID> &tablets)
{
int ret = OB_SUCCESS;
ObArray<ObTabletID> tablet_queue;
tablets.reset();
TCRLockGuard guard(lock_);
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(running_ls_to_tablets_map_.get_refactored(ls_id, tablet_queue))) {
if (OB_UNLIKELY(OB_HASH_NOT_EXIST == ret)) { // tasks in this ls have finished and reported reply
LOG_WARN("ls id is not exist in map", K(ret), K(ls_id), K(tablets));
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to get tablet queue from refactored", K(ret), K(ls_id), K(tablets));
}
} else if (OB_FAIL(tablets.assign(tablet_queue))) {
LOG_WARN("ObArray assign failed", K(ret), K(tablet_queue));
}
return ret;
}
int ObDDLTabletScheduler::get_potential_finished_lsid(const ObIArray<share::ObLSID> &running_ls_ids_now, ObIArray<share::ObLSID> &potential_finished_ls_ids)
{
int ret = OB_SUCCESS;
ObArray<share::ObLSID> current_running_ls_ids;
TCWLockGuard guard(lock_);
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(common::get_difference(running_task_ls_ids_before_, running_ls_ids_now, potential_finished_ls_ids))) {
LOG_WARN("get difference failed", K(ret), K(running_task_ls_ids_before_), K(running_ls_ids_now), K(potential_finished_ls_ids));
} else if (OB_FAIL(common::get_difference(running_task_ls_ids_before_, potential_finished_ls_ids, current_running_ls_ids))) {
LOG_WARN("get difference failed", K(ret), K(running_task_ls_ids_before_), K(potential_finished_ls_ids), K(current_running_ls_ids));
} else if (OB_FAIL(running_task_ls_ids_before_.assign(current_running_ls_ids))) {
LOG_WARN("ObArray assign failed", K(ret), K(current_running_ls_ids));
}
return ret;
}
int ObDDLTabletScheduler::determine_if_need_to_send_new_task(bool &status)
{
int ret = OB_SUCCESS;
status = false;
TCRLockGuard guard(lock_);
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (running_ls_to_tablets_map_.size() == all_ls_to_tablets_map_.size()) {
status = false;
} else if (running_ls_to_tablets_map_.size() < all_ls_to_tablets_map_.size()) {
status = true;
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sub map size bigger than map size", K(ret), K(status), K(running_ls_to_tablets_map_.size()), K(all_ls_to_tablets_map_.size()));
}
return ret;
}
int ObDDLTabletScheduler::check_target_ls_tasks_completion_status(const share::ObLSID &ls_id)
{
int ret = OB_SUCCESS;
ObArray<ObTabletID> running_tablet_queue;
common::hash::ObHashMap<uint64_t, bool> tablet_checksum_status_map;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if ((OB_UNLIKELY(!ls_id.is_valid()))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("the parameters is invalid", K(ret), K(ls_id));
} else if (OB_FAIL(get_target_running_ls_tablets(ls_id, running_tablet_queue))) {
LOG_WARN("fail to get target running ls tablets", K(ret), K(ls_id),K(running_tablet_queue));
} else if (OB_UNLIKELY(running_tablet_queue.count() < 1)) {
// do nothing, the ls tasks have finished and reported
} else if (OB_FAIL(tablet_checksum_status_map.create(running_tablet_queue.count(), ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create tablet checksum status map", K(ret), K(running_tablet_queue.count()));
} else if (OB_ISNULL(GCTX.root_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("rootservice is null", K(ret));
} else if (OB_FAIL(ObDDLChecksumOperator::get_tablet_checksum_record_without_execution_id(
tenant_id_,
table_id_,
task_id_,
running_tablet_queue,
GCTX.root_service_->get_sql_proxy(),
tablet_checksum_status_map))) {
LOG_WARN("fail to get tablet checksum status", K(ret), K(tenant_id_), K(table_id_), K(task_id_), K(running_tablet_queue));
} else {
bool is_finished_status = true;
for (int64_t i = 0; i < running_tablet_queue.count() && OB_SUCC(ret); i++) {
if (OB_FAIL(tablet_checksum_status_map.get_refactored(running_tablet_queue.at(i).id(), is_finished_status))) {
if (OB_HASH_NOT_EXIST == ret) {
LOG_WARN("tablet checksum is not exist", K(ret), K(running_tablet_queue.at(i)), K(is_finished_status));
is_finished_status = false;
ret = OB_SUCCESS;
break;
} else {
LOG_WARN("fail to get refactored", K(ret), K(running_tablet_queue.at(i)), K(is_finished_status));
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(confirm_batch_tablets_status(-1, is_finished_status, ls_id, running_tablet_queue))) {
LOG_WARN("fail to confirm batch tablets status", K(ret), K(is_finished_status), K(running_tablet_queue));
}
}
}
return ret;
}
bool ObDDLTabletScheduler::is_all_tasks_finished()
{
TCRLockGuard guard(lock_);
return all_ls_to_tablets_map_.size() < 1;
}
bool ObDDLTabletScheduler::is_running_tasks_before_finished()
{
TCRLockGuard guard(lock_);
return running_task_ls_ids_before_.count() < 1;
}
void ObDDLTabletScheduler::destroy()
{
is_inited_ = false;
tenant_id_ = 0;
table_id_ = 0;
ref_data_table_id_ = 0;
task_id_ = 0;
parallelism_ = 0;
snapshot_version_ = 0;
trace_id_.reset();
all_tablets_.reset();
running_task_ls_ids_before_.reset();
all_ls_to_tablets_map_.destroy();
running_ls_to_tablets_map_.destroy();
ls_location_map_.destroy();
running_ls_to_execution_id_.destroy();
tablet_id_to_data_size_.destroy();
tablet_id_to_data_row_cnt_.destroy();
}
int ObTabletIdUpdater::operator() (common::hash::HashMapPair<share::ObLSID, ObArray<ObTabletID>> &entry) {
int ret = OB_SUCCESS;
for (int64_t i = 0; i < tablets_->count() && OB_SUCC(ret); i++) {
for (int64_t j = 0; j < entry.second.count() && OB_SUCC(ret); j++) {
if (tablets_->at(i) == entry.second.at(j)) {
if (OB_FAIL(entry.second.remove(j))) {
LOG_WARN("failed to remove tablet id", K(ret), K(i), K(j), K(entry.second), KP(tablets_));
}
break;
}
}
}
LOG_INFO("remove tablet ids from hash map", K(entry), KP(tablets_));
return ret;
}
bool HashMapEraseIfNull::operator() (common::hash::HashMapPair<share::ObLSID, ObArray<ObTabletID>> &entry) {
return entry.second.count() == 0;
}

View File

@ -0,0 +1,98 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_ROOTSERVER_OB_DDL_TABLET_SCHEDULER_H
#define OCEANBASE_ROOTSERVER_OB_DDL_TABLET_SCHEDULER_H
#include "rootserver/ddl_task/ob_ddl_task.h"
namespace oceanbase
{
namespace rootserver
{
class ObDDLTabletScheduler final
{
public:
ObDDLTabletScheduler();
~ObDDLTabletScheduler();
int init(const uint64_t tenant_id,
const uint64_t table_id,
const uint64_t ref_data_table_id,
const int64_t task_id,
const int64_t parallelism,
const int64_t snapshot_version,
const common::ObCurTraceId::TraceId &trace_id,
const ObIArray<ObTabletID> &tablets);
int get_next_batch_tablets(int64_t &parallelism, int64_t &new_execution_id, share::ObLSID &ls_id, common::ObAddr &leader_addr, ObIArray<ObTabletID> &tablets);
int confirm_batch_tablets_status(const int64_t execution_id, const bool finish_status, const share::ObLSID &ls_id, const ObIArray<ObTabletID> &tablets);
TO_STRING_KV(K_(is_inited), K_(tenant_id), K_(table_id), K_(ref_data_table_id),
K_(task_id), K_(parallelism), K_(snapshot_version), K_(trace_id), K_(all_tablets), K_(running_task_ls_ids_before));
private:
int get_next_parallelism(int64_t &parallelism);
int get_running_sql_parallelism(int64_t &parallelism);
int get_unfinished_tablets(const int64_t execution_id, share::ObLSID &ls_id, common::ObAddr &leader_addr, ObIArray<ObTabletID> &tablets);
int get_to_be_scheduled_tablets(share::ObLSID &ls_id, common::ObAddr &leader_addr, ObIArray<ObTabletID> &tablets);
int calculate_candidate_tablets(const uint64_t left_space_size, const ObIArray<ObTabletID> &in_tablets, ObIArray<ObTabletID> &out_tablets);
int get_session_running_lsid(ObIArray<share::ObLSID> &running_ls_ids);
int get_target_running_ls_tablets(const share::ObLSID &ls_id, ObIArray<ObTabletID> &tablets);
int get_potential_finished_lsid(const ObIArray<share::ObLSID> &running_ls_ids_now, ObIArray<share::ObLSID> &potential_finished_ls_ids);
int determine_if_need_to_send_new_task(bool &status);
int check_target_ls_tasks_completion_status(const share::ObLSID &ls_id);
bool is_all_tasks_finished();
bool is_running_tasks_before_finished();
void destroy();
private:
bool is_inited_;
uint64_t tenant_id_;
uint64_t table_id_;
uint64_t ref_data_table_id_;
int64_t task_id_;
int64_t parallelism_;
int64_t snapshot_version_;
common::ObCurTraceId::TraceId trace_id_;
common::TCRWLock lock_; // this lock is used to protect read and write operations of class members: running_task_ls_ids_before_、 all_ls_to_tablets_map_、 running_ls_to_tablets_map_、 running_ls_to_execution_id_, to avoid conflicts between ddl_builder task and ddl_scheduler task.
common::ObArenaAllocator allocator_;
ObRootService *root_service_;
ObArray<ObTabletID> all_tablets_;
ObArray<share::ObLSID> running_task_ls_ids_before_; // this is the used lsid array where the tablets is located when init ObDDLTabletScheduler;
common::hash::ObHashMap<share::ObLSID, ObArray<ObTabletID>> all_ls_to_tablets_map_;
common::hash::ObHashMap<share::ObLSID, ObArray<ObTabletID>> running_ls_to_tablets_map_;
common::hash::ObHashMap<share::ObLSID, common::ObAddr> ls_location_map_;
common::hash::ObHashMap<share::ObLSID, int64_t> running_ls_to_execution_id_;
common::hash::ObHashMap<int64_t, int64_t> tablet_id_to_data_size_;
common::hash::ObHashMap<int64_t, int64_t> tablet_id_to_data_row_cnt_;
};
class ObTabletIdUpdater final
{
public:
ObTabletIdUpdater(const ObIArray<ObTabletID> &tablets) : tablets_(&tablets) {};
~ObTabletIdUpdater() {};
int operator() (common::hash::HashMapPair<share::ObLSID, ObArray<ObTabletID>> &entry);
private:
const ObIArray<ObTabletID> *tablets_;
DISALLOW_COPY_AND_ASSIGN(ObTabletIdUpdater);
};
class HashMapEraseIfNull final
{
public:
HashMapEraseIfNull() {};
~HashMapEraseIfNull() {};
bool operator() (common::hash::HashMapPair<share::ObLSID, ObArray<ObTabletID>> &entry);
private:
DISALLOW_COPY_AND_ASSIGN(HashMapEraseIfNull);
};
} // end namespace rootserver
} // end namespace oceanbase
#endif /* OCEANBASE_ROOTSERVER_OB_DDL_TABLET_SCHEDULER_H */

View File

@ -283,6 +283,7 @@ trace::ObSpanCtx* ObDDLTracing::begin_task_span()
} else {
switch (task_type) {
case DDL_CREATE_INDEX:
case DDL_CREATE_PARTITIONED_LOCAL_INDEX:
span = FLT_BEGIN_SPAN(ddl_build_index);
break;
case DDL_DROP_INDEX:
@ -366,6 +367,7 @@ trace::ObSpanCtx* ObDDLTracing::restore_task_span()
} else {
switch (task_type) {
case DDL_CREATE_INDEX:
case DDL_CREATE_PARTITIONED_LOCAL_INDEX:
span = FLT_RESTORE_DDL_SPAN(ddl_build_index, task_span_id_, task_start_ts_);
break;
case DDL_DROP_INDEX:
@ -734,6 +736,9 @@ int ObDDLTask::get_ddl_type_str(const int64_t ddl_type, const char *&ddl_type_st
case DDL_CREATE_INDEX:
ddl_type_str = "create index";
break;
case DDL_CREATE_PARTITIONED_LOCAL_INDEX:
ddl_type_str = "create partitioned local index";
break;
case DDL_MODIFY_COLUMN:
ddl_type_str = "modify column";
break;
@ -1159,7 +1164,7 @@ int ObDDLTask::switch_status(const ObDDLTaskStatus new_status, const bool enable
if (OB_CANCELED == real_ret_code || ObDDLTaskStatus::FAIL == task_status_) {
(void)ObDDLTaskRecordOperator::kill_task_inner_sql(root_service->get_sql_proxy(),
trace_id_, dst_tenant_id_, task_id_, snapshot_version_, sql_exec_addr_); // ignore return code
trace_id_, dst_tenant_id_, task_id_, snapshot_version_, sql_exec_addrs_); // ignore return code
LOG_WARN("ddl_task switch_status kill_task_inner_sql");
}
}
@ -1238,7 +1243,7 @@ int ObDDLTask::report_error_code(const ObString &forward_user_message, const int
LOG_WARN("load ddl user error failed", K(ret), K(dst_tenant_id_), K(task_id_), K(object_id_));
if (OB_ITER_END == ret) { // no single replica error message found, use ret_code_
ret = OB_SUCCESS;
if (is_oracle_mode && DDL_CREATE_INDEX != task_type_ && OB_ERR_DUPLICATED_UNIQUE_KEY == ret_code_) {
if (is_oracle_mode && DDL_CREATE_INDEX != task_type_ && DDL_CREATE_PARTITIONED_LOCAL_INDEX != task_type_ && OB_ERR_DUPLICATED_UNIQUE_KEY == ret_code_) {
ret_code_ = OB_ERR_PRIMARY_KEY_DUPLICATE;
}
const char *ddl_type_str = nullptr;
@ -1260,7 +1265,7 @@ int ObDDLTask::report_error_code(const ObString &forward_user_message, const int
LOG_WARN("print ddl user message failed", K(ret));
}
}
} else if (is_oracle_mode && DDL_CREATE_INDEX != task_type_ && OB_ERR_DUPLICATED_UNIQUE_KEY == error_message.ret_code_) {
} else if (is_oracle_mode && DDL_CREATE_INDEX != task_type_ && DDL_CREATE_PARTITIONED_LOCAL_INDEX != task_type_ && OB_ERR_DUPLICATED_UNIQUE_KEY == error_message.ret_code_) {
error_message.ret_code_ = OB_ERR_PRIMARY_KEY_DUPLICATE;
const char *str_user_error = ob_errpkt_str_user_error(ret_code_, is_oracle_mode);
const char *str_error = ob_errpkt_strerror(error_message.ret_code_, is_oracle_mode);
@ -3138,8 +3143,8 @@ int ObDDLTaskRecordOperator::check_has_index_or_mlog_task(
ObSqlString sql_string;
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
sqlclient::ObMySQLResult *result = NULL;
if (OB_FAIL(sql_string.assign_fmt("SELECT EXISTS(SELECT 1 FROM %s WHERE object_id = %lu AND target_object_id = %lu AND ddl_type IN (%d, %d, %d, %d)) as has",
OB_ALL_DDL_TASK_STATUS_TNAME, data_table_id, index_table_id, ObDDLType::DDL_CREATE_INDEX, ObDDLType::DDL_DROP_INDEX,
if (OB_FAIL(sql_string.assign_fmt("SELECT EXISTS(SELECT 1 FROM %s WHERE object_id = %lu AND target_object_id = %lu AND ddl_type IN (%d, %d, %d, %d, %d)) as has",
OB_ALL_DDL_TASK_STATUS_TNAME, data_table_id, index_table_id, ObDDLType::DDL_CREATE_INDEX, ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX, ObDDLType::DDL_DROP_INDEX,
ObDDLType::DDL_CREATE_MLOG, ObDDLType::DDL_DROP_MLOG))) {
LOG_WARN("assign sql string failed", K(ret));
} else if (OB_FAIL(proxy.read(res, tenant_id, sql_string.ptr()))) {
@ -3172,8 +3177,8 @@ int ObDDLTaskRecordOperator::get_create_index_or_mlog_task_cnt(
ObSqlString sql_string;
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
sqlclient::ObMySQLResult *result = NULL;
if (OB_FAIL(sql_string.assign_fmt("SELECT COUNT(*) as cnt FROM %s WHERE object_id = %lu AND ddl_type IN (%d, %d)",
OB_ALL_DDL_TASK_STATUS_TNAME, data_table_id, ObDDLType::DDL_CREATE_INDEX, ObDDLType::DDL_CREATE_MLOG))) {
if (OB_FAIL(sql_string.assign_fmt("SELECT COUNT(*) as cnt FROM %s WHERE object_id = %lu AND ddl_type IN (%d, %d, %d)",
OB_ALL_DDL_TASK_STATUS_TNAME, data_table_id, ObDDLType::DDL_CREATE_INDEX, ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX, ObDDLType::DDL_CREATE_MLOG))) {
LOG_WARN("assign sql string failed", K(ret));
} else if (OB_FAIL(proxy.read(res, tenant_id, sql_string.ptr()))) {
LOG_WARN("query ddl task record failed", K(ret), K(sql_string));
@ -3579,16 +3584,121 @@ int ObDDLTaskRecordOperator::kill_task_inner_sql(
const uint64_t tenant_id,
const int64_t task_id,
const int64_t snapshot_version,
const common::ObAddr &sql_exec_addr)
const ObIArray<common::ObAddr> &sql_exec_addrs)
{
int ret = OB_SUCCESS;
char ip_str[common::OB_IP_STR_BUFF];
if (OB_UNLIKELY(!proxy.is_inited() || trace_id.is_invalid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(proxy.is_inited()));
LOG_WARN("invalid argument", K(ret), K(proxy.is_inited()), K(trace_id));
} else {
LOG_INFO("start ddl kill inner sql session", K(ret), K(trace_id));
ObSqlString sql_string;
for (int64_t i = 0; i < sql_exec_addrs.count() && OB_SUCC(ret); i++) {
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
sqlclient::ObMySQLResult *result = NULL;
char trace_id_str[64] = { 0 };
char spec_charater = '%';
if (OB_UNLIKELY(0 > trace_id.to_string(trace_id_str, sizeof(trace_id_str)))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get trace id string failed", K(ret), K(trace_id), K(tenant_id));
} else if (!sql_exec_addrs.at(i).is_valid()) {
if (OB_FAIL(sql_string.assign_fmt(" SELECT id as session_id FROM %s WHERE trace_id = \"%s\" "
" and tenant = (select tenant_name from __all_tenant where tenant_id = %lu) "
" and info like \"%cINSERT%c('ddl_task_id', %ld)%cINTO%cSELECT%c%ld%c\" ",
OB_ALL_VIRTUAL_SESSION_INFO_TNAME,
trace_id_str,
tenant_id,
spec_charater,
spec_charater,
task_id,
spec_charater,
spec_charater,
spec_charater,
snapshot_version,
spec_charater))) {
LOG_WARN("assign sql string failed", K(ret));
}
} else {
if (!sql_exec_addrs.at(i).ip_to_string(ip_str, sizeof(ip_str))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ip to string failed", K(ret), K(sql_exec_addrs.at(i)));
} else if (OB_FAIL(sql_string.assign_fmt(" SELECT id as session_id FROM %s WHERE trace_id = \"%s\" "
" and tenant = (select tenant_name from __all_tenant where tenant_id = %lu) "
" and svr_ip = \"%s\" and svr_port = %d and info like \"%cINSERT%c('ddl_task_id', %ld)%cINTO%cSELECT%c%ld%c\" ",
OB_ALL_VIRTUAL_SESSION_INFO_TNAME,
trace_id_str,
tenant_id,
ip_str,
sql_exec_addrs.at(i).get_port(),
spec_charater,
spec_charater,
task_id,
spec_charater,
spec_charater,
spec_charater,
snapshot_version,
spec_charater))) {
LOG_WARN("assign sql string failed", K(ret));
}
}
LOG_INFO("kill session inner sql", K(sql_string), K(task_id), K(sql_exec_addrs.at(i)));
if (OB_FAIL(ret)) {
} else if (OB_FAIL(DDL_SIM(tenant_id, task_id, TASK_STATUS_OPERATOR_SLOW))) {
LOG_WARN("ddl sim failure: slow inner sql", K(ret), K(tenant_id), K(task_id));
} else if (OB_FAIL(DDL_SIM(tenant_id, task_id, KILL_TASK_BY_INNER_SQL_FAILED))) {
LOG_WARN("ddl sim failure", K(ret), K(tenant_id), K(task_id));
} else if (OB_FAIL(proxy.read(res, OB_SYS_TENANT_ID, sql_string.ptr(), &sql_exec_addrs.at(i)))) { // default use OB_SYS_TENANT_ID
LOG_WARN("query ddl task record failed", K(ret), K(sql_string));
} else if (OB_ISNULL((result = res.get_result()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get sql result", K(ret), KP(result));
} else {
uint64_t session_id = 0;
while (OB_SUCC(ret)) {
if (OB_FAIL(result->next())) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
break;
} else {
LOG_WARN("fail to get next row", K(ret));
}
} else {
EXTRACT_UINT_FIELD_MYSQL(*result, "session_id", session_id, uint64_t);
if (OB_SUCC(ret)) {
if (OB_FAIL(kill_inner_sql(proxy, tenant_id, session_id))){
LOG_WARN("fail to kill session", K(ret), K(session_id), K(trace_id));
} else {
LOG_WARN("succ to kill session", K(ret), K(session_id), K(trace_id));
}
}
}
}
}
}
}
}
return ret;
}
int ObDDLTaskRecordOperator::get_running_tasks_inner_sql(
common::ObMySQLProxy &proxy,
const common::ObCurTraceId::TraceId &trace_id,
const uint64_t tenant_id,
const int64_t task_id,
const int64_t snapshot_version,
const common::ObAddr &sql_exec_addr,
common::ObIAllocator &allocator,
common::ObIArray<ObString> &records)
{
int ret = OB_SUCCESS;
records.reset();
char ip_str[common::OB_IP_STR_BUFF];
if (OB_UNLIKELY(!proxy.is_inited() || trace_id.is_invalid() || OB_INVALID_ID == tenant_id || OB_INVALID_ID == task_id || snapshot_version <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(proxy.is_inited()), K(trace_id), K(tenant_id), K(task_id), K(snapshot_version));
} else {
ObSqlString sql_string;
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
sqlclient::ObMySQLResult *result = NULL;
@ -3598,9 +3708,9 @@ int ObDDLTaskRecordOperator::kill_task_inner_sql(
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get trace id string failed", K(ret), K(trace_id), K(tenant_id));
} else if (!sql_exec_addr.is_valid()) {
if (OB_FAIL(sql_string.assign_fmt(" SELECT id as session_id FROM %s WHERE trace_id = \"%s\" "
if (OB_FAIL(sql_string.assign_fmt(" SELECT info FROM %s WHERE trace_id = \"%s\" "
" and tenant = (select tenant_name from __all_tenant where tenant_id = %lu) "
" and info like \"%cINSERT%c('ddl_task_id', %ld)%cINTO%cSELECT%c%ld%c\" ",
" and info like \"%cINSERT%c('ddl_task_id', %ld)%cINTO%cSELECT%cPARTITION%c%ld%c\" ",
OB_ALL_VIRTUAL_SESSION_INFO_TNAME,
trace_id_str,
tenant_id,
@ -3610,6 +3720,7 @@ int ObDDLTaskRecordOperator::kill_task_inner_sql(
spec_charater,
spec_charater,
spec_charater,
spec_charater,
snapshot_version,
spec_charater))) {
LOG_WARN("assign sql string failed", K(ret));
@ -3618,9 +3729,9 @@ int ObDDLTaskRecordOperator::kill_task_inner_sql(
if (!sql_exec_addr.ip_to_string(ip_str, sizeof(ip_str))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ip to string failed", K(ret), K(sql_exec_addr));
} else if (OB_FAIL(sql_string.assign_fmt(" SELECT id as session_id FROM %s WHERE trace_id = \"%s\" "
} else if (OB_FAIL(sql_string.assign_fmt(" SELECT info FROM %s WHERE trace_id = \"%s\" "
" and tenant = (select tenant_name from __all_tenant where tenant_id = %lu) "
" and svr_ip = \"%s\" and svr_port = %d and info like \"%cINSERT%c('ddl_task_id', %ld)%cINTO%cSELECT%c%ld%c\" ",
" and svr_ip = \"%s\" and svr_port = %d and info like \"%cINSERT%c('ddl_task_id', %ld)%cINTO%cSELECT%cPARTITION%c%ld%c\" ",
OB_ALL_VIRTUAL_SESSION_INFO_TNAME,
trace_id_str,
tenant_id,
@ -3632,24 +3743,20 @@ int ObDDLTaskRecordOperator::kill_task_inner_sql(
spec_charater,
spec_charater,
spec_charater,
spec_charater,
snapshot_version,
spec_charater))) {
LOG_WARN("assign sql string failed", K(ret));
LOG_WARN("assign sql string failed", K(ret), K(sql_exec_addr.get_port()));
}
}
LOG_INFO("kill session inner sql", K(sql_string), K(task_id), K(sql_exec_addr));
if (OB_FAIL(ret)) {
} else if (OB_FAIL(DDL_SIM(tenant_id, task_id, TASK_STATUS_OPERATOR_SLOW))) {
LOG_WARN("ddl sim failure: slow inner sql", K(ret), K(tenant_id), K(task_id));
} else if (OB_FAIL(DDL_SIM(tenant_id, task_id, KILL_TASK_BY_INNER_SQL_FAILED))) {
LOG_WARN("ddl sim failure", K(ret), K(tenant_id), K(task_id));
} else if (OB_FAIL(proxy.read(res, OB_SYS_TENANT_ID, sql_string.ptr(), &sql_exec_addr))) { // default use OB_SYS_TENANT_ID
LOG_WARN("query ddl task record failed", K(ret), K(sql_string));
} else if (OB_ISNULL((result = res.get_result()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get sql result", K(ret), KP(result));
} else {
uint64_t session_id = 0;
ObString info;
while (OB_SUCC(ret)) {
if (OB_FAIL(result->next())) {
if (OB_ITER_END == ret) {
@ -3659,12 +3766,15 @@ int ObDDLTaskRecordOperator::kill_task_inner_sql(
LOG_WARN("fail to get next row", K(ret));
}
} else {
EXTRACT_UINT_FIELD_MYSQL(*result, "session_id", session_id, uint64_t);
EXTRACT_VARCHAR_FIELD_MYSQL(*result, "info", info);
if (OB_SUCC(ret)) {
if (OB_FAIL(kill_inner_sql(proxy, tenant_id, session_id))){
LOG_WARN("fail to kill session", K(ret), K(session_id), K(trace_id));
} else {
LOG_WARN("succ to kill session", K(ret), K(session_id), K(trace_id));
ObString deep_copy_info;
if (OB_FAIL(deep_copy_ob_string(allocator,
info,
deep_copy_info))) {
LOG_WARN("fail to deep copy partition names", K(ret), K(info));
} else if (OB_FAIL(records.push_back(deep_copy_info))) {
LOG_WARN("fail to push back executing sql info", K(ret), K(deep_copy_info));
}
}
}
@ -3672,6 +3782,7 @@ int ObDDLTaskRecordOperator::kill_task_inner_sql(
}
}
}
LOG_INFO("get running sql info", K(ret), K(trace_id), K(tenant_id), K(task_id), K(snapshot_version), K(sql_exec_addr.get_port()), K(records));
return ret;
}
@ -3688,7 +3799,53 @@ int ObDDLTask::init_ddl_task_monitor_info(const uint64_t target_table_id)
return ret;
}
int ObDDLTask::set_sql_exec_addr(const common::ObAddr &addr)
{
int ret = OB_SUCCESS;
TCWLockGuard guard(lock_);
if (OB_UNLIKELY(!addr.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(addr));
} else {
bool is_exist = false;
for (int64_t i = 0; i < sql_exec_addrs_.count(); i++) {
if (sql_exec_addrs_.at(i) == addr) {
is_exist = true;
break;
}
}
if (!is_exist) {
if (OB_FAIL(sql_exec_addrs_.push_back(addr))) {
LOG_WARN("push back sql exec addrs array failed", K(ret), K(addr), K(sql_exec_addrs_));
}
}
}
return ret;
}
int ObDDLTask::remove_sql_exec_addr(const common::ObAddr &addr)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!addr.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(addr));
} else {
int64_t i = 0;
bool is_exist = false;
for (i = 0; i < sql_exec_addrs_.count(); i++) {
if (sql_exec_addrs_.at(i) == addr) {
is_exist = true;
break;
}
}
if (is_exist) {
if (OB_FAIL(sql_exec_addrs_.remove(i))) {
LOG_WARN("failed to remove addr", K(ret), K(i), K(sql_exec_addrs_), K(addr));
}
}
}
return ret;
}
} // end namespace rootserver
} // end namespace oceanbase

View File

@ -105,10 +105,13 @@ struct ObDDLTaskInfo final
public:
ObDDLTaskInfo() : row_scanned_(0), row_inserted_(0) {}
~ObDDLTaskInfo() {}
TO_STRING_KV(K_(row_scanned), K_(row_inserted));
TO_STRING_KV(K_(row_scanned), K_(row_inserted), K_(ls_id), K_(ls_leader_addr), K_(partition_ids));
public:
int64_t row_scanned_;
int64_t row_inserted_;
share::ObLSID ls_id_;
common::ObAddr ls_leader_addr_;
ObArray<ObTabletID> partition_ids_;
};
struct ObFTSDDLChildTaskInfo final
@ -338,7 +341,18 @@ public:
const uint64_t tenant_id,
const int64_t task_id,
const int64_t snapshot_version,
const common::ObAddr &sql_exec_addr);
const ObIArray<common::ObAddr> &sql_exec_addrs);
//query the internal table __all_virtual_session_info to obtain the executing tasks sql meeting specified mode.
static int get_running_tasks_inner_sql(
common::ObMySQLProxy &proxy,
const common::ObCurTraceId::TraceId &trace_id,
const uint64_t tenant_id,
const int64_t task_id,
const int64_t snapshot_version,
const common::ObAddr &sql_exec_addr,
common::ObIAllocator &allocator,
common::ObIArray<ObString> &records);
private:
static int fill_task_record(
@ -528,7 +542,7 @@ public:
parent_task_id_(0), parent_task_key_(), task_version_(0), parallelism_(0),
allocator_(lib::ObLabel("DdlTask")), compat_mode_(lib::Worker::CompatMode::INVALID), err_code_occurence_cnt_(0),
longops_stat_(nullptr), gmt_create_(0), stat_info_(), delay_schedule_time_(0), next_schedule_ts_(0),
execution_id_(-1), sql_exec_addr_(), start_time_(0), data_format_version_(0), is_pre_split_(false)
execution_id_(-1), start_time_(0), data_format_version_(0), is_pre_split_(false)
{}
virtual ~ObDDLTask() {}
virtual int process() = 0;
@ -590,8 +604,9 @@ public:
int batch_release_snapshot(
const int64_t snapshot_version,
const common::ObIArray<common::ObTabletID> &tablet_ids);
int set_sql_exec_addr(const common::ObAddr &addr);
int remove_sql_exec_addr(const common::ObAddr &addr);
void set_sys_task_id(const TraceId &sys_task_id) { sys_task_id_ = sys_task_id; }
void set_sql_exec_addr(const common::ObAddr &addr) { sql_exec_addr_ = addr; }
const TraceId &get_sys_task_id() const { return sys_task_id_; }
virtual int collect_longops_stat(share::ObLongopsValue &value);
@ -627,7 +642,7 @@ public:
K_(ret_code), K_(task_id), K_(parent_task_id), K_(parent_task_key),
K_(task_version), K_(parallelism), K_(ddl_stmt_str), K_(compat_mode),
K_(sys_task_id), K_(err_code_occurence_cnt), K_(stat_info),
K_(next_schedule_ts), K_(delay_schedule_time), K(execution_id_), K(sql_exec_addr_), K_(data_format_version), K(consumer_group_id_),
K_(next_schedule_ts), K_(delay_schedule_time), K(execution_id_), K(sql_exec_addrs_), K_(data_format_version), K(consumer_group_id_),
K_(dst_tenant_id), K_(dst_schema_version), K_(is_pre_split));
static const int64_t MAX_ERR_TOLERANCE_CNT = 3L; // Max torlerance count for error code.
static const int64_t DEFAULT_TASK_IDLE_TIME_US = 10L * 1000L; // 10ms
@ -699,7 +714,7 @@ protected:
int64_t delay_schedule_time_;
int64_t next_schedule_ts_;
int64_t execution_id_; // guarded by lock_
common::ObAddr sql_exec_addr_;
ObArray<common::ObAddr> sql_exec_addrs_;
int64_t start_time_;
uint64_t data_format_version_;
int64_t consumer_group_id_;

View File

@ -22,6 +22,7 @@
#include "rootserver/ob_root_service.h"
#include "share/scn.h"
#include "share/schema/ob_mlog_info.h"
#include "share/tablet/ob_tablet_to_ls_operator.h"
#include "lib/mysqlclient/ob_mysql_transaction.h"
using namespace oceanbase::rootserver;
@ -53,12 +54,28 @@ int ObIndexSSTableBuildTask::set_nls_format(const ObString &nls_date_format,
return ret;
}
int ObIndexSSTableBuildTask::set_addition_info(const share::ObLSID &ls_id, const common::ObAddr &ls_leader_addr, const ObIArray<ObTabletID> &index_partition_ids)
{
int ret = OB_SUCCESS;
if (index_partition_ids.count() > 0) {
if (OB_FAIL(addition_info_.partition_ids_.assign(index_partition_ids))) {
LOG_WARN("ObArray assign failed", K(ret), K(index_partition_ids));
} else {
addition_info_.ls_id_ = ls_id;
addition_info_.ls_leader_addr_ = ls_leader_addr;
}
}
return ret;
}
int ObIndexSSTableBuildTask::process()
{
int ret = OB_SUCCESS;
ObTraceIdGuard trace_id_guard(trace_id_);
ObSqlString sql_string;
ObSchemaGetterGuard schema_guard;
ObString partition_names;
ObArray<ObString> batch_partition_names;
const ObSysVariableSchema *sys_variable_schema = NULL;
bool oracle_mode = false;
ObTabletID unused_tablet_id;
@ -66,7 +83,6 @@ int ObIndexSSTableBuildTask::process()
const ObTableSchema *index_schema = nullptr;
bool need_padding = false;
bool need_exec_new_inner_sql = true;
if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
tenant_id_, schema_guard))) {
LOG_WARN("fail to get tenant schema guard", K(ret), K(data_table_id_));
@ -94,75 +110,93 @@ int ObIndexSSTableBuildTask::process()
ret = OB_ERR_UNEXPECTED;
LOG_WARN("error unexpected, index schema must not be nullptr", K(ret), K(tenant_id_), K(dest_table_id_));
} else {
if (OB_FAIL(ObDDLUtil::generate_build_replica_sql(tenant_id_, data_table_id_,
dest_table_id_,
data_schema->get_schema_version(),
snapshot_version_,
execution_id_,
task_id_,
parallelism_,
false/*use_heap_table_ddl*/,
!data_schema->is_user_hidden_table()/*use_schema_version_hint_for_src_table*/,
nullptr,
sql_string))) {
LOG_WARN("fail to generate build replica sql", K(ret));
} else if (OB_FAIL(data_schema->is_need_padding_for_generated_column(need_padding))) {
LOG_WARN("fail to check need padding", K(ret));
} else {
common::ObCommonSqlProxy *user_sql_proxy = nullptr;
int64_t affected_rows = 0;
ObSQLMode sql_mode = SMO_STRICT_ALL_TABLES | (need_padding ? SMO_PAD_CHAR_TO_FULL_LENGTH : 0);
ObSessionParam session_param;
session_param.sql_mode_ = (int64_t *)&sql_mode;
session_param.tz_info_wrap_ = nullptr;
session_param.ddl_info_.set_is_ddl(true);
session_param.ddl_info_.set_source_table_hidden(data_schema->is_user_hidden_table());
session_param.ddl_info_.set_dest_table_hidden(index_schema->is_user_hidden_table());
session_param.nls_formats_[ObNLSFormatEnum::NLS_DATE] = nls_date_format_;
session_param.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP] = nls_timestamp_format_;
session_param.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP_TZ] = nls_timestamp_tz_format_;
session_param.use_external_session_ = true; // means session id dispatched by session mgr
session_param.consumer_group_id_ = consumer_group_id_;
common::ObAddr *sql_exec_addr = nullptr;
if (inner_sql_exec_addr_.is_valid()) {
sql_exec_addr = &inner_sql_exec_addr_;
LOG_INFO("inner sql execute addr" , K(*sql_exec_addr));
if (is_partitioned_local_index_task()) {
if (OB_FAIL(ObDDLUtil::get_index_table_batch_partition_names(tenant_id_, data_table_id_, dest_table_id_, addition_info_.partition_ids_, allocator_, batch_partition_names))) {
LOG_WARN("fail to get index table batch partition names", K(ret), K(tenant_id_), K(data_table_id_), K(dest_table_id_), K(addition_info_.partition_ids_), K(batch_partition_names));
} else if (OB_FAIL(ObDDLUtil::generate_partition_names(batch_partition_names, allocator_, partition_names))) {
LOG_WARN("fail to generate partition names", K(ret), K(batch_partition_names), K(partition_names));
}
int tmp_ret = OB_SUCCESS;
if (oracle_mode) {
user_sql_proxy = GCTX.ddl_oracle_sql_proxy_;
}
if (OB_SUCC(ret)) {
if (OB_FAIL(ObDDLUtil::generate_build_replica_sql(tenant_id_, data_table_id_,
dest_table_id_,
data_schema->get_schema_version(),
snapshot_version_,
execution_id_,
task_id_,
parallelism_,
false/*use_heap_table_ddl*/,
!data_schema->is_user_hidden_table()/*use_schema_version_hint_for_src_table*/,
nullptr,
partition_names,
sql_string))) {
LOG_WARN("fail to generate build replica sql", K(ret));
} else if (OB_FAIL(data_schema->is_need_padding_for_generated_column(need_padding))) {
LOG_WARN("fail to check need padding", K(ret));
} else {
user_sql_proxy = GCTX.ddl_sql_proxy_;
}
DEBUG_SYNC(BEFORE_INDEX_SSTABLE_BUILD_TASK_SEND_SQL);
ObTimeoutCtx timeout_ctx;
const int64_t DDL_INNER_SQL_EXECUTE_TIMEOUT = ObDDLUtil::calc_inner_sql_execute_timeout();
add_event_info(ret, "index sstable build task send innersql");
LOG_INFO("execute sql" , K(sql_string), K(data_table_id_), K(tenant_id_), K(DDL_INNER_SQL_EXECUTE_TIMEOUT), "ddl_event_info", ObDDLEventInfo());
if (OB_FAIL(timeout_ctx.set_trx_timeout_us(DDL_INNER_SQL_EXECUTE_TIMEOUT))) {
LOG_WARN("set trx timeout failed", K(ret));
} else if (OB_FAIL(timeout_ctx.set_timeout(DDL_INNER_SQL_EXECUTE_TIMEOUT))) {
LOG_WARN("set timeout failed", K(ret));
} else if (OB_FAIL(DDL_SIM(tenant_id_, task_id_, CREATE_INDEX_BUILD_SSTABLE_FAILED))) {
LOG_WARN("ddl sim failure: create index build sstable failed", K(ret), K(tenant_id_), K(task_id_));
} else if (OB_FAIL(user_sql_proxy->write(tenant_id_, sql_string.ptr(), affected_rows,
oracle_mode ? ObCompatibilityMode::ORACLE_MODE : ObCompatibilityMode::MYSQL_MODE, &session_param, sql_exec_addr))) {
LOG_WARN("fail to execute build replica sql", K(ret), K(tenant_id_));
}
if (OB_SUCC(ret)) {
if (OB_FAIL(ObCheckTabletDataComplementOp::check_finish_report_checksum(tenant_id_, dest_table_id_, execution_id_, task_id_))) {
LOG_WARN("fail to check sstable checksum_report_finish",
K(ret), K(tenant_id_), K(dest_table_id_), K(execution_id_), K(task_id_));
common::ObCommonSqlProxy *user_sql_proxy = nullptr;
int64_t affected_rows = 0;
ObSQLMode sql_mode = SMO_STRICT_ALL_TABLES | (need_padding ? SMO_PAD_CHAR_TO_FULL_LENGTH : 0);
ObSessionParam session_param;
session_param.sql_mode_ = (int64_t *)&sql_mode;
session_param.tz_info_wrap_ = nullptr;
session_param.ddl_info_.set_is_ddl(true);
session_param.ddl_info_.set_source_table_hidden(data_schema->is_user_hidden_table());
session_param.ddl_info_.set_dest_table_hidden(index_schema->is_user_hidden_table());
session_param.nls_formats_[ObNLSFormatEnum::NLS_DATE] = nls_date_format_;
session_param.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP] = nls_timestamp_format_;
session_param.nls_formats_[ObNLSFormatEnum::NLS_TIMESTAMP_TZ] = nls_timestamp_tz_format_;
session_param.use_external_session_ = true; // means session id dispatched by session mgr
session_param.consumer_group_id_ = consumer_group_id_;
common::ObAddr *sql_exec_addr = nullptr;
if (inner_sql_exec_addr_.is_valid()) {
sql_exec_addr = &inner_sql_exec_addr_;
LOG_INFO("inner sql execute addr" , K(*sql_exec_addr));
}
int tmp_ret = OB_SUCCESS;
if (oracle_mode) {
user_sql_proxy = GCTX.ddl_oracle_sql_proxy_;
} else {
user_sql_proxy = GCTX.ddl_sql_proxy_;
}
DEBUG_SYNC(BEFORE_INDEX_SSTABLE_BUILD_TASK_SEND_SQL);
ObTimeoutCtx timeout_ctx;
const int64_t DDL_INNER_SQL_EXECUTE_TIMEOUT = ObDDLUtil::calc_inner_sql_execute_timeout();
add_event_info(ret, "index sstable build task send innersql");
LOG_INFO("execute sql" , K(sql_string), K(data_table_id_), K(tenant_id_), K(DDL_INNER_SQL_EXECUTE_TIMEOUT), "ddl_event_info", ObDDLEventInfo());
if (OB_FAIL(timeout_ctx.set_trx_timeout_us(DDL_INNER_SQL_EXECUTE_TIMEOUT))) {
LOG_WARN("set trx timeout failed", K(ret));
} else if (OB_FAIL(timeout_ctx.set_timeout(DDL_INNER_SQL_EXECUTE_TIMEOUT))) {
LOG_WARN("set timeout failed", K(ret));
} else if (OB_FAIL(DDL_SIM(tenant_id_, task_id_, CREATE_INDEX_BUILD_SSTABLE_FAILED))) {
LOG_WARN("ddl sim failure: create index build sstable failed", K(ret), K(tenant_id_), K(task_id_));
} else if (OB_FAIL(user_sql_proxy->write(tenant_id_, sql_string.ptr(), affected_rows,
oracle_mode ? ObCompatibilityMode::ORACLE_MODE : ObCompatibilityMode::MYSQL_MODE, &session_param, sql_exec_addr))) {
LOG_WARN("fail to execute build replica sql", K(ret), K(tenant_id_));
}
if (OB_SUCC(ret)) {
if (is_partitioned_local_index_task()) {
bool is_checksums_all_report = false;
if (OB_FAIL(ObCheckTabletDataComplementOp::check_tablet_checksum_update_status(tenant_id_, dest_table_id_, task_id_, execution_id_, addition_info_.partition_ids_, is_checksums_all_report))) {
LOG_WARN("fail to check tablet checksum update status, maybe EAGAIN", K(ret), K(tenant_id_), K(dest_table_id_), K(task_id_), K(execution_id_), K(addition_info_.partition_ids_), K(is_checksums_all_report));
} else if (!is_checksums_all_report) {
ret = OB_EAGAIN;
LOG_WARN("tablets checksum not all report!",K(ret), K(is_checksums_all_report));
}
} else if (OB_FAIL(ObCheckTabletDataComplementOp::check_finish_report_checksum(tenant_id_, dest_table_id_, execution_id_, task_id_))) {
LOG_WARN("fail to check sstable checksum_report_finish",
K(ret), K(tenant_id_), K(dest_table_id_), K(execution_id_), K(task_id_));
}
}
}
}
}
ObDDLTaskKey task_key(tenant_id_, dest_table_id_, schema_version_);
ObDDLTaskInfo info;
int tmp_ret = root_service_->get_ddl_scheduler().on_sstable_complement_job_reply(
unused_tablet_id, task_key, snapshot_version_, execution_id_, ret, info);
int tmp_ret = 0;
tmp_ret = root_service_->get_ddl_scheduler().on_sstable_complement_job_reply(
unused_tablet_id, task_key, snapshot_version_, execution_id_, ret, addition_info_);
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("report build finish failed", K(ret), K(tmp_ret));
ret = OB_SUCCESS == ret ? tmp_ret : ret;
@ -203,12 +237,17 @@ ObAsyncTask *ObIndexSSTableBuildTask::deep_copy(char *buf, const int64_t buf_siz
consumer_group_id_,
trace_id_,
parallelism_,
is_partitioned_local_index_task_,
root_service_,
inner_sql_exec_addr_);
if (OB_SUCCESS != (task->set_nls_format(nls_date_format_, nls_timestamp_format_, nls_timestamp_tz_format_))) {
task->~ObIndexSSTableBuildTask();
task = nullptr;
LOG_WARN_RET(OB_ALLOCATE_MEMORY_FAILED, "failed to set nls format");
} else if (OB_SUCCESS != (task->set_addition_info(addition_info_.ls_id_, addition_info_.ls_leader_addr_, addition_info_.partition_ids_))) {
task->~ObIndexSSTableBuildTask();
task = nullptr;
LOG_WARN_RET(OB_ALLOCATE_MEMORY_FAILED, "failed to set index partition ids");
}
}
return task;
@ -216,8 +255,7 @@ ObAsyncTask *ObIndexSSTableBuildTask::deep_copy(char *buf, const int64_t buf_siz
/*************** ObIndexBuildTask *************/
ObIndexBuildTask::ObIndexBuildTask()
: ObDDLTask(ObDDLType::DDL_CREATE_INDEX), index_table_id_(target_object_id_),
is_unique_index_(false), is_global_index_(false), root_service_(nullptr), snapshot_held_(false),
: ObDDLTask(ObDDLType::DDL_CREATE_INDEX), index_table_id_(target_object_id_), is_unique_index_(false), is_global_index_(false), root_service_(nullptr), snapshot_held_(false),
is_sstable_complete_task_submitted_(false), sstable_complete_request_time_(0), sstable_complete_ts_(0),
check_unique_snapshot_(0), complete_sstable_job_ret_code_(INT64_MAX), create_index_arg_(), target_cg_cnt_(0)
{
@ -257,8 +295,14 @@ int ObIndexBuildTask::process()
break;
}
case ObDDLTaskStatus::REDEFINITION: {
if (OB_FAIL(wait_data_complement())) {
LOG_WARN("wait data complement failed", K(ret), K(*this));
if (is_create_partitioned_local_index()) {
if (OB_FAIL(wait_local_index_data_complement())) {
LOG_WARN("wait local index data complement failed", K(ret), K(*this));
}
} else {
if (OB_FAIL(wait_data_complement())) {
LOG_WARN("wait data complement failed", K(ret), K(*this));
}
}
break;
}
@ -358,6 +402,7 @@ int ObIndexBuildTask::init(
const int64_t consumer_group_id,
const int32_t sub_task_trace_id,
const obrpc::ObCreateIndexArg &create_index_arg,
const share::ObDDLType task_type,
const int64_t parent_task_id /* = 0 */,
const uint64_t tenant_data_version,
const int64_t task_status /* = TaskStatus::PREPARE */,
@ -396,6 +441,7 @@ int ObIndexBuildTask::init(
KR(ret), K(create_index_arg_.index_action_type_), K(index_schema->get_table_type()));
} else {
set_gmt_create(ObTimeUtility::current_time());
task_type_ = task_type;
is_global_index_ = index_schema->is_global_index_table();
is_unique_index_ = index_schema->is_unique_index();
tenant_id_ = tenant_id;
@ -486,6 +532,7 @@ int ObIndexBuildTask::init(const ObDDLTaskRecord &task_record)
LOG_WARN("index action is add_mlog but index schema is not mlog",
KR(ret), K(create_index_arg_.index_action_type_), K(index_schema->get_table_type()));
} else {
task_type_ = task_record.ddl_type_;
is_global_index_ = index_schema->is_global_index_table();
is_unique_index_ = index_schema->is_unique_index();
tenant_id_ = task_record.tenant_id_;
@ -844,47 +891,92 @@ int ObIndexBuildTask::reap_old_replica_build_task(bool &need_exec_new_inner_sql)
}
// construct ObIndexSSTableBuildTask build task
int ObIndexBuildTask::send_build_single_replica_request()
int ObIndexBuildTask::send_build_single_replica_request(const bool &is_partitioned_local_index_task, const int64_t &parallelism, const int64_t &execution_id, const share::ObLSID &ls_id, const common::ObAddr &leader_addr, const ObIArray<ObTabletID> &index_partition_ids)
{
int ret = OB_SUCCESS;
int64_t new_execution_id = 0;
int64_t new_execution_id = execution_id;
common::ObAddr ls_leader_addr = leader_addr;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("ObIndexBuildTask has not been inited", K(ret));
} else if (OB_FAIL(DDL_SIM(tenant_id_, task_id_, DDL_TASK_SEND_BUILD_REPLICA_REQUEST_FAILED))) {
LOG_WARN("ddl sim failure", K(ret), K(tenant_id_), K(task_id_));
} else if (OB_FAIL(ObDDLTask::push_execution_id(tenant_id_, task_id_, task_type_, true/*is ddl retryable*/, data_format_version_, new_execution_id))) {
LOG_WARN("failed to fetch new execution id", K(ret));
} else {
if (OB_FAIL(ObDDLUtil::get_sys_ls_leader_addr(GCONF.cluster_id, tenant_id_, create_index_arg_.inner_sql_exec_addr_))) {
LOG_WARN("get sys ls leader addr fail", K(ret), K(tenant_id_));
ret = OB_SUCCESS; // ingore ret
} else {
set_sql_exec_addr(create_index_arg_.inner_sql_exec_addr_); // set to switch_status, if task cancel, we should kill session with inner_sql_exec_addr_
}
ObIndexSSTableBuildTask task(
task_id_,
tenant_id_,
object_id_,
target_object_id_,
schema_version_,
snapshot_version_,
new_execution_id,
consumer_group_id_,
trace_id_,
parallelism_,
root_service_,
create_index_arg_.inner_sql_exec_addr_);
if (OB_FAIL(task.set_nls_format(create_index_arg_.nls_date_format_,
create_index_arg_.nls_timestamp_format_,
create_index_arg_.nls_timestamp_tz_format_))) {
LOG_WARN("failed to set nls format", K(ret), K(create_index_arg_));
} else if (OB_FAIL(root_service_->submit_ddl_single_replica_build_task(task))) {
LOG_WARN("fail to submit task", K(ret), KPC(this));
} else {
is_sstable_complete_task_submitted_ = true;
sstable_complete_request_time_ = ObTimeUtility::current_time();
if (!is_partitioned_local_index_task) {
if (OB_FAIL(ObDDLTask::push_execution_id(tenant_id_, task_id_, task_type_, true/*is ddl retryable*/, data_format_version_, new_execution_id))) {
LOG_WARN("failed to fetch new execution id", K(ret), K(tenant_id_), K(task_id_), K(task_type_), K(data_format_version_), K(new_execution_id));
} else {
ls_leader_addr = create_index_arg_.inner_sql_exec_addr_;
}
} else if (OB_UNLIKELY(index_partition_ids.count() < 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("array size less than 1", K(ret), K(index_partition_ids));
}
if (OB_SUCC(ret)) {
ObIndexSSTableBuildTask task(
task_id_,
tenant_id_,
object_id_,
target_object_id_,
schema_version_,
snapshot_version_,
new_execution_id,
consumer_group_id_,
trace_id_,
parallelism,
is_partitioned_local_index_task,
root_service_,
ls_leader_addr);
if (OB_FAIL(set_sql_exec_addr(ls_leader_addr))) {
LOG_WARN("failed to set sql execute addr", K(ret), K(ls_leader_addr));
} else if (OB_FAIL(task.set_nls_format(create_index_arg_.nls_date_format_,
create_index_arg_.nls_timestamp_format_,
create_index_arg_.nls_timestamp_tz_format_))) {
LOG_WARN("failed to set nls format", K(ret), K(create_index_arg_));
} else if (OB_FAIL(task.set_addition_info(ls_id, ls_leader_addr, index_partition_ids))) {
LOG_WARN("failed to set partition ids", K(ret), K(index_partition_ids));
} else if (OB_FAIL(root_service_->submit_ddl_single_replica_build_task(task))) {
LOG_WARN("fail to submit task", K(ret), KPC(this));
} else {
is_sstable_complete_task_submitted_ = true;
sstable_complete_request_time_ = ObTimeUtility::current_time();
}
}
}
return ret;
}
int ObIndexBuildTask::wait_and_send_single_partition_replica_task(bool &state_finished)
{
int ret = OB_SUCCESS;
int64_t parallelism = 0;
int64_t execution_id = 0;
share::ObLSID ls_id;
common::ObAddr leader_addr;
ObArray<ObTabletID> tablets;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (ObDDLTaskStatus::REDEFINITION != task_status_) {
ret = OB_STATE_NOT_MATCH;
LOG_WARN("task status not match", K(ret), K(task_status_));
} else if (OB_FAIL(tablet_scheduler_.get_next_batch_tablets(parallelism, execution_id, ls_id, leader_addr, tablets))) {
if (OB_UNLIKELY(ret == OB_EAGAIN)) {
ret = OB_SUCCESS;
} else if (OB_UNLIKELY(ret == OB_ITER_END)) {
LOG_WARN("schedule queue is null", K(ret), K(parallelism), K(execution_id), K(tablets));
ret = OB_SUCCESS;
state_finished = true;
} else {
LOG_WARN("fail to get next batch tablets", K(ret), K(parallelism), K(execution_id), K(tablets));
state_finished = true;
}
} else if (OB_FAIL(send_build_single_replica_request(true, parallelism, execution_id, ls_id, leader_addr, tablets))) {
LOG_WARN("fail to send build single partition replica request", K(ret), K(parallelism), K(execution_id), K(tablets));
}
return ret;
}
@ -922,16 +1014,48 @@ int ObIndexBuildTask::check_build_single_replica(bool &is_end)
return ret;
}
int ObIndexBuildTask::check_build_local_index_single_replica(bool &is_end)
{
int ret = OB_SUCCESS;
is_end = false;
TCWLockGuard guard(lock_);
if (INT64_MAX == complete_sstable_job_ret_code_) {
// not complete
} else if (OB_SUCCESS == complete_sstable_job_ret_code_) {
} else if (OB_SUCCESS != complete_sstable_job_ret_code_) {
ret = complete_sstable_job_ret_code_;
LOG_WARN("sstable complete job has failed", K(ret), K(object_id_), K(index_table_id_));
if (is_replica_build_need_retry(ret)) {
// retry sql job by re-submit
complete_sstable_job_ret_code_ = INT64_MAX;
ret = OB_SUCCESS;
LOG_INFO("retry complete sstable job", K(ret), K(object_id_), K(index_table_id_));
} else {
is_end = true;
}
}
return ret;
}
bool ObIndexBuildTask::is_sstable_complete_task_submitted()
{
TCRLockGuard guard(lock_);
return is_sstable_complete_task_submitted_;
}
bool ObIndexBuildTask::is_create_partitioned_local_index()
{
return ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX == task_type_;
}
int ObIndexBuildTask::wait_data_complement()
{
int ret = OB_SUCCESS;
bool state_finished = false;
bool is_request_end = false;
share::ObLSID ls_id;
common::ObAddr leader_addr;
ObArray<ObTabletID> index_partition_ids;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
@ -941,8 +1065,6 @@ int ObIndexBuildTask::wait_data_complement()
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected snapshot", K(ret), KPC(this));
}
bool is_request_end = false;
// submit a job to complete sstable for the index table on snapshot_version
if (OB_SUCC(ret) && !state_finished && !is_sstable_complete_task_submitted()) {
bool need_exec_new_inner_sql = false;
@ -954,8 +1076,8 @@ int ObIndexBuildTask::wait_data_complement()
}
} else if (!need_exec_new_inner_sql) {
state_finished = true;
} else if (OB_FAIL(send_build_single_replica_request())) {
LOG_WARN("fail to send build single replica request", K(ret));
} else if (OB_FAIL(send_build_single_replica_request(false, parallelism_, 0, ls_id, leader_addr, index_partition_ids))) {
LOG_WARN("fail to send build single replica request", K(ret), K(parallelism_), K(index_partition_ids));
}
}
@ -964,7 +1086,7 @@ int ObIndexBuildTask::wait_data_complement()
if (OB_SUCC(ret) && !state_finished && is_sstable_complete_task_submitted()) {
if (OB_FAIL(check_build_single_replica(is_request_end))) {
LOG_WARN("fail to check build single replica", K(ret));
} else if (is_request_end) {
} else if (OB_UNLIKELY(is_request_end)) {
ret = complete_sstable_job_ret_code_;
state_finished = true;
}
@ -994,6 +1116,92 @@ int ObIndexBuildTask::wait_data_complement()
return ret;
}
int ObIndexBuildTask::wait_local_index_data_complement()
{
int ret = OB_SUCCESS;
bool state_finished = false;
bool is_request_end = false;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (ObDDLTaskStatus::REDEFINITION != task_status_) {
ret = OB_STATE_NOT_MATCH;
LOG_WARN("task status not match", K(ret), K(task_status_));
} else if (OB_UNLIKELY(snapshot_version_ <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("unexpected snapshot", K(ret), KPC(this));
}
if (OB_SUCC(ret) && !state_finished && !is_sstable_complete_task_submitted()) {
if (OB_FAIL(create_schedule_queue())) {
LOG_WARN("fail to create schedule queue", K(ret), KPC(this));
}
}
if (OB_SUCC(ret) && !state_finished && is_sstable_complete_task_submitted()) {
if (OB_FAIL(wait_and_send_single_partition_replica_task(state_finished))) {
LOG_WARN("fail to send single partition replica task", K(ret), KPC(this));
}
}
DEBUG_SYNC(CREATE_INDEX_REPLICA_BUILD);
if (OB_SUCC(ret) && !state_finished && is_sstable_complete_task_submitted()) {
if (OB_FAIL(check_build_local_index_single_replica(is_request_end))) {
LOG_WARN("fail to check build single replica", K(ret));
} else if (is_request_end) {
state_finished = true;
}
}
if (OB_SUCC(ret) && state_finished && !create_index_arg_.is_spatial_index()) {
bool dummy_equal = false;
if (OB_FAIL(ObDDLChecksumOperator::check_column_checksum_without_execution_id(
tenant_id_, object_id_, index_table_id_, task_id_, false/*index build*/, dummy_equal, root_service_->get_sql_proxy()))) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to check column checksum", K(ret), K(tenant_id_), K(object_id_), K(index_table_id_), K(task_id_));
} else if (REACH_TIME_INTERVAL(1000L * 1000L)) {
LOG_INFO("index checksum has not been reported", K(ret), K(tenant_id_), K(object_id_), K(index_table_id_), K(task_id_));
}
}
}
if (state_finished || OB_FAIL(ret)) {
(void)switch_status(ObDDLTaskStatus::VALIDATE_CHECKSUM, true, ret);
LOG_INFO("wait data complement finished", K(ret), K(*this));
}
return ret;
}
int ObIndexBuildTask::create_schedule_queue()
{
int ret = OB_SUCCESS;
common::ObArray<ObTabletID> index_tablet_ids;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_UNLIKELY(ObDDLTaskStatus::REDEFINITION != task_status_)) {
ret = OB_STATE_NOT_MATCH;
LOG_WARN("task status not match", K(ret), K(task_status_));
} else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id_ || OB_INVALID_ID == object_id_ || OB_INVALID_ID == index_table_id_ ||
task_id_ == OB_INVALID_ID)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("the parameters is invalid", K(ret), K(tenant_id_), K(object_id_), K(index_table_id_), K(task_id_));
} else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, index_table_id_, index_tablet_ids))) {
LOG_WARN("failed to get index table tablet ids", K(ret), K(tenant_id_), K(index_table_id_), K(index_tablet_ids));
} else if (OB_FAIL(tablet_scheduler_.init(
tenant_id_,
index_table_id_,
object_id_,
task_id_,
parallelism_,
snapshot_version_,
trace_id_,
index_tablet_ids))) {
LOG_WARN("failed to init tablet scheduler", K(ret), K(tenant_id_), K(index_table_id_), K(index_tablet_ids));
} else {
is_sstable_complete_task_submitted_ = true;
LOG_INFO("create partitioned local index schedule queue", K(ret));
}
return ret;
}
int ObIndexBuildTask::check_need_verify_checksum(bool &need_verify)
{
int ret = OB_SUCCESS;
@ -1181,7 +1389,6 @@ int ObIndexBuildTask::update_complete_sstable_job_status(
const ObDDLTaskInfo &addition_info)
{
int ret = OB_SUCCESS;
UNUSED(addition_info);
TCWLockGuard guard(lock_);
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
@ -1197,15 +1404,27 @@ int ObIndexBuildTask::update_complete_sstable_job_status(
} else if (snapshot_version != snapshot_version_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("snapshot version not match", K(ret), K(snapshot_version), K(snapshot_version_));
} else if (execution_id < execution_id_) {
ret = OB_TASK_EXPIRED;
LOG_WARN("receive a mismatch execution result", K(ret), K(ret_code), K(execution_id), K(execution_id_));
} else {
complete_sstable_job_ret_code_ = ret_code;
sstable_complete_ts_ = ObTimeUtility::current_time();
execution_id_ = execution_id; // update ObIndexBuildTask::execution_id_ from ObIndexSSTableBuildTask::execution_id_
if (is_create_partitioned_local_index()) {
if (OB_UNLIKELY(addition_info.partition_ids_.count() < 1 || !addition_info.ls_id_.is_valid() || !addition_info.ls_leader_addr_.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(addition_info), K(ret_code));
} else if (OB_FAIL(tablet_scheduler_.confirm_batch_tablets_status(execution_id, OB_SUCCESS == ret_code, addition_info.ls_id_, addition_info.partition_ids_))) {
LOG_WARN("fail to confirm batch tablets status", K(ret), K(execution_id), K(ret_code), K(addition_info));
} else if (OB_FAIL(remove_sql_exec_addr(addition_info.ls_leader_addr_))) {
LOG_WARN("failed to remove sql execute addr", K(ret), K(addition_info));
}
} else if (OB_UNLIKELY(execution_id < execution_id_)) {
ret = OB_TASK_EXPIRED;
LOG_WARN("receive a mismatch execution result", K(ret), K(ret_code), K(execution_id), K(execution_id_));
}
if (OB_SUCC(ret)) {
complete_sstable_job_ret_code_ = ret_code;
sstable_complete_ts_ = ObTimeUtility::current_time();
execution_id_ = execution_id;
LOG_INFO("update complete sstable job return code", K(ret), K(target_object_id_), K(tablet_id), K(snapshot_version), K(ret_code), K(execution_id_), K(addition_info));
}
}
LOG_INFO("update complete sstable job return code", K(ret), K(target_object_id_), K(tablet_id), K(snapshot_version), K(ret_code), K(execution_id_));
return ret;
}

View File

@ -13,6 +13,7 @@
#ifndef OCEANBASE_ROOTSERVER_OB_INDEX_BUILD_TASK_H
#define OCEANBASE_ROOTSERVER_OB_INDEX_BUILD_TASK_H
#include "ob_ddl_tablet_scheduler.h"
#include "rootserver/ddl_task/ob_ddl_task.h"
namespace oceanbase
@ -34,12 +35,13 @@ public:
const int64_t consumer_group_id,
const common::ObCurTraceId::TraceId &trace_id,
const int64_t parallelism,
const bool is_partitioned_local_index_task,
ObRootService *root_service,
const common::ObAddr &inner_sql_exec_addr)
: task_id_(task_id), tenant_id_(tenant_id), data_table_id_(data_table_id), dest_table_id_(dest_table_id),
schema_version_(schema_version), snapshot_version_(snapshot_version), execution_id_(execution_id),
consumer_group_id_(consumer_group_id), trace_id_(trace_id), parallelism_(parallelism), allocator_("IdxSSTBuildTask"),
root_service_(root_service), inner_sql_exec_addr_(inner_sql_exec_addr)
consumer_group_id_(consumer_group_id), trace_id_(trace_id), parallelism_(parallelism), is_partitioned_local_index_task_(is_partitioned_local_index_task),
allocator_("IdxSSTBuildTask"), root_service_(root_service), inner_sql_exec_addr_(inner_sql_exec_addr)
{
set_retry_times(0);
}
@ -48,15 +50,17 @@ public:
int set_nls_format(const ObString &nls_date_format,
const ObString &nls_timestamp_format,
const ObString &nls_timestamp_tz_format);
int set_addition_info(const share::ObLSID &ls_id, const common::ObAddr &ls_leader_addr, const ObIArray<ObTabletID> &index_partition_ids);
ObDDLTaskID get_ddl_task_id() { return ObDDLTaskID(tenant_id_, task_id_); }
virtual int process() override;
virtual int64_t get_deep_copy_size() const override { return sizeof(*this); }
virtual ObAsyncTask *deep_copy(char *buf, const int64_t buf_size) const override;
void add_event_info(const int ret, const ObString &ddl_event_stmt);
TO_STRING_KV(K_(data_table_id), K_(dest_table_id), K_(schema_version), K_(snapshot_version),
K_(execution_id), K_(consumer_group_id), K_(trace_id), K_(parallelism), K_(nls_date_format),
K_(nls_timestamp_format), K_(nls_timestamp_tz_format));
K_(execution_id), K_(consumer_group_id), K_(trace_id), K_(parallelism), K_(is_partitioned_local_index_task),
K_(addition_info), K_(nls_date_format), K_(nls_timestamp_format), K_(nls_timestamp_tz_format));
private:
inline bool is_partitioned_local_index_task() const { return is_partitioned_local_index_task_ == true; }
private:
int64_t task_id_;
int64_t tenant_id_;
@ -68,12 +72,15 @@ private:
int64_t consumer_group_id_;
common::ObCurTraceId::TraceId trace_id_;
int64_t parallelism_;
bool is_partitioned_local_index_task_;
common::ObArenaAllocator allocator_;
ObString nls_date_format_;
ObString nls_timestamp_format_;
ObString nls_timestamp_tz_format_;
ObRootService *root_service_;
common::ObAddr inner_sql_exec_addr_;
ObDDLTaskInfo addition_info_;
DISALLOW_COPY_AND_ASSIGN(ObIndexSSTableBuildTask);
};
class ObIndexBuildTask : public ObDDLTask
@ -93,6 +100,7 @@ public:
const int64_t consumer_group_id,
const int32_t sub_task_trace_id,
const obrpc::ObCreateIndexArg &create_index_arg,
const share::ObDDLType task_type,
const int64_t parent_task_id /* = 0 */,
const uint64_t tenant_data_version,
const int64_t task_status = share::ObDDLTaskStatus::PREPARE,
@ -124,6 +132,8 @@ private:
int prepare();
int wait_trans_end();
int wait_data_complement();
int wait_local_index_data_complement();
int create_schedule_queue();
int verify_checksum();
int enable_index();
int clean_on_failed();
@ -135,8 +145,15 @@ private:
const share::schema::ObIndexStatus new_status);
int check_health();
int reap_old_replica_build_task(bool &need_exec_new_inner_sql);
int send_build_single_replica_request();
int send_build_single_replica_request(const bool &is_partitioned_local_index_task,
const int64_t &parallelism,
const int64_t &execution_id,
const share::ObLSID &ls_id,
const common::ObAddr &leader_addr,
const ObIArray<ObTabletID> &index_partition_ids);
int wait_and_send_single_partition_replica_task(bool &state_finished);
int check_build_single_replica(bool &is_end);
int check_build_local_index_single_replica(bool &is_end);
int check_need_verify_checksum(bool &need_verify);
int check_need_acquire_lob_snapshot(const ObTableSchema *data_table_schema,
const ObTableSchema *index_table_schema,
@ -144,6 +161,7 @@ private:
bool is_sstable_complete_task_submitted();
int check_target_cg_cnt();
int update_mlog_last_purge_scn();
bool is_create_partitioned_local_index();
private:
static const int64_t OB_INDEX_BUILD_TASK_VERSION = 1;
using ObDDLTask::is_inited_;
@ -165,6 +183,7 @@ private:
ObDDLWaitColumnChecksumCtx wait_column_checksum_ctx_;
int64_t complete_sstable_job_ret_code_;
int64_t redefinition_execution_id_;
ObDDLTabletScheduler tablet_scheduler_;
obrpc::ObCreateIndexArg create_index_arg_; // this is not a valid arg, only has nls formats for now
int64_t target_cg_cnt_;
};
@ -172,5 +191,4 @@ private:
} // end namespace rootserver
} // end namespace oceanbase
#endif // OCEANBASE_ROOTSERVER_OB_INDEX_BUILD_TASK_H
#endif // OCEANBASE_ROOTSERVER_OB_INDEX_BUILD_TASK_H

View File

@ -283,8 +283,8 @@ int ObTableRedefinitionTask::send_build_replica_request_by_sql()
if (OB_FAIL(ObDDLUtil::get_sys_ls_leader_addr(GCONF.cluster_id, tenant_id_, alter_table_arg_.inner_sql_exec_addr_))) {
LOG_WARN("get sys ls leader addr fail", K(ret), K(tenant_id_));
ret = OB_SUCCESS; // ignore ret
} else {
set_sql_exec_addr(alter_table_arg_.inner_sql_exec_addr_); // set to switch_status, if task cancel, we should kill session with inner_sql_exec_addr_
} else if (OB_FAIL(set_sql_exec_addr(alter_table_arg_.inner_sql_exec_addr_))) {
LOG_WARN("failed to set sql execute addr", K(ret), K(alter_table_arg_.inner_sql_exec_addr_));
}
ObSchemaGetterGuard schema_guard;
const ObTableSchema *orig_table_schema = nullptr;
@ -563,7 +563,7 @@ int ObTableRedefinitionTask::copy_table_indexes()
} else {
create_index_arg.index_type_ = index_schema->get_index_type();
ObCreateDDLTaskParam param(dst_tenant_id_,
ObDDLType::DDL_CREATE_INDEX,
((DATA_VERSION_4_2_2_0 <= data_format_version_ && data_format_version_ < DATA_VERSION_4_3_0_0) || data_format_version_ >= DATA_VERSION_4_3_2_0) && index_schema->is_storage_local_index_table() && index_schema->is_partitioned_table() ? ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX : ObDDLType::DDL_CREATE_INDEX,
table_schema,
index_schema,
0/*object_id*/,

View File

@ -11439,6 +11439,8 @@ const char* ObDDLService::ddl_type_str(const ObDDLType ddl_type)
const char *str = "";
if (DDL_CREATE_INDEX == ddl_type) {
str = "create index";
} else if (DDL_CREATE_PARTITIONED_LOCAL_INDEX == ddl_type) {
str = "create partitioned local index";
} else if (DDL_MODIFY_COLUMN == ddl_type) {
str = "alter column";
} else if (DDL_CHECK_CONSTRAINT == ddl_type) {
@ -12800,7 +12802,7 @@ int ObDDLService::alter_table_in_trans(obrpc::ObAlterTableArg &alter_table_arg,
}
if (OB_FAIL(ret)) {
} else if (DDL_CREATE_INDEX == ddl_type || DDL_NORMAL_TYPE == ddl_type) {
} else if (DDL_CREATE_INDEX == ddl_type || DDL_CREATE_PARTITIONED_LOCAL_INDEX == ddl_type || DDL_NORMAL_TYPE == ddl_type) {
ObIndexBuilder index_builder(*this);
const ObSArray<ObIndexArg *> &index_arg_list = alter_table_arg.index_arg_list_;
int tmp_ret = OB_SUCCESS;

View File

@ -442,7 +442,7 @@ int ObIndexBuilder::submit_build_index_task(
int ret = OB_SUCCESS;
ObTableLockOwnerID owner_id;
ObCreateDDLTaskParam param(index_schema->get_tenant_id(),
ObDDLType::DDL_CREATE_INDEX,
((DATA_VERSION_4_2_2_0 <= tenant_data_version && tenant_data_version < DATA_VERSION_4_3_0_0) || tenant_data_version >= DATA_VERSION_4_3_2_0) && index_schema->is_storage_local_index_table() && index_schema->is_partitioned_table() ? ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX : ObDDLType::DDL_CREATE_INDEX,
data_schema,
index_schema,
0/*object_id*/,

View File

@ -39,13 +39,12 @@ int ObDDLChecksumOperator::fill_one_item(
} else if (OB_FAIL(dml.add_pk_column("execution_id", item.execution_id_))
|| OB_FAIL(dml.add_pk_column("tenant_id", ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id)))
|| OB_FAIL(dml.add_pk_column("table_id", ObSchemaUtils::get_extract_schema_id(exec_tenant_id, item.table_id_)))
// currently tablet id is not necessary, so instead we save task id in this column to distinguish different DDL
// task_id is the primary key in __all_ddl_task_status, so it can uniquely identify a DDL.
|| OB_FAIL(dml.add_pk_column("ddl_task_id", item.ddl_task_id_))
|| OB_FAIL(dml.add_pk_column("column_id", item.column_id_))
|| OB_FAIL(dml.add_pk_column("task_id", item.task_id_))
|| OB_FAIL(dml.add_column("checksum", item.checksum_))) {
LOG_WARN("fail to add column", K(ret));
LOG_WARN("fail to add column", K(ret), K(data_format_version));
} else if (data_format_version >= DATA_VERSION_4_2_2_0) {
if (OB_FAIL(dml.add_column("tablet_id", item.tablet_id_))) {
LOG_WARN("fail to add tablet id column", K(ret), K(item.tablet_id_), K(data_format_version));
@ -221,6 +220,133 @@ int ObDDLChecksumOperator::get_column_checksum(const ObSqlString &sql, const uin
return ret;
}
int ObDDLChecksumOperator::get_part_column_checksum(
const uint64_t tenant_id,
const uint64_t table_id,
const uint64_t tablet_id,
const uint64_t execution_id,
const int64_t ddl_task_id,
const bool is_unique_index_checking,
common::ObMySQLProxy &sql_proxy,
common::hash::ObHashMap<int64_t, int64_t> &column_checksum_map)
{
int ret = OB_SUCCESS;
ObSqlString sql;
const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id);
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || OB_INVALID_ID == table_id || OB_INVALID_ID == tablet_id || execution_id < 0
|| OB_INVALID_ID == ddl_task_id || !sql_proxy.is_inited() || !column_checksum_map.created())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(table_id), K(execution_id), K(ddl_task_id), K(sql_proxy.is_inited()), K(column_checksum_map.created()));
} else if (OB_FAIL(sql.assign_fmt(
"SELECT column_id, checksum FROM %s "
"WHERE execution_id = %ld AND tenant_id = %ld AND table_id = %ld AND tablet_id = %ld AND ddl_task_id = %ld AND task_id %s "
"ORDER BY column_id", OB_ALL_DDL_CHECKSUM_TNAME,
execution_id, ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id), table_id, tablet_id, ddl_task_id, is_unique_index_checking ? "< 0" : ">= 0"))) {
LOG_WARN("fail to assign fmt", K(ret), K(OB_ALL_DDL_CHECKSUM_TNAME), K(execution_id), K(ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id)), K(table_id), K(tablet_id), K(ddl_task_id), K(is_unique_index_checking));
} else {
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
sqlclient::ObMySQLResult *result = NULL;
if (OB_FAIL(sql_proxy.read(res, tenant_id, sql.ptr()))) {
LOG_WARN("fail to execute sql", K(ret));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("error unexpected, query result must not be NULL", K(ret));
} else {
int64_t column_id = 0;
int64_t column_checksum = 0;
while (OB_SUCC(ret)) {
if (OB_FAIL(result->next())) {
if (OB_ITER_END == ret) {
if (0 != column_id && OB_FAIL(column_checksum_map.set_refactored(column_id, column_checksum, true))) {
LOG_WARN("fail to set column checksum to map", K(ret));
}
break;
} else {
LOG_WARN("fail to get next row", K(ret));
}
} else {
int64_t curr_column_id = 0;
int64_t curr_column_checksum = 0;
EXTRACT_INT_FIELD_MYSQL(*result, "column_id", curr_column_id, int64_t);
EXTRACT_INT_FIELD_MYSQL(*result, "checksum", curr_column_checksum, int64_t);
if (OB_SUCC(ret) && 0 == column_id) {
column_id = curr_column_id;
if (OB_FAIL(column_checksum_map.get_refactored(curr_column_id, column_checksum))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
}
}
}
if (OB_SUCC(ret)) {
if (OB_UNLIKELY(curr_column_id != column_id)) {
if (OB_FAIL(column_checksum_map.set_refactored(column_id, column_checksum, true))) {
LOG_WARN("fail to set column checksum to map", K(ret));
} else {
column_id = curr_column_id;
if (OB_FAIL(column_checksum_map.get_refactored(curr_column_id, column_checksum))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
column_checksum = curr_column_checksum;
}
} else {
column_checksum += curr_column_checksum;
}
}
} else {
column_checksum += curr_column_checksum;
}
}
}
}
}
}
}
return ret;
}
int ObDDLChecksumOperator::get_tablet_latest_execution_id(
const uint64_t tenant_id,
const uint64_t index_table_id,
const int64_t ddl_task_id,
const int64_t tablet_id,
common::ObMySQLProxy &sql_proxy,
int64_t &execution_id)
{
int ret = OB_SUCCESS;
ObSqlString sql;
execution_id = 0;
const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id);
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || OB_INVALID_ID == index_table_id || OB_INVALID_ID == ddl_task_id || OB_INVALID_ID == tablet_id || !sql_proxy.is_inited())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(index_table_id), K(ddl_task_id), K(tablet_id), K(sql_proxy.is_inited()));
} else if (OB_FAIL(sql.assign_fmt(
"SELECT max(execution_id) as execution_id FROM %s "
"WHERE tenant_id = %ld AND table_id = %ld AND ddl_task_id = %ld AND task_id = %ld",
OB_ALL_DDL_CHECKSUM_TNAME, ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id), index_table_id, ddl_task_id, tablet_id))) {
LOG_WARN("fail to assign fmt", K(ret), K(OB_ALL_DDL_CHECKSUM_TNAME), K(ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id)), K(index_table_id), K(ddl_task_id), K(tablet_id));
} else {
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
sqlclient::ObMySQLResult *result = NULL;
if (!sql.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arguments", K(ret), K(sql));
} else if (OB_FAIL(sql_proxy.read(res, tenant_id, sql.ptr()))) {
LOG_WARN("fail to execute sql", K(ret), K(tenant_id), K(sql));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("error unexpected, query result must not be NULL", K(ret));
} else if (OB_FAIL(result->next())) {
if (OB_ITER_END != ret) {
TRANS_LOG(WARN, "iterate next result fail", K(ret), K(sql));
}
} else {
EXTRACT_INT_FIELD_MYSQL(*result, "execution_id", execution_id, int64_t);
}
}
}
return ret;
}
int ObDDLChecksumOperator::get_tablet_checksum_status(
const ObSqlString &sql,
const uint64_t tenant_id,
@ -325,7 +451,56 @@ int ObDDLChecksumOperator::get_tablet_checksum_record(
}
return ret;
}
int ObDDLChecksumOperator::get_tablet_checksum_record_without_execution_id(
const uint64_t tenant_id,
const uint64_t table_id,
const int64_t ddl_task_id,
const ObIArray<ObTabletID> &tablet_ids,
ObMySQLProxy &sql_proxy,
common::hash::ObHashMap<uint64_t, bool> &tablet_checksum_status_map)
{
int ret = OB_SUCCESS;
ObSqlString sql;
const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id);
tablet_checksum_status_map.reuse();
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id|| OB_INVALID_ID == table_id || OB_INVALID_ID == ddl_task_id ||
tablet_ids.count() <= 0 || !tablet_checksum_status_map.created())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(table_id), K(ddl_task_id), K(tablet_ids.count()), K(tablet_checksum_status_map.created()));
} else {
int64_t batch_size = 100;
ObArray<uint64_t> batch_tablet_array;
// check every tablet column checksum, task_id is equal to tablet_id
for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); ++i) {
uint64_t last_tablet_id_id = tablet_ids.at(i).id();
if (OB_FAIL(batch_tablet_array.push_back(last_tablet_id_id))) {
LOG_WARN("fail to push back tablet_id_id", K(ret), K(tenant_id), K(ddl_task_id), K(last_tablet_id_id));
} else {
std::sort(batch_tablet_array.begin(), batch_tablet_array.end());
if ((i != 0 && i % batch_size == 0) /* reach batch size */ || i == tablet_ids.count() - 1 /* reach end */) {
if (OB_FAIL(sql.assign_fmt(
"SELECT task_id FROM %s "
"WHERE tenant_id = %ld AND table_id = %ld AND ddl_task_id = %ld AND task_id >= %ld and task_id <= %ld "
"GROUP BY task_id",
OB_ALL_DDL_CHECKSUM_TNAME,
ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id),
ObSchemaUtils::get_extract_schema_id(exec_tenant_id, table_id),
ddl_task_id,
batch_tablet_array.at(0), // first tablet_id in one batch
batch_tablet_array.at(batch_tablet_array.count() - 1)))) { // last tablet id in one batch
LOG_WARN("fail to assign fmt", K(ret), K(tenant_id), K(exec_tenant_id), K(ddl_task_id), K(batch_tablet_array.at(0)), K(batch_tablet_array.at(batch_tablet_array.count() - 1)));
} else if (OB_FAIL(get_tablet_checksum_status(
sql, tenant_id, batch_tablet_array, sql_proxy, tablet_checksum_status_map))) {
LOG_WARN("fail to get column checksum", K(ret), K(sql), K(tenant_id), K(batch_tablet_array));
} else {
batch_tablet_array.reset();
}
}
}
}
}
return ret;
}
int ObDDLChecksumOperator::get_table_column_checksum(
const uint64_t tenant_id,
const int64_t execution_id,
@ -360,6 +535,42 @@ int ObDDLChecksumOperator::get_table_column_checksum(
return ret;
}
int ObDDLChecksumOperator::get_table_column_checksum_without_execution_id(
const uint64_t tenant_id,
const uint64_t table_id,
const uint64_t index_table_id,
const int64_t ddl_task_id,
const bool is_unique_index_checking,
common::hash::ObHashMap<int64_t, int64_t> &column_checksum_map,
ObMySQLProxy &sql_proxy)
{
int ret = OB_SUCCESS;
ObArray<ObTabletID> tablet_ids;
ObArray<ObTabletID> index_tablet_ids;
column_checksum_map.reuse();
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || OB_INVALID_ID == table_id || OB_INVALID_ID == index_table_id
|| OB_INVALID_ID == ddl_task_id || !column_checksum_map.created() || !sql_proxy.is_inited())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(table_id), K(index_table_id), K(ddl_task_id), K(column_checksum_map.created()), K(sql_proxy.is_inited()));
} else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id, table_id, tablet_ids))) {
LOG_WARN("failed to get table tablet ids", K(ret), K(tenant_id), K(table_id), K(tablet_ids));
} else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id, index_table_id, index_tablet_ids))) {
LOG_WARN("failed to get index table tablet ids", K(ret), K(tenant_id), K(index_table_id), K(index_tablet_ids));
} else {
int64_t tablet_latest_execution_id = 0;
common::hash::ObHashSet<uint64_t>::iterator iter;
for (int64_t i = 0; i < index_tablet_ids.count() && OB_SUCC(ret); i++) {
tablet_latest_execution_id = 0;
if (OB_FAIL(get_tablet_latest_execution_id(tenant_id, index_table_id, ddl_task_id, index_tablet_ids.at(i).id(), sql_proxy, tablet_latest_execution_id))) {
LOG_WARN("fail to get tablet latest execution id", K(ret), K(tenant_id), K(index_table_id), K(ddl_task_id), K(index_tablet_ids.at(i).id()), K(tablet_latest_execution_id));
} else if (OB_FAIL(get_part_column_checksum(tenant_id, table_id, tablet_ids.at(i).id(), tablet_latest_execution_id, ddl_task_id, is_unique_index_checking, sql_proxy, column_checksum_map))) {
LOG_WARN("fail to get column checksum", K(ret), K(tenant_id), K(table_id), K(tablet_ids.at(i).id()), K(tablet_latest_execution_id), K(ddl_task_id), K(is_unique_index_checking), K(column_checksum_map.size()));
}
}
}
return ret;
}
int ObDDLChecksumOperator::check_column_checksum(
const uint64_t tenant_id,
const int64_t execution_id,
@ -383,14 +594,65 @@ int ObDDLChecksumOperator::check_column_checksum(
} else if (OB_FAIL(index_table_column_checksums.create(OB_MAX_COLUMN_NUMBER / 2, ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create column checksum map", K(ret));
} else if (OB_FAIL(get_table_column_checksum(tenant_id, execution_id, data_table_id,
ddl_task_id, is_unique_index_checking, data_table_column_checksums, sql_proxy))) {
ddl_task_id, is_unique_index_checking, data_table_column_checksums, sql_proxy))) {
LOG_WARN("fail to get table column checksum", K(ret), K(execution_id), K(data_table_id), K(ddl_task_id));
} else if (OB_FAIL(get_table_column_checksum(tenant_id, execution_id, index_table_id,
ddl_task_id, is_unique_index_checking, index_table_column_checksums, sql_proxy))) {
ddl_task_id, is_unique_index_checking, index_table_column_checksums, sql_proxy))) {
LOG_WARN("fail to get table column checksum", K(ret), K(execution_id), K(index_table_id), K(ddl_task_id));
} else {
for (hash::ObHashMap<int64_t, int64_t>::const_iterator iter = index_table_column_checksums.begin();
OB_SUCC(ret) && iter != index_table_column_checksums.end(); ++iter) {
OB_SUCC(ret) && iter != index_table_column_checksums.end(); ++iter) {
int64_t data_table_column_checksum = 0;
if (OB_FAIL(data_table_column_checksums.get_refactored(iter->first, data_table_column_checksum))) {
LOG_WARN("fail to get data table column checksum", K(ret), "column_id", iter->first);
} else if (data_table_column_checksum != iter->second) {
ret = OB_CHECKSUM_ERROR;
// In most cases, this checksum error is caused by unique constraint violation in user data, so we do not print error here.
LOG_WARN("column checksum is not equal", K(ret), K(data_table_id), K(index_table_id),
"column_id", iter->first, K(data_table_column_checksum),
"index_table_column_checksum", iter->second);
}
}
}
if (data_table_column_checksums.created()) {
data_table_column_checksums.destroy();
}
if (index_table_column_checksums.created()) {
index_table_column_checksums.destroy();
}
return ret;
}
int ObDDLChecksumOperator::check_column_checksum_without_execution_id(
const uint64_t tenant_id,
const uint64_t data_table_id,
const uint64_t index_table_id,
const int64_t ddl_task_id,
const bool is_unique_index_checking,
bool &is_equal,
common::ObMySQLProxy &sql_proxy)
{
int ret = OB_SUCCESS;
hash::ObHashMap<int64_t, int64_t> data_table_column_checksums;
hash::ObHashMap<int64_t, int64_t> index_table_column_checksums;
is_equal = true;
if (OB_UNLIKELY(OB_INVALID_ID == tenant_id || OB_INVALID_ID == data_table_id
|| OB_INVALID_ID == index_table_id || OB_INVALID_ID == ddl_task_id || !sql_proxy.is_inited())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(data_table_id), K(index_table_id), K(ddl_task_id), K(sql_proxy.is_inited()));
} else if (OB_FAIL(data_table_column_checksums.create(OB_MAX_COLUMN_NUMBER / 2, ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create column checksum map", K(ret));
} else if (OB_FAIL(index_table_column_checksums.create(OB_MAX_COLUMN_NUMBER / 2, ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create column checksum map", K(ret));
} else if (OB_FAIL(get_table_column_checksum_without_execution_id(tenant_id, data_table_id, index_table_id,
ddl_task_id, is_unique_index_checking, data_table_column_checksums, sql_proxy))) {
LOG_WARN("fail to get table column checksum", K(ret), K(data_table_id), K(index_table_id), K(ddl_task_id));
} else if (OB_FAIL(get_table_column_checksum_without_execution_id(tenant_id, index_table_id, index_table_id,
ddl_task_id, is_unique_index_checking, index_table_column_checksums, sql_proxy))) {
LOG_WARN("fail to get table column checksum", K(ret), K(index_table_id), K(ddl_task_id));
} else {
for (hash::ObHashMap<int64_t, int64_t>::const_iterator iter = index_table_column_checksums.begin();
OB_SUCC(ret) && iter != index_table_column_checksums.end(); ++iter) {
int64_t data_table_column_checksum = 0;
if (OB_FAIL(data_table_column_checksums.get_refactored(iter->first, data_table_column_checksum))) {
LOG_WARN("fail to get data table column checksum", K(ret), "column_id", iter->first);
@ -456,4 +718,4 @@ int ObDDLChecksumOperator::delete_checksum(
LOG_WARN("affected_rows is unexpected", KR(ret), K(affected_rows));
}
return ret;
}
}

View File

@ -80,6 +80,13 @@ public:
const int64_t ddl_task_id,
const bool is_unique_index_checking,
common::hash::ObHashMap<int64_t, int64_t> &column_checksums, common::ObMySQLProxy &sql_proxy);
static int get_table_column_checksum_without_execution_id(
const uint64_t tenant_id,
const uint64_t table_id,
const uint64_t index_table_id,
const int64_t ddl_task_id,
const bool is_unique_index_checking,
common::hash::ObHashMap<int64_t, int64_t> &column_checksums, common::ObMySQLProxy &sql_proxy);
static int get_tablet_checksum_record(
const uint64_t tenant_id,
const uint64_t execution_id,
@ -88,6 +95,13 @@ public:
const ObIArray<ObTabletID> &tablet_ids,
ObMySQLProxy &sql_proxy,
common::hash::ObHashMap<uint64_t, bool> &tablet_checksum_map);
static int get_tablet_checksum_record_without_execution_id(
const uint64_t tenant_id,
const uint64_t table_id,
const int64_t ddl_task_id,
const ObIArray<ObTabletID> &tablet_ids,
ObMySQLProxy &sql_proxy,
common::hash::ObHashMap<uint64_t, bool> &tablet_checksum_map);
static int check_column_checksum(
const uint64_t tenant_id,
const int64_t execution_id,
@ -97,6 +111,14 @@ public:
const bool is_unique_index_checking,
bool &is_equal,
common::ObMySQLProxy &sql_proxy);
static int check_column_checksum_without_execution_id(
const uint64_t tenant_id,
const uint64_t data_table_id,
const uint64_t index_table_id,
const int64_t ddl_task_id,
const bool is_unique_index_checking,
bool &is_equal,
common::ObMySQLProxy &sql_proxy);
static int delete_checksum(
const uint64_t tenant_id,
const int64_t execution_id,
@ -115,6 +137,22 @@ private:
const uint64_t tenant_id,
common::hash::ObHashMap<int64_t, int64_t> &column_checksum_map,
common::ObMySQLProxy &sql_proxy);
static int get_part_column_checksum(
const uint64_t tenant_id,
const uint64_t table_id,
const uint64_t tablet_id,
const uint64_t execution_id,
const int64_t ddl_task_id,
const bool is_unique_index_checking,
common::ObMySQLProxy &sql_proxy,
common::hash::ObHashMap<int64_t, int64_t> &column_checksum_map);
static int get_tablet_latest_execution_id(
const uint64_t tenant_id,
const uint64_t index_table_id,
const int64_t ddl_task_id,
const int64_t tablet_id,
common::ObMySQLProxy &sql_proxy,
int64_t &execution_id);
static int get_tablet_checksum_status(
const ObSqlString &sql,
const uint64_t tenant_id,

View File

@ -773,6 +773,7 @@ int ObDDLUtil::generate_build_replica_sql(
const bool use_heap_table_ddl_plan,
const bool use_schema_version_hint_for_src_table,
const ObColumnNameMap *col_name_map,
const ObString &partition_names,
ObSqlString &sql_string)
{
int ret = OB_SUCCESS;
@ -1021,26 +1022,30 @@ int ObDDLUtil::generate_build_replica_sql(
}
if (OB_FAIL(ret)) {
} else if (oracle_mode) {
if (OB_FAIL(sql_string.assign_fmt("INSERT /*+ monitor enable_parallel_dml parallel(%ld) opt_param('ddl_execution_id', %ld) opt_param('ddl_task_id', %ld) opt_param('enable_newsort', 'false') use_px */INTO \"%.*s\".\"%.*s\"(%.*s) SELECT /*+ index(\"%.*s\" primary) %.*s */ %.*s from \"%.*s\".\"%.*s\" as of scn %ld %.*s",
if (OB_FAIL(sql_string.assign_fmt("INSERT /*+ monitor enable_parallel_dml parallel(%ld) opt_param('ddl_execution_id', %ld) opt_param('ddl_task_id', %ld) opt_param('enable_newsort', 'false') use_px */INTO \"%.*s\".\"%.*s\" %.*s(%.*s) SELECT /*+ index(\"%.*s\" primary) %.*s */ %.*s from \"%.*s\".\"%.*s\" %.*s as of scn %ld %.*s",
real_parallelism, execution_id, task_id,
static_cast<int>(new_dest_database_name.length()), new_dest_database_name.ptr(), static_cast<int>(new_dest_table_name.length()), new_dest_table_name.ptr(),
static_cast<int>(partition_names.length()), partition_names.ptr(),
static_cast<int>(insert_column_sql_string.length()), insert_column_sql_string.ptr(),
static_cast<int>(new_source_table_name.length()), new_source_table_name.ptr(),
static_cast<int>(src_table_schema_version_hint_sql_string.length()), src_table_schema_version_hint_sql_string.ptr(),
static_cast<int>(query_column_sql_string.length()), query_column_sql_string.ptr(),
static_cast<int>(new_source_database_name.length()), new_source_database_name.ptr(), static_cast<int>(new_source_table_name.length()), new_source_table_name.ptr(),
static_cast<int>(partition_names.length()), partition_names.ptr(),
snapshot_version, static_cast<int>(rowkey_column_sql_string.length()), rowkey_column_sql_string.ptr()))) {
LOG_WARN("fail to assign sql string", K(ret));
}
} else {
if (OB_FAIL(sql_string.assign_fmt("INSERT /*+ monitor enable_parallel_dml parallel(%ld) opt_param('ddl_execution_id', %ld) opt_param('ddl_task_id', %ld) opt_param('enable_newsort', 'false') use_px */INTO `%.*s`.`%.*s`(%.*s) SELECT /*+ index(`%.*s` primary) %.*s */ %.*s from `%.*s`.`%.*s` as of snapshot %ld %.*s",
if (OB_FAIL(sql_string.assign_fmt("INSERT /*+ monitor enable_parallel_dml parallel(%ld) opt_param('ddl_execution_id', %ld) opt_param('ddl_task_id', %ld) opt_param('enable_newsort', 'false') use_px */INTO `%.*s`.`%.*s` %.*s(%.*s) SELECT /*+ index(`%.*s` primary) %.*s */ %.*s from `%.*s`.`%.*s` %.*s as of snapshot %ld %.*s",
real_parallelism, execution_id, task_id,
static_cast<int>(new_dest_database_name.length()), new_dest_database_name.ptr(), static_cast<int>(new_dest_table_name.length()), new_dest_table_name.ptr(),
static_cast<int>(partition_names.length()), partition_names.ptr(),
static_cast<int>(insert_column_sql_string.length()), insert_column_sql_string.ptr(),
static_cast<int>(new_source_table_name.length()), new_source_table_name.ptr(),
static_cast<int>(src_table_schema_version_hint_sql_string.length()), src_table_schema_version_hint_sql_string.ptr(),
static_cast<int>(query_column_sql_string.length()), query_column_sql_string.ptr(),
static_cast<int>(new_source_database_name.length()), new_source_database_name.ptr(), static_cast<int>(new_source_table_name.length()), new_source_table_name.ptr(),
static_cast<int>(partition_names.length()), partition_names.ptr(),
snapshot_version, static_cast<int>(rowkey_column_sql_string.length()), rowkey_column_sql_string.ptr()))) {
LOG_WARN("fail to assign sql string", K(ret));
}
@ -1377,6 +1382,322 @@ int ObDDLUtil::get_tablet_replica_location(
return ret;
}
int ObDDLUtil::construct_ls_tablet_id_map(
const uint64_t &tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
hash::ObHashMap<ObLSID, ObArray<ObTabletID>> &ls_tablet_id_map)
{
int ret = OB_SUCCESS;
bool is_cache_hit = false;
ObArray<ObTabletID> tablet_id_array;
if (!ls_id.is_valid() || !tablet_id.is_valid() || OB_INVALID_TENANT_ID == tenant_id) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(ls_id), K(tablet_id), K(tenant_id));
} else if (OB_FAIL(ls_tablet_id_map.get_refactored(ls_id, tablet_id_array))) {
if (OB_HASH_NOT_EXIST == ret) { // first time
ret = OB_SUCCESS;
if (OB_FAIL(tablet_id_array.push_back(tablet_id))) {
LOG_WARN("fail to push back to array", K(ret), K(tablet_id), K(tablet_id_array));
} else if (OB_FAIL(ls_tablet_id_map.set_refactored(ls_id, tablet_id_array, true /* overwrite */))) {
LOG_WARN("ls tablets map set fail", K(ret), K(ls_id), K(tablet_id_array));
}
} else {
LOG_WARN("ls tablets map get fail", K(ret), K(ls_id), K(tablet_id_array));
}
} else if (OB_FAIL(tablet_id_array.push_back(tablet_id))) {
LOG_WARN("fail to push back to array", K(ret), K(tablet_id_array), K(tablet_id));
} else if (OB_FAIL(ls_tablet_id_map.set_refactored(ls_id, tablet_id_array, true /* overwrite */))) {
LOG_WARN("ls tablets map set fail", K(ret), K(ls_id), K(tablet_id_array));
}
return ret;
}
int ObDDLUtil::get_index_table_batch_partition_names(
const uint64_t &tenant_id,
const int64_t &data_table_id,
const int64_t &index_table_id,
const ObIArray<ObTabletID> &tablets,
common::ObIAllocator &allocator,
ObIArray<ObString> &partition_names)
{
int ret = OB_SUCCESS;
if ((OB_UNLIKELY(OB_INVALID_ID == tenant_id || OB_INVALID_ID == data_table_id || OB_INVALID_ID == index_table_id || tablets.count() < 1))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("the parameters is invalid", K(ret), K(tenant_id), K(data_table_id), K(index_table_id), K(tablets.count()));
} else {
ObSchemaGetterGuard schema_guard;
const ObTableSchema *data_table_schema = nullptr;
const ObTableSchema *index_schema = nullptr;
if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(tenant_id, schema_guard))) {
LOG_WARN("fail to get schema guard", K(ret), K(tenant_id));
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, data_table_id, data_table_schema))) {
LOG_WARN("get table schema failed", K(ret), K(tenant_id), K(data_table_id));
} else if (OB_ISNULL(data_table_schema)) {
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("error unexpected, data table schema is null", K(ret), K(data_table_id));
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id, index_table_id, index_schema))) {
LOG_WARN("get table schema failed", K(ret), K(tenant_id), K(index_table_id));
} else if (OB_ISNULL(index_schema)) {
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("error unexpected, index table schema is null", K(ret), K(index_table_id));
} else {
const ObPartitionOption &data_part_option = data_table_schema->get_part_option();
const ObPartitionOption &index_part_option = index_schema->get_part_option();
if (OB_UNLIKELY(data_part_option.get_part_num() < 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("data table part num less than 1", K(ret), K(data_part_option));
} else if (OB_UNLIKELY(index_part_option.get_part_num() < 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("index table part num less than 1", K(ret), K(index_part_option));
} else if (OB_UNLIKELY(data_part_option.get_part_num() != index_part_option.get_part_num())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("error unexpected, data table partition num not equal to index table partition num", K(ret), K(data_part_option.get_part_num()), K(index_part_option.get_part_num()));
} else {
ObPartition **data_partitions = data_table_schema->get_part_array();
const ObPartitionLevel part_level = data_table_schema->get_part_level();
if (OB_ISNULL(data_partitions)) {
ret = OB_PARTITION_NOT_EXIST;
LOG_WARN("data table part array is null", K(ret));
} else {
int64_t part_index = -1;
int64_t subpart_index = -1;
for (int64_t i = 0; i < tablets.count() && OB_SUCC(ret); i++) {
if (OB_FAIL(index_schema->get_part_idx_by_tablet(tablets.at(i), part_index, subpart_index))) {
LOG_WARN("failed to get part idx by tablet", K(ret), K(tablets.at(i)), K(part_index), K(subpart_index));
} else {
ObString tmp_name;
if (PARTITION_LEVEL_ONE == part_level) {
if OB_FAIL(deep_copy_ob_string(allocator,
data_partitions[part_index]->get_part_name(),
tmp_name)) {
LOG_WARN("fail to deep copy partition names", K(ret), K(data_partitions[part_index]->get_part_name()), K(tmp_name));
} else if (OB_FAIL(partition_names.push_back(tmp_name))) {
LOG_WARN("fail to push back", K(ret), K(data_partitions[part_index]->get_part_name()), K(tmp_name), K(partition_names));
}
} else if (PARTITION_LEVEL_TWO == part_level) {
ObSubPartition **data_subpart_array = data_partitions[part_index]->get_subpart_array();
if (OB_ISNULL(data_subpart_array)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("part array is null", K(ret), K(part_index));
} else if OB_FAIL(deep_copy_ob_string(allocator,
data_subpart_array[subpart_index]->get_part_name(),
tmp_name)) {
LOG_WARN("fail to deep copy partition names", K(ret), K(data_subpart_array[subpart_index]->get_part_name()), K(tmp_name));
} else if (OB_FAIL(partition_names.push_back(tmp_name))) {
LOG_WARN("fail to push back", K(ret), K(data_subpart_array[subpart_index]->get_part_name()), K(tmp_name), K(partition_names));
}
}
}
}
}
}
}
}
return ret;
}
int ObDDLUtil::get_tablet_data_size(
const uint64_t &tenant_id,
const common::ObTabletID &tablet_id,
const share::ObLSID &ls_id,
int64_t &data_size)
{
int ret = OB_SUCCESS;
const int64_t obj_pos = 0;
ObObj result_obj;
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
data_size = 0;
if (!tablet_id.is_valid() || !ls_id.is_valid() || OB_INVALID_TENANT_ID == tenant_id) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tablet_id), K(tenant_id));
} else {
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
ObSqlString query_string;
sqlclient::ObMySQLResult *result = NULL;
if (OB_FAIL(query_string.assign_fmt("SELECT max(data_size) as data_size FROM %s WHERE tenant_id = %lu AND tablet_id = %lu AND ls_id = %lu",
OB_ALL_TABLET_META_TABLE_TNAME, tenant_id, tablet_id.id(), ls_id.id()))) {
LOG_WARN("assign sql string failed", K(ret), K(OB_ALL_TABLET_META_TABLE_TNAME), K(tenant_id), K(tablet_id), K(ls_id));
} else if (OB_FAIL(GCTX.sql_proxy_->read(res, meta_tenant_id, query_string.ptr()))) {
LOG_WARN("read record failed", K(ret), K(tenant_id), K(meta_tenant_id), K(query_string));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get sql result", K(ret), K(tenant_id), K(meta_tenant_id), K(query_string));
} else if (OB_FAIL(result->next())) {
LOG_WARN("get next result failed", K(ret), K(tenant_id), K(meta_tenant_id), K(query_string));
} else if (OB_FAIL(result->get_obj(obj_pos, result_obj))) {
LOG_WARN("failed to get object", K(ret));
} else if (result_obj.is_null()) {
data_size = 0;
LOG_WARN("data size is null", K(ret));
ret = OB_SUCCESS;
} else if (OB_UNLIKELY(!result_obj.is_integer_type())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected obj type", K(ret), K(result_obj.get_type()));
} else {
data_size = result_obj.get_int();
}
}
}
return ret;
}
int ObDDLUtil::get_tablet_data_row_cnt(
const uint64_t &tenant_id,
const common::ObTabletID &tablet_id,
const share::ObLSID &ls_id,
int64_t &data_row_cnt)
{
int ret = OB_SUCCESS;
const int64_t obj_pos = 0;
ObObj result_obj;
const uint64_t meta_tenant_id = gen_meta_tenant_id(tenant_id);
data_row_cnt = 0;
if (!tablet_id.is_valid() || !ls_id.is_valid() || OB_INVALID_TENANT_ID == tenant_id) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tablet_id), K(tenant_id));
} else {
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
ObSqlString query_string;
sqlclient::ObMySQLResult *result = NULL;
if (OB_FAIL(query_string.assign_fmt("SELECT max(row_count) as row_count FROM %s WHERE tenant_id = %lu AND tablet_id = %lu AND ls_id = %lu",
OB_ALL_TABLET_REPLICA_CHECKSUM_TNAME, tenant_id, tablet_id.id(), ls_id.id()))) {
LOG_WARN("assign sql string failed", K(ret), K(OB_ALL_TABLET_REPLICA_CHECKSUM_TNAME), K(tenant_id), K(tablet_id), K(ls_id));
} else if (OB_FAIL(GCTX.sql_proxy_->read(res, meta_tenant_id, query_string.ptr()))) {
LOG_WARN("read record failed", K(ret), K(tenant_id), K(meta_tenant_id), K(query_string));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get sql result", K(ret), K(tenant_id), K(meta_tenant_id), K(query_string));
} else if (OB_FAIL(result->next())) {
LOG_WARN("get next result failed", K(ret), K(tenant_id), K(meta_tenant_id), K(query_string));
} else if (OB_FAIL(result->get_obj(obj_pos, result_obj))) {
LOG_WARN("failed to get object", K(ret));
} else if (result_obj.is_null()) {
data_row_cnt = 0;
LOG_WARN("data size is null", K(ret));
ret = OB_SUCCESS;
} else if (OB_UNLIKELY(!result_obj.is_integer_type())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected obj type", K(ret), K(result_obj.get_type()));
} else {
data_row_cnt = result_obj.get_int();
}
}
}
return ret;
}
int ObDDLUtil::get_ls_host_left_disk_space(
const uint64_t &tenant_id,
const share::ObLSID &ls_id,
const common::ObAddr &leader_addr,
uint64_t &left_space_size)
{
int ret = OB_SUCCESS;
left_space_size = 0;
if (OB_INVALID_TENANT_ID == tenant_id || !ls_id.is_valid() || !leader_addr.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(ls_id), K(leader_addr));
} else {
char svr_ip[MAX_IP_ADDR_LENGTH] = "\0";
if (!leader_addr.ip_to_string(svr_ip, sizeof(svr_ip))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("format ip str failed", K(ret), K(leader_addr));
} else {
SMART_VAR(ObMySQLProxy::MySQLResult, res) {
ObSqlString query_string;
sqlclient::ObMySQLResult *result = NULL;
if (OB_FAIL(query_string.assign_fmt("SELECT free_size FROM %s WHERE svr_ip = \"%s\" AND svr_port = '%d'",
OB_ALL_VIRTUAL_DISK_STAT_TNAME, svr_ip, leader_addr.get_port()))) {
LOG_WARN("assign sql string failed", K(ret), K(OB_ALL_VIRTUAL_DISK_STAT_TNAME), K(svr_ip), K(leader_addr.get_port()));
} else if (OB_FAIL(GCTX.sql_proxy_->read(res, OB_SYS_TENANT_ID, query_string.ptr()))) {
LOG_WARN("read record failed", K(ret), K(tenant_id), K(ls_id), K(leader_addr), K(OB_SYS_TENANT_ID), K(query_string));
} else if (OB_ISNULL(result = res.get_result())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get sql result", K(ret), K(tenant_id), K(ls_id), K(leader_addr), K(OB_SYS_TENANT_ID), K(query_string));
} else if (OB_FAIL(result->next())) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to get next", K(ret), K(tenant_id), K(ls_id), K(leader_addr), K(OB_SYS_TENANT_ID), K(query_string));
}
} else {
EXTRACT_INT_FIELD_MYSQL(*result, "free_size", left_space_size, uint64_t);
}
}
}
}
return ret;
}
int ObDDLUtil::generate_partition_names(const common::ObIArray<ObString> &partition_names_array, common::ObIAllocator &allocator, ObString &partition_names)
{
int ret = OB_SUCCESS;
partition_names.reset();
ObSqlString sql_partition_names;
if (OB_UNLIKELY(partition_names_array.count() < 1)) {
LOG_WARN("array num is less than 1", K(ret), K(partition_names_array));
} else {
int64_t partition_nums = partition_names_array.count();
if (OB_FAIL(sql_partition_names.append("PARTITION("))) {
LOG_WARN("append partition names failed", K(ret), K(partition_names_array));
} else {
for (int64_t i = 0; i < partition_nums && OB_SUCC(ret); i++) {
if (i == partition_nums - 1) {
if (OB_FAIL(sql_partition_names.append_fmt("%.*s)", static_cast<int>(partition_names_array.at(i).length()), partition_names_array.at(i).ptr()))) {
LOG_WARN("append partition names failed", K(ret), K(partition_nums), K(partition_names_array), K(i), K(sql_partition_names));
}
} else {
if (OB_FAIL(sql_partition_names.append_fmt("%.*s,", static_cast<int>(partition_names_array.at(i).length()), partition_names_array.at(i).ptr()))) {
LOG_WARN("append partition names failed", K(ret), K(partition_nums), K(partition_names_array), K(i), K(sql_partition_names));
}
}
}
}
ObString tmp_name = sql_partition_names.string();
if (OB_SUCC(ret)) {
if OB_FAIL(deep_copy_ob_string(allocator,
tmp_name,
partition_names)) {
LOG_WARN("fail to deep copy partition names", K(ret), K(tmp_name), K(partition_names), K(partition_names_array));
}
}
}
return ret;
}
int ObDDLUtil::check_target_partition_is_running(const ObString &running_sql_info, const ObString &partition_name, common::ObIAllocator &allocator, bool &is_running_status)
{
int ret = OB_SUCCESS;
ObSqlString sql_partition_name;
ObString tmp_name;
is_running_status = false;
if (OB_UNLIKELY(running_sql_info.empty() || partition_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(running_sql_info), K(partition_name));
} else if (OB_FAIL(sql_partition_name.append_fmt("%.*s,", static_cast<int>(partition_name.length()), partition_name.ptr()))) {
LOG_WARN("append partition names failed", K(ret), K(partition_name), K(sql_partition_name));
} else {
tmp_name = sql_partition_name.string();
if (0 != ObCharset::instr(ObCollationType::CS_TYPE_UTF8MB4_BIN, running_sql_info.ptr(), running_sql_info.length(), tmp_name.ptr(), tmp_name.length())) {
is_running_status = true;
}
if (is_running_status == false) {
sql_partition_name.reuse();
tmp_name.reset();
if (OB_FAIL(sql_partition_name.append_fmt("%.*s)", static_cast<int>(partition_name.length()), partition_name.ptr()))) {
LOG_WARN("append partition names failed", K(ret), K(partition_name), K(sql_partition_name));
} else {
tmp_name = sql_partition_name.string();
if (0 != ObCharset::instr(ObCollationType::CS_TYPE_UTF8MB4_BIN, running_sql_info.ptr(), running_sql_info.length(), tmp_name.ptr(), tmp_name.length())) {
is_running_status = true;
}
}
}
}
return ret;
}
int ObDDLUtil::get_sys_ls_leader_addr(
const uint64_t cluster_id,
const uint64_t tenant_id,
@ -1583,20 +1904,22 @@ int ObDDLUtil::get_data_information(
EXTRACT_INT_FIELD_MYSQL(*result, "schema_version", schema_version, int64_t);
task_status = static_cast<share::ObDDLTaskStatus>(cur_task_status);
if (ObDDLType::DDL_CREATE_INDEX == ddl_type) {
SMART_VAR(rootserver::ObIndexBuildTask, task) {
if (OB_FAIL(task.deserialize_params_from_message(tenant_id, task_message.ptr(), task_message.length(), pos))) {
LOG_WARN("deserialize from msg failed", K(ret));
} else {
data_format_version = task.get_data_format_version();
if (OB_SUCC(ret)) {
if (is_create_index(ddl_type)) {
SMART_VAR(rootserver::ObIndexBuildTask, task) {
if (OB_FAIL(task.deserialize_params_from_message(tenant_id, task_message.ptr(), task_message.length(), pos))) {
LOG_WARN("deserialize from msg failed", K(ret));
} else {
data_format_version = task.get_data_format_version();
}
}
}
} else {
SMART_VAR(rootserver::ObTableRedefinitionTask, task) {
if (OB_FAIL(task.deserialize_params_from_message(tenant_id, task_message.ptr(), task_message.length(), pos))) {
LOG_WARN("deserialize from msg failed", K(ret));
} else {
data_format_version = task.get_data_format_version();
} else {
SMART_VAR(rootserver::ObTableRedefinitionTask, task) {
if (OB_FAIL(task.deserialize_params_from_message(tenant_id, task_message.ptr(), task_message.length(), pos))) {
LOG_WARN("deserialize from msg failed", K(ret));
} else {
data_format_version = task.get_data_format_version();
}
}
}
}
@ -2505,6 +2828,9 @@ int ObCheckTabletDataComplementOp::check_tablet_checksum_update_status(
LOG_WARN("ddl sim failure", K(ret), K(tenant_id), K(ddl_task_id));
} else if (OB_FAIL(tablet_checksum_status_map.create(tablet_count, ObModIds::OB_SSTABLE_CREATE_INDEX))) {
LOG_WARN("fail to create column checksum map", K(ret));
} else if (OB_ISNULL(GCTX.root_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("rootservice is null", K(ret));
} else if (OB_FAIL(ObDDLChecksumOperator::get_tablet_checksum_record(
tenant_id,
execution_id,

View File

@ -317,6 +317,10 @@ static inline bool is_invalid_ddl_type(const ObDDLType type)
return DDL_INVALID == type;
}
static inline bool is_create_index(const ObDDLType type)
{
return ObDDLType::DDL_CREATE_INDEX == type || ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX == type;
}
// ddl stmt or rs ddl trans has rollbacked and can retry
static inline bool is_ddl_stmt_packet_retry_err(const int ret)
{
@ -451,6 +455,7 @@ public:
const bool use_heap_table_ddl_plan,
const bool use_schema_version_hint_for_src_table,
const ObColumnNameMap *col_name_map,
const ObString &partition_names,
ObSqlString &sql_string);
static int generate_build_mview_replica_sql(
@ -528,6 +533,42 @@ public:
const uint64_t tenant_id,
const common::ObTabletID &tablet_id,
ObLSLocation &location);
static int construct_ls_tablet_id_map(
const uint64_t &tenant_id,
const share::ObLSID &ls_id,
const common::ObTabletID &tablet_id,
hash::ObHashMap<ObLSID, ObArray<ObTabletID>> &ls_tablet_id_map);
static int get_index_table_batch_partition_names(
const uint64_t &tenant_id,
const int64_t &data_table_id,
const int64_t &index_table_id,
const ObIArray<ObTabletID> &tablets,
common::ObIAllocator &allocator,
ObIArray<ObString> &partition_names);
static int get_tablet_data_size(
const uint64_t &tenant_id,
const common::ObTabletID &tablet_id,
const share::ObLSID &ls_id,
int64_t &data_size);
static int get_tablet_data_row_cnt(
const uint64_t &tenant_id,
const common::ObTabletID &tablet_id,
const share::ObLSID &ls_id,
int64_t &data_row_cnt);
static int get_ls_host_left_disk_space(
const uint64_t &tenant_id,
const share::ObLSID &ls_id,
const common::ObAddr &leader_addr,
uint64_t &left_space_size);
static int generate_partition_names(
const ObIArray<ObString> &partition_names_array,
common::ObIAllocator &allocator,
ObString &partition_names);
static int check_target_partition_is_running(
const ObString &running_sql_info,
const ObString &partition_name,
common::ObIAllocator &allocator,
bool &is_running_status);
static int check_table_exist(
const uint64_t tenant_id,
@ -740,6 +781,13 @@ public:
const uint64_t index_table_id,
const int64_t execution_id,
const uint64_t ddl_task_id);
static int check_tablet_checksum_update_status(
const uint64_t tenant_id,
const uint64_t index_table_id,
const uint64_t ddl_task_id,
const int64_t execution_id,
const ObIArray<ObTabletID> &tablet_ids,
bool &tablet_checksum_status);
private:
@ -795,14 +843,6 @@ private:
const uint64_t tenant_id,
const ObTabletID &tablet_id,
hash::ObHashMap<ObAddr, ObArray<ObTabletID>> &ip_tablets_map);
static int check_tablet_checksum_update_status(
const uint64_t tenant_id,
const uint64_t index_table_id,
const uint64_t ddl_task_id,
const int64_t execution_id,
const ObIArray<ObTabletID> &tablet_ids,
bool &tablet_checksum_status);
};
typedef common::ObCurTraceId::TraceId DDLTraceId;

View File

@ -536,6 +536,7 @@ int ObDDLErrorMessageTableOperator::generate_index_ddl_error_message(const int r
{
int ret = OB_SUCCESS;
ObBuildDDLErrorMessage error_message;
uint64_t tenant_data_format_version = 0;
const uint64_t tenant_id = index_schema.get_tenant_id();
const uint64_t data_table_id = index_schema.get_data_table_id();
const uint64_t index_table_id = index_schema.get_table_id();
@ -549,8 +550,10 @@ int ObDDLErrorMessageTableOperator::generate_index_ddl_error_message(const int r
} else if (OB_FALSE_IT(memset(error_message.user_message_, 0, OB_MAX_ERROR_MSG_LEN))) {
} else if (OB_FAIL(index_schema.get_index_name(index_name))) { //get index name
LOG_WARN("fail to get index name", K(ret), K(index_name), K(index_table_id));
} else if (OB_FAIL(ObShareUtil::fetch_current_data_version(sql_proxy, index_schema.get_tenant_id(), tenant_data_format_version))) {
LOG_WARN("get min data version failed", K(ret), K(index_schema.get_tenant_id()));
} else if (OB_FAIL(build_ddl_error_message(ret_code, index_schema.get_tenant_id(), data_table_id, error_message, index_name,
index_table_id, DDL_CREATE_INDEX, index_key, report_ret_code))) {
index_table_id, ((DATA_VERSION_4_2_2_0 <= tenant_data_format_version && tenant_data_format_version < DATA_VERSION_4_3_0_0) || tenant_data_format_version >= DATA_VERSION_4_3_2_0) && index_schema.is_storage_local_index_table() && index_schema.is_partitioned_table() ? ObDDLType::DDL_CREATE_PARTITIONED_LOCAL_INDEX : ObDDLType::DDL_CREATE_INDEX, index_key, report_ret_code))) {
LOG_WARN("build ddl error message failed", K(ret), K(data_table_id), K(index_name));
} else if (OB_FAIL(report_ddl_error_message(error_message, //report into __all_ddl_error_message
tenant_id, trace_id, task_id, parent_task_id, data_table_id, schema_version, object_id, addr, sql_proxy))) {

View File

@ -883,7 +883,7 @@ int ObAlterTableExecutor::alter_table_rpc_v2(
}
}
}
} else if (DDL_CREATE_INDEX == res.ddl_type_ || DDL_NORMAL_TYPE == res.ddl_type_) {
} else if (is_create_index(res.ddl_type_) || DDL_NORMAL_TYPE == res.ddl_type_) {
// TODO(shuangcan): alter table create index returns DDL_NORMAL_TYPE now, check if we can fix this later
// 同步等索引建成功
for (int64_t i = 0; OB_SUCC(ret) && i < add_index_arg_list.size(); ++i) {

View File

@ -967,7 +967,7 @@ int ObPxSubCoord::end_ddl(const bool need_commit)
}
}
}
FLOG_INFO("end ddl sstable", K(ret), K(need_commit), K(ls_tablet_ids));
FLOG_INFO("end ddl", "context_id", ddl_ctrl_.context_id_, K(ret), K(need_commit));
DEBUG_SYNC(END_DDL_IN_PX_SUBCOORD);
}
if (OB_EAGAIN == ret) {

View File

@ -176,6 +176,73 @@ int ObInsertLogPlan::generate_normal_raw_plan()
return ret;
}
int ObInsertLogPlan::get_index_part_ids(const ObInsertTableInfo& table_info, const ObTableSchema *&data_table_schema, const ObTableSchema *&index_schema, ObIArray<uint64_t> &index_part_ids)
{
int ret = OB_SUCCESS;
common::hash::ObHashSet<int64_t> data_part_id_set;
index_part_ids.reset();
if (OB_UNLIKELY(OB_ISNULL(data_table_schema) || OB_ISNULL(index_schema))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("the parameters is invalid", K(ret), K(table_info));
} else if (table_info.part_ids_.count() > 0) {
const ObPartitionOption &data_part_option = data_table_schema->get_part_option();
ObPartition **data_partitions = data_table_schema->get_part_array();
ObPartition **index_partitions = index_schema->get_part_array();
const ObPartitionLevel part_level = data_table_schema->get_part_level();
if (OB_ISNULL(data_partitions)) {
ret = OB_PARTITION_NOT_EXIST;
LOG_WARN("data table part array is null", K(ret));
} else if (OB_ISNULL(index_partitions)) {
ret = OB_PARTITION_NOT_EXIST;
LOG_WARN("index table part array is null", K(ret));
} else {
uint64_t part_ids_count = table_info.part_ids_.count();
if (OB_FAIL(data_part_id_set.create(part_ids_count))) {
LOG_WARN("fail to create data part id set", K(ret), K(part_ids_count));
}
for (int64_t i = 0; i < part_ids_count && OB_SUCC(ret); i++) {
if (OB_FAIL(data_part_id_set.set_refactored(table_info.part_ids_.at(i), true/*overwrite*/))) {
LOG_WARN("fail to set refactored", K(ret), K(table_info.part_ids_.at(i)), K(table_info.part_ids_));
}
}
for (int64_t i = 0; i < data_part_option.get_part_num() && OB_SUCC(ret); ++i) {
if (OB_ISNULL(data_partitions[i]) || OB_ISNULL(index_partitions[i])) {
ret = OB_PARTITION_NOT_EXIST;
LOG_WARN("NULL ptr", K(ret), K(i));
} else if (PARTITION_LEVEL_ONE == part_level) {
int64_t data_part_id = data_partitions[i]->get_part_id();
if (OB_UNLIKELY(OB_HASH_EXIST == data_part_id_set.exist_refactored(data_part_id))) {
if (OB_FAIL(index_part_ids.push_back(index_partitions[i]->get_part_id()))) {
LOG_WARN("push back error", K(ret), K(index_partitions[i]->get_part_id()));
}
}
} else if (PARTITION_LEVEL_TWO == part_level) {
ObSubPartition **data_subpart_array = data_partitions[i]->get_subpart_array();
ObSubPartition **index_subpart_array = index_partitions[i]->get_subpart_array();
int64_t subpart_num = index_partitions[i]->get_subpartition_num();
if (OB_ISNULL(data_subpart_array) || OB_ISNULL(index_subpart_array)) {
ret = OB_PARTITION_NOT_EXIST;
LOG_WARN("part array is null", K(ret), K(i));
} else if (OB_UNLIKELY(subpart_num < 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sub part num less than 1", K(ret), K(subpart_num));
} else {
for (int64_t j = 0; j < subpart_num && OB_SUCC(ret); j++) {
int64_t data_part_id = data_subpart_array[j]->get_sub_part_id();
if (OB_UNLIKELY(OB_HASH_EXIST == data_part_id_set.exist_refactored(data_part_id))) {
if (OB_FAIL(index_part_ids.push_back(index_subpart_array[j]->get_sub_part_id()))) {
LOG_WARN("push back error", K(ret), K(index_subpart_array[j]->get_sub_part_id()));
}
}
}
}
}
}
}
}
return ret;
}
int ObInsertLogPlan::generate_osg_share_info(OSGShareInfo *&info)
{
int ret = OB_SUCCESS;
@ -1311,20 +1378,26 @@ int ObInsertLogPlan::prepare_table_dml_info_for_ddl(const ObInsertTableInfo& tab
LOG_WARN("failed to get table schema", K(table_info), K(ret));
} else if (index_schema->is_index_table() && !index_schema->is_global_index_table()) {
// local index
ObArray<uint64_t> index_part_ids;
if (OB_FAIL(schema_guard->get_table_schema(session_info->get_effective_tenant_id(),
index_schema->get_data_table_id(),
data_table_schema))) {
LOG_WARN("get table schema failed", K(ret));
index_schema->get_data_table_id(),
data_table_schema))) {
LOG_WARN("get table schema failed", K(ret), K(session_info->get_effective_tenant_id()), K(index_schema->get_data_table_id()));
} else if (OB_ISNULL(data_table_schema)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get table schema", K(index_schema->get_data_table_id()), K(ret));
ret = OB_TABLE_NOT_EXIST;
LOG_WARN("failed to get table schema", K(ret), K(index_schema->get_data_table_id()));
} else if (OB_FAIL(get_index_part_ids(table_info, data_table_schema, index_schema, index_part_ids))) {
LOG_WARN("fail to get index part ids", K(ret), K(table_info), K(table_item->ddl_table_id_), K(index_schema->get_data_table_id()), K(index_part_ids));
} else if (OB_FAIL(index_dml_info->part_ids_.assign(index_part_ids))) {
LOG_WARN("fail to assign part ids", K(ret), K(index_part_ids));
} else {
index_dml_info->table_id_ = table_info.table_id_;
index_dml_info->loc_table_id_ = table_info.loc_table_id_;
index_dml_info->ref_table_id_ = index_schema->get_data_table_id();
index_dml_info->rowkey_cnt_ = index_schema->get_rowkey_column_num();
index_dml_info->spk_cnt_ = index_schema->get_shadow_rowkey_column_num();
index_dml_info->index_name_ = data_table_schema->get_table_name_str();
index_dml_info->table_id_ = table_info.table_id_;
index_dml_info->loc_table_id_ = table_info.loc_table_id_;
index_dml_info->ref_table_id_ = index_schema->get_data_table_id();
index_dml_info->rowkey_cnt_ = index_schema->get_rowkey_column_num();
index_dml_info->spk_cnt_ = index_schema->get_shadow_rowkey_column_num();
index_dml_info->index_name_ = data_table_schema->get_table_name_str();
LOG_INFO("index dml info contains part ids: ", K(ret), K(index_dml_info->part_ids_)); //在局部索引表补数据场景下,对主表part ids和索引表part ids做了关系映射
}
} else {
// global index or primary table

View File

@ -127,6 +127,7 @@ protected:
int check_contain_non_onetime_expr(const ObRawExpr *expr, bool &contain);
int check_contain_non_onetime_expr(const ObIArray<ObRawExpr *> &exprs, bool &contain);
private:
int get_index_part_ids(const ObInsertTableInfo& table_info, const ObTableSchema *&data_table_schema, const ObTableSchema *&index_schema, ObIArray<uint64_t> &index_part_ids);
int generate_osg_share_info(OSGShareInfo *&info);
int check_need_online_stats_gather(bool &need_osg);
int set_is_direct_insert();