[FEAT MERGE] Parallel create table

Co-authored-by: Tyshawn <tuyunshan@gmail.com>
This commit is contained in:
tino247 2023-08-31 10:40:35 +00:00 committed by ob-robot
parent abb2a6b573
commit 49d54bfc46
101 changed files with 10118 additions and 651 deletions

View File

@ -766,6 +766,7 @@ public:
int64_t get_sub_map_count() const { return sub_map_count_; }
int64_t get_sub_map_mem_size() const { return sub_map_mem_size_; }
void set_sub_map_mem_size(int64_t sub_map_mem_size) { sub_map_mem_size_ = sub_map_mem_size; }
TO_STRING_KV(KP(this), K_(sub_map_count), K_(sub_map_mem_size), K(allocator_.get_label()));
private:
ObPointerHashMap(const ObPointerHashMap &other, const int64_t resize_to)

View File

@ -2412,6 +2412,8 @@ OB_INLINE const char*& ob_get_origin_thread_name()
return tname;
}
static const char* PARALLEL_DDL_THREAD_NAME = "DDLPQueueTh";
// There are many clusters in arbitration server, we need a field identify the different clusters.
OB_INLINE int64_t &ob_get_cluster_id()
{

View File

@ -829,6 +829,16 @@ DEF_NAME(id, "id")
DEF_NAME(status, "status")
DEF_NAME(get_trans_result, "get_trans_result")
DEF_NAME(deserialize, "deserialize")
DEF_NAME(start_ddl_trans, "start_ddl_trans")
DEF_NAME(lock_objects, "lock_objects")
DEF_NAME(generate_schemas, "generate_schemas")
DEF_NAME(gen_task_id_and_versions, "gen_task_id_and_versions")
DEF_NAME(create_schemas, "create_schemas")
DEF_NAME(create_tablets, "create_tablets")
DEF_NAME(inc_schema_dict, "inc_schema_dict")
DEF_NAME(wait_ddl_trans, "wait_ddl_trans")
DEF_NAME(end_ddl_trans, "end_ddl_trans")
DEF_NAME_PAIR(create_view, "create view")
// location cache related
DEF_NAME_PAIR(renew_loc_by_sql, "renew loc by sql")

View File

@ -22,7 +22,7 @@ const ObLatchDesc OB_LATCHES[] __attribute__ ((init_priority(102))) = {
#undef LATCH_DEF
};
static_assert(ARRAYSIZEOF(OB_LATCHES) == 313, "DO NOT delete latch defination");
static_assert(ARRAYSIZEOF(OB_LATCHES) == 314, "DO NOT delete latch defination");
static_assert(ObLatchIds::LATCH_END == ARRAYSIZEOF(OB_LATCHES) - 1, "update id of LATCH_END before adding your defination");
}

View File

@ -327,8 +327,9 @@ LATCH_DEF(LOG_EXTERNAL_STORAGE_HANDLER_LOCK, 309, "log external storage handler
LATCH_DEF(PL_DEBUG_RUNTIMEINFO_LOCK, 310, "PL DEBUG RuntimeInfo lock", LATCH_FIFO, 2000, 0)
LATCH_DEF(MDS_TABLE_HANDLER_LOCK, 311, "mds table handler lock", LATCH_READ_PREFER, 2000, 0)
LATCH_DEF(IND_NAME_CACHE_LOCK, 312, "index name cache lock", LATCH_FIFO, 2000, 0)
LATCH_DEF(LATCH_END, 312, "latch end", LATCH_FIFO, 2000, 0)
LATCH_DEF(LATCH_END, 313, "latch end", LATCH_FIFO, 2000, 0)
#endif

View File

@ -91,6 +91,7 @@ WAIT_EVENT_DEF(START_STMT_WAIT, 16002, "wait start stmt", "trans_hash_value", "p
WAIT_EVENT_DEF(END_STMT_WAIT, 16003, "wait end stmt", "rollback", "trans_hash_value", "physic_plan_type", CLUSTER, false)
WAIT_EVENT_DEF(REMOVE_PARTITION_WAIT, 16004, "wait remove partition", "tenant_id", "table_id", "partition_id", ADMINISTRATIVE, false)
WAIT_EVENT_DEF(TABLET_LOCK_WAIT, 16016, "tablet lock wait", "", "", "", CONCURRENCY, true)
WAIT_EVENT_DEF(IND_NAME_CACHE_LOCK_WAIT, 16017, "latch:index name cache lock wait", "address", "number", "tries", CONCURRENCY, true)
WAIT_EVENT_DEF(OBCDC_PART_MGR_SCHEMA_VERSION_WAIT, 18000, "oblog part mgr schema version wait", "", "", "", CONCURRENCY, true)
// sleep

View File

@ -174,35 +174,24 @@ namespace share
{
namespace schema
{
common::SpinRWLock lock_for_schema_version;
int ObSchemaServiceSQLImpl::gen_new_schema_version(
uint64_t tenant_id,
int64_t refreshed_schema_version,
int64_t &schema_version)
{
SpinWLockGuard guard(lock_for_schema_version);
int ret = OB_SUCCESS;
schema_version = OB_INVALID_VERSION;
SpinWLockGuard guard(lock_for_schema_version);
if (OB_INVALID_TENANT_ID == tenant_id) {
const int64_t version_cnt = 1;
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tenant_id", K(ret), K(tenant_id));
} else {
if (is_sys_tenant(tenant_id)) {
if (OB_FAIL(gen_leader_sys_schema_version(tenant_id, schema_version))) {
LOG_WARN("failed to gen leader sys tenant_id schema version", K(ret), K(tenant_id));
}
} else {
// normal tenant
if (OB_FAIL(gen_leader_normal_schema_version(tenant_id, refreshed_schema_version, schema_version))) {
LOG_WARN("failed to gen leader normal schema version", K(ret), K(tenant_id), K(refreshed_schema_version));
}
}
LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id));
} else if (OB_FAIL(gen_tenant_new_schema_version_(tenant_id, refreshed_schema_version, version_cnt, schema_version))) {
LOG_WARN("fail to gen schema version", KR(ret), K(tenant_id), K(refreshed_schema_version));
}
if (OB_FAIL(ret)) {
} else {
LOG_INFO("new schema version", K(schema_version), "this", OB_P(this));
if (OB_SUCC(ret)) {
LOG_INFO("new schema version", K(tenant_id), K(schema_version));
}
return ret;
}

View File

@ -606,6 +606,15 @@ int ObMultiReplicaTestBase::create_tenant(const char *tenant_name,
SERVER_LOG(WARN, "create_tenant", K(ret));
}
}
{
ObSqlString sql;
if (FAILEDx(sql.assign_fmt("alter system set _enable_parallel_table_creation = false tenant = all"))) {
SERVER_LOG(WARN, "create_tenant", KR(ret));
} else if (OB_FAIL(sql_proxy.write(sql.ptr(), affected_rows))) {
SERVER_LOG(WARN, "create_tenant", KR(ret));
}
usleep(5 * 1000 * 1000L); // 5s
}
if (change_log_level) {
OB_LOGGER.set_log_level(log_level);
}

View File

@ -414,7 +414,7 @@ public:
|| share::schema::ObTableType::MATERIALIZED_VIEW == table_type_;
}
OB_INLINE share::schema::ObIndexType get_index_type() const { return index_type_; }
OB_INLINE bool is_index_table() const { return share::schema::ObTableSchema::is_index_table(table_type_); }
OB_INLINE bool is_index_table() const { return share::schema::is_index_table(table_type_); }
OB_INLINE bool is_normal_index() const
{
return share::schema::INDEX_TYPE_NORMAL_LOCAL == index_type_

View File

@ -65,7 +65,7 @@ struct ObCDCTableInfo
inline uint64_t get_table_id() const { return table_id_; }
inline const share::schema::ObTableType &get_table_type() const { return table_type_; }
inline bool is_index_table() const { return share::schema::ObSimpleTableSchemaV2::is_index_table(table_type_); }
inline bool is_index_table() const { return share::schema::is_index_table(table_type_); }
bool operator==(const ObCDCTableInfo &that) const {
return table_id_ == that.table_id_ && table_type_ == that.table_type_;

View File

@ -1722,6 +1722,8 @@ typedef enum ObItemType
T_ALTER_SUBPARTITION_ADD,
T_ALTER_SUBPARTITION_DROP,
T_ALTER_SUBPARTITION_TRUNCATE,
T_ALTER_SUBPARTITION_RENAME,
T_ALTER_PARTITION_RENAME,
T_ANALYZE,
T_ANALYZE_STATISTICS,

View File

@ -116,8 +116,8 @@ int ObSchemaReleaseTimeTask::init(ObServerSchemaUpdater &schema_updater, int tg_
} else {
schema_updater_ = &schema_updater;
is_inited_ = true;
if (OB_FAIL(TG_SCHEDULE(tg_id, *this, REFRESH_INTERVAL, true /*schedule repeatly*/))) {
LOG_WARN("fail to schedule task ObSchemaReleaseTimeTask", K(ret));
if (OB_FAIL(schedule_())) {
LOG_WARN("fail to schedule ObSchemaReleaseTimeTask in init", KR(ret));
}
}
return ret;
@ -129,6 +129,19 @@ void ObSchemaReleaseTimeTask::destroy()
schema_updater_ = nullptr;
}
int ObSchemaReleaseTimeTask::schedule_()
{
int ret = OB_SUCCESS;
int64_t memory_recycle_interval = GCONF._schema_memory_recycle_interval;
if (0 == memory_recycle_interval) {
memory_recycle_interval = 15L * 60L * 1000L * 1000L; //15mins
}
if (OB_FAIL(TG_SCHEDULE(lib::TGDefIDs::ServerGTimer, *this, memory_recycle_interval, false /*not schedule repeatly*/))) {
LOG_ERROR("fail to schedule task ObSchemaReleaseTimeTask", KR(ret));
}
return ret;
}
void ObSchemaReleaseTimeTask::runTimerTask()
{
int ret = OB_SUCCESS;
@ -141,6 +154,10 @@ void ObSchemaReleaseTimeTask::runTimerTask()
} else if (OB_FAIL(schema_updater_->try_release_schema())) {
LOG_WARN("ObSchemaReleaseTimeTask failed", K(ret));
}
// we should ignore error to schedule task
if (OB_FAIL(schedule_())) {
LOG_WARN("fail to schedule ObSchemaReleaseTimeTask in runTimerTask", KR(ret));
}
}
ObRemoteMasterRsUpdateTask::ObRemoteMasterRsUpdateTask(const ObGlobalContext &gctx)

View File

@ -53,7 +53,8 @@ public:
void destroy();
virtual void runTimerTask() override;
private:
const static int64_t REFRESH_INTERVAL = 30L * 60L * 1000L * 1000L; // 30min
int schedule_();
private:
ObServerSchemaUpdater *schema_updater_;
bool is_inited_;
};

View File

@ -31,6 +31,7 @@
#include "observer/omt/ob_tenant.h"
#include "observer/omt/ob_multi_tenant.h"
#include "rpc/obmysql/ob_mysql_packet.h"
#include "rootserver/ob_rs_rpc_processor.h"
#include "common/ob_clock_generator.h"
using namespace oceanbase::common;
@ -466,7 +467,7 @@ int ObSrvDeliver::init_queue_threads()
// TODO: fufeng, make it configurable
if (OB_FAIL(create_queue_thread(lib::TGDefIDs::LeaseQueueTh, "LeaseQueueTh", lease_queue_))) {
} else if (OB_FAIL(create_queue_thread(lib::TGDefIDs::DDLQueueTh, "DDLQueueTh", ddl_queue_))) {
} else if (OB_FAIL(create_queue_thread(lib::TGDefIDs::DDLPQueueTh, "DDLPQueueTh", ddl_parallel_queue_))) {
} else if (OB_FAIL(create_queue_thread(lib::TGDefIDs::DDLPQueueTh, PARALLEL_DDL_THREAD_NAME, ddl_parallel_queue_))) {
} else if (OB_FAIL(create_queue_thread(lib::TGDefIDs::MysqlQueueTh,
"MysqlQueueTh", mysql_queue_))) {
} else if (OB_FAIL(create_queue_thread(lib::TGDefIDs::DiagnoseQueueTh,
@ -524,8 +525,7 @@ int ObSrvDeliver::deliver_rpc_request(ObRequest &req)
} else if (OB_RENEW_LEASE == pkt.get_pcode()) {
queue = &lease_queue_->queue_;
} else if (10 == pkt.get_priority()) {
// for new parallel truncate table rpc
if (OB_TRUNCATE_TABLE_V2 == pkt.get_pcode()) {
if (rootserver::is_parallel_ddl(pkt.get_pcode())) {
queue = &ddl_parallel_queue_->queue_;
} else {
queue = &ddl_queue_->queue_;

View File

@ -96,6 +96,7 @@ void oceanbase::observer::init_srv_xlator_for_rootserver(ObSrvRpcXlator *xlator)
RPC_PROCESSOR(rootserver::ObRpcDropTablegroupP, *gctx_.root_service_);
RPC_PROCESSOR(rootserver::ObRpcAlterTablegroupP, *gctx_.root_service_);
RPC_PROCESSOR(rootserver::ObRpcCreateTableP, *gctx_.root_service_);
RPC_PROCESSOR(rootserver::ObRpcParallelCreateTableP, *gctx_.root_service_);
RPC_PROCESSOR(rootserver::ObRpcStartRedefTableP, *gctx_.root_service_);
RPC_PROCESSOR(rootserver::ObRpcCopyTableDependentsP, *gctx_.root_service_);
RPC_PROCESSOR(rootserver::ObRpcFinishRedefTableP, *gctx_.root_service_);

View File

@ -96,6 +96,7 @@ int ObAllVirtualSchemaMemory::inner_get_next_row(common::ObNewRow *&row)
const int64_t free_schema_mgr_cnt = schema_mem.get_free_schema_mgr_cnt();
const int64_t mem_used = schema_mem.get_mem_used();
const int64_t mem_total = schema_mem.get_mem_total();
const int64_t allocator_idx = schema_mem.get_allocator_idx();
const int64_t col_count = output_column_ids_.count();
for (int64_t i = 0; OB_SUCC(ret) && i < col_count; ++i) {
@ -136,6 +137,10 @@ int ObAllVirtualSchemaMemory::inner_get_next_row(common::ObNewRow *&row)
cur_row_.cells_[i].set_int(mem_total);
break;
}
case ALLOCATOR_IDX: {
cur_row_.cells_[i].set_int(allocator_idx);
break;
}
default : {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid col_id", KR(ret), K(col_id));

View File

@ -36,6 +36,7 @@ class ObAllVirtualSchemaMemory: public common::ObVirtualTableScannerIterator
FREE_SCHEMA_MGR_CNT,
MEM_USED,
MEM_TOTAL,
ALLOCATOR_IDX,
};
public:
explicit ObAllVirtualSchemaMemory(share::schema::ObMultiVersionSchemaService &schema_service)

View File

@ -117,7 +117,7 @@ int ObAllVirtualSchemaSlot::inner_get_next_row(common::ObNewRow *&row)
const int64_t total_ref_cnt = schema_slot.get_ref_cnt();
const int64_t schema_version = schema_slot.get_schema_version();
const int64_t schema_count = schema_slot.get_schema_count();
const int64_t allocator_idx = schema_slot.get_allocator_idx();
const int64_t col_count = output_column_ids_.count();
for (int64_t i = 0; OB_SUCC(ret) && i < col_count; ++i) {
uint64_t col_id = output_column_ids_.at(i);
@ -162,6 +162,10 @@ int ObAllVirtualSchemaSlot::inner_get_next_row(common::ObNewRow *&row)
ObCharset::get_default_charset()));
break;
}
case ALLOCATOR_IDX: {
cur_row_.cells_[i].set_int(static_cast<int64_t>(allocator_idx));
break;
}
default : {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid col_id", KR(ret), K(col_id));

View File

@ -38,6 +38,7 @@ class ObAllVirtualSchemaSlot: public common::ObVirtualTableScannerIterator
SCHEMA_COUNT,
REF_CNT,
REF_INFO,
ALLOCATOR_IDX,
};
public:
explicit ObAllVirtualSchemaSlot(share::schema::ObMultiVersionSchemaService &schema_service)

View File

@ -74,8 +74,6 @@ int ObAllVirtualServerSchemaInfo::inner_get_next_row(common::ObNewRow *&row)
LOG_WARN("fail to get schema guard", K(tmp_ret), K(tenant_id));
} else if (OB_SUCCESS != (tmp_ret = schema_guard.get_schema_count(tenant_id, schema_count))) {
LOG_WARN("fail to get schema count", K(tmp_ret), K(tenant_id));
} else if (OB_SUCCESS != (tmp_ret = schema_guard.get_schema_size(tenant_id, schema_size))) {
LOG_WARN("fail to get schema size", K(tmp_ret), K(tenant_id));
}
}
@ -110,7 +108,13 @@ int ObAllVirtualServerSchemaInfo::inner_get_next_row(common::ObNewRow *&row)
break;
}
case OB_APP_MIN_COLUMN_ID + 6: { // schema_size
cur_row_.cells_[i].set_int(schema_size);
int tmp_ret = OB_SUCCESS;
if (OB_SUCCESS != (tmp_ret = schema_guard.get_schema_size(tenant_id, schema_size))) {
cur_row_.cells_[i].set_int(OB_INVALID_ID);
LOG_WARN("fail to get schema size", K(tmp_ret), K(tenant_id));
} else {
cur_row_.cells_[i].set_int(schema_size);
}
break;
}
case OB_APP_MIN_COLUMN_ID + 7: { // min_schema_version

View File

@ -122,6 +122,13 @@ ob_set_subtarget(ob_rootserver ddl_task
ddl_task/ob_table_redefinition_task.cpp
)
ob_set_subtarget(ob_rootserver parallel_ddl
parallel_ddl/ob_ddl_helper.cpp
parallel_ddl/ob_create_table_helper.cpp
parallel_ddl/ob_create_view_helper.cpp
parallel_ddl/ob_index_name_checker.cpp
)
ob_set_subtarget(ob_rootserver freeze
freeze/ob_tenant_major_freeze.cpp
freeze/ob_major_merge_scheduler.cpp

View File

@ -88,7 +88,9 @@ int ObDDLRetryTask::deep_copy_ddl_arg(
} else if (ObDDLType::DDL_DROP_PARTITION == ddl_type
|| ObDDLType::DDL_DROP_SUB_PARTITION == ddl_type
|| ObDDLType::DDL_TRUNCATE_PARTITION == ddl_type
|| ObDDLType::DDL_TRUNCATE_SUB_PARTITION == ddl_type) {
|| ObDDLType::DDL_TRUNCATE_SUB_PARTITION == ddl_type
|| ObDDLType::DDL_RENAME_PARTITION == ddl_type
|| ObDDLType::DDL_RENAME_SUB_PARTITION == ddl_type) {
if (OB_ISNULL(ddl_arg_buf = static_cast<char *>(allocator.alloc(sizeof(obrpc::ObAlterTableArg))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
@ -145,7 +147,9 @@ int ObDDLRetryTask::init_compat_mode(const share::ObDDLType &ddl_type,
} else if (ObDDLType::DDL_DROP_PARTITION == ddl_type
|| ObDDLType::DDL_DROP_SUB_PARTITION == ddl_type
|| ObDDLType::DDL_TRUNCATE_PARTITION == ddl_type
|| ObDDLType::DDL_TRUNCATE_SUB_PARTITION == ddl_type) {
|| ObDDLType::DDL_TRUNCATE_SUB_PARTITION == ddl_type
|| ObDDLType::DDL_RENAME_PARTITION == ddl_type
|| ObDDLType::DDL_RENAME_SUB_PARTITION == ddl_type) {
compat_mode_ = static_cast<const obrpc::ObAlterTableArg *>(source_arg)->compat_mode_;
}
return ret;
@ -390,6 +394,8 @@ int ObDDLRetryTask::drop_schema(const ObDDLTaskStatus next_task_status)
}
case ObDDLType::DDL_DROP_PARTITION:
case ObDDLType::DDL_DROP_SUB_PARTITION:
case ObDDLType::DDL_RENAME_PARTITION:
case ObDDLType::DDL_RENAME_SUB_PARTITION:
case ObDDLType::DDL_TRUNCATE_PARTITION:
case ObDDLType::DDL_TRUNCATE_SUB_PARTITION: {
obrpc::ObAlterTableArg *arg = static_cast<obrpc::ObAlterTableArg *>(ddl_arg_);
@ -450,6 +456,8 @@ int ObDDLRetryTask::wait_alter_table(const ObDDLTaskStatus new_status)
}
case ObDDLType::DDL_DROP_PARTITION:
case ObDDLType::DDL_DROP_SUB_PARTITION:
case ObDDLType::DDL_RENAME_PARTITION:
case ObDDLType::DDL_RENAME_SUB_PARTITION:
case ObDDLType::DDL_TRUNCATE_PARTITION:
case ObDDLType::DDL_TRUNCATE_SUB_PARTITION: {
obrpc::ObAlterTableArg *arg = static_cast<obrpc::ObAlterTableArg *>(ddl_arg_);
@ -652,7 +660,9 @@ int ObDDLRetryTask::deserlize_params_from_message(const uint64_t tenant_id, cons
} else if (ObDDLType::DDL_DROP_PARTITION == task_type_
|| ObDDLType::DDL_DROP_SUB_PARTITION == task_type_
|| ObDDLType::DDL_TRUNCATE_PARTITION == task_type_
|| ObDDLType::DDL_TRUNCATE_SUB_PARTITION == task_type_) {
|| ObDDLType::DDL_TRUNCATE_SUB_PARTITION == task_type_
|| ObDDLType::DDL_RENAME_PARTITION == task_type_
|| ObDDLType::DDL_RENAME_SUB_PARTITION == task_type_) {
obrpc::ObAlterTableArg tmp_arg;
if (OB_FAIL(tmp_arg.deserialize(buf, buf_size, pos))) {
LOG_WARN("serialize table failed", K(ret));

View File

@ -972,6 +972,8 @@ int ObDDLScheduler::create_ddl_task(const ObCreateDDLTaskParam &param,
case DDL_TRUNCATE_TABLE:
case DDL_DROP_PARTITION:
case DDL_DROP_SUB_PARTITION:
case DDL_RENAME_PARTITION:
case DDL_RENAME_SUB_PARTITION:
case DDL_TRUNCATE_PARTITION:
case DDL_TRUNCATE_SUB_PARTITION:
default:
@ -1798,6 +1800,8 @@ int ObDDLScheduler::schedule_ddl_task(const ObDDLTaskRecord &record)
case DDL_TRUNCATE_TABLE:
case DDL_DROP_PARTITION:
case DDL_DROP_SUB_PARTITION:
case DDL_RENAME_PARTITION:
case DDL_RENAME_SUB_PARTITION:
case DDL_TRUNCATE_PARTITION:
case DDL_TRUNCATE_SUB_PARTITION:
ret = schedule_ddl_retry_task(record);

View File

@ -295,6 +295,8 @@ trace::ObSpanCtx* ObDDLTracing::begin_task_span()
case DDL_TRUNCATE_TABLE:
case DDL_DROP_PARTITION:
case DDL_DROP_SUB_PARTITION:
case DDL_RENAME_PARTITION:
case DDL_RENAME_SUB_PARTITION:
case DDL_TRUNCATE_PARTITION:
case DDL_TRUNCATE_SUB_PARTITION:
span = FLT_BEGIN_SPAN(ddl_retry_task);
@ -375,6 +377,8 @@ trace::ObSpanCtx* ObDDLTracing::restore_task_span()
case DDL_TRUNCATE_TABLE:
case DDL_DROP_PARTITION:
case DDL_DROP_SUB_PARTITION:
case DDL_RENAME_PARTITION:
case DDL_RENAME_SUB_PARTITION:
case DDL_TRUNCATE_PARTITION:
case DDL_TRUNCATE_SUB_PARTITION:
span = FLT_RESTORE_DDL_SPAN(ddl_retry_task, task_span_id_, task_start_ts_);
@ -757,6 +761,12 @@ int ObDDLTask::get_ddl_type_str(const int64_t ddl_type, const char *&ddl_type_st
case DDL_DROP_SUB_PARTITION:
ddl_type_str = "drop sub partition";
break;
case DDL_RENAME_PARTITION:
ddl_type_str = "rename partition";
break;
case DDL_RENAME_SUB_PARTITION:
ddl_type_str = "rename sub partition";
break;
case DDL_TRUNCATE_PARTITION:
ddl_type_str = "truncate partition";
break;

View File

@ -406,7 +406,6 @@ ObNewTableTabletAllocator::ObNewTableTabletAllocator(
: tenant_id_(tenant_id),
schema_guard_(schema_guard),
sql_proxy_(sql_proxy),
trans_(),
bg_ls_stat_operator_(),
status_(MyStatus::INVALID),
ls_id_array_(),
@ -431,8 +430,6 @@ int ObNewTableTabletAllocator::init()
LOG_WARN("sql proxy ptr is null", KR(ret), KP(sql_proxy_));
} else if (OB_FAIL(bg_ls_stat_operator_.init(sql_proxy_))) {
LOG_WARN("fail to init bg_ls_stat_operator_", KR(ret));
} else if (OB_FAIL(trans_.start(sql_proxy_, meta_tenant_id))) {
LOG_WARN("fail to start trans", KR(ret), K(meta_tenant_id));
} else {
status_ = MyStatus::WAIT_TO_PREPARE;
is_add_partition_ = false;
@ -561,19 +558,8 @@ int ObNewTableTabletAllocator::get_ls_id_array(
int ObNewTableTabletAllocator::finish(
const bool commit)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
// by pass, maybe invoked with !inited_
} else if (!trans_.is_started()) {
// bypass
} else {
int tmp_ret = OB_SUCCESS;
if (OB_SUCCESS != (tmp_ret = trans_.end(commit))) {
LOG_WARN("fail to end trans", KR(ret), KR(tmp_ret), K(commit));
ret = (OB_SUCCESS == ret ? tmp_ret : ret);
}
}
return ret;
UNUSED(commit);
return OB_SUCCESS;
}
int ObNewTableTabletAllocator::get_tablet_id_array(
@ -716,7 +702,7 @@ int ObNewTableTabletAllocator::alloc_tablet_for_create_balance_group(
if (OB_SUCC(ret)) {
if (OB_FAIL(bg_ls_stat_operator_.insert_update_balance_group_ls_stat(
THIS_WORKER.get_timeout_remain(),
trans_,
*sql_proxy_,
tenant_id_,
bg_id,
bg_ls_stat_array))) {
@ -829,7 +815,7 @@ int ObNewTableTabletAllocator::alloc_tablet_for_add_balance_group(
if (OB_SUCC(ret)) {
if (OB_FAIL(bg_ls_stat_operator_.insert_update_balance_group_ls_stat(
THIS_WORKER.get_timeout_remain(),
trans_,
*sql_proxy_,
tenant_id_,
bg_id,
final_ls_stat_array))) {
@ -873,10 +859,10 @@ int ObNewTableTabletAllocator::alloc_tablet_for_one_level_partitioned_balance_gr
common::ObArray<ObBalanceGroupLSStat> bg_ls_stat_array;
if (OB_FAIL(bg_ls_stat_operator_.get_balance_group_ls_stat(
THIS_WORKER.get_timeout_remain(),
trans_,
*sql_proxy_,
tenant_id_,
bg.id(),
true, /*for update*/
false, /*for update*/
bg_ls_stat_array))) {
LOG_WARN("fail to get balance group ls stat", KR(ret),
K(tenant_id_), K(bg));
@ -937,10 +923,10 @@ int ObNewTableTabletAllocator::alloc_tablet_for_two_level_partitioned_balance_gr
common::ObArray<ObBalanceGroupLSStat> bg_ls_stat_array;
if (OB_FAIL(bg_ls_stat_operator_.get_balance_group_ls_stat(
THIS_WORKER.get_timeout_remain(),
trans_,
*sql_proxy_,
tenant_id_,
bg.id(),
true, /*for update*/
false, /*for update*/
bg_ls_stat_array))) {
LOG_WARN("fail to get balance group ls stat", KR(ret),
K(tenant_id_), K(bg));
@ -983,10 +969,10 @@ int ObNewTableTabletAllocator::alloc_tablet_for_non_partitioned_balance_group(
LOG_WARN("fail to get available ls", KR(ret));
} else if (OB_FAIL(bg_ls_stat_operator_.get_balance_group_ls_stat(
THIS_WORKER.get_timeout_remain(),
trans_,
*sql_proxy_,
tenant_id_,
bg.id(),
true, /*for update*/
false, /*for update*/
bg_ls_stat_array))) {
LOG_WARN("fail to get balance group ls stat", KR(ret), K(tenant_id_), K(bg));
} else if (OB_FAIL(alloc_tablet_for_add_balance_group(

View File

@ -256,7 +256,6 @@ private:
uint64_t tenant_id_;
share::schema::ObSchemaGetterGuard &schema_guard_;
common::ObMySQLProxy *sql_proxy_;
common::ObMySQLTransaction trans_;
ObBalanceGroupLSStatOperator bg_ls_stat_operator_;
MyStatus status_;
common::ObArray<share::ObLSID> ls_id_array_;

View File

@ -708,7 +708,6 @@ int ObBootstrap::create_all_core_table_partition()
ObMySQLProxy &sql_proxy = ddl_service_.get_sql_proxy();
ObTableCreator table_creator(OB_SYS_TENANT_ID,
SCN::base_scn(),
lst_operator_,
trans);
if (OB_FAIL(trans.start(&sql_proxy, OB_SYS_TENANT_ID))) {
LOG_WARN("fail to start trans", KR(ret));
@ -757,7 +756,6 @@ int ObBootstrap::create_all_partitions()
ObMySQLProxy &sql_proxy = ddl_service_.get_sql_proxy();
ObTableCreator table_creator(OB_SYS_TENANT_ID,
SCN::base_scn(),
lst_operator_,
trans);
if (OB_FAIL(trans.start(&sql_proxy, OB_SYS_TENANT_ID))) {
LOG_WARN("fail to start trans", KR(ret));

View File

@ -1612,7 +1612,7 @@ int ObDDLOperator::create_sequence_in_create_table(ObTableSchema &table_schema,
ObSequenceDDLProxy ddl_operator(schema_service_);
char temp_sequence_name[OB_MAX_SEQUENCE_NAME_LENGTH + 1] = { 0 };
int32_t len = snprintf(temp_sequence_name, sizeof(temp_sequence_name), "%s%lu%c%lu",
"ISEQ$$_",
IDENTITY_COLUMN_SEQUENCE_OBJECT_NAME_PREFIX,
ObSchemaUtils::get_extract_schema_id(table_schema.get_tenant_id(), table_schema.get_table_id()),
'_',
column_schema.get_column_id());
@ -2517,6 +2517,52 @@ int ObDDLOperator::check_part_equal(
return ret;
}
int ObDDLOperator::rename_table_partitions(const ObTableSchema &orig_table_schema,
ObTableSchema &inc_table_schema,
ObTableSchema &new_table_schema,
ObMySQLTransaction &trans)
{
int ret = OB_SUCCESS;
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
int64_t new_schema_version = OB_INVALID_VERSION;
ObSchemaService *schema_service = schema_service_.get_schema_service();
if (OB_ISNULL(schema_service)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema_service is NULL", KR(ret));
} else if (OB_FAIL(schema_service_.gen_new_schema_version(tenant_id, new_schema_version))) {
LOG_WARN("fail to gen new schema version", KR(ret), K(tenant_id));
} else if (OB_FAIL(schema_service->get_table_sql_service().rename_inc_part_info(trans,
orig_table_schema,
inc_table_schema,
new_schema_version))) {
LOG_WARN("rename inc part info failed", KR(ret));
}
return ret;
}
int ObDDLOperator::rename_table_subpartitions(const ObTableSchema &orig_table_schema,
ObTableSchema &inc_table_schema,
ObTableSchema &new_table_schema,
ObMySQLTransaction &trans)
{
int ret = OB_SUCCESS;
const uint64_t tenant_id = orig_table_schema.get_tenant_id();
int64_t new_schema_version = OB_INVALID_VERSION;
ObSchemaService *schema_service = schema_service_.get_schema_service();
if (OB_ISNULL(schema_service)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema_service is NULL", KR(ret));
} else if (OB_FAIL(schema_service_.gen_new_schema_version(tenant_id, new_schema_version))) {
LOG_WARN("fail to gen new schema version", KR(ret), K(tenant_id));
} else if (OB_FAIL(schema_service->get_table_sql_service().rename_inc_subpart_info(trans,
orig_table_schema,
inc_table_schema,
new_schema_version))) {
LOG_WARN("rename inc subpart info failed", KR(ret));
}
return ret;
}
int ObDDLOperator::drop_table_partitions(const ObTableSchema &orig_table_schema,
ObTableSchema &inc_table_schema,
ObTableSchema &new_table_schema,

View File

@ -281,6 +281,14 @@ public:
share::schema::ObTableSchema &inc_table_schema,
share::schema::ObTableSchema &new_table_schema,
common::ObMySQLTransaction &trans);
int rename_table_partitions(const share::schema::ObTableSchema &orig_table_schema,
share::schema::ObTableSchema &inc_table_schema,
share::schema::ObTableSchema &new_table_schema,
common::ObMySQLTransaction &trans);
int rename_table_subpartitions(const share::schema::ObTableSchema &orig_table_schema,
share::schema::ObTableSchema &inc_table_schema,
share::schema::ObTableSchema &new_table_schema,
common::ObMySQLTransaction &trans);
int get_part_array_from_table(const share::schema::ObTableSchema &orig_table_schema,
const share::schema::ObTableSchema &inc_table_schema,
common::ObIArray<share::schema::ObPartition*> &part_array);

File diff suppressed because it is too large Load Diff

View File

@ -37,6 +37,7 @@
#include "share/ob_freeze_info_proxy.h"
#include "common/ob_common_utility.h"
#include "share/config/ob_config.h" // ObConfigPairs
#include "rootserver/parallel_ddl/ob_index_name_checker.h"
namespace oceanbase
{
@ -47,7 +48,6 @@ using ObAddrArray = ObSEArray<ObAddr, 3>;
class ObMySQLProxy;
class ObAddr;
class ObMySQLTransaction;
class ObDDLSQLTransaction;
}
namespace obrpc
{
@ -95,27 +95,6 @@ class ObTableGroupHelp;
//class ObFreezeInfoManager;
class ObSnapshotInfoManager;
class ObDDLTaskController
{
public:
ObDDLTaskController() : inited_(false) {}
int init();
static const int DDL_TASK_COND_SLOT = 16;
int register_task_and_assign_schema_version(share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id,
const uint64_t schema_version_count,
ObIArray<int64_t> &schema_version_res);
int wait_task_ready(int64_t schema_version);
int del_task_schema_version(int64_t schema_version);
private:
int check_task_ready(int64_t schema_version, bool &ready);
private:
bool inited_;
ObThreadCond cond_slot_[DDL_TASK_COND_SLOT];
ObSEArray<int64_t, 32> tasks_;
common::SpinRWLock lock_;
};
class ObDDLService
{
public:
@ -146,6 +125,7 @@ public:
ObZoneManager &get_zone_mgr() { return *zone_mgr_; }
ObSnapshotInfoManager &get_snapshot_mgr() { return *snapshot_mgr_; }
share::ObLSTableOperator &get_lst_operator() { return *lst_operator_; }
share::schema::ObIndexNameChecker &get_index_name_checker() { return index_name_checker_; }
// create_index_table will fill table_id and frozen_version to table_schema
virtual int create_index_table(const obrpc::ObCreateIndexArg &arg,
@ -292,14 +272,15 @@ public:
common::ObArenaAllocator &allocator,
obrpc::ObAlterTableRes &res,
ObIArray<ObDDLTaskRecord> &ddl_tasks);
template <typename PARTITION_SCHEMA>
int generate_object_id_for_partition_schemas(
ObIArray<PARTITION_SCHEMA> &partition_schemas);
ObIArray<ObTableSchema> &partition_schemas);
int generate_object_id_for_partition_schema(
ObPartitionSchema &partition_schema,
const bool gen_subpart_only = false);
const bool gen_subpart_only = false,
share::ObIDGenerator *batch_id_generator = NULL);
int generate_tables_tablet_id(ObIArray<ObTableSchema> &table_schemas);
int generate_tablet_id(ObTableSchema &schema);
int generate_tablet_id(ObTableSchema &schema,
share::ObIDGenerator *batch_id_generator = NULL);
int alter_table_column(
const share::schema::ObTableSchema &origin_table_schema,
const share::schema::AlterTableSchema & alter_table_schema,
@ -420,7 +401,8 @@ public:
share::schema::ObTableSchema &new_table_schema,
ObDDLOperator &ddl_operator,
ObSchemaGetterGuard &schema_guard,
ObMySQLTransaction &trans);
ObMySQLTransaction &trans,
const ObTableSchema &orig_data_table_schema);
virtual int alter_table_constraints(const obrpc::ObAlterTableArg::AlterConstraintType type,
share::schema::ObSchemaGetterGuard &schema_guard,
const share::schema::ObTableSchema &orig_table_schema,
@ -1072,6 +1054,10 @@ int check_table_udt_id_is_exist(share::schema::ObSchemaGetterGuard &schema_guard
int recompile_view(const ObTableSchema &view_schema, const bool reset_view_column_infos, ObDDLSQLTransaction &trans);
int recompile_all_views_batch(const uint64_t tenant_id, const common::ObIArray<uint64_t > &view_ids);
int try_add_dep_info_for_all_synonyms_batch(const uint64_t tenant_id, const common::ObIArray<uint64_t> &synonym_ids);
int try_check_and_set_table_schema_in_tablegroup(
share::schema::ObSchemaGetterGuard &schema_guard,
share::schema::ObTableSchema &schema);
private:
enum PartitionBornMethod : int64_t
{
@ -1120,6 +1106,15 @@ private:
LOCALITY_NOT_CHANGED,
ALTER_LOCALITY_INVALID,
};
int calc_partition_object_id_cnt_(
const ObPartitionSchema &partition_schema,
const bool gen_subpart_only,
int64_t &object_cnt);
int calc_table_tablet_id_cnt_(
const ObTableSchema &table_schema,
uint64_t &tablet_cnt);
int check_has_index_operation(
ObSchemaGetterGuard &schema_guard,
const uint64_t teannt_id,
@ -2245,6 +2240,10 @@ private:
const bool is_split);
int check_alter_partitions(const share::schema::ObTableSchema &orig_table_schema,
obrpc::ObAlterTableArg &alter_table_arg);
int check_alter_rename_partitions_(const share::schema::ObTableSchema &orig_table_schema,
const obrpc::ObAlterTableArg &alter_table_arg);
int check_alter_rename_subpartitions_(const share::schema::ObTableSchema &orig_table_schema,
const obrpc::ObAlterTableArg &alter_table_arg);
int check_alter_drop_partitions(const share::schema::ObTableSchema &orig_table_schema,
const obrpc::ObAlterTableArg &alter_table_arg,
const bool is_truncate);
@ -2270,9 +2269,6 @@ private:
const share::schema::ObSysVariableSchema &old_sys_variable,
share::schema::ObSysVariableSchema &new_sys_variable,
bool& value_changed);
int try_check_and_set_table_schema_in_tablegroup(
share::schema::ObSchemaGetterGuard &schema_guard,
share::schema::ObTableSchema &schema);
virtual int reconstruct_table_schema_from_recyclebin(share::schema::ObTableSchema &index_table_schema,
const share::schema::ObRecycleObject &recycle_obj,
@ -2331,6 +2327,9 @@ private:
int gen_inc_table_schema_for_drop_part(
const share::schema::ObTableSchema &orig_table_schema,
share::schema::AlterTableSchema &inc_table_schema);
int gen_inc_table_schema_for_rename_part_(
const share::schema::ObTableSchema &orig_table_schema,
share::schema::AlterTableSchema &inc_table_schema);
int gen_inc_table_schema_for_trun_part(
const share::schema::ObTableSchema &orig_table_schema,
share::schema::AlterTableSchema &inc_table_schema,
@ -2341,6 +2340,9 @@ private:
int gen_inc_table_schema_for_drop_subpart(
const share::schema::ObTableSchema &orig_table_schema,
share::schema::AlterTableSchema &inc_table_schema);
int gen_inc_table_schema_for_rename_subpart_(
const share::schema::ObTableSchema &orig_table_schema,
share::schema::AlterTableSchema &inc_table_schema);
int gen_inc_table_schema_for_trun_subpart(
const share::schema::ObTableSchema &orig_table_schema,
share::schema::AlterTableSchema &inc_table_schema,
@ -2379,11 +2381,37 @@ private:
const ObIArray<const ObTableSchema*> &orig_table_schemas,
const ObIArray<ObTableSchema*> &new_table_schemas,
ObMySQLTransaction &trans);
int check_alter_tenant_when_rebalance_is_disabled_(
const share::schema::ObTenantSchema &orig_tenant_schema,
const share::schema::ObTenantSchema &new_tenant_schema);
//not check belong to the same table
int check_same_partition_(const bool is_oracle_mode, const ObPartition &l, const ObPartition &r,
const ObPartitionFuncType part_type, bool &is_matched) const;
//not check belong to the same table
int check_same_subpartition_(const bool is_oracle_mode, const ObSubPartition &l, const ObSubPartition &r,
const ObPartitionFuncType part_type, bool &is_matched) const;
//After renaming a partition/subpartition, the consistency of the partition name between the data table and aux table is no longer guaranteed.
//Therefore, the partition names in the inc aux table must be synchronized with the ori aux table after assigning the data table's partition
//schema to the inc aux table.
//This function relies on the assumption that the inc table schema has a valid partition name.
int fix_local_idx_part_name_(const ObSimpleTableSchemaV2 &ori_data_table_schema,
const ObSimpleTableSchemaV2 &ori_table_schema,
ObSimpleTableSchemaV2 &inc_table_schema);
//This function relies on the assumption that the inc table schema has a valid subpartition name.
int fix_local_idx_subpart_name_(const ObSimpleTableSchemaV2 &ori_data_table_schema,
const ObSimpleTableSchemaV2 &ori_table_schema,
ObSimpleTableSchemaV2 &inc_table_schema);
//During the process of adding a partition/subpartition, we only check whether the partition schema of the argument is valid.
//It's possible for the inc aux table's partition name to duplicate with an existing partition name if one renames a partition/subpartition
//to another name and then adds a partition/subpartition with the same name.
//In this case, we will generate a name with a part/subpart id to replace the inc part/subpart name to avoid duplication.
int fix_local_idx_part_name_for_add_part_(const ObSimpleTableSchemaV2 &ori_table_schema,
ObSimpleTableSchemaV2 &inc_table_schema);
int fix_local_idx_part_name_for_add_subpart_(const ObSimpleTableSchemaV2 &ori_table_schema,
ObSimpleTableSchemaV2 &inc_table_schema);
private:
int check_locality_compatible_(ObTenantSchema &schema);
@ -2471,9 +2499,10 @@ private:
ObZoneManager *zone_mgr_;
ObUnitManager *unit_mgr_;
ObSnapshotInfoManager *snapshot_mgr_;
mutable common::SpinRWLock pz_entity_cnt_lock_;
ObDDLTaskController ddl_task_controller_;
ObLatch ddl_lock_; // for ddl concurrent control
// for paralled ddl to cache oracle's index name map
share::schema::ObIndexNameChecker index_name_checker_;
private:
DISALLOW_COPY_AND_ASSIGN(ObDDLService);
};
@ -2693,20 +2722,6 @@ int ObDDLService::check_partition_name_valid(const SCHEMA &orig_schema,
return ret;
}
template <typename PARTITION_SCHEMA>
int ObDDLService::generate_object_id_for_partition_schemas(
ObIArray<PARTITION_SCHEMA> &partition_schemas)
{
int ret = common::OB_SUCCESS;
for (int64_t i = 0; i < partition_schemas.count() && OB_SUCC(ret); i++) {
if (OB_FAIL(generate_object_id_for_partition_schema(partition_schemas.at(i)))) {
RS_LOG(WARN, "fail to generate object_id for partitions",
KR(ret), K(partition_schemas.at(i)));
}
}
return ret;
}
} // end namespace rootserver
} // end namespace oceanbasr
#endif // _OCEANBASE_ROOTSERVER_OB_DDL_SERVICE_H_

View File

@ -294,7 +294,8 @@ int ObIndexBuilder::do_create_global_index(
new_arg, new_table_schema, allocator, gen_columns))) {
LOG_WARN("fail to adjust expr index args", K(ret));
} else if (OB_FAIL(generate_schema(
new_arg, new_table_schema, global_index_without_column_info, index_schema))) {
new_arg, new_table_schema, global_index_without_column_info,
true/*generate_id*/, index_schema))) {
LOG_WARN("fail to generate schema", K(ret), K(new_arg));
} else {
if (gen_columns.empty()) {
@ -472,7 +473,8 @@ int ObIndexBuilder::do_create_local_index(
my_arg, new_table_schema, allocator, gen_columns))) {
LOG_WARN("fail to adjust expr index args", K(ret));
} else if (OB_FAIL(generate_schema(
my_arg, new_table_schema, global_index_without_column_info, index_schema))) {
my_arg, new_table_schema, global_index_without_column_info,
true/*generate_id*/, index_schema))) {
LOG_WARN("fail to generate schema", K(ret), K(my_arg));
} else if (OB_FAIL(new_table_schema.check_create_index_on_hidden_primary_key(index_schema))) {
LOG_WARN("failed to check create index on table", K(ret), K(index_schema));
@ -626,6 +628,7 @@ int ObIndexBuilder::generate_schema(
const ObCreateIndexArg &arg,
ObTableSchema &data_schema,
const bool global_index_without_column_info,
const bool generate_id,
ObTableSchema &schema)
{
int ret = OB_SUCCESS;
@ -804,10 +807,12 @@ int ObIndexBuilder::generate_schema(
LOG_WARN("fail to assign partition schema", K(schema), K(ret));
} else if (OB_FAIL(ddl_service_.try_format_partition_schema(schema))) {
LOG_WARN("fail to format partition schema", KR(ret), K(schema));
} else if (OB_FAIL(ddl_service_.generate_object_id_for_partition_schema(schema))) {
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(schema));
} else if (OB_FAIL(ddl_service_.generate_tablet_id(schema))) {
LOG_WARN("fail to fetch new table id", K(schema), K(ret));
} else if (generate_id) {
if (OB_FAIL(ddl_service_.generate_object_id_for_partition_schema(schema))) {
LOG_WARN("fail to generate object_id for partition schema", KR(ret), K(schema));
} else if (OB_FAIL(ddl_service_.generate_tablet_id(schema))) {
LOG_WARN("fail to fetch new table id", KR(ret), K(schema));
}
}
}
return ret;

View File

@ -76,6 +76,7 @@ public:
int generate_schema(const obrpc::ObCreateIndexArg &arg,
share::schema::ObTableSchema &data_schema,
const bool global_index_without_column_info,
const bool generate_id,
share::schema::ObTableSchema &index_schema);
int submit_drop_index_task(
common::ObMySQLTransaction &trans,

View File

@ -36,6 +36,7 @@ ObLobMetaBuilder::~ObLobMetaBuilder()
int ObLobMetaBuilder::generate_aux_lob_meta_schema(
ObSchemaService *schema_service,
const share::schema::ObTableSchema &data_schema,
const uint64_t specified_table_id,
share::schema::ObTableSchema &aux_lob_meta_schema,
bool need_generate_id)
{
@ -47,14 +48,15 @@ int ObLobMetaBuilder::generate_aux_lob_meta_schema(
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(data_schema), K(ret));
} else {
uint64_t new_table_id = OB_INVALID_ID;
uint64_t new_table_id = specified_table_id;
const int64_t buf_size = 64;
char buf[buf_size];
MEMSET(buf, 0, buf_size);
int64_t pos = 0;
if (OB_FAIL(generate_schema(data_schema, aux_lob_meta_schema))) {
LOG_WARN("generate_schema for aux vp table failed", K(data_schema), K(ret));
} else if (OB_FAIL(schema_service->fetch_new_table_id(data_schema.get_tenant_id(), new_table_id))) {
} else if (OB_INVALID_ID == new_table_id
&& OB_FAIL(schema_service->fetch_new_table_id(data_schema.get_tenant_id(), new_table_id))) {
LOG_WARN("failed to fetch_new_table_id", "tenant_id", data_schema.get_tenant_id(), K(ret));
} else if (OB_FAIL(generate_lob_meta_table_name(new_table_id, buf, buf_size, pos))) {
LOG_WARN("failed to generate_lob_meta_table_name", K(ret), K(new_table_id));

View File

@ -36,9 +36,11 @@ public:
explicit ObLobMetaBuilder(ObDDLService &ddl_service);
virtual ~ObLobMetaBuilder();
// won't fetch new table id if specified_table_id is valid
int generate_aux_lob_meta_schema(
share::schema::ObSchemaService *schema_service,
const share::schema::ObTableSchema &data_schema,
const uint64_t specified_table_id,
share::schema::ObTableSchema &aux_lob_meta_schema,
bool generate_id);

View File

@ -36,6 +36,7 @@ ObLobPieceBuilder::~ObLobPieceBuilder()
int ObLobPieceBuilder::generate_aux_lob_piece_schema(
ObSchemaService *schema_service,
const share::schema::ObTableSchema &data_schema,
const uint64_t specified_table_id,
share::schema::ObTableSchema &aux_lob_piece_schema,
bool need_generate_id)
{
@ -47,14 +48,15 @@ int ObLobPieceBuilder::generate_aux_lob_piece_schema(
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(data_schema), K(ret));
} else {
uint64_t new_table_id = OB_INVALID_ID;
uint64_t new_table_id = specified_table_id;
const int64_t buf_size = 64;
char buf[buf_size];
MEMSET(buf, 0, buf_size);
int64_t pos = 0;
if (OB_FAIL(generate_schema(data_schema, aux_lob_piece_schema))) {
LOG_WARN("generate_schema for aux vp table failed", K(data_schema), K(ret));
} else if (OB_FAIL(schema_service->fetch_new_table_id(data_schema.get_tenant_id(), new_table_id))) {
} else if (OB_INVALID_ID == new_table_id
&& OB_FAIL(schema_service->fetch_new_table_id(data_schema.get_tenant_id(), new_table_id))) {
LOG_WARN("failed to fetch_new_table_id", "tenant_id", data_schema.get_tenant_id(), K(ret));
} else if (OB_FAIL(generate_lob_piece_table_name(new_table_id, buf, buf_size, pos))) {
LOG_WARN("failed to generate_lob_piece_table_name", K(ret), K(new_table_id));

View File

@ -27,9 +27,11 @@ public:
explicit ObLobPieceBuilder(ObDDLService &ddl_service);
virtual ~ObLobPieceBuilder();
// won't fetch new table id if specified_table_id is valid
int generate_aux_lob_piece_schema(
share::schema::ObSchemaService *schema_service,
const share::schema::ObTableSchema &data_schema,
const uint64_t specified_table_id,
share::schema::ObTableSchema &aux_vp_table_schema,
bool generate_id);

View File

@ -108,6 +108,9 @@
#include "logservice/ob_log_service.h"
#include "rootserver/ob_heartbeat_service.h"
#include "parallel_ddl/ob_create_table_helper.h" // ObCreateTableHelper
#include "parallel_ddl/ob_create_view_helper.h" // ObCreateViewHelper
namespace oceanbase
{
@ -1279,6 +1282,8 @@ void ObRootService::wait()
FLOG_INFO("rebalance task mgr exit success");
TG_WAIT(lib::TGDefIDs::GlobalCtxTimer);
FLOG_INFO("global ctx timer exit success");
ddl_service_.get_index_name_checker().reset_all_cache();
FLOG_INFO("reset index name checker success");
ObUpdateRsListTask::clear_lock();
THE_RS_JOB_TABLE.reset_max_job_id();
int64_t cost = ObTimeUtility::current_time() - start_time;
@ -2970,6 +2975,67 @@ int ObRootService::handle_security_audit(const ObSecurityAuditArg &arg)
return ret;
}
int ObRootService::parallel_ddl_pre_check_(const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
bool is_dropped = false;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id));
} else if (OB_FAIL(schema_service_->check_if_tenant_has_been_dropped(tenant_id, is_dropped))) {
LOG_WARN("fail to check if tenant has been dropped", KR(ret), K(tenant_id));
} else if (is_dropped) {
ret = OB_TENANT_HAS_BEEN_DROPPED;
LOG_WARN("tenant has been dropped", KR(ret), K(tenant_id));
} else if (!schema_service_->is_tenant_refreshed(tenant_id)) {
// use this err to trigger DDL retry and release current thread.
ret = OB_ERR_PARALLEL_DDL_CONFLICT;
LOG_WARN("tenant' schema not refreshed yet, need retry", KR(ret), K(tenant_id));
}
return ret;
}
int ObRootService::parallel_create_table(const ObCreateTableArg &arg, ObCreateTableRes &res)
{
LOG_TRACE("receive create table arg", K(arg));
int64_t begin_time = ObTimeUtility::current_time();
const uint64_t tenant_id = arg.exec_tenant_id_;
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else if (OB_UNLIKELY(!arg.is_valid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arg", KR(ret), K(arg));
} else if (OB_FAIL(parallel_ddl_pre_check_(tenant_id))) {
LOG_WARN("pre check failed before parallel ddl execute", KR(ret), K(tenant_id));
} else if (arg.schema_.is_view_table()) {
ObCreateViewHelper create_view_helper(schema_service_, tenant_id, arg, res);
if (OB_FAIL(create_view_helper.init(ddl_service_))) {
LOG_WARN("fail to init create view helper", KR(ret), K(tenant_id));
} else if (OB_FAIL(create_view_helper.execute())) {
LOG_WARN("fail to execute create view", KR(ret), K(tenant_id));
}
} else {
ObCreateTableHelper create_table_helper(schema_service_, tenant_id, arg, res);
if (OB_FAIL(create_table_helper.init(ddl_service_))) {
LOG_WARN("fail to init create table helper", KR(ret), K(tenant_id));
} else if (OB_FAIL(create_table_helper.execute())) {
LOG_WARN("fail to execute create table", KR(ret), K(tenant_id));
}
}
int64_t cost = ObTimeUtility::current_time() - begin_time;
LOG_TRACE("finish create table", KR(ret), K(arg), K(cost));
ROOTSERVICE_EVENT_ADD("ddl", "parallel_create_table",
K(ret), K(tenant_id),
"table_id", res.table_id_, K(cost));
return ret;
}
int ObRootService::create_table(const ObCreateTableArg &arg, ObCreateTableRes &res)
{
LOG_DEBUG("receive create table arg", K(arg));
@ -3197,6 +3263,7 @@ int ObRootService::create_table(const ObCreateTableArg &arg, ObCreateTableRes &r
} else if (OB_FAIL(index_builder.generate_schema(index_arg,
table_schema,
global_index_without_column_info,
true, /*generate_id*/
index_schema))) {
LOG_WARN("generate_schema for index failed", K(index_arg), K(table_schema), K(ret));
}
@ -4093,6 +4160,10 @@ int ObRootService::alter_table(const obrpc::ObAlterTableArg &arg, obrpc::ObAlter
ddl_type = ObDDLType::DDL_TRUNCATE_PARTITION;
} else if (obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == nonconst_arg.alter_part_type_) {
ddl_type = ObDDLType::DDL_TRUNCATE_SUB_PARTITION;
} else if (obrpc::ObAlterTableArg::RENAME_PARTITION == nonconst_arg.alter_part_type_) {
ddl_type = ObDDLType::DDL_RENAME_PARTITION;
} else if (obrpc::ObAlterTableArg::RENAME_SUB_PARTITION == nonconst_arg.alter_part_type_) {
ddl_type = ObDDLType::DDL_RENAME_SUB_PARTITION;
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected ddl type", K(ret), K(nonconst_arg.alter_part_type_), K(nonconst_arg));

View File

@ -477,6 +477,7 @@ public:
int create_database(const obrpc::ObCreateDatabaseArg &arg, obrpc::UInt64 &db_id);
int create_tablegroup(const obrpc::ObCreateTablegroupArg &arg, obrpc::UInt64 &tg_id);
int handle_security_audit(const obrpc::ObSecurityAuditArg &arg);
int parallel_create_table(const obrpc::ObCreateTableArg &arg, obrpc::ObCreateTableRes &res);
int create_table(const obrpc::ObCreateTableArg &arg, obrpc::ObCreateTableRes &res);
int alter_database(const obrpc::ObAlterDatabaseArg &arg);
int alter_table(const obrpc::ObAlterTableArg &arg, obrpc::ObAlterTableRes &res);
@ -856,7 +857,7 @@ private:
int handle_cancel_backup_backup(const obrpc::ObBackupManageArg &arg);
int handle_cancel_all_backup_force(const obrpc::ObBackupManageArg &arg);
int clean_global_context();
private:
bool is_sys_tenant(const common::ObString &tenant_name);
int table_allow_ddl_operation(const obrpc::ObAlterTableArg &arg);
int get_table_schema(uint64_t tenant_id,
@ -874,11 +875,13 @@ private:
void update_cpu_quota_concurrency_in_memory_();
int set_cpu_quota_concurrency_config_();
int try_notify_switch_leader(const obrpc::ObNotifySwitchLeaderArg::SwitchLeaderComment &comment);
private:
int precheck_interval_part(const obrpc::ObAlterTableArg &arg);
int old_add_server(const obrpc::ObAdminServerArg &arg);
int old_delete_server(const obrpc::ObAdminServerArg &arg);
int old_cancel_delete_server(const obrpc::ObAdminServerArg &arg);
int parallel_ddl_pre_check_(const uint64_t tenant_id);
private:
static const int64_t OB_MAX_CLUSTER_REPLICA_COUNT = 10000000;
static const int64_t OB_ROOT_SERVICE_START_FAIL_COUNT_UPPER_LIMIT = 5;

View File

@ -27,6 +27,11 @@ namespace oceanbase
{
namespace rootserver
{
bool is_parallel_ddl(const obrpc::ObRpcPacketCode pcode)
{
return obrpc::OB_TRUNCATE_TABLE_V2 == pcode
|| obrpc::OB_PARALLEL_CREATE_TABLE == pcode;
}
// precondition: enable_ddl = false
bool is_allow_when_disable_ddl(const obrpc::ObRpcPacketCode pcode, const obrpc::ObDDLArg *ddl_arg)
@ -129,10 +134,16 @@ protected:
RS_LOG(WARN, "ddl operation not allow in standby", KR(ret), KPC(ddl_arg_));
} else {
auto *tsi_value = GET_TSI(share::schema::TSIDDLVar);
// used for parallel ddl
auto *tsi_generator = GET_TSI(share::schema::TSISchemaVersionGenerator);
if (OB_ISNULL(tsi_value)) {
ret = OB_ERR_UNEXPECTED;
RS_LOG(WARN, "Failed to get TSIDDLVar", K(ret), K(pcode));
} else if (OB_ISNULL(tsi_generator)) {
ret = OB_ERR_UNEXPECTED;
RS_LOG(WARN, "Failed to get TSISchemaVersionGenerator", KR(ret), K(pcode));
} else {
tsi_generator->reset();
tsi_value->exec_tenant_id_ = ddl_arg_->exec_tenant_id_;
tsi_value->ddl_id_str_ = NULL;
const common::ObString &ddl_id_str = ddl_arg_->ddl_id_str_;
@ -183,7 +194,7 @@ protected:
int64_t start_ts = ObTimeUtility::current_time();
bool with_ddl_lock = false;
if (is_ddl_like_) {
if (obrpc::OB_TRUNCATE_TABLE_V2 == pcode) {
if (is_parallel_ddl(pcode)) {
if (OB_FAIL(root_service_.get_ddl_service().ddl_rlock())) {
RS_LOG(WARN, "root service ddl lock fail", K(ret), K(ddl_arg_));
}
@ -329,6 +340,7 @@ DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_CREATE_TABLEGROUP, ObRpcCreateTablegroupP,
DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_DROP_TABLEGROUP, ObRpcDropTablegroupP, drop_tablegroup(arg_));
DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_ALTER_TABLEGROUP, ObRpcAlterTablegroupP, alter_tablegroup(arg_));
DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_CREATE_TABLE, ObRpcCreateTableP, create_table(arg_, result_));
DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_PARALLEL_CREATE_TABLE, ObRpcParallelCreateTableP, parallel_create_table(arg_, result_));
DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_ALTER_TABLE, ObRpcAlterTableP, alter_table(arg_, result_));
DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_DROP_TABLE, ObRpcDropTableP, drop_table(arg_, result_));
DEFINE_DDL_RS_RPC_PROCESSOR(obrpc::OB_RENAME_TABLE, ObRpcRenameTableP, rename_table(arg_));

View File

@ -32,10 +32,9 @@ public:
ObTableCreator(
const uint64_t tenant_id,
const share::SCN &frozen_scn,
share::ObLSTableOperator &lst_operator,
ObMySQLTransaction &trans)
: tenant_id_(tenant_id),
tablet_creator_(tenant_id, frozen_scn, lst_operator, trans),
tablet_creator_(tenant_id, frozen_scn, trans),
trans_(trans),
ls_id_array_(),
inited_(false) {}

View File

@ -19,7 +19,6 @@
#include "common/ob_tablet_id.h"//ObTabletID
#include "share/ob_rpc_struct.h"//ObBatchCreateTabletArg
#include "share/ob_ls_id.h"//share::ObLSID
#include "share/ls/ob_ls_table_operator.h"
namespace oceanbase
{
@ -27,10 +26,6 @@ namespace rpc
{
class ObBatchCreateTabletArg;
}
namespace share
{
class ObLSTableOperator;
}
namespace rootserver
{
struct ObTabletCreatorArg
@ -102,11 +97,9 @@ const static int64_t BATCH_ARG_SIZE = 1024 * 1024; // 1M
ObTabletCreator(
const uint64_t tenant_id,
const share::SCN &major_frozen_scn,
share::ObLSTableOperator &lst_operator,
ObMySQLTransaction &trans)
: tenant_id_(tenant_id),
major_frozen_scn_(major_frozen_scn),
lst_operator_(&lst_operator),
allocator_("TbtCret"),
args_map_(),
trans_(trans),
@ -125,7 +118,6 @@ private:
private:
const uint64_t tenant_id_;
const share::SCN major_frozen_scn_;
share::ObLSTableOperator *lst_operator_;
ObArenaAllocator allocator_;
common::hash::ObHashMap<share::ObLSID, ObBatchCreateTabletHelper*> args_map_;
ObMySQLTransaction &trans_;

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,129 @@
/**
* 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_CREATE_TABLE_HELPER_H_
#define OCEANBASE_ROOTSERVER_OB_CREATE_TABLE_HELPER_H_
#include "rootserver/parallel_ddl/ob_ddl_helper.h"
#include "lib/hash/ob_hashmap.h"
namespace oceanbase
{
namespace share
{
namespace schema
{
class ObMultiVersionSchemaService;
class ObMockFKParentTableSchema;
}
}
namespace obrpc
{
class ObCreateTableArg;
class ObCreateTableRes;
}
namespace rootserver
{
class ObCreateTableHelper : public ObDDLHelper
{
public:
class MockFkParentTableNameWrapper {
public:
MockFkParentTableNameWrapper()
: parent_database_(),
parent_table_() {}
MockFkParentTableNameWrapper(
const common::ObString &parent_database,
const common::ObString &parent_table)
: parent_database_(parent_database),
parent_table_(parent_table) {}
~MockFkParentTableNameWrapper() {}
uint64_t hash() const;
int hash(uint64_t &hash_val) const;
bool operator==(const MockFkParentTableNameWrapper &rv) const;
TO_STRING_KV(K_(parent_database), K_(parent_table));
private:
common::ObString parent_database_;
common::ObString parent_table_;
};
public:
ObCreateTableHelper(
share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id,
const obrpc::ObCreateTableArg &arg,
obrpc::ObCreateTableRes &res);
virtual ~ObCreateTableHelper();
virtual int execute() override;
private:
int init_();
int lock_objects_();
int generate_schemas_();
int calc_schema_version_cnt_();
int create_schemas_();
int create_tablets_();
int add_index_name_to_cache_();
int lock_database_by_obj_name_();
int lock_objects_by_name_();
int lock_objects_by_id_();
int post_lock_objects_by_id_();
int check_ddl_conflict_();
int prefetch_schemas_();
int check_and_set_database_id_();
int check_table_name_();
int set_tablegroup_id_();
int check_and_set_parent_table_id_();
int generate_table_schema_();
int generate_aux_table_schemas_();
int generate_foreign_keys_();
int generate_sequence_object_();
int generate_audit_schema_();
int get_mock_fk_parent_table_info_(
const obrpc::ObCreateForeignKeyArg &foreign_key_arg,
share::schema::ObForeignKeyInfo &foreign_key_info,
share::schema::ObMockFKParentTableSchema *&new_mock_fk_parent_table_schema);
int try_replace_mock_fk_parent_table_(
share::schema::ObMockFKParentTableSchema *&new_mock_fk_parent_table);
int check_fk_columns_type_for_replacing_mock_fk_parent_table_(
const share::schema::ObTableSchema &parent_table_schema,
const share::schema::ObMockFKParentTableSchema &mock_parent_table_schema);
int create_sequence_();
int create_tables_();
int create_audits_();
int deal_with_mock_fk_parent_tables_();
private:
const obrpc::ObCreateTableArg &arg_;
obrpc::ObCreateTableRes &res_;
// replace_mock_fk_parent_table_id_ is valid if table name is same with existed mock fk parent table
uint64_t replace_mock_fk_parent_table_id_;
// new table schema for data/index/lob tables
common::ObArray<ObTableSchema> new_tables_;
common::ObArray<ObMockFKParentTableSchema *> new_mock_fk_parent_tables_;
common::hash::ObHashMap<MockFkParentTableNameWrapper, share::schema::ObMockFKParentTableSchema*> new_mock_fk_parent_table_map_;
common::ObArray<ObSAuditSchema *> new_audits_;
common::ObArray<ObSequenceSchema *> new_sequences_;
bool has_index_;
private:
DISALLOW_COPY_AND_ASSIGN(ObCreateTableHelper);
};
} // end namespace rootserver
} // end namespace oceanbase
#endif//OCEANBASE_ROOTSERVER_OB_CREATE_TABLE_HELPER_H_

View File

@ -0,0 +1,94 @@
/**
* 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 "rootserver/parallel_ddl/ob_create_view_helper.h"
#include "share/schema/ob_multi_version_schema_service.h"
#include "share/ob_rpc_struct.h"
using namespace oceanbase::lib;
using namespace oceanbase::common;
using namespace oceanbase::share;
using namespace oceanbase::share::schema;
using namespace oceanbase::rootserver;
ObCreateViewHelper::ObCreateViewHelper(
share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id,
const obrpc::ObCreateTableArg &arg,
obrpc::ObCreateTableRes &res)
: ObDDLHelper(schema_service, tenant_id),
arg_(arg),
res_(res)
{}
ObCreateViewHelper::~ObCreateViewHelper()
{
}
int ObCreateViewHelper::execute()
{
RS_TRACE(create_view_begin);
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(start_ddl_trans_())) {
LOG_WARN("fail to start ddl trans", KR(ret));
} else if (OB_FAIL(lock_objects_())) {
LOG_WARN("fail to lock objects", KR(ret));
} else if (OB_FAIL(generate_schemas_())) {
LOG_WARN("fail to generate schemas", KR(ret));
} else if (OB_FAIL(gen_task_id_and_schema_versions_())) {
LOG_WARN("fail to gen task id and schema versions", KR(ret));
} else if (OB_FAIL(create_schemas_())) {
LOG_WARN("fail create schemas", KR(ret));
} else if (OB_FAIL(serialize_inc_schema_dict_())) {
LOG_WARN("fail to serialize inc schema dict", KR(ret));
} else if (OB_FAIL(wait_ddl_trans_())) {
LOG_WARN("fail to wait ddl trans", KR(ret));
}
if (OB_FAIL(end_ddl_trans_(ret))) { // won't overwrite ret
LOG_WARN("fail to end ddl trans", KR(ret));
}
RS_TRACE(create_view_end);
FORCE_PRINT_TRACE(THE_RS_TRACE, "[parallel create view]");
return ret;
}
//TODO:(yanmu.ztl) to implement
int ObCreateViewHelper::lock_objects_()
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
}
return ret;
}
//TODO:(yanmu.ztl) to implement
int ObCreateViewHelper::generate_schemas_()
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
}
return ret;
}
//TODO:(yanmu.ztl) to implement
int ObCreateViewHelper::create_schemas_()
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
}
return ret;
}

View File

@ -0,0 +1,59 @@
/**
* 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_CREATE_VIEW_HELPER_H_
#define OCEANBASE_ROOTSERVER_OB_CREATE_VIEW_HELPER_H_
#include "rootserver/parallel_ddl/ob_ddl_helper.h"
namespace oceanbase
{
namespace share
{
namespace schema
{
class ObMultiVersionSchemaService;
}
}
namespace obrpc
{
class ObCreateTableArg;
class ObCreateTableRes;
}
namespace rootserver
{
class ObCreateViewHelper : public ObDDLHelper
{
public:
ObCreateViewHelper(
share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id,
const obrpc::ObCreateTableArg &arg,
obrpc::ObCreateTableRes &res);
virtual ~ObCreateViewHelper();
virtual int execute() override;
private:
int lock_objects_();
int generate_schemas_();
int create_schemas_();
private:
const obrpc::ObCreateTableArg &arg_;
obrpc::ObCreateTableRes &res_;
private:
DISALLOW_COPY_AND_ASSIGN(ObCreateViewHelper);
};
} // end namespace rootserver
} // end namespace oceanbase
#endif//OCEANBASE_ROOTSERVER_OB_CREATE_VIEW_HELPER_H_

View File

@ -0,0 +1,633 @@
/**
* 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 "observer/ob_inner_sql_connection.h" //ObInnerSQLConnection
#include "rootserver/parallel_ddl/ob_ddl_helper.h"
#include "share/schema/ob_multi_version_schema_service.h"
#include "share/schema/ob_ddl_sql_service.h"
#include "share/ob_max_id_fetcher.h"
#include "storage/tablelock/ob_table_lock_rpc_struct.h" //ObLockObjRequest
#include "storage/tablelock/ob_lock_inner_connection_util.h" //ObInnerConnectionLockUtil
using namespace oceanbase::lib;
using namespace oceanbase::common;
using namespace oceanbase::share;
using namespace oceanbase::share::schema;
using namespace oceanbase::rootserver;
ObDDLHelper::ObLockObjPair::ObLockObjPair()
: obj_id_(0),
lock_mode_(transaction::tablelock::MAX_LOCK_MODE)
{
}
ObDDLHelper::ObLockObjPair::ObLockObjPair(
const uint64_t obj_id,
transaction::tablelock::ObTableLockMode lock_mode)
: obj_id_(obj_id),
lock_mode_(lock_mode)
{
}
int ObDDLHelper::ObLockObjPair::init(
const uint64_t obj_id,
transaction::tablelock::ObTableLockMode lock_mode)
{
int ret = OB_SUCCESS;
reset();
obj_id_ = obj_id;
lock_mode_ = lock_mode;
return ret;
}
void ObDDLHelper::ObLockObjPair::reset()
{
obj_id_ = 0;
lock_mode_ = transaction::tablelock::MAX_LOCK_MODE;
}
bool ObDDLHelper::ObLockObjPair::less_than(
const ObLockObjPair &left,
const ObLockObjPair &right)
{
bool bret = false;
if (left.get_obj_id() != right.get_obj_id()) {
bret = (left.get_obj_id() < right.get_obj_id());
} else {
bret = (left.get_lock_mode() < right.get_lock_mode());
}
return bret;
}
ObDDLHelper::ObDDLHelper(
share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id)
: inited_(false),
schema_service_(schema_service),
ddl_service_(NULL),
sql_proxy_(NULL),
ddl_trans_controller_(NULL),
tenant_id_(tenant_id),
task_id_(common::OB_INVALID_ID),
schema_version_cnt_(0),
object_id_cnt_(0),
trans_(schema_service_,
false, /*need_end_signal*/
false, /*enable_query_stash*/
true /*enable_ddl_parallel*/),
lock_database_name_map_(),
lock_object_name_map_(),
lock_object_id_map_(),
latest_schema_guard_(schema_service, tenant_id)
{}
ObDDLHelper::~ObDDLHelper()
{
}
int ObDDLHelper::init(rootserver::ObDDLService &ddl_service)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(inited_)) {
ret = OB_INIT_TWICE;
LOG_WARN("ddl_helper already inited", KR(ret));
} else if (OB_ISNULL(schema_service_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("schema_service is null", KR(ret));
} else if (OB_FAIL(lock_database_name_map_.create(
OBJECT_BUCKET_NUM, "LockDBNameMap", "LockDBNameMap"))) {
LOG_WARN("fail to create lock database name map", KR(ret));
} else if (OB_FAIL(lock_object_name_map_.create(
OBJECT_BUCKET_NUM, "LockObjNameMap", "LockObjNameMap"))) {
LOG_WARN("fail to create lock object name map", KR(ret));
} else if (OB_FAIL(lock_object_id_map_.create(
OBJECT_BUCKET_NUM, "LockObjIDMap", "LockObjIDMap"))) {
LOG_WARN("fail to create lock object id map", KR(ret));
} else {
ddl_service_ = &ddl_service;
sql_proxy_ = &(ddl_service.get_sql_proxy());
ddl_trans_controller_ = &(schema_service_->get_ddl_trans_controller());
task_id_ = OB_INVALID_ID;
schema_version_cnt_ = 0;
object_id_cnt_ = 0;
inited_ = true;
}
return ret;
}
int ObDDLHelper::check_inner_stat_()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("ddl_helper not init yet", KR(ret));
} else if (OB_ISNULL(ddl_service_)
|| OB_ISNULL(sql_proxy_)
|| OB_ISNULL(schema_service_)
|| OB_ISNULL(ddl_trans_controller_)) {
ret = OB_NOT_INIT;
LOG_WARN("ptr is null", KR(ret), KP_(ddl_service), KP_(schema_service),
KP_(sql_proxy), K_(ddl_trans_controller));
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id_)) {
ret = OB_NOT_INIT;
LOG_WARN("invalid tenant_id", KR(ret), K_(tenant_id));
}
return ret;
}
int ObDDLHelper::start_ddl_trans_()
{
int ret = OB_SUCCESS;
bool with_snapshot = false;
int64_t fake_schema_version = 1000;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(trans_.start(sql_proxy_, tenant_id_, fake_schema_version, with_snapshot))) {
LOG_WARN("fail to start trans", KR(ret), K_(tenant_id), K(fake_schema_version), K(with_snapshot));
}
RS_TRACE(start_ddl_trans);
return ret;
}
int ObDDLHelper::gen_task_id_and_schema_versions_()
{
int ret = OB_SUCCESS;
// just for interface compatibility, schema version can be fetched from TSISchemaVersionGenerator
ObArray<int64_t> schema_versions;
int64_t version_cnt = OB_INVALID_INDEX;
auto *tsi_generator = GET_TSI(TSISchemaVersionGenerator);
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(ddl_trans_controller_->create_task_and_assign_schema_version(
tenant_id_, schema_version_cnt_, task_id_, schema_versions))) {
LOG_WARN("fail to gen task id and schema_versions", KR(ret), K_(tenant_id), K_(schema_version_cnt));
} else if (OB_UNLIKELY(OB_INVALID_ID == task_id_
|| schema_version_cnt_ != schema_versions.count())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("task_id or schema version cnt not match", KR(ret), K_(tenant_id), K_(task_id),
K_(schema_version_cnt), "schema_versions_cnt", schema_versions.count());
} else if (OB_ISNULL(tsi_generator)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tsi schema version generator is null", KR(ret));
} else if (OB_FAIL(tsi_generator->get_version_cnt(version_cnt))) {
LOG_WARN("fail to get id cnt", KR(ret));
} else if (OB_UNLIKELY(schema_version_cnt_ != version_cnt)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("schema version cnt not match", KR(ret), K_(tenant_id), K_(task_id),
K_(schema_version_cnt), K(version_cnt));
}
RS_TRACE(gen_task_id_and_versions);
return ret;
}
int ObDDLHelper::serialize_inc_schema_dict_()
{
int ret = OB_SUCCESS;
auto *tsi_generator = GET_TSI(TSISchemaVersionGenerator);
int64_t start_schema_version = OB_INVALID_VERSION;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_ISNULL(tsi_generator)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tsi schema version generator is null", KR(ret));
} else if (OB_FAIL(tsi_generator->get_start_version(start_schema_version))) {
LOG_WARN("fail to get start schema version", KR(ret));
} else if (OB_FAIL(trans_.serialize_inc_schemas(start_schema_version - 1))) {
LOG_WARN("fail to serialize inc schemas", KR(ret), K_(tenant_id),
"start_schema_version", start_schema_version - 1);
}
RS_TRACE(inc_schema_dict);
return ret;
}
int ObDDLHelper::wait_ddl_trans_()
{
int ret = OB_SUCCESS;
ObTimeoutCtx ctx;
const int64_t DEFAULT_TS = 10 * 1000 * 1000L; // 10s
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, DEFAULT_TS))) {
LOG_WARN("fail to set default ts", KR(ret));
} else if (OB_FAIL(ddl_trans_controller_->wait_task_ready(tenant_id_, task_id_, ctx.get_timeout()))) {
LOG_WARN("fail to wait ddl trans", KR(ret), K_(tenant_id), K_(task_id));
}
RS_TRACE(wait_ddl_trans);
return ret;
}
// this function should be always called
int ObDDLHelper::end_ddl_trans_(const int return_ret)
{
int ret = return_ret;
// write 1503 ddl operation
if (OB_SUCC(ret)) {
auto *tsi_generator = GET_TSI(TSISchemaVersionGenerator);
int64_t version_cnt = OB_INVALID_INDEX;
int64_t boundary_schema_version = OB_INVALID_VERSION;
share::schema::ObSchemaService *schema_service_impl = NULL;
if (OB_ISNULL(tsi_generator)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tsi schema version generator is null", KR(ret));
} else if (OB_FAIL(tsi_generator->get_version_cnt(version_cnt))) {
LOG_WARN("fail to get version cnt", KR(ret), K(version_cnt));
} else if (0 == version_cnt) {
// no schema change, just skip
} else if (OB_UNLIKELY(version_cnt < 2)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("not enough version cnt for boudary ddl operation", KR(ret), K(version_cnt));
} else if (OB_ISNULL(schema_service_)
|| OB_ISNULL(schema_service_impl = schema_service_->get_schema_service())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ptr is null", KR(ret), KP_(schema_service));
} else if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id_, boundary_schema_version))) {
LOG_WARN("fail to gen new schema version", KR(ret), K_(tenant_id));
} else {
share::schema::ObDDLSqlService ddl_sql_service(*schema_service_impl);
obrpc::ObDDLNopOpreatorArg arg;
arg.schema_operation_.op_type_ = OB_DDL_END_SIGN;
arg.schema_operation_.tenant_id_ = tenant_id_;
if (OB_FAIL(ddl_sql_service.log_nop_operation(arg.schema_operation_,
boundary_schema_version,
NULL,
trans_))) {
LOG_WARN("fail to log ddl operation", KR(ret), K(arg));
}
}
}
if (trans_.is_started()) {
int tmp_ret = OB_SUCCESS;
bool is_commit = OB_SUCC(ret);
if (OB_TMP_FAIL(trans_.end(is_commit))) {
LOG_WARN("trans end failed", KR(ret), KR(tmp_ret), K(is_commit));
ret = is_commit ? tmp_ret : ret;
}
}
if (OB_NOT_NULL(ddl_trans_controller_) && OB_INVALID_ID != task_id_) {
ddl_trans_controller_->remove_task(tenant_id_, task_id_);
}
RS_TRACE(end_ddl_trans);
return ret;
}
int ObDDLHelper::execute()
{
return OB_NOT_IMPLEMENT;
/*
* Implement of parallel ddl should has following actions:
*
* ----------------------------------------------
* 1. start ddl trans:
* - to be exclusive with non-parallel ddl.
* - to be concurrent with other parallel ddl.
*
* if (OB_FAIL(start_ddl_trans_())) {
* LOG_WARN("fail to start ddl trans", KR(ret));
* }
*
* ----------------------------------------------
* 2. lock object by name/object_id
* - to be exclusive with other parallel ddl which involving the same objects.
* - lock object in trans
* Attension:
* 1) lock objects just for mutual exclusion, should check if related objects changed after acquire locks.
* 2) For same object, lock object by name first. After that, lock object by id if it's neccessary.
*
* ----------------------------------------------
* 3. fetch & generate schema:
* - fetch the latest schemas from inner table.
* - generate schema with arg and the latests schemas.
*
* ----------------------------------------------
* 4. register task id & generate schema versions:
* - generate an appropriate number of schema versions for this DDL and register task id.
* - concurrent DDL trans will be committed in descending order of version later.
*
* if (FAILEDx(gen_task_id_and_schema_versions_())) {
* LOG_WARN("fail to gen task id and schema versions", KR(ret));
* }
*
* ----------------------------------------------
* 5. create schema:
* - persist schema in inner table.
*
* ----------------------------------------------
* 6. [optional] serialize increment data dictionary:
* - if table/database/tenant schema changed, records changed schemas in log and commits with DDL trans.
*
* if (FAILEDx(serialize_inc_schema_dict_())) {
* LOG_WARN("fail to serialize inc schema dict", KR(ret));
* }
*
* ----------------------------------------------
* 7. wait concurrent ddl trans ended:
* - wait concurrent DDL trans with smallest schema version ended.
*
* if (FAILEDx(wait_ddl_trans_())) {
* LOG_WARN(fail to wait ddl trans, KR(ret));
* }
*
* ----------------------------------------------
* 8. end ddl trans:
* - abort/commit ddl trans.
*
* if (OB_FAIL(end_ddl_trans_(ret))) { // won't overwrite ret
* LOG_WARN("fail to end ddl trans", KR(ret));
* }
*/
}
int ObDDLHelper::add_lock_object_to_map_(
const uint64_t lock_obj_id,
const transaction::tablelock::ObTableLockMode lock_mode,
ObjectLockMap &lock_map)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(transaction::tablelock::SHARE != lock_mode
&& transaction::tablelock::EXCLUSIVE != lock_mode)) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("not support lock mode to lock object by name", KR(ret), K(lock_mode));
} else {
bool need_update = false;
transaction::tablelock::ObTableLockMode existed_lock_mode = transaction::tablelock::MAX_LOCK_MODE;
if (OB_FAIL(lock_map.get_refactored(lock_obj_id, existed_lock_mode))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
need_update = true;
} else {
LOG_WARN("fail to get lock object from map", KR(ret), K(lock_obj_id));
}
} else if (transaction::tablelock::SHARE == existed_lock_mode
&& transaction::tablelock::EXCLUSIVE == lock_mode) {
// upgrade lock
need_update = true;
}
if (OB_SUCC(ret) && need_update) {
int overwrite = 1;
if (OB_FAIL(lock_map.set_refactored(lock_obj_id, lock_mode, overwrite))) {
LOG_WARN("fail to set lock object to map", KR(ret), K(lock_obj_id), K(lock_mode));
}
}
}
return ret;
}
int ObDDLHelper::lock_objects_in_map_(
const transaction::tablelock::ObLockOBJType obj_type,
ObjectLockMap &lock_map)
{
int ret = OB_SUCCESS;
ObArray<ObLockObjPair> lock_pairs;
const int64_t lock_cnt = lock_map.size();
ObTimeoutCtx ctx;
observer::ObInnerSQLConnection *conn = NULL;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(lock_cnt < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("unexpected lock cnt", KR(ret), K(lock_cnt));
} else if (0 == lock_cnt) {
// skip
} else if (OB_FAIL(lock_pairs.reserve(lock_cnt))) {
LOG_WARN("fail to reserve lock pairs", KR(ret), K(lock_cnt));
} else if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, GCONF.rpc_timeout))) {
LOG_WARN("fail to set timeout ctx", KR(ret));
} else if (OB_ISNULL(conn = dynamic_cast<observer::ObInnerSQLConnection *>
(trans_.get_connection()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("trans conn is NULL", KR(ret));
} else {
ObLockObjPair pair;
FOREACH_X(it, lock_map, OB_SUCC(ret)) {
if (OB_FAIL(pair.init(it->first, it->second))) {
LOG_WARN("fail to init lock pair", KR(ret),
"obj_id", it->first, "lock_mode", it->second);
} else if (OB_FAIL(lock_pairs.push_back(pair))) {
LOG_WARN("fail to push back lock pair", KR(ret), K(pair));
}
} // end foreach
if (OB_SUCC(ret)) {
std::sort(lock_pairs.begin(), lock_pairs.end(), ObLockObjPair::less_than);
FOREACH_X(it, lock_pairs, OB_SUCC(ret)) {
const int64_t timeout = ctx.get_timeout();
if (OB_UNLIKELY(timeout <= 0)) {
ret = OB_TIMEOUT;
LOG_WARN("already timeout", KR(ret), K(timeout));
} else {
transaction::tablelock::ObLockObjRequest lock_arg;
lock_arg.obj_type_ = obj_type;
lock_arg.owner_id_ = ObTableLockOwnerID(0);
lock_arg.obj_id_ = it->get_obj_id();
lock_arg.lock_mode_ = it->get_lock_mode();
lock_arg.op_type_ = ObTableLockOpType::IN_TRANS_COMMON_LOCK;
lock_arg.timeout_us_ = timeout;
LOG_INFO("try lock object", KR(ret), K(lock_arg));
if (OB_FAIL(ObInnerConnectionLockUtil::lock_obj(tenant_id_, lock_arg, conn))) {
LOG_WARN("lock obj failed", KR(ret), K_(tenant_id), K(lock_arg));
}
}
} // end foreach
}
}
(void) lock_map.clear();
return ret;
}
int ObDDLHelper::add_lock_object_by_database_name_(
const ObString &database_name,
const transaction::tablelock::ObTableLockMode lock_mode)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(database_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_name is invalid", KR(ret), K(database_name));
} else {
// use OB_ORIGIN_AND_INSENSITIVE and ignore end space to make more conficts for safety.
common::ObCollationType cs_type = ObSchema::get_cs_type_with_cmp_mode(OB_ORIGIN_AND_INSENSITIVE);
bool calc_end_space = false;
uint64_t lock_obj_id = 0;
lock_obj_id = common::ObCharset::hash(
cs_type, database_name.ptr(), database_name.length(),
lock_obj_id, calc_end_space, NULL);
if (OB_FAIL(add_lock_object_to_map_(lock_obj_id, lock_mode, lock_database_name_map_))) {
LOG_WARN("fail to add lock object to map", KR(ret), K(lock_obj_id), K(lock_mode));
}
LOG_INFO("add lock object by database name", KR(ret), K(database_name), K(lock_mode), K(lock_obj_id));
}
return ret;
}
int ObDDLHelper::lock_databases_by_name_()
{
return lock_objects_in_map_(ObLockOBJType::OBJ_TYPE_DATABASE_NAME, lock_database_name_map_);
}
int ObDDLHelper::add_lock_object_by_name_(
const ObString &database_name,
const ObString &object_name,
const share::schema::ObSchemaType schema_type,
const transaction::tablelock::ObTableLockMode lock_mode)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(database_name.empty() || object_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_name/object_name is invalid", KR(ret), K(database_name), K(object_name));
} else {
// 1. use OB_ORIGIN_AND_INSENSITIVE and ignore end space to make more conficts for safety.
// 2. encoded with database name to make less conficts between different databases/users.
common::ObCollationType cs_type = ObSchema::get_cs_type_with_cmp_mode(OB_ORIGIN_AND_INSENSITIVE);
bool calc_end_space = false;
uint64_t lock_obj_id = 0;
lock_obj_id = common::ObCharset::hash(
cs_type, database_name.ptr(), database_name.length(),
lock_obj_id, calc_end_space, NULL);
lock_obj_id = common::ObCharset::hash(
cs_type, object_name.ptr(), object_name.length(),
lock_obj_id, calc_end_space, NULL);
if (OB_FAIL(add_lock_object_to_map_(lock_obj_id, lock_mode, lock_object_name_map_))) {
LOG_WARN("fail to add lock object to map", KR(ret), K(lock_obj_id), K(lock_mode));
}
LOG_INFO("add lock object by name", KR(ret), K(database_name),
K(object_name), K(schema_type), K(lock_mode), K(lock_obj_id));
}
return ret;
}
int ObDDLHelper::lock_existed_objects_by_name_()
{
return lock_objects_in_map_(ObLockOBJType::OBJ_TYPE_OBJECT_NAME, lock_object_name_map_);
}
int ObDDLHelper::add_lock_object_by_id_(
const uint64_t lock_obj_id,
const share::schema::ObSchemaType schema_type,
const transaction::tablelock::ObTableLockMode lock_mode)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == lock_obj_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("object_id is invalid", KR(ret), K(lock_obj_id));
} else if (OB_FAIL(add_lock_object_to_map_(lock_obj_id, lock_mode, lock_object_id_map_))) {
LOG_WARN("fail to add lock object to map", KR(ret), K(lock_obj_id), K(lock_mode));
}
LOG_INFO("add lock object by id", KR(ret), K(lock_obj_id), K(schema_type), K(lock_mode));
return ret;
}
int ObDDLHelper::lock_existed_objects_by_id_()
{
return lock_objects_in_map_(ObLockOBJType::OBJ_TYPE_COMMON_OBJ, lock_object_id_map_);
}
// 1. constraint name and foreign key name are in the same namespace in oracle tenant.
// 2. constraint name and foreign key name are in different namespace in mysql tenant.
int ObDDLHelper::check_constraint_name_exist_(
const share::schema::ObTableSchema &table_schema,
const common::ObString &constraint_name,
const bool is_foreign_key,
bool &exist)
{
int ret = OB_SUCCESS;
bool is_oracle_mode = false;
uint64_t constraint_id = OB_INVALID_ID;
const uint64_t database_id = table_schema.get_database_id();
exist = false;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
LOG_WARN("check if oracle compat mode failed", KR(ret), K_(tenant_id));
} else {
const bool check_fk = (is_oracle_mode || is_foreign_key);
if (OB_SUCC(ret) && check_fk) {
if (OB_FAIL(latest_schema_guard_.get_foreign_key_id(
database_id, constraint_name, constraint_id))) {
LOG_WARN("fail to get foreign key id", KR(ret), K_(tenant_id), K(database_id), K(constraint_name));
} else if (OB_INVALID_ID != constraint_id) {
exist = true;
}
}
const bool check_cst = (is_oracle_mode || !is_foreign_key);
if (OB_SUCC(ret) && !exist && check_cst) {
if (table_schema.is_mysql_tmp_table()) {
// tmp table in mysql mode, do nothing
} else if (OB_FAIL(latest_schema_guard_.get_constraint_id(
database_id, constraint_name, constraint_id))) {
LOG_WARN("fail to get constraint id", KR(ret), K_(tenant_id), K(database_id), K(constraint_name));
} else if (OB_INVALID_ID != constraint_id) {
exist = true;
}
}
}
return ret;
}
int ObDDLHelper::gen_object_ids_(
const int64_t object_cnt,
share::ObIDGenerator &id_generator)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id_
|| object_cnt < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tenant_id or object_cnt", KR(ret), K_(tenant_id), K(object_cnt));
} else if (0 == object_cnt) {
// skip
} else {
ObMaxIdFetcher id_fetcher(*sql_proxy_);
uint64_t max_object_id = OB_INVALID_ID;
uint64_t min_object_id = OB_INVALID_ID;
if (OB_FAIL(id_fetcher.fetch_new_max_id(tenant_id_, OB_MAX_USED_OBJECT_ID_TYPE,
max_object_id, UINT64_MAX/*initial value should exist*/, object_cnt))) {
LOG_WARN("fail to fetch object id", KR(ret), K_(tenant_id), K(object_cnt));
} else if (OB_UNLIKELY(OB_INVALID_ID == max_object_id)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("object_id is invalid", KR(ret), K_(tenant_id), K(object_cnt));
} else if (0 >= (min_object_id = max_object_id - object_cnt + 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("min_object_id should be greator than 0",
KR(ret), K_(tenant_id), K(min_object_id), K(max_object_id), K(object_cnt));
} else if (OB_FAIL(id_generator.init(1 /*step*/, min_object_id, max_object_id))) {
LOG_WARN("fail to init id generator", KR(ret), K_(tenant_id),
K(min_object_id), K(max_object_id), K(object_cnt));
}
}
return ret;
}
int ObDDLHelper::gen_partition_object_and_tablet_ids_(
ObIArray<ObTableSchema> &table_schemas)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(ddl_service_->generate_object_id_for_partition_schemas(table_schemas))) {
LOG_WARN("fail to generate object_ids", KR(ret));
} else if (OB_FAIL(ddl_service_->generate_tables_tablet_id(table_schemas))) {
LOG_WARN("fail to generate tablet_ids", KR(ret));
}
return ret;
}

View File

@ -0,0 +1,150 @@
/**
* 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_HELPER_H_
#define OCEANBASE_ROOTSERVER_OB_DDL_HELPER_H_
#include "lib/hash/ob_hashmap.h"
#include "rootserver/ob_ddl_service.h" // ObDDLTransController
#include "share/schema/ob_latest_schema_guard.h" // ObLatestSchemaGuard
namespace oceanbase
{
namespace common
{
class ObMySQLProxy;
}
namespace share
{
class ObIDGenerator;
namespace schema
{
class ObMultiVersionSchemaService;
class ObDDLTransController;
class ObLatestSchemaGuard;
}
}
namespace rootserver
{
class ObDDLSQLTransaction;
class ObDDLService;
class ObDDLHelper
{
public:
class ObLockObjPair{
public:
ObLockObjPair();
ObLockObjPair(const uint64_t obj_id,
transaction::tablelock::ObTableLockMode lock_mode);
~ObLockObjPair() {}
int init(
const uint64_t obj_id,
transaction::tablelock::ObTableLockMode lock_mode);
void reset();
static bool less_than(const ObLockObjPair &left, const ObLockObjPair &right);
uint64_t get_obj_id() const { return obj_id_; }
transaction::tablelock::ObTableLockMode get_lock_mode() const { return lock_mode_; }
TO_STRING_KV(K_(obj_id), K_(lock_mode));
private:
uint64_t obj_id_;
transaction::tablelock::ObTableLockMode lock_mode_;
};
typedef common::hash::ObHashMap<uint64_t, transaction::tablelock::ObTableLockMode> ObjectLockMap;
public:
ObDDLHelper(
share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id);
virtual ~ObDDLHelper();
int init(rootserver::ObDDLService &ddl_service);
virtual int execute();
protected:
virtual int check_inner_stat_();
/* main actions */
int start_ddl_trans_();
int gen_task_id_and_schema_versions_();
int serialize_inc_schema_dict_();
int wait_ddl_trans_();
int end_ddl_trans_(const int return_ret);
/*--------------*/
protected:
// lock database name
int add_lock_object_by_database_name_(
const ObString &database_name,
const transaction::tablelock::ObTableLockMode lock_mode);
int lock_databases_by_name_();
// lock object name
int add_lock_object_by_name_(
const ObString &database_name,
const ObString &object_name,
const share::schema::ObSchemaType schema_type,
const transaction::tablelock::ObTableLockMode lock_mode);
int lock_existed_objects_by_name_();
// lock object id
int add_lock_object_by_id_(
const uint64_t lock_obj_id,
const share::schema::ObSchemaType schema_type,
const transaction::tablelock::ObTableLockMode lock_mode);
int lock_existed_objects_by_id_();
int gen_object_ids_(
const int64_t object_cnt,
share::ObIDGenerator &id_generator);
int gen_partition_object_and_tablet_ids_(
ObIArray<ObTableSchema> &table_schemas);
int check_constraint_name_exist_(
const share::schema::ObTableSchema &table_schema,
const common::ObString &constraint_name,
const bool is_foreign_key,
bool &exist);
private:
int add_lock_object_to_map_(
const uint64_t lock_obj_id,
const transaction::tablelock::ObTableLockMode lock_mode,
ObjectLockMap &lock_map);
int lock_objects_in_map_(
const transaction::tablelock::ObLockOBJType obj_type,
ObjectLockMap &lock_map);
protected:
bool inited_;
share::schema::ObMultiVersionSchemaService *schema_service_;
rootserver::ObDDLService *ddl_service_;
common::ObMySQLProxy *sql_proxy_;
share::schema::ObDDLTransController *ddl_trans_controller_;
uint64_t tenant_id_; // normally, ObDDLHelper only deal with ddl in one tenant
int64_t task_id_; // allocated by ObDDLTransController
int64_t schema_version_cnt_; // used to allocate schema versions for this DDL
int64_t object_id_cnt_; // used to allocate object ids for this DDL
rootserver::ObDDLSQLTransaction trans_;
// used to lock databases by name
ObjectLockMap lock_database_name_map_;
// used to lock objects by name
ObjectLockMap lock_object_name_map_;
// used to lock objects by id
ObjectLockMap lock_object_id_map_;
// should use this guard after related objects are locked
share::schema::ObLatestSchemaGuard latest_schema_guard_;
common::ObArenaAllocator allocator_;
private:
static const int64_t OBJECT_BUCKET_NUM = 1024;
DISALLOW_COPY_AND_ASSIGN(ObDDLHelper);
};
} // end namespace rootserver
} // end namespace oceanbase
#endif//OCEANBASE_ROOTSERVER_OB_DDL_HELPER_H_

View File

@ -0,0 +1,486 @@
/**
* 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 SHARE_SCHEMA
#include "rootserver/parallel_ddl/ob_index_name_checker.h"
#include "share/schema/ob_schema_service_sql_impl.h"
#include "share/schema/ob_multi_version_schema_service.h"
using namespace oceanbase::lib;
using namespace oceanbase::common;
using namespace oceanbase::share;
using namespace oceanbase::share::schema;
using namespace oceanbase::rootserver;
ObIndexNameCache::ObIndexNameCache(
const uint64_t tenant_id,
common::ObMySQLProxy &sql_proxy)
: mutex_(common::ObLatchIds::IND_NAME_CACHE_LOCK),
tenant_id_(tenant_id),
sql_proxy_(sql_proxy),
allocator_(ObMemAttr(OB_SYS_TENANT_ID, "IndNameInfo", ObCtxIds::SCHEMA_SERVICE)),
cache_(ModulePageAllocator(allocator_)),
loaded_(false)
{
}
void ObIndexNameCache::reset_cache()
{
lib::ObMutexGuard guard(mutex_);
(void) inner_reset_cache_();
}
void ObIndexNameCache::inner_reset_cache_()
{
cache_.destroy();
allocator_.reset();
loaded_ = false;
FLOG_INFO("[INDEX NAME CACHE] reset index name map", K_(tenant_id));
}
int ObIndexNameCache::check_index_name_exist(
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &index_name,
bool &is_exist)
{
int ret = OB_SUCCESS;
is_exist = false;
bool is_oracle_mode = false;
if (OB_UNLIKELY(
OB_INVALID_TENANT_ID == tenant_id
|| tenant_id_ != tenant_id
|| OB_INVALID_ID == database_id
|| index_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arg", KR(ret), K(tenant_id_), K(tenant_id), K(database_id), K(index_name));
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
tenant_id, is_oracle_mode))) {
LOG_WARN("fail to check is oracle mode", KR(ret), K(tenant_id));
} else {
lib::ObMutexGuard guard(mutex_);
ObString idx_name;
uint64_t data_table_id = OB_INVALID_ID;
if (OB_FAIL(try_load_cache_())) {
LOG_WARN("fail to load index name cache", KR(ret), K(tenant_id));
} else if (is_recyclebin_database_id(database_id)) {
idx_name = index_name;
data_table_id = OB_INVALID_ID;
} else {
uint64_t data_table_id = ObSimpleTableSchemaV2::extract_data_table_id_from_index_name(index_name);
if (OB_INVALID_ID == database_id) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid index name", KR(ret), K(index_name));
} else if (OB_FAIL(ObSimpleTableSchemaV2::get_index_name(index_name, idx_name))) {
LOG_WARN("fail to get original index name", KR(ret), K(index_name));
} else {
data_table_id = (is_oracle_mode && !is_mysql_sys_database_id(database_id)) ?
OB_INVALID_ID : data_table_id;
}
}
if (OB_SUCC(ret)) {
ObIndexSchemaHashWrapper index_name_wrapper(
tenant_id,
database_id,
data_table_id,
idx_name);
ObIndexNameInfo *index_name_info = NULL;
if (OB_FAIL(cache_.get_refactored(index_name_wrapper, index_name_info))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to get index name info", KR(ret), K(index_name_wrapper));
}
} else if (OB_ISNULL(index_name_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("index name info is null", KR(ret), K(index_name_wrapper));
} else {
is_exist = true;
LOG_INFO("index name exist", KR(ret), KPC(index_name_info),
K(database_id), K(index_name), K(data_table_id), K(idx_name));
// Before call check_index_name_exist(), index_name will be locked by trans first.
// And add_index_name() will be called before trans commit.
//
// It may has garbage when trans commit failed after add_index_name() is called.
// So, we need to double check if index name actually exists in inner table when confict occurs.
ObSchemaService *schema_service_impl = NULL;
uint64_t index_id = OB_INVALID_ID;
if (OB_ISNULL(schema_service_impl = GSCHEMASERVICE.get_schema_service())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema service impl is null", KR(ret));
} else if (OB_FAIL(schema_service_impl->get_index_id(
sql_proxy_, tenant_id, database_id,
index_name_info->get_index_name(), index_id))) {
LOG_WARN("fail to get index id", KR(ret), KPC(index_name_info));
} else if (OB_INVALID_ID != index_id) {
is_exist = true;
} else {
is_exist = false;
FLOG_INFO("garbage index name exist, should be erased", KPC(index_name_info),
K(database_id), K(index_name), K(data_table_id), K(idx_name));
if (OB_FAIL(cache_.erase_refactored(index_name_wrapper))) {
LOG_WARN("fail to erase key", KR(ret), K(index_name_wrapper));
if (OB_HASH_NOT_EXIST != ret) {
(void) inner_reset_cache_();
}
}
}
}
}
}
return ret;
}
int ObIndexNameCache::add_index_name(
const share::schema::ObTableSchema &index_schema)
{
int ret = OB_SUCCESS;
const uint64_t tenant_id = index_schema.get_tenant_id();
const uint64_t database_id = index_schema.get_database_id();
const ObString &index_name = index_schema.get_table_name_str();
const ObTableType table_type = index_schema.get_table_type();
uint64_t data_table_id = index_schema.get_data_table_id();
bool is_oracle_mode = false;
if (OB_UNLIKELY(
OB_INVALID_TENANT_ID == tenant_id
|| tenant_id_ != tenant_id
|| OB_INVALID_ID == database_id
|| index_name.empty()
|| !is_index_table(table_type))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arg", KR(ret), K(tenant_id_),
K(tenant_id), K(database_id), K(index_name), K(table_type));
} else if (OB_UNLIKELY(!is_recyclebin_database_id(database_id)
&& index_schema.get_origin_index_name_str().empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid index schema", KR(ret), K(index_schema));
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
tenant_id, is_oracle_mode))) {
LOG_WARN("fail to check is oracle mode", KR(ret), K(tenant_id));
} else {
lib::ObMutexGuard guard(mutex_);
if (OB_FAIL(try_load_cache_())) {
LOG_WARN("fail to load index name cache", KR(ret), K(tenant_id));
} else {
void *buf = NULL;
ObIndexNameInfo *index_name_info = NULL;
ObString idx_name;
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObIndexNameInfo)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc index name info", KR(ret));
} else if (FALSE_IT(index_name_info = new (buf) ObIndexNameInfo())) {
} else if (OB_FAIL(index_name_info->init(allocator_, index_schema))) {
LOG_WARN("fail to init index name info", KR(ret), K(index_schema));
} else if (is_recyclebin_database_id(database_id)) {
data_table_id = OB_INVALID_ID;
idx_name = index_name_info->get_index_name();
} else {
data_table_id = (is_oracle_mode && !is_mysql_sys_database_id(database_id)) ?
OB_INVALID_ID : index_name_info->get_data_table_id();
idx_name = index_name_info->get_original_index_name();
}
if (OB_SUCC(ret)) {
int overwrite = 0;
ObIndexSchemaHashWrapper index_name_wrapper(index_name_info->get_tenant_id(),
database_id,
data_table_id,
idx_name);
if (OB_FAIL(cache_.set_refactored(index_name_wrapper, index_name_info, overwrite))) {
LOG_WARN("fail to set refactored", KR(ret), KPC(index_name_info));
if (OB_HASH_EXIST == ret) {
ObIndexNameInfo **exist_index_info = cache_.get(index_name_wrapper);
if (OB_NOT_NULL(exist_index_info) && OB_NOT_NULL(*exist_index_info)) {
FLOG_ERROR("[INDEX NAME CACHE] duplicated index info exist",
KR(ret), KPC(index_name_info), KPC(*exist_index_info));
}
} else {
(void) inner_reset_cache_();
}
} else {
FLOG_INFO("[INDEX NAME CACHE] add index name to cache", KR(ret), KPC(index_name_info));
}
}
}
}
return ret;
}
// need protect by mutex_
int ObIndexNameCache::try_load_cache_()
{
int ret = OB_SUCCESS;
if (loaded_) {
// do nothing
} else {
(void) inner_reset_cache_();
ObRefreshSchemaStatus schema_status;
schema_status.tenant_id_ = tenant_id_;
int64_t schema_version = OB_INVALID_VERSION;
int64_t timeout_ts = OB_INVALID_TIMESTAMP;
if (OB_FAIL(GSCHEMASERVICE.get_schema_version_in_inner_table(
sql_proxy_, schema_status, schema_version))) {
LOG_WARN("fail to get schema version", KR(ret), K(schema_status));
} else if (!ObSchemaService::is_formal_version(schema_version)) {
ret = OB_EAGAIN;
LOG_WARN("schema version is informal, need retry", KR(ret), K(schema_status), K(schema_version));
} else if (OB_FAIL(ObShareUtil::get_ctx_timeout(GCONF.internal_sql_execute_timeout, timeout_ts))) {
LOG_WARN("fail to get timeout", KR(ret));
} else {
int64_t original_timeout_ts = THIS_WORKER.get_timeout_ts();
int64_t schema_version = OB_INVALID_VERSION;
THIS_WORKER.set_timeout_ts(timeout_ts);
ObSchemaGetterGuard guard;
int64_t start_time = ObTimeUtility::current_time();
if (OB_FAIL(GSCHEMASERVICE.async_refresh_schema(tenant_id_, schema_version))) {
LOG_WARN("fail to refresh schema", KR(ret), K_(tenant_id), K(schema_version));
} else if (OB_FAIL(GSCHEMASERVICE.get_tenant_schema_guard(tenant_id_, guard))) {
LOG_WARN("fail to get schema guard", KR(ret), K_(tenant_id));
} else if (OB_FAIL(guard.get_schema_version(tenant_id_, schema_version))) {
LOG_WARN("fail to get schema version", KR(ret), K_(tenant_id));
} else if (OB_FAIL(guard.deep_copy_index_name_map(allocator_, cache_))) {
LOG_WARN("fail to deep copy index name map", KR(ret), K_(tenant_id));
} else {
loaded_ = true;
FLOG_INFO("[INDEX NAME CACHE] load index name map", KR(ret), K_(tenant_id),
K(schema_version), "cost", ObTimeUtility::current_time() - start_time);
}
if (OB_FAIL(ret)) {
(void) inner_reset_cache_();
LOG_WARN("load index name map failed", KR(ret), K_(tenant_id),
K(schema_version), "cost", ObTimeUtility::current_time() - start_time);
}
THIS_WORKER.set_timeout_ts(original_timeout_ts);
}
}
return ret;
}
ObIndexNameChecker::ObIndexNameChecker()
: rwlock_(),
allocator_(ObMemAttr(OB_SYS_TENANT_ID, "IndNameCache", ObCtxIds::SCHEMA_SERVICE)),
index_name_cache_map_(),
sql_proxy_(NULL),
inited_(false)
{
}
ObIndexNameChecker::~ObIndexNameChecker()
{
destroy();
}
int ObIndexNameChecker::init(common::ObMySQLProxy &sql_proxy)
{
int ret = OB_SUCCESS;
SpinWLockGuard guard(rwlock_);
if (inited_) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", KR(ret));
} else {
const int64_t BUCKET_NUM = 1024;
if (OB_FAIL(index_name_cache_map_.create(BUCKET_NUM, "IndNameMap", "IndNameMap"))) {
LOG_WARN("fail to create hash map", KR(ret));
} else {
sql_proxy_ = &sql_proxy;
inited_ = true;
}
}
return ret;
}
void ObIndexNameChecker::destroy()
{
SpinWLockGuard guard(rwlock_);
if (inited_) {
FOREACH(it, index_name_cache_map_) {
if (OB_NOT_NULL(it->second)) {
(it->second)->~ObIndexNameCache();
it->second = NULL;
}
}
index_name_cache_map_.destroy();
allocator_.reset();
sql_proxy_ = NULL;
inited_ = false;
}
}
void ObIndexNameChecker::reset_all_cache()
{
int ret = OB_SUCCESS;
SpinRLockGuard guard(rwlock_);
if (inited_) {
FOREACH(it, index_name_cache_map_) {
if (OB_NOT_NULL(it->second)) {
(void) (it->second)->reset_cache();
}
}
}
}
int ObIndexNameChecker::reset_cache(const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
bool can_skip = false;
SpinRLockGuard guard(rwlock_);
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else {
ObIndexNameCache *cache = NULL;
if (OB_FAIL(index_name_cache_map_.get_refactored(tenant_id, cache))) {
if (OB_HASH_NOT_EXIST != ret) {
LOG_WARN("fail to get refactored", KR(ret), K(tenant_id));
} else {
// tenant not in cache, just skip
ret = OB_SUCCESS;
}
} else if (OB_ISNULL(cache)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cache is null", KR(ret), K(tenant_id));
} else {
(void) cache->reset_cache();
}
}
return ret;
}
int ObIndexNameChecker::check_index_name_exist(
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &index_name,
bool &is_exist)
{
int ret = OB_SUCCESS;
bool can_skip = false;
is_exist = false;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else if (OB_FAIL(check_tenant_can_be_skipped_(tenant_id, can_skip))) {
LOG_WARN("fail to check tenant", KR(ret), K(tenant_id));
} else if (can_skip) {
// do nothing
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id
|| OB_INVALID_ID == database_id
|| index_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arg", KR(ret), K(tenant_id), K(database_id), K(index_name));
} else if (OB_FAIL(try_init_index_name_cache_map_(tenant_id))) {
LOG_WARN("fail to init index name cache", KR(ret), K(tenant_id));
} else {
SpinRLockGuard guard(rwlock_);
ObIndexNameCache *cache = NULL;
if (OB_FAIL(index_name_cache_map_.get_refactored(tenant_id, cache))) {
LOG_WARN("fail to get refactored", KR(ret), K(tenant_id));
} else if (OB_ISNULL(cache)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cache is null", KR(ret));
} else if (OB_FAIL(cache->check_index_name_exist(
tenant_id, database_id, index_name, is_exist))) {
LOG_WARN("fail to check index name exist",
KR(ret), K(tenant_id), K(database_id), K(index_name));
}
}
return ret;
}
int ObIndexNameChecker::add_index_name(
const share::schema::ObTableSchema &index_schema)
{
int ret = OB_SUCCESS;
const uint64_t tenant_id = index_schema.get_tenant_id();
bool can_skip = false;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else if (OB_FAIL(check_tenant_can_be_skipped_(tenant_id, can_skip))) {
LOG_WARN("fail to check tenant", KR(ret), K(tenant_id));
} else if (can_skip) {
// do nothing
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arg", KR(ret), K(tenant_id));
} else if (OB_FAIL(try_init_index_name_cache_map_(tenant_id))) {
LOG_WARN("fail to init index name cache", KR(ret), K(tenant_id));
} else {
SpinRLockGuard guard(rwlock_);
ObIndexNameCache *cache = NULL;
if (OB_FAIL(index_name_cache_map_.get_refactored(tenant_id, cache))) {
LOG_WARN("fail to get refactored", KR(ret), K(tenant_id));
} else if (OB_ISNULL(cache)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("cache is null", KR(ret));
} else if (OB_FAIL(cache->add_index_name(index_schema))) {
LOG_WARN("fail to add index name", KR(ret), K(index_schema));
}
}
return ret;
}
// only cache oracle tenant's index name map
int ObIndexNameChecker::check_tenant_can_be_skipped_(
const uint64_t tenant_id,
bool &can_skip)
{
int ret = OB_SUCCESS;
bool is_oracle_mode = false;
can_skip = false;
if (is_sys_tenant(tenant_id)
|| is_meta_tenant(tenant_id)) {
can_skip = true;
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
tenant_id, is_oracle_mode))) {
LOG_WARN("fail to check is oracle mode", KR(ret), K(tenant_id));
} else {
can_skip = !is_oracle_mode;
}
return ret;
}
int ObIndexNameChecker::try_init_index_name_cache_map_(const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
SpinWLockGuard guard(rwlock_);
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", KR(ret));
} else if (OB_ISNULL(sql_proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_proxy is null", KR(ret));
} else {
ObIndexNameCache *cache = NULL;
if (OB_FAIL(index_name_cache_map_.get_refactored(tenant_id, cache))) {
if (OB_HASH_NOT_EXIST != ret) {
LOG_WARN("fail to get cache", KR(ret), K(tenant_id));
} else {
ret = OB_SUCCESS;
cache = NULL;
void *buf = NULL;
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObIndexNameCache)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc memory", KR(ret));
} else if (FALSE_IT(cache = new (buf) ObIndexNameCache(tenant_id, *sql_proxy_))) {
} else if (OB_FAIL(index_name_cache_map_.set_refactored(tenant_id, cache))) {
LOG_WARN("fail to set cache", KR(ret), K(tenant_id));
}
}
} else {
// cache exist, just skip
}
}
return ret;
}

View File

@ -0,0 +1,108 @@
/**
* 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_INDEX_NAME_CHECKER_H_
#define OCEANBASE_ROOTSERVER_OB_INDEX_NAME_CHECKER_H_
#include "lib/hash/ob_pointer_hashmap.h"
#include "share/schema/ob_schema_struct.h"
#include "share/schema/ob_table_schema.h"
namespace oceanbase
{
namespace share
{
namespace schema
{
/*
* ObIndexNameCache will be protected by mutex
*
* Any failure when call related function will reset cache immediately for correctness.
*/
class ObIndexNameCache
{
public:
ObIndexNameCache() = delete;
ObIndexNameCache(const uint64_t tenant_id,
common::ObMySQLProxy &sql_proxy);
~ObIndexNameCache() {}
void reset_cache();
int check_index_name_exist(const uint64_t tenant_id,
const uint64_t database_id,
const ObString &index_name,
bool &is_exist);
int add_index_name(const share::schema::ObTableSchema &index_schema);
private:
void inner_reset_cache_();
int try_load_cache_();
private:
lib::ObMutex mutex_;
uint64_t tenant_id_;
common::ObMySQLProxy &sql_proxy_;
common::ObArenaAllocator allocator_;
ObIndexNameMap cache_;
bool loaded_;
DISALLOW_COPY_AND_ASSIGN(ObIndexNameCache);
};
/*
* design doc: ob/rootservice/ffa2062ce3n1tvi0#SLpFs
*
* usage: for parallel ddl to check if index name is duplicated in oracle mode
*
* notice:
* 1. ObIndexNameChecker should be used in ObDDLSQLTransaction only.
* 2. nothing will be done for mysql tenant.
*/
class ObIndexNameChecker
{
public:
ObIndexNameChecker();
~ObIndexNameChecker();
int init(common::ObMySQLProxy &sql_proxy);
void destroy();
// release memory when RS changed.
void reset_all_cache();
// will be called in the following cases:
// 1. before non parallel ddl commit.
// 2. after non parallel ddl commit failed.
int reset_cache(const uint64_t tenant_id);
// lock object by original index name first before call check_index_name_exist().
int check_index_name_exist(const uint64_t tenant_id,
const uint64_t database_id,
const ObString &index_name,
bool &is_exist);
// call add_index_name() before parallel ddl trans commit.
int add_index_name(const share::schema::ObTableSchema &index_schema);
private:
int check_tenant_can_be_skipped_(const uint64_t tenant_id, bool &can_skip);
int try_init_index_name_cache_map_(const uint64_t tenant_id);
private:
common::SpinRWLock rwlock_;
common::ObArenaAllocator allocator_;
// index_name_cache_map_ won't be erased()
common::hash::ObHashMap<uint64_t, ObIndexNameCache*, common::hash::ReadWriteDefendMode> index_name_cache_map_;
common::ObMySQLProxy *sql_proxy_;
bool inited_;
};
} // end schema
} // end share
} // end oceanbase
#endif//OCEANBASE_ROOTSERVER_OB_INDEX_NAME_CHECKER_H_

View File

@ -423,6 +423,7 @@ ob_set_subtarget(ob_share schema
schema/ob_sequence_sql_service.cpp
schema/ob_server_schema_service.cpp
schema/ob_table_schema.cpp
schema/ob_latest_schema_guard.cpp
)
ob_set_subtarget(ob_share stat

View File

@ -8057,6 +8057,21 @@ int ObInnerTableSchema::all_virtual_schema_memory_schema(ObTableSchema &table_sc
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("allocator_idx", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
table_schema.get_part_option().set_part_num(1);
table_schema.set_part_level(PARTITION_LEVEL_ONE);
@ -8231,6 +8246,21 @@ int ObInnerTableSchema::all_virtual_schema_slot_schema(ObTableSchema &table_sche
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("allocator_idx", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
table_schema.get_part_option().set_part_num(1);
table_schema.set_part_level(PARTITION_LEVEL_ONE);

View File

@ -11612,6 +11612,7 @@ def_table_schema(
('free_schema_mgr_cnt', 'int'),
('mem_used', 'int'),
('mem_total', 'int'),
('allocator_idx', 'int'),
],
partition_columns = ['svr_ip', 'svr_port'],
vtable_route_policy = 'distributed',
@ -11636,6 +11637,7 @@ def_table_schema(
('schema_count', 'int'),
('total_ref_cnt', 'int'),
('ref_info','varchar:OB_MAX_SCHEMA_REF_INFO'),
('allocator_idx', 'int'),
],
partition_columns = ['svr_ip', 'svr_port'],
vtable_route_policy = 'distributed',

View File

@ -64,6 +64,7 @@ public:
RPC_S(PRD create_database, obrpc::OB_CREATE_DATABASE, (ObCreateDatabaseArg), UInt64);
RPC_S(PRD create_tablegroup, obrpc::OB_CREATE_TABLEGROUP, (ObCreateTablegroupArg), UInt64);
RPC_S(PRD create_table, obrpc::OB_CREATE_TABLE, (ObCreateTableArg), ObCreateTableRes);
RPC_S(PRD parallel_create_table, obrpc::OB_PARALLEL_CREATE_TABLE, (ObCreateTableArg), ObCreateTableRes);
RPC_S(PRD alter_table, obrpc::OB_ALTER_TABLE, (ObAlterTableArg), ObAlterTableRes);
RPC_S(PRD create_hidden_table, obrpc::OB_CREATE_HIDDEN_TABLE, (obrpc::ObCreateHiddenTableArg), ObCreateHiddenTableRes);
RPC_S(PRD alter_database, obrpc::OB_ALTER_DATABASE, (ObAlterDatabaseArg));

View File

@ -64,7 +64,8 @@ enum ObDDLType
DDL_DROP_SUB_PARTITION = 505,
DDL_TRUNCATE_PARTITION = 506,
DDL_TRUNCATE_SUB_PARTITION = 507,
DDL_RENAME_PARTITION = 508,
DDL_RENAME_SUB_PARTITION = 509,
///< @note add new double table long running ddl type before this line
DDL_DOUBLE_TABLE_OFFLINE = 1000,
DDL_MODIFY_COLUMN = 1001, // only modify columns

View File

@ -542,6 +542,7 @@ class ObString;
ACT(STOP_RECOVERY_LS_THREAD1,)\
ACT(STOP_TRANSFER_LS_LOGICAL_TABLE_REPLACED,)\
ACT(BEFORE_TRANSFER_DOING,)\
ACT(AFTER_PARALLEL_DDL_LOCK_OBJ_BY_NAME,)\
ACT(BEFORE_BUILD_LS_MIGRATION_DAG_NET,)\
ACT(AFTER_JOIN_LEARNER_LIST,)\
ACT(BEFORE_TRANSFER_START_COMMIT,)\

View File

@ -2775,7 +2775,9 @@ DEF_TO_STRING(ObCreateForeignKeyArg)
K_(is_modify_validate_flag),
K_(rely_flag),
K_(is_modify_rely_flag),
K_(is_modify_fk_state));
K_(is_modify_fk_state),
K_(parent_database_id),
K_(parent_table_id));
J_OBJ_END();
return pos;
}
@ -2798,7 +2800,9 @@ OB_SERIALIZE_MEMBER((ObCreateForeignKeyArg, ObIndexArg),
is_modify_rely_flag_,
is_modify_fk_state_,
need_validate_data_,
is_parent_table_mock_);
is_parent_table_mock_,
parent_database_id_,
parent_table_id_);
bool ObDropForeignKeyArg::is_valid() const
{

View File

@ -1914,6 +1914,8 @@ public:
SET_INTERVAL,
// cnovert interval to range
INTERVAL_TO_RANGE,
RENAME_PARTITION,
RENAME_SUB_PARTITION,
NO_OPERATION = 1000
};
enum AlterConstraintType
@ -2426,7 +2428,9 @@ public:
is_modify_rely_flag_(false),
is_modify_fk_state_(false),
need_validate_data_(true),
is_parent_table_mock_(false)
is_parent_table_mock_(false),
parent_database_id_(common::OB_INVALID_ID),
parent_table_id_(common::OB_INVALID_ID)
{}
virtual ~ObCreateForeignKeyArg()
{}
@ -2452,6 +2456,8 @@ public:
is_modify_fk_state_ = false;
need_validate_data_ = true;
is_parent_table_mock_ = false;
parent_database_id_ = common::OB_INVALID_ID;
parent_table_id_ = common::OB_INVALID_ID;
}
bool is_valid() const;
int assign(const ObCreateForeignKeyArg &other) {
@ -2479,6 +2485,8 @@ public:
is_modify_fk_state_ = other.is_modify_fk_state_;
need_validate_data_ = other.need_validate_data_;
is_parent_table_mock_ = other.is_parent_table_mock_;
parent_database_id_ = other.parent_database_id_;
parent_table_id_ = other.parent_table_id_;
}
return ret;
}
@ -2503,6 +2511,8 @@ public:
bool is_modify_fk_state_;
bool need_validate_data_;
bool is_parent_table_mock_;
uint64_t parent_database_id_; // used in ddl_service to store related object_id
uint64_t parent_table_id_; // used in ddl_service to store related object_id
};
struct ObDropForeignKeyArg : public ObIndexArg

View File

@ -29,6 +29,103 @@ namespace oceanbase
using namespace common;
namespace share
{
void ObIDGenerator::reset()
{
inited_ = false;
step_ = 0;
start_id_ = common::OB_INVALID_ID;
end_id_ = common::OB_INVALID_ID;
current_id_ = common::OB_INVALID_ID;
}
int ObIDGenerator::init(
const uint64_t step,
const uint64_t start_id,
const uint64_t end_id)
{
int ret = OB_SUCCESS;
reset();
if (OB_UNLIKELY(start_id > end_id || 0 == step)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid start_id/end_id", KR(ret), K(start_id), K(end_id), K(step));
} else {
step_ = step;
start_id_ = start_id;
end_id_ = end_id;
current_id_ = start_id - step_;
inited_ = true;
}
return ret;
}
int ObIDGenerator::next(uint64_t &current_id)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("generator is not inited", KR(ret), KPC(this));
} else if (current_id_ >= end_id_) {
ret = OB_ITER_END;
} else {
current_id_ += step_;
current_id = current_id_;
}
return ret;
}
int ObIDGenerator::get_start_id(uint64_t &start_id) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("generator is not inited", KR(ret), KPC(this));
} else {
start_id = start_id_;
}
return ret;
}
int ObIDGenerator::get_current_id(uint64_t &current_id) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("generator is not inited", KR(ret), KPC(this));
} else {
current_id = current_id_;
}
return ret;
}
int ObIDGenerator::get_end_id(uint64_t &end_id) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("generator is not inited", KR(ret), KPC(this));
} else {
end_id = end_id_;
}
return ret;
}
int ObIDGenerator::get_id_cnt(uint64_t &cnt) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("generator is not inited", KR(ret), KPC(this));
} else if (OB_UNLIKELY(end_id_ < start_id_
|| step_ <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid start_id/end_id/step", KR(ret), KPC(this));
} else {
cnt = (end_id_ - start_id_) / step_ + 1;
}
return ret;
}
int ObShareUtil::set_default_timeout_ctx(ObTimeoutCtx &ctx, const int64_t default_timeout)
{
int ret = OB_SUCCESS;

View File

@ -27,6 +27,47 @@ namespace storage
}
namespace share
{
// available range is [start_id, end_id]
class ObIDGenerator
{
public:
ObIDGenerator()
: inited_(false),
step_(0),
start_id_(common::OB_INVALID_ID),
end_id_(common::OB_INVALID_ID),
current_id_(common::OB_INVALID_ID)
{}
ObIDGenerator(const uint64_t step)
: inited_(false),
step_(step),
start_id_(common::OB_INVALID_ID),
end_id_(common::OB_INVALID_ID),
current_id_(common::OB_INVALID_ID)
{}
virtual ~ObIDGenerator() {}
void reset();
int init(const uint64_t step,
const uint64_t start_id,
const uint64_t end_id);
int next(uint64_t &current_id);
int get_start_id(uint64_t &start_id) const;
int get_current_id(uint64_t &current_id) const;
int get_end_id(uint64_t &end_id) const;
int get_id_cnt(uint64_t &cnt) const;
TO_STRING_KV(K_(inited), K_(step), K_(start_id), K_(end_id), K_(current_id));
protected:
bool inited_;
uint64_t step_;
uint64_t start_id_;
uint64_t end_id_;
uint64_t current_id_;
};
class ObShareUtil
{
public:

View File

@ -253,7 +253,7 @@ int ObUpgradeUtils::check_schema_sync(
" OR (a.refreshed_schema_version mod %ld) != 0) ",
OB_ALL_VIRTUAL_SERVER_SCHEMA_INFO_TNAME,
OB_ALL_VIRTUAL_SERVER_SCHEMA_INFO_TNAME,
schema::ObSchemaService::SCHEMA_VERSION_INC_STEP))) {
schema::ObSchemaVersionGenerator::SCHEMA_VERSION_INC_STEP))) {
LOG_WARN("fail to assign sql", KR(ret));
} else if (OB_INVALID_TENANT_ID != tenant_id
&& OB_FAIL(sql.append_fmt(" AND a.tenant_id = %ld", tenant_id))) {

View File

@ -477,6 +477,9 @@ DEF_VERSION(compatible, OB_TENANT_PARAMETER, "4.2.1.0", "compatible version for
DEF_BOOL(enable_ddl, OB_CLUSTER_PARAMETER, "True", "specifies whether DDL operation is turned on. "
"Value: True:turned on; False: turned off",
ObParameterAttr(Section::ROOT_SERVICE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_BOOL(_enable_parallel_table_creation, OB_TENANT_PARAMETER, "True", "specifies whether create table parallelly. "
"Value: True: create table parallelly; False: create table serially",
ObParameterAttr(Section::ROOT_SERVICE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_BOOL(enable_major_freeze, OB_CLUSTER_PARAMETER, "True", "specifies whether major_freeze function is turned on. "
"Value: True:turned on; False: turned off",
ObParameterAttr(Section::ROOT_SERVICE, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
@ -1200,9 +1203,9 @@ DEF_STR_WITH_CHECKER(_audit_mode, OB_TENANT_PARAMETER, "NONE",
"MYSQL: use mysql audit"
"ORACLE: use oracle audit",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_INT(_max_schema_slot_num, OB_CLUSTER_PARAMETER, "128", "[2,8192]",
DEF_INT(_max_schema_slot_num, OB_CLUSTER_PARAMETER, "128", "[2,256]",
"the max schema slot number for each tenant, "
"Range: [2, 8192] in integer",
"Range: [2, 256] in integer",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_INT_WITH_CHECKER(_ob_query_rate_limit, OB_TENANT_PARAMETER, "-1",
common::ObConfigQueryRateLimitChecker,
@ -1565,6 +1568,12 @@ DEF_BOOL(_force_explict_500_malloc, OB_CLUSTER_PARAMETER, "False",
DEF_CAP(range_optimizer_max_mem_size, OB_TENANT_PARAMETER, "128M", "[16M,1G]",
"to limit the memory consumption for the query range optimizer. Range: [16M,1G]",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_TIME(_schema_memory_recycle_interval, OB_CLUSTER_PARAMETER, "15m", "[0s,)",
"the time interval between the schedules of schema memory recycle task. "
"0 means only turn off gc current allocator, "
"and other schema memory recycle task's interval will be 15mins. "
"Range [0s,)",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
#ifdef ENABLE_500_MEMORY_LIMIT
DEF_BOOL(_enable_system_tenant_memory_limit, OB_CLUSTER_PARAMETER, "True",
"specifies whether allowed to limit the memory of tenant 500",

View File

@ -178,10 +178,11 @@ int ObDDLTransController::broadcast_consensus_version(const int64_t tenant_id,
return ret;
}
int ObDDLTransController::create_task_and_assign_schema_version(const uint64_t tenant_id,
const uint64_t schema_version_count,
int64_t &task_id,
ObIArray<int64_t> &schema_version_res)
int ObDDLTransController::create_task_and_assign_schema_version(
const uint64_t tenant_id,
const uint64_t schema_version_count,
int64_t &task_id,
ObIArray<int64_t> &schema_version_res)
{
int ret = OB_SUCCESS;
if (!inited_) {
@ -194,13 +195,22 @@ int ObDDLTransController::create_task_and_assign_schema_version(const uint64_t t
ret = OB_INVALID_ARGUMENT;
LOG_WARN("register_task_and_assign_schema_version", KR(ret), K(tenant_id), K(schema_version_count), K(schema_version_res));
} else {
int64_t new_schema_version = 0;
int64_t end_schema_version = OB_INVALID_VERSION;
SpinWLockGuard guard(lock_);
for (int i = 0; OB_SUCC(ret) && i < schema_version_count; i++) {
if (OB_FAIL(schema_service_->gen_new_schema_version(tenant_id, new_schema_version))) {
LOG_WARN("register_task_and_assign_schema_version", KR(ret), K(tenant_id));
} else if (OB_FAIL(schema_version_res.push_back(new_schema_version))) {
LOG_WARN("register_task_and_assign_schema_version", KR(ret));
if (OB_FAIL(schema_service_->gen_batch_new_schema_versions(
tenant_id, schema_version_count, end_schema_version))) {
LOG_WARN("fail to gen batch new schema versions", KR(ret), K(schema_version_count));
} else if (OB_FAIL(schema_version_res.reserve(schema_version_count))) {
LOG_WARN("fail to reserve memory", KR(ret), K(schema_version_count));
} else {
int64_t new_schema_version = end_schema_version -
(schema_version_count - 1) * ObSchemaVersionGenerator::SCHEMA_VERSION_INC_STEP;
for (int i = 0; OB_SUCC(ret) && i < schema_version_count; i++) {
if (OB_FAIL(schema_version_res.push_back(new_schema_version))) {
LOG_WARN("register_task_and_assign_schema_version", KR(ret));
} else {
new_schema_version += ObSchemaVersionGenerator::SCHEMA_VERSION_INC_STEP;
}
}
}
if (OB_SUCC(ret)) {
@ -228,7 +238,10 @@ int ObDDLTransController::create_task_and_assign_schema_version(const uint64_t t
return ret;
}
int ObDDLTransController::check_task_ready(uint64_t tenant_id, int64_t task_id, bool &ready)
int ObDDLTransController::check_task_ready_(
const uint64_t tenant_id,
const int64_t task_id,
bool &ready)
{
int ret = OB_SUCCESS;
int idx = OB_INVALID_INDEX;
@ -237,14 +250,10 @@ int ObDDLTransController::check_task_ready(uint64_t tenant_id, int64_t task_id,
for (int i = 0; i < tasks_.count(); i++) {
if (tasks_.at(i).tenant_id_ == tenant_id) {
pre_task_count++;
}
if (tasks_.at(i).task_id_ == task_id) {
if (tenant_id != tasks_.at(i).tenant_id_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("task tenant_id mismatch", KR(ret), K(tenant_id), K(tasks_));
if (tasks_.at(i).task_id_ == task_id) {
idx = i;
break;
}
idx = i;
break;
}
}
ready = false;
@ -260,9 +269,13 @@ int ObDDLTransController::check_task_ready(uint64_t tenant_id, int64_t task_id,
} else {
// gc end task
for (int i = 0; i < 10; i++) {
if (!tasks_.empty() && tasks_.at(0).task_id_ != task_id && tasks_.at(0).task_end_) {
int tmp_ret = tasks_.remove(0);
if (tmp_ret != OB_SUCCESS) {
if (!tasks_.empty()
&& tasks_.at(0).task_end_
&& !(tasks_.at(0).tenant_id_ == tenant_id
&& tasks_.at(0).task_id_ == task_id)) {
LOG_INFO("gc parallel ddl task", K(tasks_.at(0)));
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(tasks_.remove(0))) {
LOG_WARN("check_task_ready", KR(tmp_ret));
}
} else {
@ -273,15 +286,18 @@ int ObDDLTransController::check_task_ready(uint64_t tenant_id, int64_t task_id,
return ret;
}
int ObDDLTransController::wait_task_ready(uint64_t tenant_id, int64_t task_id, int64_t wait_us)
int ObDDLTransController::wait_task_ready(
const uint64_t tenant_id,
const int64_t task_id,
const int64_t wait_us)
{
int ret = OB_SUCCESS;
bool ready = false;
uint64_t cond_idx = task_id % DDL_TASK_COND_SLOT;
int64_t start_time = ObTimeUtility::current_time();
while (OB_SUCC(ret) && ObTimeUtility::current_time() - start_time < wait_us) {
if (OB_FAIL(check_task_ready(tenant_id, task_id, ready))) {
LOG_WARN("wait_task_ready", KR(ret), K(task_id), K(ready));
if (OB_FAIL(check_task_ready_(tenant_id, task_id, ready))) {
LOG_WARN("wait_task_ready", KR(ret), K(tenant_id), K(task_id), K(ready));
} else if (ready) {
break;
} else {
@ -291,26 +307,29 @@ int ObDDLTransController::wait_task_ready(uint64_t tenant_id, int64_t task_id, i
}
if (OB_FAIL(ret)) {
} else if (!ready) {
remove_task(task_id);
ret = OB_TIMEOUT;
LOG_WARN("wait_task_ready", KR(ret), K(task_id), K(tasks_), K(ready));
if (OB_FAIL(remove_task(tenant_id, task_id))) {
LOG_WARN("fail to remove task", KR(ret), K(tenant_id), K(task_id));
} else {
ret = OB_TIMEOUT;
}
LOG_WARN("wait_task_ready", KR(ret), K(tenant_id), K(task_id), K(tasks_), K(ready));
}
return ret;
}
int ObDDLTransController::remove_task(int64_t task_id)
int ObDDLTransController::remove_task(const uint64_t tenant_id, const int64_t task_id)
{
int ret = OB_SUCCESS;
int idx = OB_INVALID_INDEX;
uint64_t tenant_id = OB_INVALID_TENANT_ID;
SpinWLockGuard guard(lock_);
for (int i = 0; i < tasks_.count(); i++) {
if (tasks_.at(i).task_id_ == task_id) {
if (tasks_.at(i).tenant_id_ == tenant_id
&& tasks_.at(i).task_id_ == task_id) {
tasks_.at(i).task_end_ = true;
idx = i;
tenant_id = tasks_.at(i).tenant_id_;
LOG_INFO("remove parallel ddl task", K(tasks_.at(i)));
if (OB_FAIL(tasks_.remove(i))) {
LOG_WARN("remove_task fail", KR(ret), K(task_id));
LOG_WARN("remove_task fail", KR(ret), K(tenant_id), K(task_id));
} else if (OB_FAIL(tenants_.set_refactored(tenant_id, 1, 0, 1))) {
LOG_WARN("set_refactored fail", KR(ret), K(tenant_id));
} else {
@ -322,10 +341,10 @@ int ObDDLTransController::remove_task(int64_t task_id)
if (OB_FAIL(ret)) {
} else if (OB_INVALID_INDEX == idx) {
ret = OB_ENTRY_NOT_EXIST;
LOG_WARN("task_id not found", KR(ret), K(task_id), K(tasks_));
LOG_WARN("task_id not found", KR(ret), K(tenant_id), K(task_id), K(tasks_));
} else if (OB_INVALID_TENANT_ID == tenant_id) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tenant_id is invalid", KR(ret), K(task_id), K(tasks_));
LOG_WARN("tenant_id is invalid", KR(ret), K(tenant_id), K(task_id), K(tasks_));
} else {
// wake up next
for (int next = idx; next < tasks_.count(); next++) {
@ -340,7 +359,8 @@ int ObDDLTransController::remove_task(int64_t task_id)
return ret;
}
int ObDDLTransController::check_enable_ddl_trans_new_lock(uint64_t tenant_id, bool &res)
int ObDDLTransController::check_enable_ddl_trans_new_lock(
const uint64_t tenant_id, bool &res)
{
int ret = OB_SUCCESS;
if (!inited_) {
@ -362,7 +382,7 @@ int ObDDLTransController::check_enable_ddl_trans_new_lock(uint64_t tenant_id, bo
return ret;
}
int ObDDLTransController::set_enable_ddl_trans_new_lock(uint64_t tenant_id)
int ObDDLTransController::set_enable_ddl_trans_new_lock(const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
if (!inited_) {

View File

@ -47,20 +47,21 @@ public:
void wait();
void destroy();
static const int DDL_TASK_COND_SLOT = 128;
int create_task_and_assign_schema_version(const uint64_t tenant_id,
int create_task_and_assign_schema_version(
const uint64_t tenant_id,
const uint64_t schema_version_count,
int64_t &task_id,
ObIArray<int64_t> &schema_version_res);
int wait_task_ready(uint64_t tenant_id, int64_t task_id, int64_t wait_us);
int remove_task(int64_t task_id);
int check_enable_ddl_trans_new_lock(uint64_t tenant_id, bool &res);
int set_enable_ddl_trans_new_lock(uint64_t tenant_id);
int wait_task_ready(const uint64_t tenant_id, const int64_t task_id, const int64_t wait_us);
int remove_task(const uint64_t tenant_id, const int64_t task_id);
int check_enable_ddl_trans_new_lock(const uint64_t tenant_id, bool &res);
int set_enable_ddl_trans_new_lock(const uint64_t tenant_id);
int broadcast_consensus_version(const int64_t tenant_id,
const int64_t schema_version,
const ObArray<ObAddr> &server_list);
private:
virtual void run1() override;
int check_task_ready(uint64_t tenant_id, int64_t task_id, bool &ready);
int check_task_ready_(const uint64_t tenant_id, const int64_t task_id, bool &ready);
private:
bool inited_;
common::ObThreadCond cond_slot_[DDL_TASK_COND_SLOT];

View File

@ -0,0 +1,806 @@
/**
* 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 SHARE_SCHEMA
#include "share/schema/ob_latest_schema_guard.h"
#include "share/schema/ob_multi_version_schema_service.h"
using namespace oceanbase::lib;
using namespace oceanbase::common;
using namespace oceanbase::share;
using namespace oceanbase::share::schema;
ObLatestSchemaGuard::ObLatestSchemaGuard(
share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id)
: schema_service_(schema_service),
tenant_id_(tenant_id),
local_allocator_("LastestSchGuard"),
schema_objs_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(local_allocator_))
{
}
ObLatestSchemaGuard::~ObLatestSchemaGuard()
{
}
int ObLatestSchemaGuard::check_inner_stat_()
{
int ret = OB_SUCCESS;
int64_t schema_version = OB_INVALID_VERSION;
if (OB_ISNULL(schema_service_)) {
ret = OB_NOT_INIT;
LOG_WARN("schema service is null", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tenant_id is invalid", KR(ret), K_(tenant_id));
} else if (OB_FAIL(schema_service_->get_tenant_refreshed_schema_version(tenant_id_, schema_version))) {
if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_TENANT_NOT_EXIST;
LOG_WARN("tenant not exist", KR(ret), K_(tenant_id));
} else {
LOG_WARN("fail to get tenant refreshed schema version", KR(ret), K_(tenant_id));
}
}
return ret;
}
int ObLatestSchemaGuard::check_and_get_service_(
ObSchemaService *&schema_service_impl,
ObMySQLProxy *&sql_proxy)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret), K_(tenant_id));
} else if (OB_ISNULL(schema_service_impl = schema_service_->get_schema_service())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema service impl is null", KR(ret), K_(tenant_id));
} else if (OB_ISNULL(sql_proxy = schema_service_->get_sql_proxy())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql proxy is null", KR(ret), K_(tenant_id));
}
return ret;
}
template<typename T>
int ObLatestSchemaGuard::get_schema_(
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const T *&schema)
{
int ret = OB_SUCCESS;
const ObSchema *base_schema = NULL;
schema = NULL;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(TENANT_SCHEMA == schema_type && !is_sys_tenant(tenant_id))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tenant_id for TENANT_SCHEMA", KR(ret), K(tenant_id), K(schema_id));
} else if (OB_UNLIKELY(SYS_VARIABLE_SCHEMA == schema_type && tenant_id != schema_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tenant_id and schema_id not match for TENANT_SCHEMA",
KR(ret), K(tenant_id), K(schema_id));
} else if (OB_UNLIKELY(!is_normal_schema(schema_type)
|| OB_INVALID_ID == schema_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else if (OB_FAIL(get_from_local_cache_(schema_type, tenant_id, schema_id, schema))) {
if (OB_ENTRY_NOT_EXIST != ret) {
LOG_WARN("fail to get schema from cache", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else if (OB_FAIL(schema_service_->get_latest_schema(
local_allocator_, schema_type, tenant_id, schema_id, base_schema))) {
LOG_WARN("fail to get latest schema", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else if (OB_ISNULL(base_schema)) {
// schema not exist
} else if (OB_FAIL(put_to_local_cache_(schema_type, tenant_id, schema_id, base_schema))) {
LOG_WARN("fail to put to local cache", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else {
schema = static_cast<const T*>(base_schema);
}
}
return ret;
}
template<typename T>
int ObLatestSchemaGuard::get_from_local_cache_(
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const T *&schema)
{
int ret = OB_SUCCESS;
schema = NULL;
if (OB_UNLIKELY(OB_INVALID_ID == schema_id
|| OB_INVALID_TENANT_ID == tenant_id
|| !is_normal_schema(schema_type))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else {
const ObSchema *tmp_schema = NULL;
bool found = false;
FOREACH_CNT_X(id_schema, schema_objs_, !found) {
if (id_schema->schema_type_ == schema_type
&& id_schema->tenant_id_ == tenant_id
&& id_schema->schema_id_ == schema_id) {
tmp_schema = id_schema->schema_;
found = true;
}
}
if (!found) {
ret = OB_ENTRY_NOT_EXIST;
LOG_TRACE("local cache miss [id to schema]", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else if (OB_ISNULL(tmp_schema)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tmp schema is NULL", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else {
schema = static_cast<const T *>(tmp_schema);
LOG_TRACE("schema cache hit", K(schema_type), K(tenant_id), K(schema_id));
}
}
return ret;
}
template<typename T>
int ObLatestSchemaGuard::put_to_local_cache_(
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const T *&schema)
{
int ret = OB_SUCCESS;
SchemaObj schema_obj;
schema_obj.schema_type_ = schema_type;
schema_obj.tenant_id_ = tenant_id;
schema_obj.schema_id_ = schema_id;
schema_obj.schema_ = const_cast<ObSchema*>(schema);
if (OB_FAIL(schema_objs_.push_back(schema_obj))) {
LOG_WARN("add schema object failed", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
}
return ret;
}
int ObLatestSchemaGuard::get_tablegroup_id(
const common::ObString &tablegroup_name,
uint64_t &tablegroup_id)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
tablegroup_id = OB_INVALID_ID;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(tablegroup_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tablegroup_name is empty", KR(ret), K_(tenant_id), K(tablegroup_name));
} else if (OB_FAIL(schema_service_impl->get_tablegroup_id(
*sql_proxy, tenant_id_, tablegroup_name, tablegroup_id))) {
LOG_WARN("fail to get tablegroup id", KR(ret), K_(tenant_id), K(tablegroup_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == tablegroup_id)) {
LOG_INFO("tablegroup not exist", KR(ret), K_(tenant_id), K(tablegroup_name));
}
return ret;
}
int ObLatestSchemaGuard::get_database_id(
const common::ObString &database_name,
uint64_t &database_id)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
database_id = OB_INVALID_ID;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(database_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_name is empty", KR(ret), K_(tenant_id), K(database_name));
} else if (OB_FAIL(schema_service_impl->get_database_id(
*sql_proxy, tenant_id_, database_name, database_id))) {
LOG_WARN("fail to get database id", KR(ret), K_(tenant_id), K(database_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id)) {
LOG_INFO("database not exist", KR(ret), K_(tenant_id), K(database_name));
}
return ret;
}
int ObLatestSchemaGuard::get_table_id(
const uint64_t database_id,
const uint64_t session_id,
const ObString &table_name,
uint64_t &table_id,
ObTableType &table_type,
int64_t &schema_version)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
table_id = OB_INVALID_ID;
table_type = ObTableType::MAX_TABLE_TYPE;
schema_version = OB_INVALID_VERSION;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| table_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/table_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(table_name));
} else if (OB_FAIL(schema_service_impl->get_table_id(
*sql_proxy, tenant_id_, database_id, session_id,
table_name, table_id, table_type, schema_version))) {
LOG_WARN("fail to get database id", KR(ret), K_(tenant_id), K(database_id), K(session_id), K(table_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == table_id)) {
LOG_INFO("table not exist", KR(ret), K_(tenant_id), K(database_id), K(session_id), K(table_name));
}
return ret;
}
int ObLatestSchemaGuard::get_mock_fk_parent_table_id(
const uint64_t database_id,
const ObString &table_name,
uint64_t &mock_fk_parent_table_id)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| table_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/table_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(table_name));
} else if (OB_FAIL(schema_service_impl->get_mock_fk_parent_table_id(
*sql_proxy, tenant_id_, database_id, table_name, mock_fk_parent_table_id))) {
LOG_WARN("fail to get mock parent table id", KR(ret), K_(tenant_id), K(database_id), K(table_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == mock_fk_parent_table_id)) {
LOG_INFO("mock parent table not exist", KR(ret), K_(tenant_id), K(database_id), K(table_name));
}
return ret;
}
int ObLatestSchemaGuard::get_synonym_id(
const uint64_t database_id,
const ObString &synonym_name,
uint64_t &synonym_id)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
synonym_id = OB_INVALID_ID;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| synonym_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/synonym_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(synonym_name));
} else if (OB_FAIL(schema_service_impl->get_synonym_id(
*sql_proxy, tenant_id_, database_id, synonym_name, synonym_id))) {
LOG_WARN("fail to get synonym id", KR(ret), K_(tenant_id), K(database_id), K(synonym_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == synonym_id)) {
LOG_INFO("synonym not exist", KR(ret), K_(tenant_id), K(database_id), K(synonym_name));
}
return ret;
}
int ObLatestSchemaGuard::get_constraint_id(
const uint64_t database_id,
const ObString &constraint_name,
uint64_t &constraint_id)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
constraint_id = OB_INVALID_ID;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| constraint_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/constraint_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(constraint_name));
} else if (OB_FAIL(schema_service_impl->get_constraint_id(
*sql_proxy, tenant_id_, database_id, constraint_name, constraint_id))) {
LOG_WARN("fail to get constraint id", KR(ret), K_(tenant_id), K(database_id), K(constraint_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == constraint_id)) {
LOG_INFO("constraint not exist", KR(ret), K_(tenant_id), K(database_id), K(constraint_name));
}
return ret;
}
int ObLatestSchemaGuard::get_foreign_key_id(
const uint64_t database_id,
const ObString &foreign_key_name,
uint64_t &foreign_key_id)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
foreign_key_id = OB_INVALID_ID;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| foreign_key_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/foreign_key_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(foreign_key_name));
} else if (OB_FAIL(schema_service_impl->get_foreign_key_id(
*sql_proxy, tenant_id_, database_id, foreign_key_name, foreign_key_id))) {
LOG_WARN("fail to get foreign_key id", KR(ret), K_(tenant_id), K(database_id), K(foreign_key_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == foreign_key_id)) {
LOG_INFO("foreign_key not exist", KR(ret), K_(tenant_id), K(database_id), K(foreign_key_name));
}
return ret;
}
int ObLatestSchemaGuard::get_sequence_id(
const uint64_t database_id,
const ObString &sequence_name,
uint64_t &sequence_id,
bool &is_system_generated)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
sequence_id = OB_INVALID_ID;
is_system_generated = false;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| sequence_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/sequence_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(sequence_name));
} else if (OB_FAIL(schema_service_impl->get_sequence_id(
*sql_proxy, tenant_id_, database_id,
sequence_name, sequence_id, is_system_generated))) {
LOG_WARN("fail to get sequence id", KR(ret), K_(tenant_id), K(database_id), K(sequence_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == sequence_id)) {
LOG_INFO("sequence not exist", KR(ret), K_(tenant_id), K(database_id), K(sequence_id));
}
return ret;
}
int ObLatestSchemaGuard::get_package_id(
const uint64_t database_id,
const ObString &package_name,
const ObPackageType package_type,
const int64_t compatible_mode,
uint64_t &package_id)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
package_id = OB_INVALID_ID;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| package_name.empty()
|| INVALID_PACKAGE_TYPE == package_type
|| compatible_mode < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/package_name/package_type/compatible_mode is invalid",
KR(ret), K_(tenant_id), K(database_id), K(package_name),
K(package_type), K(compatible_mode));
} else if (OB_FAIL(schema_service_impl->get_package_id(
*sql_proxy, tenant_id_, database_id, package_name,
package_type, compatible_mode, package_id))) {
LOG_WARN("fail to get package id", KR(ret), K_(tenant_id),
K(database_id), K(package_name), K(compatible_mode));
} else if (OB_UNLIKELY(OB_INVALID_ID == package_id)) {
LOG_INFO("package not exist", KR(ret), K_(tenant_id), K(database_id),
K(package_name), K(package_type), K(compatible_mode));
}
return ret;
}
int ObLatestSchemaGuard::get_routine_id(
const uint64_t database_id,
const uint64_t package_id,
const uint64_t overload,
const ObString &routine_name,
common::ObIArray<std::pair<uint64_t, share::schema::ObRoutineType>> &routine_pairs)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
routine_pairs.reset();
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| routine_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/routine_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(routine_name));
} else if (OB_FAIL(schema_service_impl->get_routine_id(
*sql_proxy, tenant_id_, database_id, package_id,
overload, routine_name, routine_pairs))) {
LOG_WARN("fail to get routine id", KR(ret), K_(tenant_id),
K(database_id), K(package_id), K(overload), K(routine_name));
} else if (OB_UNLIKELY(routine_pairs.empty())) {
LOG_INFO("routine not exist", KR(ret), K_(tenant_id), K(database_id),
K(package_id), K(routine_name), K(overload), K(routine_name));
}
return ret;
}
int ObLatestSchemaGuard::check_udt_exist(
const uint64_t database_id,
const uint64_t package_id,
const ObUDTTypeCode type_code,
const ObString &udt_name,
bool &exist)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
exist = false;
uint64_t udt_id = OB_INVALID_ID;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id
|| UDT_INVALID_TYPE_CODE == type_code
|| udt_name.empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id/type_code/udt_name is invalid",
KR(ret), K_(tenant_id), K(database_id), K(type_code), K(udt_name));
} else if (OB_FAIL(schema_service_impl->get_udt_id(
*sql_proxy, tenant_id_, database_id, package_id, udt_name, udt_id))) {
LOG_WARN("fail to get udt id", KR(ret), K_(tenant_id), K(database_id),
K(package_id), K(udt_name));
} else if (OB_UNLIKELY(OB_INVALID_ID == udt_id)) {
exist = false;
} else if (UDT_TYPE_OBJECT_BODY != type_code) {
exist = true;
} else {
const ObUDTTypeInfo *udt_info = NULL;
if (OB_FAIL(get_udt_info(udt_id, udt_info))) {
LOG_WARN("fail to get udt", KR(ret), K_(tenant_id), K(udt_id));
} else if (OB_ISNULL(udt_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("udt not exist", KR(ret), K_(tenant_id), K(udt_id));
} else if (2 == udt_info->get_object_type_infos().count()) {
if (udt_info->is_object_type_legal()) {
exist = true;
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("illegal object type which has object body",
KR(ret), KPC(udt_info), K(udt_name), K(type_code));
}
} else {
exist = false;
}
}
if (OB_SUCC(ret) && !exist) {
LOG_INFO("udt not exist", KR(ret), K_(tenant_id),
K(package_id), K(type_code), K(udt_name));
}
return ret;
}
int ObLatestSchemaGuard::get_table_schema_versions(
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(table_ids.count() <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("table_ids is empty", KR(ret));
} else if (OB_FAIL(schema_service_impl->get_table_schema_versions(
*sql_proxy, tenant_id_, table_ids, versions))) {
LOG_WARN("fail to get table schema versions", KR(ret), K_(tenant_id), K(table_ids));
}
return ret;
}
int ObLatestSchemaGuard::get_mock_fk_parent_table_schema_versions(
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_UNLIKELY(table_ids.count() <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("table_ids is empty", KR(ret));
} else if (OB_FAIL(schema_service_impl->get_mock_fk_parent_table_schema_versions(
*sql_proxy, tenant_id_, table_ids, versions))) {
LOG_WARN("fail to get mock fk parent table schema versions", KR(ret), K_(tenant_id), K(table_ids));
}
return ret;
}
int ObLatestSchemaGuard::get_default_audit_schemas(
common::ObIArray<ObSAuditSchema> &audit_schemas)
{
int ret = OB_SUCCESS;
ObSchemaService *schema_service_impl = NULL;
ObMySQLProxy *sql_proxy = NULL;
if (OB_FAIL(check_and_get_service_(schema_service_impl, sql_proxy))) {
LOG_WARN("fail to check and get service", KR(ret));
} else if (OB_FAIL(schema_service_impl->get_audits_in_owner(
*sql_proxy, tenant_id_, AUDIT_OBJ_DEFAULT, OB_AUDIT_MOCK_USER_ID,
audit_schemas))) {
LOG_WARN("fail to get audits in owner", KR(ret), K_(tenant_id));
}
return ret;
}
int ObLatestSchemaGuard::check_oracle_object_exist(
const uint64_t database_id,
const uint64_t session_id,
const ObString &object_name,
const ObSchemaType &schema_type,
const ObRoutineType &routine_type,
const bool is_or_replace)
{
int ret = OB_SUCCESS;
bool is_oracle_mode = false;
if (OB_UNLIKELY(
OB_INVALID_TENANT_ID == tenant_id_
|| OB_INVALID_ID == database_id
|| object_name.empty()
|| OB_MAX_SCHEMA == schema_type)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arg", KR(ret), K_(tenant_id),
K(database_id), K(object_name), K(schema_type));
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
tenant_id_, is_oracle_mode))) {
LOG_WARN("fail to check oracle mode", KR(ret), K_(tenant_id));
} else if (!is_oracle_mode) {
// skip
} else {
// table
if (OB_SUCC(ret)) {
uint64_t table_id = OB_INVALID_ID;
ObTableType table_type = MAX_TABLE_TYPE;
int64_t schema_version = OB_INVALID_VERSION;
if (OB_FAIL(get_table_id(database_id, session_id, object_name,
table_id, table_type, schema_version))) {
LOG_WARN("fail to get table id", KR(ret), K(database_id), K(session_id), K(object_name));
} else if (OB_INVALID_ID != table_id) {
if (TABLE_SCHEMA == schema_type && is_view_table(table_type) && is_or_replace) {
// create or replace view
} else {
ret = OB_ERR_EXIST_OBJECT;
LOG_WARN("Name is already used by table in oracle mode", KR(ret), K(table_id));
}
}
}
// sequence
if (OB_SUCC(ret)) {
uint64_t sequence_id = OB_INVALID_ID;
bool is_system_generated = false;
if (OB_FAIL(get_sequence_id(database_id, object_name, sequence_id, is_system_generated))) {
LOG_WARN("fail to get sequence id", KR(ret), K(database_id), K(object_name));
} else if (OB_INVALID_ID != sequence_id) {
ret = OB_ERR_EXIST_OBJECT;
LOG_WARN("Name is already used by sequence in oracle mode", KR(ret), K(sequence_id));
}
}
// synonym
if (OB_SUCC(ret)) {
uint64_t synonym_id = OB_INVALID_ID;
if (OB_FAIL(get_synonym_id(database_id, object_name, synonym_id))) {
LOG_WARN("fail to get synonym id", KR(ret), K(database_id), K(object_name));
} else if (OB_INVALID_ID != synonym_id) {
if (SYNONYM_SCHEMA == schema_type && is_or_replace) {
// create or replace synonym
} else {
ret = OB_ERR_EXIST_OBJECT;
LOG_WARN("Name is already used by synonym in oracle mode", KR(ret), K(synonym_id));
}
}
}
// package
if (OB_SUCC(ret)) {
uint64_t package_id = OB_INVALID_ID;
if (OB_FAIL(get_package_id(database_id, object_name,
PACKAGE_TYPE, COMPATIBLE_ORACLE_MODE, package_id))) {
LOG_WARN("fail to get package id", KR(ret), K(database_id), K(object_name));
} else if (OB_INVALID_ID != package_id) {
if (PACKAGE_SCHEMA == schema_type && is_or_replace) {
// create or replace package
} else {
ret = OB_ERR_EXIST_OBJECT;
LOG_WARN("Name is already used by package in oracle mode", KR(ret), K(package_id));
}
}
}
// standalone procedure/function
if (OB_SUCC(ret)) {
ObArray<std::pair<uint64_t, ObRoutineType>> routine_pairs;
const uint64_t package_id = OB_INVALID_ID;
const uint64_t overload = 0;
if (OB_FAIL(get_routine_id(database_id, package_id, overload, object_name, routine_pairs))) {
LOG_WARN("fail to get routine id", KR(ret), K(database_id), K(package_id), K(overload), K(object_name));
} else if (!routine_pairs.empty()) {
for (int64_t i = 0; OB_SUCC(ret) && i < routine_pairs.count(); i++) {
std::pair<uint64_t, ObRoutineType> &routine_pair = routine_pairs.at(i);
if (ROUTINE_SCHEMA == schema_type && is_or_replace) {
if (ROUTINE_PROCEDURE_TYPE == routine_type
&& ROUTINE_PROCEDURE_TYPE == routine_pair.second) {
// create or replace standalone procedure
} else if (ROUTINE_PROCEDURE_TYPE != routine_type
&& ROUTINE_FUNCTION_TYPE == routine_pair.second) {
// create or replace standalone function
} else {
ret = OB_ERR_EXIST_OBJECT;
}
} else {
ret = OB_ERR_EXIST_OBJECT;
}
if (OB_ERR_EXIST_OBJECT == ret) {
LOG_WARN("Name is already used by routine in oracle mode", KR(ret),
"routine_id", routine_pair.first, "routine_type", routine_pair.second);
}
} // end for
}
}
// udt
if (OB_SUCC(ret)) {
const uint64_t package_id = OB_INVALID_ID;
const ObUDTTypeCode type_code = ObUDTTypeCode::UDT_TYPE_OBJECT;
bool exist = false;
if (OB_FAIL(check_udt_exist(database_id, package_id, type_code, object_name, exist))) {
LOG_WARN("fail to check udt exist", KR(ret), K(database_id), K(package_id), K(type_code));
} else if (exist) {
if (UDT_SCHEMA == schema_type && is_or_replace) {
// create or replace udt
} else {
ret = OB_ERR_EXIST_OBJECT;
LOG_WARN("Name is already used by udt in oracle mode", KR(ret), K(object_name));
}
}
}
}
return ret;
}
int ObLatestSchemaGuard::get_table_schema(
const uint64_t table_id,
const ObTableSchema *&table_schema)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(get_schema_(TABLE_SCHEMA,
tenant_id_, table_id, table_schema))) {
LOG_WARN("fail to get table table", KR(ret), K_(tenant_id), K(table_id));
} else if (OB_ISNULL(table_schema)) {
LOG_INFO("table not exist", KR(ret), K_(tenant_id), K(table_id));
}
return ret;
}
int ObLatestSchemaGuard::get_mock_fk_parent_table_schema(
const uint64_t mock_fk_parent_table_id,
const ObMockFKParentTableSchema *&mock_fk_parent_table_schema)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_FAIL(get_schema_(MOCK_FK_PARENT_TABLE_SCHEMA,
tenant_id_, mock_fk_parent_table_id, mock_fk_parent_table_schema))) {
LOG_WARN("fail to get mock fk parent table", KR(ret), K_(tenant_id), K(mock_fk_parent_table_id));
} else if (OB_ISNULL(mock_fk_parent_table_schema)) {
LOG_INFO("mock fk parent table not exist", KR(ret), K_(tenant_id), K(mock_fk_parent_table_id));
}
return ret;
}
int ObLatestSchemaGuard::get_tablegroup_schema(
const uint64_t tablegroup_id,
const ObTablegroupSchema *&tablegroup_schema)
{
int ret = OB_SUCCESS;
tablegroup_schema = NULL;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == tablegroup_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tablegroup_id is invalid", KR(ret), K_(tenant_id), K(tablegroup_id));
} else if (OB_FAIL(get_schema_(TABLEGROUP_SCHEMA,
tenant_id_, tablegroup_id, tablegroup_schema))) {
LOG_WARN("fail to get tablegroup", KR(ret), K_(tenant_id), K(tablegroup_id));
} else if (OB_ISNULL(tablegroup_schema)) {
LOG_INFO("tablegroup not exist", KR(ret), K_(tenant_id), K(tablegroup_id));
}
return ret;
}
int ObLatestSchemaGuard::get_database_schema(
const uint64_t database_id,
const ObDatabaseSchema *&database_schema)
{
int ret = OB_SUCCESS;
database_schema = NULL;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == database_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("database_id is invalid", KR(ret), K_(tenant_id), K(database_id));
} else if (OB_FAIL(get_schema_(DATABASE_SCHEMA,
tenant_id_, database_id, database_schema))) {
LOG_WARN("fail to get database", KR(ret), K_(tenant_id), K(database_id));
} else if (OB_ISNULL(database_schema)) {
LOG_INFO("database not exist", KR(ret), K_(tenant_id), K(database_id));
}
return ret;
}
int ObLatestSchemaGuard::get_tenant_schema(
const uint64_t tenant_id,
const ObTenantSchema *&tenant_schema)
{
int ret = OB_SUCCESS;
tenant_schema = NULL;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == tenant_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tenant_id is invalid", KR(ret), K_(tenant_id), K(tenant_id));
} else if (OB_FAIL(get_schema_(TENANT_SCHEMA,
OB_SYS_TENANT_ID, tenant_id, tenant_schema))) {
LOG_WARN("fail to get tenant", KR(ret), K_(tenant_id), K(tenant_id));
} else if (OB_ISNULL(tenant_schema)) {
LOG_INFO("tenant not exist", KR(ret), K_(tenant_id), K(tenant_id));
}
return ret;
}
int ObLatestSchemaGuard::get_tablespace_schema(
const uint64_t tablespace_id,
const ObTablespaceSchema *&tablespace_schema)
{
int ret = OB_SUCCESS;
tablespace_schema = NULL;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == tablespace_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tablespace_id is invalid", KR(ret), K_(tenant_id), K(tablespace_id));
} else if (OB_FAIL(get_schema_(TABLESPACE_SCHEMA,
tenant_id_, tablespace_id, tablespace_schema))) {
LOG_WARN("fail to get tablespace", KR(ret), K_(tenant_id), K(tablespace_id));
} else if (OB_ISNULL(tablespace_schema)) {
LOG_INFO("tablespace not exist", KR(ret), K_(tenant_id), K(tablespace_id));
}
return ret;
}
int ObLatestSchemaGuard::get_udt_info(
const uint64_t udt_id,
const ObUDTTypeInfo *udt_info)
{
int ret = OB_SUCCESS;
udt_info = NULL;
if (OB_FAIL(check_inner_stat_())) {
LOG_WARN("fail to check inner stat", KR(ret));
} else if (OB_UNLIKELY(OB_INVALID_ID == udt_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("udt_id is invalid", KR(ret), K_(tenant_id), K(udt_id));
} else if (OB_FAIL(get_schema_(UDT_SCHEMA, tenant_id_, udt_id, udt_info))) {
LOG_WARN("fail to get udt", KR(ret), K_(tenant_id), K(udt_id));
} else if (OB_ISNULL(udt_info)) {
LOG_INFO("udt not exist", KR(ret), K_(tenant_id), K(udt_info));
}
return ret;
}

View File

@ -0,0 +1,399 @@
/**
* 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 OB_OCEANBASE_SCHEMA_OB_LATEST_SCHEMA_GUARD_H_
#define OB_OCEANBASE_SCHEMA_OB_LATEST_SCHEMA_GUARD_H_
#include "lib/allocator/page_arena.h" //ObArenaAllocator
#include "share/schema/ob_schema_struct.h"
#include "share/schema/ob_package_info.h"
#include "share/schema/ob_routine_info.h"
#include "share/schema/ob_udt_info.h"
namespace oceanbase
{
namespace common
{
class ObMySQLProxy;
}
namespace share
{
namespace schema
{
class ObMultiVersionSchemaService;
class ObSchemaService;
struct SchemaObj;
// NOTICE:
// 1. Not thread safety.
// 2. Don't assume that objects are fetched under the same schema version.
// 3. Objects will be cached in guard when fetch object by id in the first time.
// 4. Should be used after related objects are locked by name/id.
class ObLatestSchemaGuard
{
const static int DEFAULT_RESERVE_SIZE = 32;
typedef common::ObSEArray<SchemaObj, DEFAULT_RESERVE_SIZE> SchemaObjs;
public:
ObLatestSchemaGuard() = delete;
ObLatestSchemaGuard(share::schema::ObMultiVersionSchemaService *schema_service,
const uint64_t tenant_id);
~ObLatestSchemaGuard();
public:
/* -------------- interfaces without cache ---------------*/
// 1. won't cache tablegroup_id by name.
//
// @param[in]:
// - tablegroup_name : string comparsion is case sensitive in mysql/oracle tenant.
// @param[out]:
// - tablegroup_id : OB_INVALID_ID means tablegroup not exist
int get_tablegroup_id(const ObString &tablegroup_name,
uint64_t &tablegroup_id);
// 1. won't cache database_id by name.
//
// @param[in]:
// - datasbase_name:
// 1) If database name is "oceanbase", string comparsion is case insensitive.
// 2) string comparsion is case sensitive in oracle tenant.
// 3) string comparsion is case insensitive in mysql tenant.
// @param[out]:
// - database_id: OB_INVALID_ID means databse not exist
int get_database_id(
const common::ObString &database_name,
uint64_t &database_id);
// ATTENTION!!!!:
//
// 1. hidden/lob meta/lob piece tables are not visible in user namespace, so we can't get related objects from this interface.
//
// 2.TODO(yanmu.ztl): This interface doesn't support to get index id by index name.
//
// 3. we will match table name with the following priorities:
// (rules with smaller sequence numbers have higher priority)
// - 3.1. if session_id > 0, match table with specified session_id. (mysql tmp table or ctas table)
// - 3.2. match table with session_id = 0.
// - 3.3. if table name is inner table name (comparsion insensitive), match related inner table.
// - 3.4. string comparsion is case sensitive in oracle tenant and is case insensitive in mysql tenant.
//
// 4. mock parent table is not visible in this interface.
//
// 5. won't cache database_id by name.
//
// @param[in]:
// - database_id
// - session_id
// - table_name
// @param[out]:
// - table_id: OB_INVALID_ID means table not exist
// - table_type
// - schema_version
int get_table_id(
const uint64_t database_id,
const uint64_t session_id,
const ObString &table_name,
uint64_t &table_id,
ObTableType &table_type,
int64_t &schema_version);
// check if table is a mock parent table
// 1. table name comparsion is case sensitive.
// 2. won't cache mock_fk_parent_table_id by name.
// 3. TODO(yanmu.ztl): May has poor performance when tenant has many mock parent tables
// because related table is lack of index on name.
// @param[in]:
// - database_id
// - table_name
// @param[out]:
// - mock_fk_parent_table_id : OB_INVALID_ID means databse not exist
int get_mock_fk_parent_table_id(
const uint64_t database_id,
const ObString &table_name,
uint64_t &mock_fk_parent_table_id);
// 1. synonym_name comparsion is case sensitive.
// 2. won't cache id by name.
//
// @param[in]:
// - database_id
// - synonym_name
// @param[out]:
// - synonym_id : OB_INVALID_ID means synonym not exist
int get_synonym_id(const uint64_t database_id,
const ObString &synonym_name,
uint64_t &synonym_id);
// 1. constraint name comparsion:
// - case sensitive: oracle
// - case insensitive: mysql
// 2. won't cache id by name.
//
// @param[in]:
// - database_id
// - constraint_name
// @param[out]:
// - : OB_INVALID_ID means constraint not exist
int get_constraint_id(const uint64_t database_id,
const ObString &constraint_name,
uint64_t &constraint_id);
// 1. foreign key name comparsion:
// - case sensitive: oracle
// - case insensitive: mysql
// 2. won't cache id by name.
//
// @param[in]:
// - database_id
// - foreign_key_name
// @param[out]:
// - : OB_INVALID_ID means foreign key not exist
int get_foreign_key_id(const uint64_t database_id,
const ObString &foreign_key_name,
uint64_t &foreign_key_id);
// 1. sequence name comparsion: case sensitive
// 2. won't cache id by name.
//
// @param[in]:
// - database_id
// - sequence_name
// @param[out]:
// - is_system_generated : true means it's a inner sequence object
// - sequence_id : OB_INVALID_ID means constraint not exist
int get_sequence_id(const uint64_t database_id,
const ObString &sequence_name,
uint64_t &sequence_id,
bool &is_system_generated);
// 1. package name comparsion:
// - case sensitive: oracle
// - case insensitive: mysql
// 2. won't cache id by name.
//
// @param[in]:
// - database_id
// - package_name
// - package_type
// - compatible_mode
// @param[out]:
// - package_id : OB_INVALID_ID means package not exist
int get_package_id(const uint64_t database_id,
const ObString &package_name,
const ObPackageType package_type,
const int64_t compatible_mode,
uint64_t &package_id);
// 1. routine name comparsion:
// - case sensitive: oracle
// - case insensitive: mysql
// 2. won't cache id by name.
//
// @param[in]:
// - database_id
// - package_id: can be OB_INVALID_ID
// - overload: can be 0
// - routine_name
// @param[out]:
// - routine_pairs : empty means routine not exist
int get_routine_id(
const uint64_t database_id,
const uint64_t package_id,
const uint64_t overload,
const ObString &routine_name,
common::ObIArray<std::pair<uint64_t, share::schema::ObRoutineType>> &routine_pairs);
// 1. udt name comparsion:
// - case sensitive: oracle
// - case insensitive: mysql
// 2. won't cache id by name.
//
// @param[in]:
// - database_id
// - package_id: can be OB_INVALID_ID
// - type_code
// - udt_name
// @param[out]:
// - exist
int check_udt_exist(
const uint64_t database_id,
const uint64_t package_id,
const ObUDTTypeCode type_code,
const ObString &udt_name,
bool &exist);
// 1. won't cache versions.
// @param[in]:
// - table_ids
// @param[out]:
// - versions
int get_table_schema_versions(
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions);
// 1. won't cache versions.
// @param[in]:
// - table_ids
// @param[out]:
// - versions
int get_mock_fk_parent_table_schema_versions(
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions);
// 1. won't cache.
// 2. return audits which is AUDIT_OBJ_DEFAULT and owner is OB_AUDIT_MOCK_USER_ID.
// @param[in]:
// - allocator
// @param[out]:
// - audit_schemas
int get_default_audit_schemas(
common::ObIArray<ObSAuditSchema> &audit_schemas);
// 1. won't cache
//
// https://docs.oracle.com/cd/E18283_01/server.112/e17118/sql_elements008.htm
// Within a namespace, no two objects can have the same name.
// In oracle mode, the following schema objects share one namespace:
// Tables(create, rename, flashback)
// Views(create, create or replace, rename, flashback)
// Sequences(create, rename)
// Private synonyms(create, create or replace, rename)
// Stand-alone procedures(create, create or replace)
// Stand-alone stored functions(create, create or replace)
// Packages(create, create or replace)
// Materialized views (OB oracle mode is not supported now)
// User-defined types(create, create or replace)
// This function is used to check object name is duplicate in other different schemas in oracle mode.
// This function should be as a supplement to the original oracle detection logic of duplicate object name.
// @param [in] database_id
// @param [in] session_id : for temporary table
// @param [in] object_name
// @param [in] schema_type : schema type of object to be checked
// @param [in] routine_type : If schema_type is ROUTINE_SCHEMA, routine_type is used to
// distinguish whether object is procedure or function.
// @param [in] is_or_replace : distinguish whether create schema with create_or_replace option
//
// @return : return OB_ERR_EXIST_OBJECT when object name conflicts
int check_oracle_object_exist(
const uint64_t database_id,
const uint64_t session_id,
const ObString &object_name,
const ObSchemaType &schema_type,
const ObRoutineType &routine_type,
const bool is_or_replace);
/* -------------- interfaces without cache end ---------------*/
/* -------------- interfaces with cache ---------------*/
// 1. will cache schema in guard
// @param[in]:
// - table_id
// @param[out]:
// - table_schema: return NULL if table not exist
int get_table_schema(
const uint64_t table_id,
const ObTableSchema *&table_schema);
// 1. will cache schema in guard
// @param[in]:
// - mock_fk_parent_table_id
// @param[out]:
// - mock_fk_parent_table_schema: return NULL if mock fk parent table not exist
int get_mock_fk_parent_table_schema(
const uint64_t mock_fk_parent_table_id,
const ObMockFKParentTableSchema *&mock_fk_parent_table_schema);
// 1. will cache tablegroup schema in guard
// @param[in]:
// - tablegroup_id
// @param[out]:
// - tablegroup_schema: return NULL if tablegroup not exist
int get_tablegroup_schema(
const uint64_t tablegroup_id,
const ObTablegroupSchema *&tablegroup_schema);
// 1. will cache database schema in guard
// @param[in]:
// - database_id
// @param[out]:
// - database_schema: return NULL if database not exist
int get_database_schema(
const uint64_t database_id,
const ObDatabaseSchema *&database_schema);
// 1. will cache tenant schema in guard
// @param[in]:
// - tenant_id
// @param[out]:
// - tenant_schema: return NULL if tenant not exist
int get_tenant_schema(
const uint64_t tenant_id,
const ObTenantSchema *&tenant_schema);
// 1. will cache tablespace schema in guard
// @param[in]:
// - tablespace_id
// @param[out]:
// - tablespace_schema: return NULL if tablespace not exist
int get_tablespace_schema(
const uint64_t tablespace_id,
const ObTablespaceSchema *&tablespace_schema);
// 1. will cache udt schema in guard
// @param[in]:
// - udt_id
// @param[out]:
// - udt_info: return NULL if udt not exist
int get_udt_info(
const uint64_t udt_id,
const ObUDTTypeInfo *udt_info);
/* -------------- interfaces with cache end ---------------*/
private:
int check_inner_stat_();
int check_and_get_service_(
ObSchemaService *&schema_service_impl,
common::ObMySQLProxy *&sql_proxy);
// For TENANT_SCHEMA, tenant_id should be OB_SYS_TENANT_ID;
// For SYS_VARIABLE_SCHEMA, tenant_id should be equal with schema_id;
template<typename T>
int get_schema_(
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const T *&schema);
template<typename T>
int get_from_local_cache_(
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const T *&schema);
template<typename T>
int put_to_local_cache_(
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const T *&schema);
private:
ObMultiVersionSchemaService *schema_service_;
uint64_t tenant_id_;
common::ObArenaAllocator local_allocator_;
SchemaObjs schema_objs_;
private:
DISALLOW_COPY_AND_ASSIGN(ObLatestSchemaGuard);
};
} //end of namespace schema
} //end of namespace share
} //end of namespace oceanbase
#endif //OB_OCEANBASE_SCHEMA_OB_LATEST_SCHEMA_GUARD_H_

View File

@ -371,6 +371,90 @@ int ObMultiVersionSchemaService::build_full_materalized_view_schema(
return ret;
}
// for ObLatestSchemaGuard
int ObMultiVersionSchemaService::get_latest_schema(
common::ObIAllocator &allocator,
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const ObSchema *&schema)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(TENANT_SCHEMA == schema_type && !is_sys_tenant(tenant_id))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tenant_id for TENANT_SCHEMA", KR(ret), K(tenant_id), K(schema_id));
} else if (OB_UNLIKELY(SYS_VARIABLE_SCHEMA == schema_type && tenant_id != schema_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tenant_id and schema_id not match for TENANT_SCHEMA",
KR(ret), K(tenant_id), K(schema_id));
} else if (OB_UNLIKELY(!is_normal_schema(schema_type)
|| OB_INVALID_ID == schema_id)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(schema_type), K(tenant_id), K(schema_id));
} else if ((TABLE_SCHEMA == schema_type
|| TABLE_SIMPLE_SCHEMA == schema_type)
&& OB_ALL_CORE_TABLE_TID == schema_id) {
const ObTableSchema *hard_code_schema = schema_cache_.get_all_core_table();
if (OB_ISNULL(hard_code_schema)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("all core table schema is null", KR(ret));
} else if (is_sys_tenant(tenant_id)) {
schema = hard_code_schema;
} else {
ObTableSchema* new_table = NULL;
if (OB_FAIL(ObSchemaUtils::alloc_schema(allocator, new_table))) {
LOG_WARN("fail to alloc table schema", KR(ret));
} else if (OB_FAIL(new_table->assign(*hard_code_schema))) {
LOG_WARN("fail to assign all core schema", KR(ret), K(tenant_id));
} else if (OB_FAIL(ObSchemaUtils::construct_tenant_space_full_table(
tenant_id, *new_table))) {
LOG_WARN("fail to construct tenant's __all_core_table schema", KR(ret), K(tenant_id));
} else {
schema = static_cast<const ObSchema*>(new_table);
}
}
} else {
ObRefreshSchemaStatus schema_status;
schema_status.tenant_id_ = tenant_id;
const int64_t schema_version = INT64_MAX;
ObSchema *new_schema = NULL;
if (OB_FAIL(schema_fetcher_.fetch_schema(schema_type,
schema_status,
schema_id,
schema_version,
allocator,
new_schema))) {
LOG_WARN("fail to fetch schema", KR(ret), K(schema_type),
K(tenant_id), K(schema_id), K(schema_version));
} else if (OB_ISNULL(new_schema)) {
// schema not exist or schema history is recycled.
} else if (TABLE_SCHEMA != schema_type) {
schema = new_schema;
} else {
ObTableSchema *new_table = static_cast<ObTableSchema *>(new_schema);
if (MATERIALIZED_VIEW == new_table->get_table_type()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("not support to fetch latest mv", KR(ret), "table_id", schema_id);
} else if (OB_ALL_CORE_TABLE_TID == schema_id) {
// do-nothing
} else if (!need_construct_aux_infos_(*new_table)) {
// do-nothing
} else if (ObSysTableChecker::is_sys_table_has_index(schema_id)) {
if (OB_FAIL(ObSysTableChecker::fill_sys_index_infos(*new_table))) {
LOG_WARN("fail to fill sys indexes", KR(ret), K(tenant_id), "table_id", schema_id);
}
} else if (OB_FAIL(construct_aux_infos_(*sql_proxy_,
schema_status, tenant_id, *new_table))) {
LOG_WARN("fail to construct aux infos", KR(ret), K(tenant_id), "table_id", schema_id);
}
if (OB_SUCC(ret)) {
schema = static_cast<const ObSchema*>(new_table);
}
}
}
return ret;
}
// In order to facilitate the processing of the tenant_space table and index, the logic of updating the schema_cache
// is placed in the upper layer, and at the same time, the schema_guard is passed in to obtain the tenant schema
// Whether it is lazy mode is distinguished by whether mgr is NULL
@ -2073,9 +2157,11 @@ int ObMultiVersionSchemaService::add_schema(
ObSchemaMgr *schema_mgr_for_cache = NULL;
ObSchemaMemMgr *mem_mgr = NULL;
ObSchemaMgrCache *schema_mgr_cache = NULL;
int64_t new_schema_version = OB_INVALID_VERSION;
int64_t refreshed_schema_version = OB_INVALID_VERSION;
int64_t received_broadcast_version = OB_INVALID_VERSION;
ObSchemaStore* schema_store = NULL;
const int64_t start_time = ObTimeUtility::current_time();
if (!force_add && !check_inner_stat()) {
ret = OB_INNER_STAT_ERROR;
LOG_WARN("inner stat error", K(ret));
@ -2097,7 +2183,7 @@ int ObMultiVersionSchemaService::add_schema(
LOG_WARN("mem_mgr is null", K(ret), K(tenant_id));
} else {
schema_mgr_cache = &schema_store->schema_mgr_cache_;
int64_t new_schema_version = schema_mgr_for_cache->get_schema_version();
new_schema_version = schema_mgr_for_cache->get_schema_version();
refreshed_schema_version = schema_store->get_refreshed_version();
if (OB_ISNULL(schema_mgr_cache)) {
ret = OB_ERR_UNEXPECTED;
@ -2126,21 +2212,26 @@ int ObMultiVersionSchemaService::add_schema(
LOG_WARN("fail to check can switch allocator", KR(ret));
} else if (can_switch) {
// Switch allocator && rewrite schema_mgr_for_cache_
LOG_INFO("try to switch allocator", KR(ret), K(tenant_id), K(new_schema_version));
if (OB_FAIL(switch_allocator_(*mem_mgr, schema_mgr_for_cache))) {
LOG_WARN("fail to alloc schema mgr after switch allocator", KR(ret), K(tenant_id), K(new_schema_version));
}
}
}
if (OB_SUCC(ret)) {
schema_store->update_refreshed_version(new_schema_version);
FLOG_INFO("[REFRESH_SCHEMA] change refreshed_schema_version with new mode", K(tenant_id), K(new_schema_version));
// Because RS only notifies other observers through RPC, the received_broadcast_version of the local observer
// is not updated
// This variable will be copied in obmp_query to ob_latest_schema_version in the session variable
// The proxy will use the variable ob_latest_schema_version to ensure that
// multiple observers are connected to the same schema version.
schema_store->update_refreshed_version(new_schema_version);
FLOG_INFO("[REFRESH_SCHEMA] change refreshed_schema_version with new mode", K(tenant_id), K(new_schema_version));
// To reduce allocator's memory more frequently
if (OB_FAIL(try_gc_allocator_when_add_schema_(tenant_id, mem_mgr, schema_mgr_cache))) {
LOG_WARN("fail to gc current allocator when add schema", KR(ret), K(tenant_id), K(new_schema_version));
}
}
int64_t end_time = ObTimeUtility::current_time();
LOG_INFO("finish add schema", KR(ret), K(tenant_id), K(new_schema_version), "cost_ts", start_time - end_time);
}
return ret;
}
@ -2207,6 +2298,7 @@ int ObMultiVersionSchemaService::switch_allocator_(
ObSchemaMgr *&latest_schema_mgr)
{
int ret = OB_SUCCESS;
int64_t start_time = ObTimeUtility::current_time();
if (OB_ISNULL(latest_schema_mgr)) {
ret = OB_ERR_UNEXPECTED;
@ -2221,6 +2313,8 @@ int ObMultiVersionSchemaService::switch_allocator_(
ObSchemaMgr *old_mgr = latest_schema_mgr;
const uint64_t tenant_id = latest_schema_mgr->get_tenant_id();
const int64_t schema_version = latest_schema_mgr->get_schema_version();
LOG_INFO("try to switch allocator", KR(ret), K(tenant_id), K(schema_version));
if (OB_FAIL(mem_mgr.alloc_schema_mgr(new_mgr, alloc_for_liboblog))) {
LOG_WARN("fail to alloc mem", KR(ret));
} else {
@ -2261,6 +2355,8 @@ int ObMultiVersionSchemaService::switch_allocator_(
LOG_ERROR("fail to free new_mgr", KR(tmp_ret), K(schema_version));
}
}
int64_t end_time = ObTimeUtility::current_time();
LOG_INFO("finish switch allocator", KR(ret), K(tenant_id), K(schema_version), "cost_ts", end_time - start_time);
}
return ret;
}
@ -3416,7 +3512,7 @@ int ObMultiVersionSchemaService::try_gc_tenant_schema_mgr_for_refresh(
if (OB_FAIL(ret)) {
} else if (OB_FAIL(try_gc_tenant_schema_mgr(mem_mgr, schema_mgr_cache))) {
LOG_WARN("fail to elimiante schema mgr", K(ret), K(tenant_id));
LOG_WARN("fail to eliminate schema mgr", K(ret), K(tenant_id));
} else if (OB_FAIL(destroy_schema_struct(tenant_id))) {
LOG_WARN("fail to destroy schema struct", K(ret), K(tenant_id));
}
@ -3451,7 +3547,7 @@ int ObMultiVersionSchemaService::try_gc_tenant_schema_mgr_for_fallback(
} else {
schema_mgr_cache = &schema_store->schema_mgr_cache_for_liboblog_;
if (OB_FAIL(try_gc_tenant_schema_mgr(mem_mgr, schema_mgr_cache))) {
LOG_WARN("fail to elimiante schema mgr", K(ret), K(tenant_id));
LOG_WARN("fail to eliminate schema mgr", K(ret), K(tenant_id));
}
}
}
@ -3530,6 +3626,10 @@ int ObMultiVersionSchemaService::try_gc_existed_tenant_schema_mgr()
} else if (OB_FAIL(try_gc_tenant_schema_mgr_for_fallback(tenant_id))) {
LOG_WARN("fail to gc tenant schema mgr for fallback", KR(ret), K(tenant_id));
}
// 3.let schema mgr to free slot's memory
if (OB_FAIL(try_gc_current_allocator(tenant_id, mem_mgr, schema_mgr_cache))) {
LOG_WARN("fail to gc current slot", KR(ret), K(tenant_id));
}
}
}
}
@ -3580,8 +3680,8 @@ int ObMultiVersionSchemaService::try_gc_another_allocator(
if (OB_ISNULL(another_ptrs.at(i))) {
LOG_WARN("ptrs is null", K(ret), K(tenant_id), K(i));
} else if (FALSE_IT(eli_schema_mgr = static_cast<ObSchemaMgr *>(another_ptrs.at(i)))) {
} else if (OB_FAIL(schema_mgr_cache->try_elimiante_schema_mgr(eli_schema_mgr))) {
LOG_WARN("fail to elimiante schema_mgr", K(ret), K(tenant_id), K(eli_schema_mgr));
} else if (OB_FAIL(schema_mgr_cache->try_eliminate_schema_mgr(eli_schema_mgr))) {
LOG_WARN("fail to eliminate schema_mgr", K(ret), K(tenant_id), K(eli_schema_mgr));
} else if (OB_FAIL(mem_mgr->free_schema_mgr(eli_schema_mgr))) {
LOG_ERROR("free eli schema mgr falied", KR(ret), K(tenant_id));
}
@ -3596,6 +3696,188 @@ int ObMultiVersionSchemaService::try_gc_another_allocator(
return ret;
}
// try to gc current allocator's schema mgr, it can reduce the number of schema mgr in the background
int ObMultiVersionSchemaService::try_gc_current_allocator(
const uint64_t tenant_id,
ObSchemaMemMgr *&mem_mgr,
ObSchemaMgrCache *&schema_mgr_cache)
{
int ret = OB_SUCCESS;
int64_t recycle_interval = GCONF._schema_memory_recycle_interval;
if (!check_inner_stat()) {
ret = OB_INNER_STAT_ERROR;
LOG_WARN("inner stat error", KR(ret));
} else if (OB_ISNULL(mem_mgr) || OB_ISNULL(schema_mgr_cache)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("mem_mgr or schema_mgr_cahe is null",
KR(ret), K(tenant_id), KP(mem_mgr), KP(schema_mgr_cache));
} else if (tenant_id != mem_mgr->get_tenant_id()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tenant_id not match", KR(ret), K(tenant_id), K(mem_mgr->get_tenant_id()));
} else if (0 == recycle_interval) {
// 0 means turn off gc current allocator
LOG_TRACE("_schema_memory_recycle_interval is zero, turn off gc current allocator", K(tenant_id));
} else {
int64_t start_time = ObTimeUtility::current_time();
ObArray<void *> current_ptrs;
int64_t local_version = OB_INVALID_VERSION;
lib::ObMutexGuard guard(schema_refresh_mutex_);
if (OB_FAIL(mem_mgr->get_current_ptrs(current_ptrs))) {
LOG_WARN("fail to get another ptrs", KR(ret), K(tenant_id));
} else if (OB_FAIL(get_tenant_refreshed_schema_version(tenant_id, local_version))) {
LOG_WARN("fail to get local refreshed schema version", KR(ret), K(tenant_id));
} else if (!ObSchemaService::is_formal_version(local_version)) {
LOG_TRACE("ignore to free current allocator when refreshed version is not formal", K(tenant_id), K(local_version));
} else {
int64_t eli_timestamp = 0;
ObSchemaMgr *eli_schema_mgr = NULL;
int64_t eli_schema_version = OB_INVALID_VERSION;
for (int64_t i = 0; OB_SUCC(ret) && i < current_ptrs.count(); i++) {
if (OB_ISNULL(current_ptrs.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ptrs is null", KR(ret), K(tenant_id), K(i));
} else {
eli_schema_mgr = static_cast<ObSchemaMgr *>(current_ptrs.at(i));
eli_timestamp = eli_schema_mgr->get_timestamp_in_slot();
eli_schema_version = eli_schema_mgr->get_schema_version();
if (eli_schema_version >= local_version
|| (recycle_interval > ObClockGenerator::getClock() - eli_timestamp)) {
LOG_TRACE("no need to gc current allocator's schema mgr", K(tenant_id), K(eli_timestamp),
K(eli_schema_version), K(local_version), K(recycle_interval));
} else {
//gc only those that have been put in the slot for more than recycle_interval
LOG_INFO("try to gc current allocator's schema mgr which is in slot",
K(tenant_id), K(eli_schema_version), K(local_version),
K(eli_timestamp), K(recycle_interval));
if (OB_FAIL(schema_mgr_cache->try_eliminate_schema_mgr(eli_schema_mgr))) {
if (OB_EAGAIN == ret) {
// schema mgr in use, just ignore
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to eliminate schema_mgr", KR(ret), K(tenant_id),
K(eli_schema_version), K(eli_timestamp));
}
} else if (OB_FAIL(mem_mgr->free_schema_mgr(eli_schema_mgr))) {
LOG_ERROR("free eli schema mgr falied", KR(ret), K(tenant_id));
}
}
}
}
}
int64_t end_time = ObTimeUtility::current_time();
LOG_INFO("finish gc current allocator's schema mgr which is in slot", KR(ret), K(tenant_id),
"cost_ts", end_time - start_time);
}
return ret;
}
bool ObMultiVersionSchemaService::compare_schema_mgr_info_(
const ObSchemaMgr *lhs,
const ObSchemaMgr *rhs)
{
return lhs->get_schema_version() < rhs->get_schema_version();
}
// need protected by schema_refresh_mutex_
// try to gc current and another allocators' schema mgr, it can reduce the number of schema mgr in the foreground
// 1.reserve_mgr_count can let us reserve the number of total schema mgr
// 2.we can turn this off by set _schema_memory_recycle_interval to zero
int ObMultiVersionSchemaService::try_gc_allocator_when_add_schema_(
const uint64_t tenant_id,
ObSchemaMemMgr *&mem_mgr,
ObSchemaMgrCache *&schema_mgr_cache)
{
int ret = OB_SUCCESS;
ObArray<void *> all_ptrs;
int64_t local_version = OB_INVALID_VERSION;
int64_t reserve_version = OB_INVALID_VERSION;
int64_t start_time = ObTimeUtility::current_time();
const int64_t reserve_mgr_count = RESERVE_SCHEMA_MGR_CNT;
if (OB_ISNULL(mem_mgr) || OB_ISNULL(schema_mgr_cache)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("mem_mgr or schema_mgr_cahe is null",
KR(ret), K(tenant_id), KP(mem_mgr), KP(schema_mgr_cache));
} else if (0 > reserve_mgr_count) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("reserve_mgr_count is less than zero", KR(ret));
} else if (0 == GCONF._schema_memory_recycle_interval) {
// ignore
} else if (OB_FAIL(mem_mgr->get_all_ptrs(all_ptrs))) {
LOG_WARN("fail to get another ptrs", KR(ret), K(tenant_id));
} else if (OB_FAIL(get_tenant_refreshed_schema_version(tenant_id, local_version))) {
LOG_WARN("fail to get local refreshed schema version", KR(ret), K(tenant_id));
} else if (!ObSchemaService::is_formal_version(local_version)) {
LOG_TRACE("ignore to free current allocator when refreshed version is not formal", K(tenant_id), K(local_version));
} else {
SchemaMgrIterator iter;
SchemaMgrInfos schema_mgr_infos;
ObSchemaMgr *eli_schema_mgr = NULL;
for (int64_t i = 0; OB_SUCC(ret) && i < all_ptrs.count(); i++) {
iter = schema_mgr_infos.end();
eli_schema_mgr = static_cast<ObSchemaMgr *>(all_ptrs.at(i));
if (OB_ISNULL(eli_schema_mgr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("eli_schema_mgr is null", KR(ret), K(tenant_id), K(i));
} else if (OB_FAIL(schema_mgr_infos.insert(eli_schema_mgr, iter, compare_schema_mgr_info_))) {
LOG_WARN("fail to insert schema mgr info", KR(ret), K(tenant_id));
}
}
if (OB_FAIL(ret)) {
// ignore
} else if (all_ptrs.count() != schema_mgr_infos.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("all_ptrs and schema_mgr_infos count not equal", KR(ret), K(tenant_id),
K(all_ptrs.count()), K(schema_mgr_infos.count()));
} else {
int64_t schema_mgr_cnt = schema_mgr_infos.count();
int64_t reserve_index = schema_mgr_cnt > reserve_mgr_count ?
schema_mgr_cnt - reserve_mgr_count - 1 : OB_INVALID_INDEX;
// we should skip free schema mgr when schema_mgr_cnt less than reserve_mgr_count
if (reserve_index >= 0 && 0 != schema_mgr_cnt) {
reserve_version = schema_mgr_infos.at(reserve_index)->get_schema_version();
}
}
int64_t eli_schema_version = 0;
int64_t total_schema_ptr_cnt = all_ptrs.count();
int64_t remain_schema_ptr_cnt = all_ptrs.count();
for (int64_t i = 0; OB_SUCC(ret) && i < total_schema_ptr_cnt; i++) {
eli_schema_mgr = static_cast<ObSchemaMgr *>(all_ptrs.at(i));
if (OB_ISNULL(eli_schema_mgr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("eli_schema_mgr is null", KR(ret), K(tenant_id), K(i));
} else {
eli_schema_version = eli_schema_mgr->get_schema_version();
if (eli_schema_version >= local_version
|| eli_schema_version >= reserve_version) {
LOG_TRACE("no need to gc allocator's schema mgr", K(tenant_id),
K(eli_schema_version), K(local_version), K(reserve_version));
} else {
LOG_INFO("try to gc allocator's schema mgr which schema version is less than reserve_version",
K(tenant_id), K(eli_schema_version), K(local_version), K(reserve_version));
if (OB_FAIL(schema_mgr_cache->try_eliminate_schema_mgr(eli_schema_mgr))) {
if (OB_EAGAIN == ret) {
// schema mgr in use, just ignore
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to eliminate schema_mgr", KR(ret), K(tenant_id), K(eli_schema_version));
}
} else if (OB_FAIL(mem_mgr->free_schema_mgr(eli_schema_mgr))) {
LOG_ERROR("free eli schema mgr falied", KR(ret), K(tenant_id));
} else {
remain_schema_ptr_cnt--;
}
}
}
} // for
int64_t end_time = ObTimeUtility::current_time();
LOG_INFO("finish gc allocator's schema mgr when add schema", KR(ret), K(tenant_id),
K(total_schema_ptr_cnt), K(remain_schema_ptr_cnt), K(reserve_version),
"cost_ts", end_time - start_time);
}
return ret;
}
bool ObMultiVersionSchemaService::is_sys_full_schema() const
{
return is_tenant_full_schema(OB_SYS_TENANT_ID);
@ -3861,6 +4143,30 @@ int ObMultiVersionSchemaService::gen_new_schema_version(
return ret;
}
int ObMultiVersionSchemaService::gen_batch_new_schema_versions(
const uint64_t tenant_id,
const int64_t version_cnt,
int64_t &schema_version)
{
int ret = OB_SUCCESS;
int64_t refreshed_schema_version = OB_INVALID_VERSION;
schema_version = OB_INVALID_VERSION;
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id
|| version_cnt < 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid tenant_id", KR(ret), K(tenant_id));
} else if (OB_FAIL(get_tenant_refreshed_schema_version(tenant_id, refreshed_schema_version))) {
LOG_WARN("fail to get refreshed schema version", KR(ret), K(tenant_id), K(refreshed_schema_version));
} else if (OB_ISNULL(schema_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema service is null", KR(ret), K(tenant_id));
} else if (OB_FAIL(schema_service_->gen_batch_new_schema_versions(
tenant_id, refreshed_schema_version, version_cnt, schema_version))) {
LOG_WARN("fail to gen new schema_version", KR(ret), K(tenant_id), K(version_cnt), K(refreshed_schema_version));
}
return ret;
}
int ObMultiVersionSchemaService::get_new_schema_version(uint64_t tenant_id, int64_t &schema_version) {
int ret = OB_SUCCESS;
schema_version = OB_INVALID_VERSION;
@ -4086,6 +4392,36 @@ int ObMultiVersionSchemaService::check_restore_tenant_exist(
return ret;
}
int ObMultiVersionSchemaService::get_tenant_name_case_mode(
const uint64_t tenant_id,
ObNameCaseMode &name_case_mode)
{
int ret = OB_SUCCESS;
ObSchemaGetterGuard guard;
const ObSimpleSysVariableSchema *sys_variable = NULL;
bool is_oracle_mode = false;
name_case_mode = OB_NAME_CASE_INVALID;
if (is_sys_tenant(tenant_id)
|| is_meta_tenant(tenant_id)) {
name_case_mode = OB_ORIGIN_AND_INSENSITIVE;
} else if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(tenant_id, is_oracle_mode))) {
LOG_WARN("fail to check oracle mode", KR(ret), K(tenant_id));
} else if (is_oracle_mode) {
// name_case_mode is incorrent and meaningless for oracle tenant.
name_case_mode = OB_ORIGIN_AND_SENSITIVE;
} else if (OB_FAIL(get_tenant_schema_guard(tenant_id, guard))) {
LOG_WARN("fail to get tenant schema guard", KR(ret), K(tenant_id));
} else if (OB_FAIL(guard.get_sys_variable_schema(tenant_id, sys_variable))) {
LOG_WARN("fail to get sys variable schema", KR(ret), K(tenant_id));
} else if (OB_ISNULL(sys_variable)) {
ret = OB_ENTRY_NOT_EXIST;
LOG_WARN("sys variable not exist", KR(ret), K(tenant_id));
} else {
name_case_mode = sys_variable->get_name_case_mode();
}
return ret;
}
// Obtain the least referenced schema version of the observer for schema recycling
int ObMultiVersionSchemaService::get_recycle_schema_version(
const uint64_t tenant_id,

View File

@ -108,6 +108,8 @@ class ObMultiVersionSchemaService;
class ObSchemaGetterGuard;
class ObMultiVersionSchemaService : public ObServerSchemaService
{
typedef common::ObSortedVector<ObSchemaMgr *> SchemaMgrInfos;
typedef SchemaMgrInfos::iterator SchemaMgrIterator;
public:
static bool g_skip_resolve_materialized_view_definition_;
@ -171,6 +173,12 @@ public:
common::ObKVCacheHandle &handle,
const ObSchema *&schema);
int get_latest_schema(common::ObIAllocator &allocator,
const ObSchemaType schema_type,
const uint64_t tenant_id,
const uint64_t schema_id,
const ObSchema *&schema);
const ObSimpleTenantSchema* get_simple_gts_tenant() const
{
return schema_cache_.get_simple_gts_tenant();
@ -234,6 +242,13 @@ public:
virtual int set_last_refreshed_schema_info(const ObRefreshSchemaInfo &schema_info);
int update_baseline_schema_version(const uint64_t tenant_id, const int64_t baseline_schema_version);
int gen_new_schema_version(uint64_t tenant_id, int64_t &schema_version);
// gen schema versions in [start_version, end_version] with specified schema version cnt.
// @param[out]:
// - schema_version: end_version
int gen_batch_new_schema_versions(
const uint64_t tenant_id,
const int64_t version_cnt,
int64_t &schema_version);
/*----------- check schema interface -----------------*/
bool is_sys_full_schema() const;
@ -324,6 +339,7 @@ public:
bool &is_restore);
int check_restore_tenant_exist(const common::ObIArray<uint64_t> &tenant_ids, bool &exist);
int get_tenant_name_case_mode(const uint64_t tenant_id, ObNameCaseMode &name_case_mode);
/*------------- refresh schema interface -----------------*/
int broadcast_tenant_schema(
const uint64_t tenant_id,
@ -403,6 +419,10 @@ private:
int try_gc_another_allocator(const uint64_t tenant_id,
ObSchemaMemMgr *&mem_mgr,
ObSchemaMgrCache *&schema_mgr_cache);
// try release slot's schema mgr which is in current allocator and without reference
int try_gc_current_allocator(const uint64_t tenant_id,
ObSchemaMemMgr *&mem_mgr,
ObSchemaMgrCache *&schema_mgr_cache);
int get_schema_status(
const common::ObArray<ObRefreshSchemaStatus> &schema_status_array,
@ -453,12 +473,18 @@ private:
ObSchemaMgrCache &schema_mgr_cache);
int switch_allocator_(ObSchemaMemMgr &mem_mgr,
ObSchemaMgr *&latest_schema_mgr);
inline static bool compare_schema_mgr_info_(const ObSchemaMgr *lhs,
const ObSchemaMgr *rhs);
int try_gc_allocator_when_add_schema_(const uint64_t tenant_id,
ObSchemaMemMgr *&mem_mgr,
ObSchemaMgrCache *&schema_mgr_cache);
private:
static const int64_t MAX_VERSION_COUNT = 64;
static const int64_t MAX_VERSION_COUNT_FOR_LIBOBLOG = 6;
static const int32_t MAX_RETRY_TIMES = 10;
static const int64_t RETRY_INTERVAL_US = 1000 * 1000; //1s
static const int64_t DEFAULT_TENANT_SET_SIZE = 64;
static const int64_t RESERVE_SCHEMA_MGR_CNT = 10;
bool init_;
mutable lib::ObMutex schema_refresh_mutex_;//assert only one thread can refresh schema

View File

@ -1539,6 +1539,153 @@ int ObDropIncSubPartHelper::drop_subpartition_info()
return ret;
}
int ObRenameIncPartHelper::rename_partition_info()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(ori_table_) || OB_ISNULL(inc_table_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table is null", KR(ret), KP(ori_table_), KP(inc_table_));
} else if (!ori_table_->is_user_partition_table()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("unsupport behavior on not user partition table", KR(ret), KPC(ori_table_));
} else {
const uint64_t tenant_id = ori_table_->get_tenant_id();
const uint64_t table_id = ori_table_->get_table_id();
const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id);
ObDMLSqlSplicer dml;
ObSqlString part_sql;
ObPartition **part_array = inc_table_->get_part_array();
ObPartition *inc_part = nullptr;
const int64_t inc_part_num = inc_table_->get_partition_num();
int64_t affected_rows = 0;
if (OB_ISNULL(part_array)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inc table part_array is null", KR(ret), KP(inc_table_));
} else if (OB_UNLIKELY(1 != inc_part_num)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inc part num should be 1", KR(ret), K(inc_part_num));
} else if (OB_ISNULL(inc_part = part_array[0])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inc_part is null", KR(ret));
} else if (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, table_id)))
|| OB_FAIL(dml.add_pk_column("part_id", inc_part->get_part_id()))
|| OB_FAIL(dml.add_column("schema_version", schema_version_))
|| OB_FAIL(dml.add_column("part_name", inc_part->get_part_name().ptr()))) {
LOG_WARN("dml add column failed", KR(ret));
} else if (OB_FAIL(dml.splice_update_sql(share::OB_ALL_PART_TNAME, part_sql))) {
LOG_WARN("dml splice update sql failed", KR(ret));
} else if (OB_FAIL(sql_client_.write(exec_tenant_id, part_sql.ptr(), affected_rows))) {
LOG_WARN("fail to execute sql",KR(ret), K(tenant_id), K(part_sql));
} else if (OB_UNLIKELY(inc_part_num != affected_rows)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected affected rows", KR(ret), K(inc_part_num), K(affected_rows));
} else {
ObDMLSqlSplicer history_dml;
ObSqlString part_history_sql;
affected_rows = 0;
HEAP_VAR(ObAddIncPartDMLGenerator, part_dml_gen,
ori_table_, *inc_part, inc_part_num, inc_part->get_part_idx(), schema_version_) {
if (OB_FAIL(part_dml_gen.gen_dml(history_dml))) {
LOG_WARN("gen dml failed", KR(ret));
} else if (OB_FAIL(history_dml.add_column("is_deleted", false))) {
LOG_WARN("add column failed", KR(ret));
} else if (OB_FAIL(history_dml.splice_insert_sql(share::OB_ALL_PART_HISTORY_TNAME,
part_history_sql))) {
LOG_WARN("failed to splice batch insert sql", KR(ret), K(part_history_sql));
} else if (OB_FAIL(sql_client_.write(exec_tenant_id, part_history_sql.ptr(), affected_rows))) {
LOG_WARN("execute sql failed", KR(ret), K(part_history_sql), KPC(inc_part));
} else if (OB_UNLIKELY(inc_part_num != affected_rows)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("history affected_rows is unexpected", KR(ret), K(inc_part_num), K(affected_rows));
}
}
}
}
return ret;
}
int ObRenameIncSubpartHelper::rename_subpartition_info()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(ori_table_) || OB_ISNULL(inc_table_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table is null", KR(ret), KP(ori_table_), KP(inc_table_));
} else if (!ori_table_->is_user_subpartition_table()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("unsupport behavior on not user subpartition table", KR(ret), KPC(ori_table_));
} else {
int64_t tenant_id = ori_table_->get_tenant_id();
const uint64_t table_id = ori_table_->get_table_id();
const uint64_t exec_tenant_id = ObSchemaUtils::get_exec_tenant_id(tenant_id);
ObDMLSqlSplicer dml;
ObSqlString subpart_sql;
ObPartition **part_array = inc_table_->get_part_array();
ObPartition *inc_part = nullptr;
const int64_t inc_part_num = inc_table_->get_partition_num();
if (OB_ISNULL(part_array)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("partition array is null", KR(ret), KP(inc_table_));
} else if (OB_UNLIKELY(1 != inc_part_num)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inc part num should be 1", KR(ret), K(inc_part_num));
} else if (OB_ISNULL(inc_part = part_array[0])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inc part is null", KR(ret));
} else {
ObSubPartition **subpart_array = inc_part->get_subpart_array();
ObSubPartition *inc_subpart = nullptr;
const int64_t inc_subpart_num = inc_part->get_subpartition_num();
int64_t affected_rows = 0;
if (OB_ISNULL(subpart_array)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("subpart_array is null", KR(ret));
} else if (OB_UNLIKELY(1 != inc_subpart_num)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inc subpart num should be 1", KR(ret), K(inc_subpart_num));
} else if (OB_ISNULL(inc_subpart = subpart_array[0])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("inc_subpart is null", KR(ret));
} else if (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, table_id)))
||OB_FAIL(dml.add_pk_column("part_id", inc_part->get_part_id()))
||OB_FAIL(dml.add_pk_column("sub_part_id", inc_subpart->get_sub_part_id()))
||OB_FAIL(dml.add_column("schema_version", schema_version_))
||OB_FAIL(dml.add_column("sub_part_name", inc_subpart->get_part_name().ptr()))) {
LOG_WARN("dml add column failed", KR(ret));
} else if (OB_FAIL(dml.splice_update_sql(share::OB_ALL_SUB_PART_TNAME, subpart_sql))) {
LOG_WARN("dml splice update sql failed", KR(ret));
} else if (OB_FAIL(sql_client_.write(exec_tenant_id, subpart_sql.ptr(), affected_rows))) {
LOG_WARN("fail to execute sql",KR(ret), K(tenant_id), K(subpart_sql));
} else if (OB_UNLIKELY(inc_subpart_num != affected_rows)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected affected rows", KR(ret), K(inc_subpart_num), K(affected_rows));
} else {
ObDMLSqlSplicer history_sub_dml;
ObSqlString subpart_history_sql;
affected_rows = 0;
HEAP_VAR(ObAddIncSubPartDMLGenerator, sub_part_dml_gen,
ori_table_, *inc_part, *inc_subpart, inc_part_num, inc_part->get_part_idx(),
inc_subpart->get_sub_part_idx(), schema_version_) {
if (OB_FAIL(sub_part_dml_gen.gen_dml(history_sub_dml))) {
LOG_WARN("gen dml history failed", KR(ret));
} else if (OB_FAIL(history_sub_dml.add_column("is_deleted", false))) {
LOG_WARN("add column failed", KR(ret));
} else if (OB_FAIL(history_sub_dml.splice_insert_sql(share::OB_ALL_SUB_PART_HISTORY_TNAME,
subpart_history_sql))) {
LOG_WARN("failed to splice insert sql", KR(ret), K(subpart_history_sql));
} else if (OB_FAIL(sql_client_.write(exec_tenant_id,subpart_history_sql.ptr(), affected_rows))) {
LOG_WARN("execute sql failed", KR(ret), K(subpart_history_sql));
} else if (OB_UNLIKELY(inc_subpart_num != affected_rows)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("history affected_rows is unexpected", KR(ret), K(inc_part_num), K(affected_rows));
}
}
}
}
}
return ret;
}
} //end of schema
} //end of share
} //end of oceanbase

View File

@ -406,6 +406,48 @@ private:
DISALLOW_COPY_AND_ASSIGN(ObDropIncSubPartHelper);
};
//rename partition
class ObRenameIncPartHelper
{
public:
ObRenameIncPartHelper(const ObPartitionSchema *ori_table,
const ObPartitionSchema *inc_table,
const int64_t schema_version,
common::ObISQLClient &sql_client)
: ori_table_(ori_table),
inc_table_(inc_table),
schema_version_(schema_version),
sql_client_(sql_client) {}
virtual ~ObRenameIncPartHelper() {}
int rename_partition_info();
private:
const ObPartitionSchema *ori_table_;
const ObPartitionSchema *inc_table_;
int64_t schema_version_;
common::ObISQLClient &sql_client_;
DISALLOW_COPY_AND_ASSIGN(ObRenameIncPartHelper);
};
//rename subpartition
class ObRenameIncSubpartHelper
{
public:
ObRenameIncSubpartHelper(const ObPartitionSchema *ori_table,
const ObPartitionSchema *inc_table,
const int64_t schema_version,
common::ObISQLClient &sql_client)
: ori_table_(ori_table),
inc_table_(inc_table),
schema_version_(schema_version),
sql_client_(sql_client) {}
virtual ~ObRenameIncSubpartHelper() {}
int rename_subpartition_info();
private:
const ObPartitionSchema *ori_table_;
const ObPartitionSchema *inc_table_;
int64_t schema_version_;
common::ObISQLClient &sql_client_;
DISALLOW_COPY_AND_ASSIGN(ObRenameIncSubpartHelper);
};
} //end of schema
} //end of share
} //end of oceanbase

View File

@ -555,19 +555,24 @@ bool ObSchemaCache::need_use_sys_cache(const ObSchemaCacheKey &cache_key) const
{
bool is_need = false;
if (TENANT_SCHEMA == cache_key.schema_type_
&& (OB_SYS_TENANT_ID == cache_key.schema_id_
&& (is_sys_tenant(cache_key.schema_id_)
|| OB_GTS_TENANT_ID == cache_key.schema_id_)) {
is_need = true;
} else if (USER_SCHEMA == cache_key.schema_type_
&& is_sys_tenant(cache_key.tenant_id_)) {
is_need = true;
} else if (TABLE_SCHEMA == cache_key.schema_type_
&& is_inner_table(cache_key.schema_id_)) {
// Ensure cache of full sys table schemas won't be washed while create tenant or bootstrap.
// FIXME:(yanmu.ztl) Recycle user/meta tenant's full table schema when tenant is dropped.
is_need = true;
} else if (TABLE_SCHEMA == cache_key.schema_type_) {
if (is_inner_table(cache_key.schema_id_)) {
if (is_sys_tenant(cache_key.tenant_id_)) {
is_need = true;
} else if (!is_virtual_table(cache_key.schema_id_)
&& !is_sys_view(cache_key.schema_id_)) {
is_need = true;
}
// normal tenant's virtual table and view schema should use kvcache to store
}
} else if (SYS_VARIABLE_SCHEMA == cache_key.schema_type_
&& OB_SYS_TENANT_ID == cache_key.schema_id_) {
&& is_sys_tenant(cache_key.schema_id_)) {
is_need = true;
}
return is_need;
@ -929,7 +934,9 @@ int ObSchemaFetcher::fetch_schema(ObSchemaType schema_type,
} else {
do {
observer::ObUseWeakGuard use_weak_guard;
if (OB_FAIL(schema_service_->can_read_schema_version(schema_status, schema_version))) {
if (INT64_MAX == schema_version) {
// skip inspection while fetch latest schema
} else if (OB_FAIL(schema_service_->can_read_schema_version(schema_status, schema_version))) {
LOG_WARN("incremant schema is not readable now, waiting and retry", K(ret), K(retry_times), K(schema_version));
if (OB_SCHEMA_EAGAIN == ret) {
retry = (retry_times++ < RETRY_TIMES_MAX);

View File

@ -9035,6 +9035,22 @@ int ObSchemaGetterGuard::check_global_index_exist(const uint64_t tenant_id, cons
return ret;
}
int ObSchemaGetterGuard::deep_copy_index_name_map(
common::ObIAllocator &allocator,
ObIndexNameMap &index_name_cache)
{
int ret = OB_SUCCESS;
const ObSchemaMgr *mgr = NULL;
if (OB_FAIL(check_lazy_guard(tenant_id_, mgr))) {
LOG_WARN("fail to check lazy guard", KR(ret), K_(tenant_id));
// const_cast to iterate index_name_map_, mgr won't be changed actually
} else if (OB_FAIL(const_cast<ObSchemaMgr*>(mgr)
->deep_copy_index_name_map(allocator, index_name_cache))) {
LOG_WARN("fail to deep copy index name map", KR(ret), K_(tenant_id));
}
return ret;
}
} //end of namespace schema
} //end of namespace share
} //end of namespace oceanbase

View File

@ -17,7 +17,6 @@
#include "lib/container/ob_se_array.h"
#include "lib/allocator/ob_mod_define.h"
#include "lib/allocator/page_arena.h"
#include "share/cache/ob_kv_storecache.h"
#include "share/schema/ob_schema_mgr_cache.h"
#include "share/schema/ob_package_info.h"
#include "share/schema/ob_routine_info.h"
@ -118,21 +117,6 @@ class ObSchemaGetterGuard
{
friend class ObMultiVersionSchemaService;
friend class MockSchemaService;
struct SchemaObj
{
SchemaObj()
: schema_type_(OB_MAX_SCHEMA),
tenant_id_(common::OB_INVALID_ID),
schema_id_(common::OB_INVALID_ID),
schema_(NULL)
{}
ObSchemaType schema_type_;
uint64_t tenant_id_;
uint64_t schema_id_;
ObSchema *schema_;
common::ObKVCacheHandle handle_;
TO_STRING_KV(K_(schema_type), K_(tenant_id), K_(schema_id), KP_(schema));
};
const static int DEFAULT_RESERVE_SIZE = 2;
typedef common::ObSEArray<SchemaObj, DEFAULT_RESERVE_SIZE> SchemaObjs;
typedef common::ObSEArray<ObSchemaMgrInfo, DEFAULT_RESERVE_SIZE> SchemaMgrInfos;
@ -1050,6 +1034,9 @@ public:
const uint64_t table_id,
const uint64_t column_id,
bool &is_key);
int deep_copy_index_name_map(common::ObIAllocator &allocator,
ObIndexNameMap &index_name_cache);
private:
int check_ssl_access(const ObUserInfo &user_info,
SSL *ssl_st);

View File

@ -27,18 +27,21 @@ void ObSchemaMemory::reset() {
mem_total_ = OB_INVALID_COUNT;
used_schema_mgr_cnt_ = OB_INVALID_COUNT;
free_schema_mgr_cnt_ = OB_INVALID_COUNT;
allocator_idx_ = OB_INVALID_INDEX;
}
void ObSchemaMemory::init(const int64_t pos, const uint64_t &tenant_id,
const int64_t &mem_used, const int64_t &mem_total,
const int64_t &used_schema_mgr_cnt,
const int64_t &free_schema_mgr_cnt) {
const int64_t &free_schema_mgr_cnt,
const int64_t &allocator_idx) {
pos_ = pos;
tenant_id_ = tenant_id;
mem_used_ = mem_used;
mem_total_ = mem_total;
used_schema_mgr_cnt_ = used_schema_mgr_cnt;
free_schema_mgr_cnt_ = free_schema_mgr_cnt;
allocator_idx_ = allocator_idx;
}
namespace share
{
@ -146,6 +149,7 @@ int ObSchemaMemMgr::alloc_schema_mgr(ObSchemaMgr *&schema_mgr, bool alloc_for_li
schema_mgr = new (tmp_ptr) ObSchemaMgr();
} else {
schema_mgr = new (tmp_ptr) ObSchemaMgr(*allocator);
schema_mgr->set_allocator_idx(pos_);
}
if (OB_SUCC(ret) && OB_ISNULL(schema_mgr)) {
ret = OB_ERR_UNEXPECTED;
@ -250,9 +254,10 @@ int ObSchemaMemMgr::free_schema_mgr(ObSchemaMgr *&schema_mgr)
LOG_WARN("turn on error injection ERRSIM_FREE_SCEHMA_MGR", KR(ret));
} else if (OB_NOT_NULL(schema_mgr)) {
const uint64_t tenant_id = schema_mgr->get_tenant_id();
const int64_t timestamp_in_slot = schema_mgr->get_timestamp_in_slot();
const int64_t schema_version = schema_mgr->get_schema_version();
schema_mgr->~ObSchemaMgr();
FLOG_INFO("[SCHEMA_RELEASE] free schema mgr", K(tenant_id), K(schema_version));
FLOG_INFO("[SCHEMA_RELEASE] free schema mgr", K(tenant_id), K(schema_version), K(timestamp_in_slot));
SpinWLockGuard guard(schema_mem_rwlock_);
if (OB_FAIL(free_(static_cast<void *>(schema_mgr)))) {
LOG_ERROR("free schema_mgr failed", KR(ret), K(tenant_id), K(schema_version));
@ -301,7 +306,7 @@ int ObSchemaMemMgr::get_all_alloc_info(common::ObIArray<ObSchemaMemory> &schema_
mem_total = allocator_[pos_].total();
used_schema_mgr_cnt = ptrs_[pos_].count();
free_schema_mgr_cnt = all_ptrs_[pos_] - used_schema_mgr_cnt;
schema_mem.init(0, tenant_id, mem_used, mem_total, used_schema_mgr_cnt, free_schema_mgr_cnt);
schema_mem.init(0, tenant_id, mem_used, mem_total, used_schema_mgr_cnt, free_schema_mgr_cnt, pos_);
if (OB_FAIL(schema_mem_infos.push_back(schema_mem))) {
LOG_WARN("fail to push back schema_mem", KR(ret));
} else {
@ -309,7 +314,7 @@ int ObSchemaMemMgr::get_all_alloc_info(common::ObIArray<ObSchemaMemory> &schema_
mem_total = allocator_[1 - pos_].total();
used_schema_mgr_cnt = ptrs_[1 - pos_].count();
free_schema_mgr_cnt = all_ptrs_[1 - pos_] - used_schema_mgr_cnt;
schema_mem.init(1, tenant_id, mem_used, mem_total, used_schema_mgr_cnt, free_schema_mgr_cnt);
schema_mem.init(1, tenant_id, mem_used, mem_total, used_schema_mgr_cnt, free_schema_mgr_cnt, 1 - pos_);
if (OB_FAIL(schema_mem_infos.push_back(schema_mem))) {
LOG_WARN("fail to push back schema_mem", KR(ret));
}
@ -537,6 +542,7 @@ int ObSchemaMemMgr::try_reset_another_allocator()
int ObSchemaMemMgr::get_another_ptrs(common::ObArray<void *> &ptrs)
{
int ret = OB_SUCCESS;
ptrs.reset();
SpinRLockGuard guard(schema_mem_rwlock_);
if (!check_inner_stat()) {
@ -553,6 +559,52 @@ int ObSchemaMemMgr::get_another_ptrs(common::ObArray<void *> &ptrs)
return ret;
}
int ObSchemaMemMgr::get_current_ptrs(common::ObArray<void *> &ptrs)
{
int ret = OB_SUCCESS;
ptrs.reset();
SpinRLockGuard guard(schema_mem_rwlock_);
if (!check_inner_stat()) {
ret = OB_INNER_STAT_ERROR;
LOG_WARN("inner stat error", KR(ret));
} else {
common::ObArray<void *> &current_ptrs = ptrs_[pos_];
for (int64_t i = 0; OB_SUCC(ret) && i < current_ptrs.count(); i++) {
if (OB_FAIL(ptrs.push_back(current_ptrs.at(i)))) {
LOG_WARN("fail to push back ptr", KR(ret), K(i));
}
}
}
return ret;
}
int ObSchemaMemMgr::get_all_ptrs(common::ObArray<void *> &ptrs)
{
int ret = OB_SUCCESS;
ptrs.reset();
SpinRLockGuard guard(schema_mem_rwlock_);
if (!check_inner_stat()) {
ret = OB_INNER_STAT_ERROR;
LOG_WARN("inner stat error", KR(ret));
} else {
common::ObArray<void *> &current_ptrs = ptrs_[pos_];
common::ObArray<void *> &another_ptrs = ptrs_[1 - pos_];
for (int64_t i = 0; OB_SUCC(ret) && i < current_ptrs.count(); i++) {
if (OB_FAIL(ptrs.push_back(current_ptrs.at(i)))) {
LOG_WARN("fail to push back current ptr", KR(ret), K(i));
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < another_ptrs.count(); i++) {
if (OB_FAIL(ptrs.push_back(another_ptrs.at(i)))) {
LOG_WARN("fail to push back another ptr", KR(ret), K(i));
}
}
}
return ret;
}
} //end of namespace schema
} //end of namespace share
} //end of namespace oceanbase

View File

@ -27,19 +27,22 @@ public:
ObSchemaMemory():pos_(OB_INVALID_INDEX), tenant_id_(OB_INVALID_TENANT_ID),
mem_used_(OB_INVALID_COUNT), mem_total_(OB_INVALID_COUNT),
used_schema_mgr_cnt_(OB_INVALID_COUNT),
free_schema_mgr_cnt_(OB_INVALID_COUNT) {}
free_schema_mgr_cnt_(OB_INVALID_COUNT),
allocator_idx_(OB_INVALID_INDEX) {}
~ObSchemaMemory() {}
void reset();
void init(const int64_t pos, const uint64_t &tenant_id,
const int64_t &mem_used, const int64_t &mem_total,
const int64_t &used_schema_mgr_cnt,
const int64_t &free_schema_mgr_cnt);
const int64_t &free_schema_mgr_cnt,
const int64_t &allocator_idx);
int64_t get_pos() const { return pos_; }
uint64_t get_tenant_id() const { return tenant_id_; }
int64_t get_mem_used() const { return mem_used_; }
int64_t get_mem_total() const { return mem_total_; }
int64_t get_used_schema_mgr_cnt() const { return used_schema_mgr_cnt_; }
int64_t get_free_schema_mgr_cnt() const { return free_schema_mgr_cnt_; }
int64_t get_allocator_idx() const { return allocator_idx_; }
TO_STRING_KV(K_(pos), K_(mem_used), K_(mem_total),
K_(used_schema_mgr_cnt), K_(free_schema_mgr_cnt));
private:
@ -49,6 +52,7 @@ private:
int64_t mem_total_;
int64_t used_schema_mgr_cnt_;
int64_t free_schema_mgr_cnt_;
int64_t allocator_idx_;
};
namespace common
{
@ -79,6 +83,8 @@ public:
uint64_t get_tenant_id() const { return tenant_id_; }
int try_reset_another_allocator();
int get_another_ptrs(common::ObArray<void *> &ptrs);
int get_current_ptrs(common::ObArray<void *> &ptrs);
int get_all_ptrs(common::ObArray<void *> &ptrs);
int get_all_alloc_info(common::ObIArray<ObSchemaMemory> &tenant_mem_infos);
int alloc_schema_mgr(ObSchemaMgr *&schema_mgr, bool alloc_for_liboblog);
int free_schema_mgr(ObSchemaMgr *&schema_mgr);

View File

@ -510,7 +510,9 @@ ObSchemaMgr::ObSchemaMgr()
mock_fk_parent_table_mgr_(allocator_),
rls_policy_mgr_(allocator_),
rls_group_mgr_(allocator_),
rls_context_mgr_(allocator_)
rls_context_mgr_(allocator_),
timestamp_in_slot_(0),
allocator_idx_(OB_INVALID_INDEX)
{
}
@ -562,7 +564,9 @@ ObSchemaMgr::ObSchemaMgr(ObIAllocator &allocator)
mock_fk_parent_table_mgr_(allocator_),
rls_policy_mgr_(allocator_),
rls_group_mgr_(allocator_),
rls_context_mgr_(allocator_)
rls_context_mgr_(allocator_),
timestamp_in_slot_(0),
allocator_idx_(OB_INVALID_INDEX)
{
}
@ -654,6 +658,7 @@ void ObSchemaMgr::reset()
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else {
timestamp_in_slot_ = 0;
schema_version_ = OB_INVALID_VERSION;
is_consistent_ = true;
@ -3538,39 +3543,6 @@ int ObSchemaMgr::remove_aux_table(const ObSimpleTableSchemaV2 &schema_to_del)
return ret;
}
uint64_t ObSchemaMgr::extract_data_table_id_from_index_name(const ObString &index_name) const
{
int64_t pos = 0;
ObString data_table_id_str;
uint64_t data_table_id = OB_INVALID_ID;
if (!index_name.prefix_match(OB_INDEX_PREFIX)) {
LOG_WARN_RET(OB_INVALID_ARGUMENT, "index table name not in valid format", K(index_name));
} else {
pos = strlen(OB_INDEX_PREFIX);
while (NULL != index_name.ptr() &&
isdigit(*(index_name.ptr() + pos)) &&
pos < index_name.length()) {
++pos;
}
if (pos + 1 >= index_name.length()) {
LOG_WARN_RET(OB_INVALID_ARGUMENT, "index table name not in valid format", K(pos), K(index_name), K(index_name.length()));
} else if ('_' != *(index_name.ptr() + pos)) {
LOG_WARN_RET(OB_INVALID_ARGUMENT, "index table name not in valid format", K(pos), K(index_name), K(index_name.length()));
} else {
data_table_id_str.assign_ptr(
index_name.ptr() + strlen(OB_INDEX_PREFIX),
static_cast<ObString::obstr_size_t>(pos) - strlen(OB_INDEX_PREFIX));
int ret = (common_string_unsigned_integer(
0, ObVarcharType, CS_TYPE_UTF8MB4_GENERAL_CI, data_table_id_str, false, data_table_id));
if (OB_FAIL(ret)) {
data_table_id = OB_INVALID_ID;
LOG_WARN("convert string to uint failed", K(ret), K(data_table_id_str), K(index_name));
}
}
}
return data_table_id;
}
int ObSchemaMgr::get_table_schema(
const uint64_t tenant_id,
const uint64_t table_id,
@ -3793,15 +3765,10 @@ int ObSchemaMgr::get_index_schema(
// FIXME: oracle mode not support drop user/database to recyclebin yet, now
// can determine whether the index is in the recycle bin based on database_id
ObString cutted_index_name;
ObSimpleTableSchemaV2 tmp_schema_for_cutting_ind_name;
tmp_schema_for_cutting_ind_name.reset();
tmp_schema_for_cutting_ind_name.set_table_type(USER_INDEX);
uint64_t data_table_id = extract_data_table_id_from_index_name(table_name);
uint64_t data_table_id = ObSimpleTableSchemaV2::extract_data_table_id_from_index_name(table_name);
if (OB_INVALID_ID == data_table_id) {
// nothing to do, need to go on and it will get a empty ptr of dst table_schema
} else if (OB_FAIL(tmp_schema_for_cutting_ind_name.set_table_name(table_name))){
LOG_WARN("fail to set index name", K(ret));
} else if (OB_FAIL(tmp_schema_for_cutting_ind_name.get_index_name(cutted_index_name))) {
} else if (OB_FAIL(ObSimpleTableSchemaV2::get_index_name(table_name, cutted_index_name))) {
if (OB_SCHEMA_ERROR == ret) {
// If the input table_name of the function does not conform to the prefixed index name format of'__idx_DataTableId_IndexName',
// an empty table schema pointer should be returned, and no error should be reported, so reset the error code to OB_SUCCESS
@ -3830,6 +3797,80 @@ int ObSchemaMgr::get_index_schema(
return ret;
}
int ObSchemaMgr::deep_copy_index_name_map(
common::ObIAllocator &allocator,
ObIndexNameMap &index_name_cache)
{
int ret = OB_SUCCESS;
bool is_oracle_mode = false;
if (OB_FAIL(ObCompatModeGetter::check_is_oracle_mode_with_tenant_id(
tenant_id_, is_oracle_mode))) {
LOG_WARN("fail to get tenant mode", KR(ret), K_(tenant_id));
} else {
// index_name_cache will destory or not init, so sub_map_mem_size should be set first
// to reduce dynamic memory allocation and avoid error.
(void) index_name_cache.set_sub_map_mem_size(index_name_map_.get_sub_map_mem_size());
if (OB_FAIL(index_name_cache.init())) {
LOG_WARN("init index name cache failed", KR(ret));
}
}
for (int64_t sub_map_id = 0;
OB_SUCC(ret) && sub_map_id < index_name_map_.get_sub_map_count();
sub_map_id++) {
auto it = index_name_map_.begin(sub_map_id);
auto end = index_name_map_.end(sub_map_id);
for (; OB_SUCC(ret) && it != end; ++it) {
const ObSimpleTableSchemaV2 *index_schema = *it;
void *buf = NULL;
ObIndexNameInfo *index_name_info = NULL;
uint64_t data_table_id = OB_INVALID_ID;
uint64_t database_id = OB_INVALID_ID;
ObString index_name;
if (OB_ISNULL(index_schema)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("index schema is null", KR(ret));
} else if (FALSE_IT(database_id = index_schema->get_database_id())) {
} else if (OB_UNLIKELY(!is_recyclebin_database_id(database_id)
&& index_schema->get_origin_index_name_str().empty())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid index schema", KR(ret), KPC(index_schema));
} else if (OB_ISNULL(buf = allocator.alloc(sizeof(ObIndexNameInfo)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc index name info", KR(ret));
} else if (FALSE_IT(index_name_info = new (buf) ObIndexNameInfo())) {
} else if (OB_FAIL(index_name_info->init(allocator, *index_schema))) {
LOG_WARN("fail to init index name info", KR(ret), KPC(index_schema));
} else if (is_recyclebin_database_id(database_id)) {
data_table_id = OB_INVALID_ID;
index_name = index_name_info->get_index_name();
} else {
data_table_id = (is_oracle_mode && !is_mysql_sys_database_id(database_id)) ?
OB_INVALID_ID : index_name_info->get_data_table_id();
index_name = index_name_info->get_original_index_name();
}
if (OB_SUCC(ret)) {
int overwrite = 0;
ObIndexSchemaHashWrapper index_name_wrapper(index_name_info->get_tenant_id(),
database_id,
data_table_id,
index_name);
if (OB_FAIL(index_name_cache.set_refactored(
index_name_wrapper, index_name_info, overwrite))) {
LOG_WARN("fail to set refactored", KR(ret), KPC(index_name_info));
if (OB_HASH_EXIST == ret) {
ObIndexNameInfo **exist_index_info = index_name_cache.get(index_name_wrapper);
if (OB_NOT_NULL(exist_index_info) && OB_NOT_NULL(*exist_index_info)) {
FLOG_ERROR("duplicated index info exist", KR(ret),
KPC(index_name_info), KPC(*exist_index_info));
}
}
}
}
} // end for
} // end for
return ret;
}
int ObSchemaMgr::get_table_schema(const uint64_t tenant_id,
const uint64_t database_id,
const uint64_t session_id,

View File

@ -804,6 +804,14 @@ public:
// get virtual table id or sys view id
int get_non_sys_table_ids(const uint64_t tenant_id, ObIArray<uint64_t> &non_sys_table_ids) const;
int64_t get_timestamp_in_slot() const { return timestamp_in_slot_; };
void set_timestamp_in_slot(const int64_t timestamp) { timestamp_in_slot_ = timestamp; }
int64_t get_allocator_idx() const { return allocator_idx_; }
void set_allocator_idx(const int64_t allocator_idx) { allocator_idx_ = allocator_idx; }
int deep_copy_index_name_map(
common::ObIAllocator &allocator,
ObIndexNameMap &index_name_cache);
private:
inline bool check_inner_stat() const;
@ -887,7 +895,6 @@ private:
int rebuild_schema_meta_if_not_consistent();
int rebuild_table_hashmap(uint64_t &fk_cnt, uint64_t &cst_cnt);
int rebuild_db_hashmap();
uint64_t extract_data_table_id_from_index_name(const common::ObString &index_name) const;
/*schema statistics*/
int get_tenant_statistics(ObSchemaStatisticsInfo &schema_info) const;
@ -947,6 +954,8 @@ private:
ObRlsPolicyMgr rls_policy_mgr_;
ObRlsGroupMgr rls_group_mgr_;
ObRlsContextMgr rls_context_mgr_;
int64_t timestamp_in_slot_; // when schema mgr put in slot, we will set the timestamp
int64_t allocator_idx_;
};
}//end of namespace schema

View File

@ -29,17 +29,19 @@ void ObSchemaSlot::reset() {
schema_count_ = OB_INVALID_COUNT;
mod_ref_infos_.reset();
ref_cnt_ = OB_INVALID_COUNT;
allocator_idx_ = OB_INVALID_INDEX;
}
void ObSchemaSlot::init(const uint64_t &tenant_id, const int64_t &slot_id,
const int64_t &schema_version, const int64_t &schema_count,
const int64_t &ref_cnt, const common::ObString &str) {
const int64_t &ref_cnt, const common::ObString &str, const int64_t &allocator_idx) {
tenant_id_ = tenant_id;
slot_id_ = slot_id;
schema_version_ = schema_version;
schema_count_ = schema_count;
ref_cnt_ = ref_cnt;
mod_ref_infos_ = str;
allocator_idx_ = allocator_idx;
}
namespace share
@ -348,30 +350,14 @@ static const char* ref_info_type_strs[] = {
"STACK",
"VTABLE_SCAN_PARAM",
"INNER_SQL_RESULT",
"TABLE_API_ROW_ITER",
"STAT_CONV_INFO",
"SHUFFLE_TASK_HANDLE",
"LOAD_DATA_IMPL",
"PX_TASK_PROCESSS",
"TABLE_SCAN",
"DIST_EXECUTER",
"MINI_TASK_BASE",
"REMOTE_EXE",
"CACHED_GUARD",
"UNIQ_CHECK",
"LOGIC_ROW",
"TAILORED_ROW_ITER",
"SSTABLE_MERGE_CTX",
"SSTABLE_SPLIT_CTX",
"RELATIVE_TABLE",
"RECOVER_POINT",
"PART_SCHEMA_RECORDER",
"VIRTUAL_TABLE",
"PHY_RES_STAT",
"TENANT_PT_ITER",
"INDEX_PARAM",
"BACKUP_CHECKER",
"DIS_TASK_SPLITER",
"DAS_CTX",
"SCHEMA_RECORDER",
"SPI_RESULT_SET",
@ -441,6 +427,7 @@ int ObSchemaMgrCache::get_slot_info(common::ObIAllocator &allocator, common::ObI
int64_t schema_version = OB_INVALID_VERSION;
int64_t schema_count = OB_INVALID_COUNT;
int64_t ref_cnt = OB_INVALID_COUNT;
int64_t allocator_idx = OB_INVALID_INDEX;
ObSchemaSlot schema_slot;
ObString tmp_str;
ObString ref_infos;
@ -461,6 +448,7 @@ int ObSchemaMgrCache::get_slot_info(common::ObIAllocator &allocator, common::ObI
slot_id = i;
tenant_id = schema_mgr->get_tenant_id();
schema_version = schema_mgr->get_schema_version();
allocator_idx = schema_mgr->get_allocator_idx();
ref_cnt = schema_mgr_items_[i].ref_cnt_;
mod_ref = schema_mgr_items_[i].mod_ref_cnt_;
if (OB_FAIL(schema_mgr->get_schema_count(schema_count))) {
@ -479,7 +467,7 @@ int ObSchemaMgrCache::get_slot_info(common::ObIAllocator &allocator, common::ObI
}
if (OB_SUCC(ret)) {
schema_slot.init(tenant_id, slot_id, schema_version,
schema_count, ref_cnt, ref_infos);
schema_count, ref_cnt, ref_infos, allocator_idx);
if (OB_FAIL(schema_slot_infos.push_back(schema_slot))) {
LOG_WARN("push back to schema_slot_infos failed", KR(ret), K(tenant_id), K(schema_version));
}
@ -570,8 +558,13 @@ int ObSchemaMgrCache::put(ObSchemaMgr *schema_mgr,
}
} else {
eli_schema_mgr = dst_item->schema_mgr_;
schema_mgr->set_timestamp_in_slot(ObClockGenerator::getClock());
dst_item->schema_mgr_ = schema_mgr;
LOG_INFO("dst schema mgr item ptr", K(dst_item), K(dst_item->schema_mgr_->get_schema_version()), K(target_pos));
uint64_t tenant_id = schema_mgr->get_tenant_id();
int64_t dst_timestamp = schema_mgr->get_timestamp_in_slot();
int64_t dst_schema_version = schema_mgr->get_schema_version();
LOG_INFO("dst schema mgr item ptr", K(tenant_id), K(dst_item),
K(dst_timestamp), K(dst_schema_version), K(target_pos));
(void)ATOMIC_STORE(&dst_item->ref_cnt_, 0);
for (int64_t i = 0; i < ObSchemaMgrItem::MOD_MAX; i++) {
(void)ATOMIC_STORE(&dst_item->mod_ref_cnt_[i], 0);
@ -636,7 +629,7 @@ int ObSchemaMgrCache::try_gc_tenant_schema_mgr(ObSchemaMgr *&eli_schema_mgr)
return ret;
}
int ObSchemaMgrCache::try_elimiante_schema_mgr(ObSchemaMgr *&eli_schema_mgr)
int ObSchemaMgrCache::try_eliminate_schema_mgr(ObSchemaMgr *&eli_schema_mgr)
{
int ret = OB_SUCCESS;
if (!check_inner_stat()) {
@ -658,7 +651,12 @@ int ObSchemaMgrCache::try_elimiante_schema_mgr(ObSchemaMgr *&eli_schema_mgr)
} else if (eli_schema_mgr != tmp_schema_mgr) {
} else if (ATOMIC_LOAD(&schema_mgr_item.ref_cnt_) > 0) {
ret = OB_EAGAIN;
LOG_WARN("schema mgr is in use, try elimiante later", K(ret), K(tmp_schema_mgr));
uint64_t tenant_id = tmp_schema_mgr->get_tenant_id();
int64_t ref_cnt = ATOMIC_LOAD(&schema_mgr_item.ref_cnt_);
int64_t timestamp = tmp_schema_mgr->get_timestamp_in_slot();
int64_t schema_version = tmp_schema_mgr->get_schema_version();
LOG_WARN("schema mgr is in use, try eliminate later", KR(ret), K(tenant_id),
K(ref_cnt), K(schema_version), K(timestamp));
} else {
eli_schema_mgr = tmp_schema_mgr;
schema_mgr_item.schema_mgr_ = NULL;
@ -694,6 +692,7 @@ void ObSchemaMgrCache::dump() const
const ObSchemaMgr *schema_mgr = schema_mgr_item.schema_mgr_;
uint64_t tenant_id = OB_INVALID_TENANT_ID;
int64_t schema_version = OB_INVALID_VERSION;
int64_t timestamp_in_slot = 0;
int64_t schema_count = 0;
int64_t schema_size = 0;
if (OB_NOT_NULL(schema_mgr)) {
@ -704,10 +703,12 @@ void ObSchemaMgrCache::dump() const
ret = OB_SUCC(ret) ? tmp_ret : ret;
tenant_id = schema_mgr->get_tenant_id();
schema_version = schema_mgr->get_schema_version();
timestamp_in_slot = schema_mgr->get_timestamp_in_slot();
total_count += schema_count;
total_size += schema_size;
FLOG_INFO("[SCHEMA_STATISTICS] dump schema_mgr_item", "i", i, K(ret),
K(tenant_id), K(schema_version), K(schema_count), K(schema_size),
K(tenant_id), K(schema_version), K(schema_count),
K(schema_size), K(timestamp_in_slot),
"ref_cnt", schema_mgr_item.ref_cnt_,
"mod_ref_cnt", ObArrayWrap<int64_t>(schema_mgr_item.mod_ref_cnt_,
ObSchemaMgrItem::MOD_MAX));

View File

@ -25,18 +25,19 @@ class ObSchemaSlot
public:
ObSchemaSlot(): tenant_id_(OB_INVALID_TENANT_ID), slot_id_(OB_INVALID_INDEX),
schema_version_(OB_INVALID_VERSION), schema_count_(OB_INVALID_COUNT),
ref_cnt_(OB_INVALID_COUNT), mod_ref_infos_() {}
ref_cnt_(OB_INVALID_COUNT), mod_ref_infos_(), allocator_idx_(OB_INVALID_INDEX) {}
~ObSchemaSlot() {}
void reset();
void init(const uint64_t &tenant_id, const int64_t &slot_id,
const int64_t &schema_version, const int64_t &schema_count,
const int64_t &ref_cnt, const common::ObString &str);
const int64_t &ref_cnt, const common::ObString &str, const int64_t &allocator_idx);
uint64_t get_tenant_id() const { return tenant_id_; }
int64_t get_slot_id() const { return slot_id_; }
int64_t get_schema_version() const { return schema_version_; }
int64_t get_schema_count() const { return schema_count_; }
int64_t get_ref_cnt() const { return ref_cnt_;}
const common::ObString& get_mod_ref_infos() const { return mod_ref_infos_; }
int64_t get_allocator_idx() const { return allocator_idx_; }
TO_STRING_KV(K_(tenant_id), K_(slot_id), K_(schema_version),
K_(schema_count), K_(ref_cnt), K_(mod_ref_infos));
private:
@ -46,6 +47,7 @@ private:
int64_t schema_count_;
int64_t ref_cnt_;
common::ObString mod_ref_infos_;
int64_t allocator_idx_;
};
namespace common
{
@ -63,35 +65,19 @@ struct ObSchemaMgrItem
MOD_STACK = 0,
MOD_VTABLE_SCAN_PARAM = 1,
MOD_INNER_SQL_RESULT = 2,
MOD_TABLE_API_ROW_ITER = 3,
MOD_STAT_CONV_INFO = 4,
MOD_SHUFFLE_TASK_HANDLE = 5,
MOD_LOAD_DATA_IMPL = 6,
MOD_PX_TASK_PROCESSS = 7,
MOD_TABLE_SCAN = 8,
MOD_DIST_EXECUTER = 9,
MOD_MINI_TASK_BASE = 10,
MOD_REMOTE_EXE = 11,
MOD_CACHED_GUARD = 12,
MOD_UNIQ_CHECK = 13,
MOD_LOGIC_ROW = 14,
MOD_TAILORED_ROW_ITER = 15,
MOD_SSTABLE_MERGE_CTX = 16,
MOD_SSTABLE_SPLIT_CTX = 17,
MOD_RELATIVE_TABLE = 18,
MOD_RECOVER_POINT = 19,
MOD_PART_SCHEMA_RECORDER = 20,
MOD_VIRTUAL_TABLE = 21,
MOD_PHY_RES_STAT = 22,
MOD_TENANT_PT_ITER = 23,
MOD_INDEX_PARAM = 24,
MOD_BACKUP_CHECKER = 25,
MOD_DIS_TASK_SPLITER = 26,
MOD_DAS_CTX = 27,
MOD_SCHEMA_RECORDER = 28,
MOD_SPI_RESULT_SET = 29,
MOD_PL_PREPARE_RESULT = 30,
MOD_PARTITION_BALANCE = 31,
MOD_LOAD_DATA_IMPL = 3,
MOD_PX_TASK_PROCESSS = 4,
MOD_REMOTE_EXE = 5,
MOD_CACHED_GUARD = 6,
MOD_UNIQ_CHECK = 7,
MOD_SSTABLE_SPLIT_CTX = 8,
MOD_RELATIVE_TABLE = 9,
MOD_VIRTUAL_TABLE = 10,
MOD_DAS_CTX = 11,
MOD_SCHEMA_RECORDER = 12,
MOD_SPI_RESULT_SET = 13,
MOD_PL_PREPARE_RESULT = 14,
MOD_PARTITION_BALANCE = 15,
MOD_MAX
};
ObSchemaMgrItem()
@ -151,10 +137,10 @@ public:
ObSchemaMgr *&eli_schema_mgr,
ObSchemaMgrHandle *handle = NULL);
int try_gc_tenant_schema_mgr(ObSchemaMgr *&eli_schema_mgr);
int try_elimiante_schema_mgr(ObSchemaMgr *&eli_schema_mgr);
int try_eliminate_schema_mgr(ObSchemaMgr *&eli_schema_mgr);
void dump() const;
public:
const static int64_t MAX_SCHEMA_SLOT_NUM = 8 * 1024L; // 8192
const static int64_t MAX_SCHEMA_SLOT_NUM = 256; // 256
private:
bool check_inner_stat() const;
// need process in wlock

View File

@ -18,6 +18,7 @@
#include "lib/utility/serialization.h"
#include "lib/oblog/ob_log_module.h"
#include "ob_schema_macro_define.h"
#include "share/schema/ob_schema_struct.h"
namespace oceanbase
{
@ -224,6 +225,8 @@ DEFINE_SERIALIZE(AlterTableSchema)
SHARE_SCHEMA_LOG(WARN, "fail to serialize sql_mode_", K(ret));
} else if (OB_FAIL(split_partition_name_.serialize(buf, buf_len, pos))) {
SHARE_SCHEMA_LOG(WARN, "fail to serialize partition_name", K(ret));
} else if (OB_FAIL(new_part_name_.serialize(buf, buf_len, pos))) {
SHARE_SCHEMA_LOG(WARN, "fail to serialize new_part_name", K(ret));
}
return ret;
}
@ -251,6 +254,8 @@ DEFINE_DESERIALIZE(AlterTableSchema)
SHARE_SCHEMA_LOG(WARN, "fail to deserialize sql mode", K(ret));
} else if (OB_FAIL(split_partition_name_.deserialize(buf, data_len, pos))) {
SHARE_SCHEMA_LOG(WARN, "fail to deserialize split_partition_name", K(ret));
} else if (OB_FAIL(new_part_name_.deserialize(buf, data_len, pos))) {
SHARE_SCHEMA_LOG(WARN, "fail to serialize new_part_name", K(ret));
}
return ret;
}
@ -268,6 +273,7 @@ void AlterTableSchema::reset()
split_partition_name_.reset();
split_high_bound_val_.reset();
split_list_row_values_.reset();
new_part_name_.reset();
}
int64_t AlterTableSchema::to_string(char *buf, const int64_t buf_len) const
@ -280,7 +286,8 @@ int64_t AlterTableSchema::to_string(char *buf, const int64_t buf_len) const
K_(origin_database_name),
K_(split_partition_name),
K_(split_high_bound_val),
K_(split_list_row_values));
K_(split_list_row_values),
K_(new_part_name));
J_COMMA();
J_NAME(N_ALTER_TABLE_SCHEMA);
J_COLON();
@ -581,17 +588,19 @@ DEFINE_GET_SERIALIZE_SIZE(AlterTableSchema)
size += alter_option_bitset_.get_serialize_size();
size += serialization::encoded_length_vi64(sql_mode_);
size += split_partition_name_.get_serialize_size();
size += new_part_name_.get_serialize_size();
return size;
}
bool ObSchemaService::is_formal_version(const int64_t schema_version)
{
return schema_version % SCHEMA_VERSION_INC_STEP == 0;
return schema_version % ObSchemaVersionGenerator::SCHEMA_VERSION_INC_STEP == 0;
}
bool ObSchemaService::is_sys_temp_version(const int64_t schema_version)
{
return schema_version % SCHEMA_VERSION_INC_STEP == (SCHEMA_VERSION_INC_STEP - 1);
return schema_version % ObSchemaVersionGenerator::SCHEMA_VERSION_INC_STEP
== (ObSchemaVersionGenerator::SCHEMA_VERSION_INC_STEP - 1);
}
int ObSchemaService::gen_core_temp_version(const int64_t schema_version,

View File

@ -23,6 +23,7 @@
#include "share/ob_define.h"
#include "share/schema/ob_table_schema.h"
#include "share/schema/ob_column_schema.h"
#include "share/schema/ob_routine_info.h"
namespace oceanbase
{
@ -113,6 +114,8 @@ enum ObSchemaOperationCategory
ACT(OB_DDL_SET_INTERVAL, = 57) \
ACT(OB_DDL_INTERVAL_TO_RANGE, = 58) \
ACT(OB_DDL_TRUNCATE_TABLE, = 59) \
ACT(OB_DDL_RENAME_PARTITION, = 60) \
ACT(OB_DDL_RENAME_SUB_PARTITION, = 61) \
ACT(OB_DDL_TABLE_OPERATION_END, = 100) \
ACT(OB_DDL_TENANT_OPERATION_BEGIN, = 101) \
ACT(OB_DDL_ADD_TENANT,) \
@ -542,7 +545,8 @@ public:
sql_mode_(SMO_DEFAULT),
split_partition_name_(),
split_high_bound_val_(),
split_list_row_values_()
split_list_row_values_(),
new_part_name_()
{
}
AlterTableSchema(common::ObIAllocator *allocator)
@ -556,7 +560,8 @@ public:
sql_mode_(SMO_DEFAULT),
split_partition_name_(),
split_high_bound_val_(),
split_list_row_values_()
split_list_row_values_(),
new_part_name_()
{
}
inline const common::ObString &get_origin_table_name() const { return origin_table_name_; }
@ -575,6 +580,8 @@ public:
inline const common::ObRowkey& get_split_list_row_values() const {
return split_list_row_values_;
}
inline const common::ObString &get_new_part_name() const { return new_part_name_; }
inline int set_new_part_name(const common::ObString &new_part_name);
int assign_subpartiton_key_info(const common::ObPartitionKeyInfo& src_info);
int add_alter_column(const AlterColumnSchema &column, const bool need_allocate);
@ -595,6 +602,7 @@ public:
// for tablegroup
common::ObRowkey split_high_bound_val_;
common::ObRowkey split_list_row_values_;
common::ObString new_part_name_;
int assign(const ObTableSchema &src_schema);
//virtual int add_partition(const ObPartition &part);
//virtual int add_subpartition(const ObSubPartition &sub_part);
@ -610,6 +618,11 @@ int AlterTableSchema::set_split_partition_name(const common::ObString &partition
return deep_copy_str(partition_name, split_partition_name_);
}
int AlterTableSchema::set_new_part_name(const common::ObString &new_part_name)
{
return deep_copy_str(new_part_name, new_part_name_);
}
int AlterTableSchema::set_origin_table_name(const common::ObString &origin_table_name)
{
return deep_copy_str(origin_table_name, origin_table_name_);
@ -949,7 +962,11 @@ public:
virtual int fetch_new_directory_id(const uint64_t tenant_id, uint64_t &new_directory_id) = 0;
virtual int fetch_new_normal_rowid_table_tablet_ids(const uint64_t tenant_id, uint64_t &tablet_id, const uint64_t size) = 0;
virtual int fetch_new_extended_rowid_table_tablet_ids(const uint64_t tenant_id, uint64_t &tablet_id, const uint64_t size) = 0;
virtual int fetch_new_tablet_ids(const ObTableSchema &table_schema, uint64_t &tablet_id, const uint64_t size) = 0;
virtual int fetch_new_tablet_ids(
const uint64_t tenant_id,
const bool gen_normal_tablet,
const uint64_t size,
uint64_t &min_tablet_id) = 0;
virtual int fetch_new_context_id(const uint64_t tenant_id, uint64_t &new_context_id) = 0;
virtual int fetch_new_rls_policy_id(const uint64_t tenant_id, uint64_t &new_rls_policy_id) = 0;
virtual int fetch_new_rls_group_id(const uint64_t tenant_id, uint64_t &new_rls_group_id) = 0;
@ -1082,10 +1099,25 @@ public:
const uint64_t table_id,
common::ObISQLClient &sql_client,
ObMockFKParentTableSchema &mock_fk_parent_table_schema) = 0;
virtual int get_audits_in_owner(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const ObSAuditType audit_type,
const uint64_t owner_id,
common::ObIArray<ObSAuditSchema> &audit_schemas) = 0;
virtual void set_refreshed_schema_version(const int64_t schema_version) = 0;
virtual int gen_new_schema_version(uint64_t tenant_id,
int64_t refreshed_schema_version,
virtual int gen_new_schema_version(const uint64_t tenant_id,
const int64_t refreshed_schema_version,
int64_t &schema_version) = 0;
// gen schema versions in [start_version, end_version] with specified schema version cnt.
// @param[out]:
// - schema_version: end_version
virtual int gen_batch_new_schema_versions(
const uint64_t tenant_id,
const int64_t refreshed_schema_version,
const int64_t version_cnt,
int64_t &schema_version) = 0;
virtual int get_new_schema_version(uint64_t tenant_id, int64_t &schema_version) = 0;
virtual int get_ori_schema_version(const ObRefreshSchemaStatus &schema_status,
@ -1143,24 +1175,6 @@ public:
const common::ObString &dblink_name,
bool is_reverse_link,
uint64_t *current_scn) = 0;
// when refresh schema, if new ddl operations are as following:
// (ALTER USER TABLE, v1), (ALTER SYS TABLE, v2),
// if we replay new ddl operation one by one, when we execute sql to read sys table
// to fetch user table schema, leader partition server find sys table version not match,
// read new user table schema will fail, so we need first refresh sys table schema,
// then publish, then refresh new user table schemas and publish,
// but what version we used to publish sys table schema when we haven't refresh use table,
// we use a temporary version which means it don't contain all schema item whose version
// is small than temporary version. now we have temporary core versin for core table,
// temporary system version for system table, we set SCHEMA_VERSION_INC_STEP to 8 so that
// it is enough when we add more temporary version
static const int64_t SCHEMA_VERSION_INC_STEP = 8;
// After support standalone cluster, the generation of the schema version of the standalone cluster depends on
// the schema version of the primary cluster. In order to ensure that the schema version of the primary and standalone cluster
// can be globally and uniquely incremented, the schema version of the primary cluster is set to the second level,
// and the schema version of the standalone cluster is generated. Remove the reference to time
// In millisecond increments
static const int64_t SYS_SCHEMA_VERSION_INC_STEP = 1 * 1000LL;
static bool is_formal_version(const int64_t schema_version);
static bool is_sys_temp_version(const int64_t schema_version);
@ -1171,6 +1185,112 @@ public:
static int alloc_table_schema(const ObTableSchema &table, common::ObIAllocator &allocator,
ObTableSchema *&allocated_table_schema);
/*----------- interfaces for latest schema start -----------*/
virtual int get_tablegroup_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const ObString &tablegroup_name,
uint64_t &tablegroup_id) = 0;
virtual int get_database_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const ObString &database_name,
uint64_t &database_id) = 0;
virtual int get_table_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const uint64_t session_id,
const ObString &table_name,
uint64_t &table_id,
ObTableType &table_type,
int64_t &schema_version) = 0;
virtual int get_index_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &index_name,
uint64_t &index_id) = 0;
virtual int get_mock_fk_parent_table_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &table_name,
uint64_t &mock_fk_parent_table_id) = 0;
virtual int get_synonym_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &synonym_name,
uint64_t &synonym_id) = 0;
virtual int get_constraint_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &constraint_name,
uint64_t &constraint_id) = 0;
virtual int get_foreign_key_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &foreign_key_name,
uint64_t &foreign_key_id) = 0;
virtual int get_sequence_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &sequence_name,
uint64_t &sequence_id,
bool &is_system_generated) = 0;
virtual int get_package_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &package_name,
const ObPackageType package_type,
const int64_t compatible_mode,
uint64_t &package_id) = 0;
virtual int get_routine_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const uint64_t package_id,
const uint64_t overload,
const ObString &routine_name,
common::ObIArray<std::pair<uint64_t, share::schema::ObRoutineType>> &routine_pairs) = 0;
virtual int get_udt_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const uint64_t package_id,
const ObString &udt_name,
uint64_t &udt_id) = 0;
virtual int get_table_schema_versions(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions) = 0;
virtual int get_mock_fk_parent_table_schema_versions(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions) = 0;
/*----------- interfaces for latest schema end -------------*/
};
}//namespace schema
}//namespace share

File diff suppressed because it is too large Load Diff

View File

@ -299,7 +299,11 @@ public:
virtual int fetch_new_directory_id(const uint64_t tenant_id, uint64_t &new_directory_id);
virtual int fetch_new_normal_rowid_table_tablet_ids(const uint64_t tenant_id, uint64_t &tablet_id, const uint64_t size);
virtual int fetch_new_extended_rowid_table_tablet_ids(const uint64_t tenant_id, uint64_t &tablet_id, const uint64_t size);
virtual int fetch_new_tablet_ids(const ObTableSchema &table_schema, uint64_t &tablet_id, const uint64_t size);
virtual int fetch_new_tablet_ids(
const uint64_t tenant_id,
const bool gen_normal_tablet,
const uint64_t size,
uint64_t &min_tablet_id) override;
virtual int fetch_new_context_id(const uint64_t tenant_id, uint64_t &new_context_id);
virtual int fetch_new_rls_policy_id(const uint64_t tenant_id, uint64_t &new_rls_policy_id);
virtual int fetch_new_rls_group_id(const uint64_t tenant_id, uint64_t &new_rls_group_id);
@ -540,6 +544,15 @@ public:
const int64_t refreshed_schema_version,
int64_t &schema_version);
// gen schema versions in [start_version, end_version] with specified schema version cnt.
// @param[out]:
// - schema_version: end_version
virtual int gen_batch_new_schema_versions(
const uint64_t tenant_id,
const int64_t refreshed_schema_version,
const int64_t version_cnt,
int64_t &schema_version);
virtual int get_new_schema_version(uint64_t tenant_id, int64_t &schema_version);
@ -648,6 +661,7 @@ public:
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
int64_t &schema_version);
static int sort_table_partition_info_v2(ObTableSchema &table_schema);
// Get latest schema version from inner table for each table_id.
@ -658,6 +672,128 @@ public:
const uint64_t tenant_id,
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObTableLatestSchemaVersion> &table_schema_versions);
/*----------- interfaces for latest schema start -----------*/
virtual int get_tablegroup_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const ObString &tablegroup_name,
uint64_t &tablegroup_id) override;
virtual int get_database_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const ObString &database_name,
uint64_t &database_id) override;
virtual int get_table_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const uint64_t session_id,
const ObString &table_name,
uint64_t &table_id,
ObTableType &table_type,
int64_t &schema_version) override;
// index_name comparsion:
// 1. oracle & !is_mysql_sys_database : case sensitive
// 2. mysql || is_mysql_sys_database : case insensitive
//
// @param[int]:
// - index_name : should be a "full" index name, can't be a original index name
// @param[out]:
// - index_id : OB_INVALID_ID means that index doesn't exist
virtual int get_index_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &index_name,
uint64_t &index_id) override;
virtual int get_mock_fk_parent_table_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &table_name,
uint64_t &mock_fk_parent_table_id) override;
virtual int get_synonym_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &synonym_name,
uint64_t &synonym_id) override;
virtual int get_constraint_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &constraint_name,
uint64_t &constraint_id) override;
virtual int get_foreign_key_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &foreign_key_name,
uint64_t &foreign_key_id) override;
virtual int get_sequence_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &sequence_name,
uint64_t &sequence_id,
bool &is_system_generated) override;
virtual int get_package_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const ObString &package_name,
const ObPackageType package_type,
const int64_t compatible_mode,
uint64_t &package_id) override;
virtual int get_routine_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const uint64_t package_id,
const uint64_t overload,
const ObString &routine_name,
common::ObIArray<std::pair<uint64_t, share::schema::ObRoutineType>> &routine_pairs) override;
virtual int get_udt_id(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const uint64_t database_id,
const uint64_t package_id,
const ObString &udt_name,
uint64_t &udt_id) override;
virtual int get_table_schema_versions(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions) override;
virtual int get_mock_fk_parent_table_schema_versions(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const common::ObIArray<uint64_t> &table_ids,
common::ObIArray<ObSchemaIdVersion> &versions) override;
virtual int get_audits_in_owner(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const ObSAuditType audit_type,
const uint64_t owner_id,
common::ObIArray<ObSAuditSchema> &audit_schemas) override;
/*----------- interfaces for latest schema end -------------*/
private:
bool check_inner_stat();
int fetch_new_schema_id(const uint64_t tenant_id, const share::ObMaxIdType max_id_type, uint64_t &new_schema_id);
@ -669,7 +805,7 @@ private:
common::ObArray<ObTableSchema> &core_schemas);
int get_core_table_columns(common::ObISQLClient &sql_client,
const ObRefreshSchemaStatus &schema_status,
common::ObArray<ObTableSchema> &core_schemaas);
common::ObArray<ObTableSchema> &core_schemas);
// get schemas of sys tables and user tables, read from schema related core tables
int get_not_core_table_schemas(const ObRefreshSchemaStatus &schema_status,
@ -1003,10 +1139,27 @@ private:
static uint64_t fill_exec_tenant_id(const ObRefreshSchemaStatus &schema_status);
static uint64_t fill_extract_tenant_id(const ObRefreshSchemaStatus &schema_status, const uint64_t tenant_id);
static uint64_t fill_extract_schema_id(const ObRefreshSchemaStatus &schema_status, const uint64_t schema_id);
int gen_leader_sys_schema_version(const int64_t tenant_id, int64_t &schema_version);
int gen_leader_normal_schema_version(const uint64_t tenant_id,
const int64_t refreshed_schema_version,
int64_t &schema_version);
int gen_tenant_new_schema_version_(
const uint64_t tenant_id,
const int64_t refreshed_schema_version,
const int64_t version_cnt,
int64_t &schema_version);
int gen_new_schema_version_(
const int64_t refreshed_schema_version,
const int64_t gen_schema_version,
const int64_t version_cnt,
int64_t &schema_version);
int retrieve_schema_id_with_name_(
common::ObISQLClient &sql_client,
const uint64_t tenant_id,
const ObSqlString &sql,
const char* id_col_name,
const char* name_col_name,
const ObString &schema_name,
const bool case_compare,
uint64_t &schema_id);
int fetch_table_latest_schema_versions_(
common::ObISQLClient &sql_client,

View File

@ -85,6 +85,112 @@ lib::Worker::CompatMode get_worker_compat_mode(const ObCompatibilityMode &mode)
return worker_mode;
}
int ObSchemaIdVersion::init(
const uint64_t schema_id,
const int64_t schema_version)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(OB_INVALID_ID == schema_id
|| schema_version <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("schema_id/schema_version is invalid",
KR(ret), K(schema_id), K(schema_version));
} else {
schema_id_ = schema_id;
schema_version_ = schema_version;
}
return ret;
}
void ObSchemaIdVersion::reset()
{
schema_id_ = OB_INVALID_ID;
schema_version_ = OB_INVALID_VERSION;
}
bool ObSchemaIdVersion::is_valid() const
{
return OB_INVALID_ID != schema_id_ && schema_version_ > 0;
}
int ObSchemaVersionGenerator::init(
const int64_t start_version,
const int64_t end_version)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(start_version <= 0
|| end_version <= 0
|| start_version > end_version)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid version", KR(ret), K(start_version), K(end_version));
} else if (OB_FAIL(ObIDGenerator::init(SCHEMA_VERSION_INC_STEP,
static_cast<uint64_t>(start_version),
static_cast<uint64_t>(end_version)))) {
LOG_WARN("fail to init id generator", KR(ret), K(start_version), K(end_version));
}
return ret;
}
int ObSchemaVersionGenerator::next_version(int64_t &current_version)
{
int ret = OB_SUCCESS;
uint64_t id = OB_INVALID_ID;
if (OB_FAIL(ObIDGenerator::next(id))) {
LOG_WARN("fail to get next id", KR(ret));
} else {
current_version = static_cast<int64_t>(id);
}
return ret;
}
int ObSchemaVersionGenerator::get_start_version(int64_t &start_version) const
{
int ret = OB_SUCCESS;
uint64_t id = OB_INVALID_ID;
if (OB_FAIL(ObIDGenerator::get_start_id(id))) {
LOG_WARN("fail to get start id", KR(ret));
} else {
start_version = static_cast<int64_t>(id);
}
return ret;
}
int ObSchemaVersionGenerator::get_current_version(int64_t &current_version) const
{
int ret = OB_SUCCESS;
uint64_t id = OB_INVALID_ID;
if (OB_FAIL(ObIDGenerator::get_current_id(id))) {
LOG_WARN("fail to get current id", KR(ret));
} else {
current_version = static_cast<int64_t>(id);
}
return ret;
}
int ObSchemaVersionGenerator::get_end_version(int64_t &end_version) const
{
int ret = OB_SUCCESS;
uint64_t id = OB_INVALID_ID;
if (OB_FAIL(ObIDGenerator::get_end_id(id))) {
LOG_WARN("fail to get end id", KR(ret));
} else {
end_version = static_cast<int64_t>(id);
}
return ret;
}
int ObSchemaVersionGenerator::get_version_cnt(int64_t &version_cnt) const
{
int ret = OB_SUCCESS;
uint64_t id_cnt = OB_INVALID_ID;
if (OB_FAIL(ObIDGenerator::get_id_cnt(id_cnt))) {
LOG_WARN("fail to get id cnt", KR(ret));
} else {
version_cnt = static_cast<int64_t>(id_cnt);
}
return ret;
}
uint64_t ObSysTableChecker::TableNameWrapper::hash() const
{
uint64_t hash_ret = 0;
@ -4430,6 +4536,105 @@ int ObPartitionSchema::mock_list_partition_array()
return ret;
}
int ObPartitionSchema::get_partition_by_name(const ObString &name, const ObPartition *&part) const
{
int ret = OB_SUCCESS;
part = nullptr;
const ObPartitionLevel part_level = this->get_part_level();
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
ObPartitionSchemaIter iter(*this, check_partition_mode);
ObPartitionSchemaIter::Info info;
if (PARTITION_LEVEL_ZERO == part_level) {
ret = OB_UNKNOWN_PARTITION;
LOG_WARN("could not get partition on nonpartitioned table", KR(ret), K(part_level));
} else {
while (OB_SUCC(ret)) {
if (OB_FAIL(iter.next_partition_info(info))) {
if (OB_ITER_END == ret) {
ret = OB_UNKNOWN_PARTITION;
LOG_WARN("could not find the partition by given name", KR(ret), K(name), KPC(this));
} else {
LOG_WARN("unexpected erro happened when get partition by name", KR(ret));
}
} else if (OB_ISNULL(info.part_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("info.part_ is null", KR(ret), KPC(this));
} else if (ObCharset::case_insensitive_equal(name, info.part_->get_part_name())) {
part = info.part_;
break;
}
}
}
return ret;
}
int ObPartitionSchema::get_subpartition_by_name(const ObString &name, const ObPartition *&part, const ObSubPartition *&subpart) const
{
int ret = OB_SUCCESS;
part = nullptr;
subpart = nullptr;
const ObPartitionLevel part_level = this->get_part_level();
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
ObPartitionSchemaIter iter(*this, check_partition_mode);
ObPartitionSchemaIter::Info info;
if (PARTITION_LEVEL_TWO != part_level) {
ret = OB_UNKNOWN_SUBPARTITION;
LOG_WARN("could not get subpartition on not composite partition table", KR(ret), K(part_level));
} else {
while (OB_SUCC(ret)) {
if (OB_FAIL(iter.next_partition_info(info))) {
if (OB_ITER_END == ret) {
//subpart not exist errno is same with the part right now
ret = OB_UNKNOWN_SUBPARTITION;
LOG_WARN("could not find the subpartition by given name", KR(ret), K(name), KPC(this));
} else {
LOG_WARN("unexpected erro happened when get subpartition by name", KR(ret));
}
} else if (OB_ISNULL(info.part_) || OB_ISNULL(info.partition_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("info.part_ or info.partition_ is null", KR(ret), KP(info.part_), KP(info.partition_), KPC(this));
} else if (ObCharset::case_insensitive_equal(name, info.partition_->get_part_name())) {
part = info.part_;
subpart = reinterpret_cast<const ObSubPartition*>(info.partition_);
break;
}
}
}
return ret;
}
int ObPartitionSchema::check_partition_duplicate_with_name(const ObString &name) const
{
int ret = OB_SUCCESS;
ObCheckPartitionMode check_partition_mode = CHECK_PARTITION_MODE_NORMAL;
ObPartitionSchemaIter iter(*this, check_partition_mode);
ObPartitionSchemaIter::Info info;
const ObPartitionLevel part_level = this->get_part_level();
if (PARTITION_LEVEL_ZERO == part_level) {
//nonpartitioned tabel doesn't have any partition
} else {
while (OB_SUCC(ret)) {
if (OB_FAIL(iter.next_partition_info(info))) {
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
break;
} else {
LOG_WARN("unexpected erro happened when get check partition duplicate with name", KR(ret));
}
} else if (OB_ISNULL(info.part_) || OB_ISNULL(info.partition_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("info.part_ is null", KR(ret), KP(info.part_), KP(info.partition_), KPC(this));
} else if (ObCharset::case_insensitive_equal(name, info.part_->get_part_name())
|| ObCharset::case_insensitive_equal(name, info.partition_->get_part_name())) {
ret = OB_DUPLICATE_OBJECT_NAME_EXIST;
LOG_WARN("there is a partition or subpartition have the same name", KR(ret), KPC(info.part_), KPC(info.partition_));
break;
}
}
}
return ret;
}
/*-------------------------------------------------------------------------------------------------
* ------------------------------ObTablegroupSchema-------------------------------------------
----------------------------------------------------------------------------------------------------*/
@ -6010,6 +6215,56 @@ int ObPartition::preserve_subpartition(const int64_t &capacity) {
return ret;
}
int ObPartition::get_normal_subpartition_index_by_id(const int64_t subpart_id,
int64_t &subpartition_index) const
{
int ret = OB_SUCCESS;
subpartition_index = OB_INVALID_INDEX;
bool finded = false;
if (OB_INVALID_ID == subpart_id) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("subpart_id is invalid", KR(ret), K(subpart_id));
} else {
if (OB_ISNULL(subpartition_array_) || OB_UNLIKELY(subpartition_num_ <= 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid subpartition array", KR(ret), K(subpartition_array_), K(subpartition_num_));
}
for (int64_t i = 0; !finded && i < subpartition_num_ && OB_SUCC(ret); i++) {
if (OB_ISNULL(subpartition_array_[i])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid subpartition", KR(ret), K(i));
} else if (subpart_id == subpartition_array_[i]->get_sub_part_id()) {
subpartition_index = i;
finded = true;
}
}
}
if (OB_SUCC(ret) && !finded) {
ret = OB_ENTRY_NOT_EXIST;
LOG_WARN("fail to find subpartition index", KR(ret), K(subpart_id));
}
return ret;
}
int ObPartition::get_normal_subpartition_by_subpartition_index(const int64_t subpartition_index,
const ObSubPartition *&subpartition) const
{
int ret = OB_SUCCESS;
subpartition = nullptr;
const int64_t subpart_num = subpartition_num_;
if (0 <= subpartition_index && subpart_num > subpartition_index) {
if (OB_ISNULL(get_subpart_array())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("subpartition array is null", KR(ret), K(subpartition_index));
} else {
subpartition = get_subpart_array()[subpartition_index];
}
} else {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid subpartition index", KR(ret), K(subpartition_index));
}
return ret;
}
ObSubPartition::ObSubPartition()
: ObBasePartition()
{
@ -9005,6 +9260,38 @@ ObTableType get_inner_table_type_by_id(const uint64_t tid) {
return type;
}
bool is_mysql_tmp_table(const ObTableType table_type)
{
return ObTableType::TMP_TABLE == table_type;
}
bool is_view_table(const ObTableType table_type)
{
return ObTableType::USER_VIEW == table_type
|| ObTableType::SYSTEM_VIEW == table_type
|| ObTableType::MATERIALIZED_VIEW == table_type;
}
bool is_index_table(const ObTableType table_type)
{
return ObTableType::USER_INDEX == table_type;
}
bool is_aux_lob_meta_table(const ObTableType table_type)
{
return ObTableType::AUX_LOB_META == table_type;
}
bool is_aux_lob_piece_table(const ObTableType table_type)
{
return ObTableType::AUX_LOB_PIECE == table_type;
}
bool is_aux_lob_table(const ObTableType table_type)
{
return is_aux_lob_meta_table(table_type) || is_aux_lob_piece_table(table_type);
}
const char *schema_type_str(const ObSchemaType schema_type)
{
const char *str = "";
@ -11107,8 +11394,8 @@ ObForeignKeyInfo::ObForeignKeyInfo(ObIAllocator *allocator)
foreign_key_id_(common::OB_INVALID_ID),
child_table_id_(common::OB_INVALID_ID),
parent_table_id_(common::OB_INVALID_ID),
child_column_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
parent_column_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
child_column_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
parent_column_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
update_action_(ACTION_INVALID),
delete_action_(ACTION_INVALID),
foreign_key_name_(),
@ -12621,8 +12908,8 @@ ObMockFKParentTableSchema::ObMockFKParentTableSchema()
ObMockFKParentTableSchema::ObMockFKParentTableSchema(ObIAllocator *allocator)
: ObSimpleMockFKParentTableSchema(allocator),
foreign_key_infos_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
column_array_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator))
foreign_key_infos_(SCHEMA_BIG_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
column_array_(SCHEMA_MID_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator))
{
reset();
}
@ -13482,6 +13769,83 @@ int ObForeignKeyInfo::get_parent_column_id(const uint64_t child_column_id, uint6
return ret;
}
ObIndexSchemaHashWrapper GetIndexNameKey<ObIndexSchemaHashWrapper, ObIndexNameInfo*>::operator()(
const ObIndexNameInfo *index_name_info) const
{
if (OB_NOT_NULL(index_name_info)) {
bool is_oracle_mode = false;
if (OB_UNLIKELY(OB_SUCCESS != ObCompatModeGetter::check_is_oracle_mode_with_table_id(
index_name_info->get_tenant_id(), index_name_info->get_index_id(), is_oracle_mode))) {
ObIndexSchemaHashWrapper null_wrap;
return null_wrap;
} else if (is_recyclebin_database_id(index_name_info->get_database_id())) {
ObIndexSchemaHashWrapper index_schema_hash_wrapper(
index_name_info->get_tenant_id(),
index_name_info->get_database_id(),
common::OB_INVALID_ID,
index_name_info->get_index_name());
return index_schema_hash_wrapper;
} else {
ObIndexSchemaHashWrapper index_schema_hash_wrapper(
index_name_info->get_tenant_id(),
index_name_info->get_database_id(),
is_oracle_mode ? common::OB_INVALID_ID : index_name_info->get_data_table_id(),
index_name_info->get_original_index_name());
return index_schema_hash_wrapper;
}
} else {
ObIndexSchemaHashWrapper null_wrap;
return null_wrap;
}
}
ObIndexNameInfo::ObIndexNameInfo()
: tenant_id_(OB_INVALID_TENANT_ID),
database_id_(OB_INVALID_ID),
data_table_id_(OB_INVALID_ID),
index_id_(OB_INVALID_ID),
index_name_(),
original_index_name_()
{
}
void ObIndexNameInfo::reset()
{
tenant_id_ = OB_INVALID_TENANT_ID;
database_id_ = OB_INVALID_ID;
data_table_id_ = OB_INVALID_ID;
index_id_ = OB_INVALID_ID;
index_name_.reset();
original_index_name_.reset();
}
int ObIndexNameInfo::init(
common::ObIAllocator &allocator,
const share::schema::ObSimpleTableSchemaV2 &index_schema)
{
int ret = OB_SUCCESS;
reset();
const bool c_style = true;
if (OB_FAIL(ob_write_string(allocator,
index_schema.get_table_name_str(), index_name_, c_style))) {
LOG_WARN("fail to write string", KR(ret), K(index_schema));
} else {
tenant_id_ = index_schema.get_tenant_id();
database_id_ = index_schema.get_database_id();
data_table_id_ = index_schema.get_data_table_id();
index_id_ = index_schema.get_table_id();
// use shallow copy to reduce memory allocation
if (is_recyclebin_database_id(database_id_)) {
original_index_name_ = index_name_;
} else {
if (OB_FAIL(ObSimpleTableSchemaV2::get_index_name(index_name_, original_index_name_))) {
LOG_WARN("fail to generate index name", KR(ret), K_(index_name));
}
}
}
return ret;
}
//
//
} //namespace schema

View File

@ -29,6 +29,8 @@
#include "share/ob_priv_common.h"
#include "lib/worker.h"
#include "objit/common/ob_item_type.h"
#include "share/ob_share_util.h" // ObIDGenerator
#include "share/cache/ob_kv_storecache.h" // ObKVCacheHandle
#include "lib/hash/ob_pointer_hashmap.h"
#include "lib/string/ob_sql_string.h"
@ -49,6 +51,7 @@ class ObIAllocator;
class ObSqlString;
class ObString;
class ObDataTypeCastParams;
class ObKVCacheHandle;
}
namespace sql
{
@ -137,6 +140,12 @@ static const uint64_t OB_MIN_ID = 0;//used for lower_bound
// table_flags stored in __all_table.table_flag
#define CASCADE_RLS_OBJECT_FLAG (INT64_C(1) << 0)
// schema array size
static const int64_t SCHEMA_SMALL_MALLOC_BLOCK_SIZE = 64;
static const int64_t SCHEMA_MALLOC_BLOCK_SIZE = 128;
static const int64_t SCHEMA_MID_MALLOC_BLOCK_SIZE = 256;
static const int64_t SCHEMA_BIG_MALLOC_BLOCK_SIZE = 1024;
//-------enum defenition
enum ObTableLoadType
{
@ -259,6 +268,13 @@ const char *ob_mysql_table_type_str(ObTableType type);
ObTableType get_inner_table_type_by_id(const uint64_t tid);
bool is_mysql_tmp_table(const ObTableType table_type);
bool is_view_table(const ObTableType table_type);
bool is_index_table(const ObTableType table_type);
bool is_aux_lob_meta_table(const ObTableType table_type);
bool is_aux_lob_piece_table(const ObTableType table_type);
bool is_aux_lob_table(const ObTableType table_type);
enum ObIndexType
{
INDEX_TYPE_IS_NOT = 0,//is not index table
@ -408,6 +424,54 @@ public:
};
class ObSchemaIdVersion
{
public:
ObSchemaIdVersion()
: schema_id_(common::OB_INVALID_ID),
schema_version_(common::OB_INVALID_VERSION)
{}
~ObSchemaIdVersion() {}
int init(const uint64_t schema_id, const int64_t schema_version);
void reset();
bool is_valid() const;
uint64_t get_schema_id() const { return schema_id_; }
int64_t get_schema_version() const { return schema_version_; }
TO_STRING_KV(K_(schema_id), K_(schema_version));
private:
uint64_t schema_id_;
int64_t schema_version_;
};
class ObSchemaVersionGenerator : public ObIDGenerator
{
public:
// when refresh schema, if new ddl operations are as following:
// (ALTER USER TABLE, v1), (ALTER SYS TABLE, v2),
// if we replay new ddl operation one by one, when we execute sql to read sys table
// to fetch user table schema, leader partition server find sys table version not match,
// read new user table schema will fail, so we need first refresh sys table schema,
// then publish, then refresh new user table schemas and publish,
// but what version we used to publish sys table schema when we haven't refresh use table,
// we use a temporary version which means it don't contain all schema item whose version
// is small than temporary version. now we have temporary core versin for core table,
// temporary system version for system table, we set SCHEMA_VERSION_INC_STEP to 8 so that
// it is enough when we add more temporary version
static const int64_t SCHEMA_VERSION_INC_STEP = 8;
public:
ObSchemaVersionGenerator()
: ObIDGenerator(SCHEMA_VERSION_INC_STEP) {}
virtual ~ObSchemaVersionGenerator() {}
int init(const int64_t start_version, const int64_t end_version);
int next_version(int64_t &current_version);
int get_start_version(int64_t &start_version) const;
int get_current_version(int64_t &current_version) const;
int get_end_version(int64_t &end_version) const;
int get_version_cnt(int64_t &version_cnt) const;
};
typedef ObSchemaVersionGenerator TSISchemaVersionGenerator;
struct ObRefreshSchemaInfo
{
OB_UNIS_VERSION(1);
@ -511,12 +575,6 @@ inline bool is_index_local_storage(ObIndexType index_type)
|| INDEX_TYPE_SPATIAL_GLOBAL_LOCAL_STORAGE == index_type;
}
inline bool is_aux_lob_table(const ObTableType &table_type)
{
return AUX_LOB_META == table_type
|| AUX_LOB_PIECE == table_type;
}
inline bool is_related_table(
const ObTableType &table_type,
const ObIndexType &index_type)
@ -690,6 +748,8 @@ typedef enum {
RLS_POLICY_SCHEMA = 36,
RLS_GROUP_SCHEMA = 37,
RLS_CONTEXT_SCHEMA = 38,
CONSTRAINT_SCHEMA = 39, // not dependent schema
FOREIGN_KEY_SCHEMA = 40, // not dependent schema
///<<< add schema type before this line
OB_MAX_SCHEMA
} ObSchemaType;
@ -1196,7 +1256,6 @@ public:
inline int get_err_ret() const { return error_ret_; }
protected:
static const int64_t STRING_ARRAY_EXTEND_CNT = 7;
static const int64_t SCHEMA_MALLOC_BLOCK_SIZE = 128;
void *alloc(const int64_t size);
void free(void *ptr);
int deep_copy_str(const char *src, common::ObString &dest);
@ -1265,6 +1324,23 @@ int ObSchema::set_charset_and_collation_options(common::ObCharsetType src_charse
return ret;
}
struct SchemaObj
{
SchemaObj()
: schema_type_(OB_MAX_SCHEMA),
tenant_id_(common::OB_INVALID_ID),
schema_id_(common::OB_INVALID_ID),
schema_(NULL),
handle_()
{}
ObSchemaType schema_type_;
uint64_t tenant_id_;
uint64_t schema_id_;
ObSchema *schema_;
common::ObKVCacheHandle handle_;
TO_STRING_KV(K_(schema_type), K_(tenant_id), K_(schema_id), KP_(schema));
};
class ObLocality
{
OB_UNIS_VERSION(1);
@ -1958,7 +2034,7 @@ public:
int64_t get_schema_version() const
{ return schema_version_; }
int set_part_name(common::ObString &part_name)
int set_part_name(const common::ObString &part_name)
{ return deep_copy_str(part_name, name_); }
const common::ObString &get_part_name() const
{ return name_; }
@ -2081,6 +2157,10 @@ public:
ObSubPartition **get_hidden_subpart_array() const { return hidden_subpartition_array_; }
int64_t get_hidden_subpartition_num() const { return hidden_subpartition_num_; }
int preserve_subpartition(const int64_t &capacity);
int get_normal_subpartition_by_subpartition_index(const int64_t subpartition_index,
const ObSubPartition *&partition) const;
int get_normal_subpartition_index_by_id(const int64_t subpart_id,
int64_t &subpartition_index) const;
INHERIT_TO_STRING_KV(
"BasePartition", ObBasePartition,
@ -2433,6 +2513,29 @@ public:
// only used for generate part_name
int get_max_part_id(int64_t &part_id) const;
int get_max_part_idx(int64_t &part_idx) const;
//@param[in] name: the partition name which you want to get partition by
//@param[out] part: the partition get by the name, when this function could not find the partition
// by the name, this param would be nullptr
//@param[ret] when this function traversal all the partition but could not find the partition by name,
// the ret would be OB_UNKNOWN_PARTITION.
//note this function would only check partition
int get_partition_by_name(const ObString &name, const ObPartition *&part) const;
//@param[in] name: the subpartition name which you want to get subpartition by
//@param[out] part: the partition that the subpartition get by the name belongs to, when this function could not
// find the subpartition by the name, this param would be nullptr
// subpart: the subpartition get by the name, when this function could not find the subpartition
// by the name, this param would be nullptr
//@param[ret] when this function traversal all the subpartition but could not find the subpartition by name,
// the ret would be OB_UNKNOWN_SUBPARTITION.
//note this function would only check subpartition
int get_subpartition_by_name(const ObString &name, const ObPartition *&part, const ObSubPartition *&subpart) const;
//@param[in] name: the partition or subpartition name you want to check
// whether there is already a partition or subpartition have the same name
//@param[ret] when this function traversal all partitions and subpartitions and find the name is duplicate with
// existed (sub)partition it will return OB_DUPLICATE_OBJECT_EXIST
//note this function would check both partitions and subpartitions
int check_partition_duplicate_with_name(const ObString &name) const;
protected:
int inner_add_partition(const ObPartition &part);
template<class T>
@ -6210,8 +6313,8 @@ public:
uint64_t foreign_key_id_;
uint64_t child_table_id_;
uint64_t parent_table_id_;
common::ObSEArray<uint64_t, 8> child_column_ids_;
common::ObSEArray<uint64_t, 8> parent_column_ids_;
common::ObSEArray<uint64_t, 4> child_column_ids_;
common::ObSEArray<uint64_t, 4> parent_column_ids_;
ObReferenceAction update_action_;
ObReferenceAction delete_action_;
common::ObString foreign_key_name_;
@ -6523,6 +6626,7 @@ private:
common::ObString encrypted_key_;
};
static const char* IDENTITY_COLUMN_SEQUENCE_OBJECT_NAME_PREFIX = "ISEQ$$_";
class ObSequenceSchema: public ObSchema
{
OB_UNIS_VERSION(1);
@ -8115,6 +8219,52 @@ private:
bool is_deleted_;
};
class ObIndexNameInfo
{
public:
ObIndexNameInfo();
~ObIndexNameInfo() {}
void reset();
int init(common::ObIAllocator &allocator,
const share::schema::ObSimpleTableSchemaV2 &index_schema);
uint64_t get_tenant_id() const { return tenant_id_; }
uint64_t get_database_id() const { return database_id_; }
uint64_t get_data_table_id() const { return data_table_id_; }
uint64_t get_index_id() const { return index_id_; }
const ObString& get_index_name() const { return index_name_; }
const ObString& get_original_index_name() const { return original_index_name_; }
TO_STRING_KV(K_(tenant_id), K_(database_id),
K_(data_table_id), K_(index_id),
K_(index_name), K_(original_index_name));
private:
uint64_t tenant_id_;
uint64_t database_id_;
uint64_t data_table_id_;
uint64_t index_id_;
ObString index_name_;
ObString original_index_name_;
DISALLOW_COPY_AND_ASSIGN(ObIndexNameInfo);
};
template<class K, class V>
struct GetIndexNameKey
{
void operator()(const K &k, const V &v)
{
UNUSED(k);
UNUSED(v);
}
};
template<>
struct GetIndexNameKey<ObIndexSchemaHashWrapper, ObIndexNameInfo*>
{
ObIndexSchemaHashWrapper operator()(const ObIndexNameInfo *index_name_info) const;
};
typedef common::hash::ObPointerHashMap<ObIndexSchemaHashWrapper, ObIndexNameInfo*, GetIndexNameKey, 1024> ObIndexNameMap;
}//namespace schema
}//namespace share
}//namespace oceanbase

View File

@ -492,6 +492,47 @@ int ObSchemaUtils::construct_inner_table_schemas(
}
return ret;
}
int ObSchemaUtils::try_check_parallel_ddl_schema_in_sync(
const ObTimeoutCtx &ctx,
const uint64_t tenant_id,
const int64_t schema_version,
const int64_t consensus_timeout)
{
int ret = OB_SUCCESS;
int64_t start_time = ObTimeUtility::current_time();
ObMultiVersionSchemaService *schema_service = NULL;
if (OB_UNLIKELY(OB_INVALID_TENANT_ID == tenant_id
|| schema_version <= 0
|| consensus_timeout < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid arg", KR(ret), K(tenant_id), K(schema_version), K(consensus_timeout));
} else if (OB_ISNULL(schema_service = GCTX.schema_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema_service is null", KR(ret));
}
while (OB_SUCC(ret) && ctx.get_timeout() > 0) {
int64_t refreshed_schema_version = OB_INVALID_VERSION;
int64_t consensus_schema_version = OB_INVALID_VERSION;
if (OB_FAIL(schema_service->get_tenant_refreshed_schema_version(tenant_id, refreshed_schema_version))) {
LOG_WARN("get refreshed schema_version fail", KR(ret), K(tenant_id));
} else if (OB_FAIL(schema_service->get_tenant_broadcast_consensus_version(tenant_id, consensus_schema_version))) {
LOG_WARN("get consensus schema_version fail", KR(ret), K(tenant_id));
} else if (refreshed_schema_version >= schema_version
&& consensus_schema_version >= schema_version) {
break;
} else if (refreshed_schema_version >= schema_version
&& ObTimeUtility::current_time() - start_time >= consensus_timeout) {
break;
} else {
if (REACH_TIME_INTERVAL(1000 * 1000L)) { // 1s
LOG_WARN("schema version not sync", K(tenant_id),
K(refreshed_schema_version), K(consensus_schema_version), K(schema_version));
}
ob_usleep(10 * 1000L); // 10ms
}
}
return ret;
}
int ObSchemaUtils::batch_get_latest_table_schemas(
common::ObISQLClient &sql_client,

View File

@ -120,6 +120,7 @@ public:
uint64_t tenant_id,
uint64_t data_table_id,
ObIArray<ObTableSchema> &table_schemas);
// Optimized method to batch get latest table schemas from cache or inner_table automatically.
//
// @param[in] sql_client: ObISQLClient
@ -151,6 +152,12 @@ public:
const uint64_t tenant_id,
const ObObjectID &table_id,
ObSimpleTableSchemaV2 *&table_schema);
static int try_check_parallel_ddl_schema_in_sync(
const ObTimeoutCtx &ctx,
const uint64_t tenant_id,
const int64_t schema_version,
const int64_t consensus_timeout);
private:
static int get_tenant_variable(schema::ObSchemaGetterGuard &schema_guard,
uint64_t tenant_id,

View File

@ -861,6 +861,7 @@ public:
int64_t get_table_count() const;
//the schema service should be thread safe
ObSchemaService *get_schema_service(void) const;
common::ObMySQLProxy *get_sql_proxy(void) const { return sql_proxy_; }
void dump_schema_manager() const;
// public utils

View File

@ -57,8 +57,8 @@ public:
OB_INLINE int64_t get_column_count() const { return columns_.count(); }
OB_INLINE const Columns &get_columns() const { return columns_; }
OB_INLINE const ColumnMap &get_col_map() const { return col_map_; }
OB_INLINE bool is_index_table() const { return ObTableSchema::is_index_table(table_type_); }
OB_INLINE bool is_lob_meta_table() const { return ObTableSchema::is_aux_lob_meta_table(table_type_); }
OB_INLINE bool is_index_table() const { return share::schema::is_index_table(table_type_); }
OB_INLINE bool is_lob_meta_table() const { return share::schema::is_aux_lob_meta_table(table_type_); }
OB_INLINE bool is_materialized_view() const
{ return ObTableSchema::is_materialized_view(table_type_); }
OB_INLINE bool is_storage_index_table() const

View File

@ -108,8 +108,8 @@ ObSimpleTableSchemaV2::ObSimpleTableSchemaV2()
ObSimpleTableSchemaV2::ObSimpleTableSchemaV2(ObIAllocator *allocator)
: ObPartitionSchema(allocator),
simple_foreign_key_info_array_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
simple_constraint_info_array_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator))
simple_foreign_key_info_array_(SCHEMA_MID_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
simple_constraint_info_array_(SCHEMA_MID_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator))
{
reset();
}
@ -1415,22 +1415,22 @@ ObTableSchema::ObTableSchema()
ObTableSchema::ObTableSchema(ObIAllocator *allocator)
: ObSimpleTableSchemaV2(allocator),
view_schema_(allocator),
base_table_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
depend_table_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
simple_index_infos_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
aux_vp_tid_array_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
base_table_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
depend_table_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
simple_index_infos_(SCHEMA_MID_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
aux_vp_tid_array_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
rowkey_info_(allocator),
shadow_rowkey_info_(allocator),
index_info_(allocator),
partition_key_info_(allocator),
subpartition_key_info_(allocator),
foreign_key_infos_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
label_se_column_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
trigger_list_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
depend_mock_fk_parent_table_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
rls_policy_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
rls_group_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator)),
rls_context_ids_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(*allocator))
foreign_key_infos_(SCHEMA_BIG_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
label_se_column_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
trigger_list_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
depend_mock_fk_parent_table_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
rls_policy_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
rls_group_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator)),
rls_context_ids_(SCHEMA_SMALL_MALLOC_BLOCK_SIZE, ModulePageAllocator(*allocator))
{
reset();
}
@ -5774,6 +5774,40 @@ int ObSimpleTableSchemaV2::get_index_name(const ObString &table_name, ObString &
return ret;
}
uint64_t ObSimpleTableSchemaV2::extract_data_table_id_from_index_name(const ObString &index_name)
{
int64_t pos = 0;
ObString data_table_id_str;
uint64_t data_table_id = OB_INVALID_ID;
if (!index_name.prefix_match(OB_INDEX_PREFIX)) {
LOG_WARN_RET(OB_INVALID_ARGUMENT, "index table name not in valid format", K(index_name));
} else {
pos = strlen(OB_INDEX_PREFIX);
while (NULL != index_name.ptr() &&
isdigit(*(index_name.ptr() + pos)) &&
pos < index_name.length()) {
++pos;
}
if (pos + 1 >= index_name.length()) {
LOG_WARN_RET(OB_INVALID_ARGUMENT, "index table name not in valid format", K(pos), K(index_name), K(index_name.length()));
} else if ('_' != *(index_name.ptr() + pos)) {
LOG_WARN_RET(OB_INVALID_ARGUMENT, "index table name not in valid format", K(pos), K(index_name), K(index_name.length()));
} else {
data_table_id_str.assign_ptr(
index_name.ptr() + strlen(OB_INDEX_PREFIX),
static_cast<ObString::obstr_size_t>(pos) - strlen(OB_INDEX_PREFIX));
int ret = (common_string_unsigned_integer(
0, ObVarcharType, CS_TYPE_UTF8MB4_GENERAL_CI, data_table_id_str, false, data_table_id));
if (OB_FAIL(ret)) {
data_table_id = OB_INVALID_ID;
LOG_WARN("convert string to uint failed", KR(ret), K(data_table_id_str), K(index_name));
}
}
}
return data_table_id;
}
int ObSimpleTableSchemaV2::generate_origin_index_name()
{
int ret = OB_SUCCESS;

View File

@ -263,6 +263,8 @@ public:
{
return (ObViewColumnFilledFlag)((table_mode >> TM_VIEW_COLUMN_FILLED_OFFSET) & VIEW_COLUMN_FILLED_MASK);
}
inline bool is_user_hidden_table() const
{ return TABLE_STATE_IS_HIDDEN_MASK & state_flag_; }
TO_STRING_KV("table_mode_flag", mode_flag_,
"pk_mode", pk_mode_,
"table_state_flag", state_flag_,
@ -560,7 +562,7 @@ public:
inline bool is_offline_ddl_table() const
{ return TABLE_STATE_IS_DDL_MASK & table_mode_.state_flag_; }
inline bool is_user_hidden_table() const
{ return TABLE_STATE_IS_HIDDEN_MASK & table_mode_.state_flag_; }
{ return table_mode_.is_user_hidden_table(); }
inline bool is_offline_ddl_original_table() const
{ return is_offline_ddl_table() && !is_user_hidden_table(); }
inline bool check_can_do_ddl() const
@ -709,6 +711,8 @@ public:
static int get_index_name(Allocator &allocator, uint64_t table_id,
const common::ObString &src, common::ObString &dst);
static int get_index_name(const common::ObString &src, common::ObString &dst);
static uint64_t extract_data_table_id_from_index_name(const common::ObString &index_name);
int generate_origin_index_name();
virtual int check_if_oracle_compat_mode(bool &is_oracle_mode) const;
// interface derived
@ -720,26 +724,23 @@ public:
inline bool is_sys_view() const { return share::schema::ObTableType::SYSTEM_VIEW == table_type_; }
inline bool is_storage_index_table() const override { return is_index_table() || is_materialized_view(); }
inline static bool is_storage_index_table(share::schema::ObTableType table_type)
{ return is_index_table(table_type) || is_materialized_view(table_type);}
{ return share::schema::is_index_table(table_type) || is_materialized_view(table_type);}
inline bool is_storage_local_index_table() const { return is_index_local_storage() || is_materialized_view(); }
inline bool is_user_table() const { return share::schema::ObTableType::USER_TABLE == table_type_; }
inline bool is_sys_table() const { return share::schema::ObTableType::SYSTEM_TABLE == table_type_; }
inline bool is_vir_table() const { return share::schema::ObTableType::VIRTUAL_TABLE == table_type_; }
inline bool is_view_table() const { return share::schema::ObTableType::USER_VIEW == table_type_ || share::schema::ObTableType::SYSTEM_VIEW == table_type_ || share::schema::ObTableType::MATERIALIZED_VIEW == table_type_; }
inline bool is_index_table() const { return is_index_table(table_type_); }
inline static bool is_index_table(share::schema::ObTableType table_type) { return share::schema::ObTableType::USER_INDEX == table_type; }
inline bool is_tmp_table() const { return share::schema::ObTableType::TMP_TABLE == table_type_ || share::schema::ObTableType::TMP_TABLE_ORA_SESS == table_type_ || share::schema::ObTableType::TMP_TABLE_ORA_TRX == table_type_; }
inline bool is_view_table() const { return share::schema::is_view_table(table_type_); }
inline bool is_index_table() const { return share::schema::is_index_table(table_type_); }
inline bool is_tmp_table() const { return is_mysql_tmp_table() || share::schema::ObTableType::TMP_TABLE_ORA_SESS == table_type_ || share::schema::ObTableType::TMP_TABLE_ORA_TRX == table_type_; }
inline bool is_ctas_tmp_table() const { return 0 != session_id_ && !is_tmp_table(); }
inline bool is_mysql_tmp_table() const { return share::schema::ObTableType::TMP_TABLE == table_type_; }
inline bool is_mysql_tmp_table() const { return share::schema::is_mysql_tmp_table(table_type_); }
inline bool is_oracle_tmp_table() const { return share::schema::ObTableType::TMP_TABLE_ORA_SESS == table_type_ || share::schema::ObTableType::TMP_TABLE_ORA_TRX == table_type_; }
inline bool is_oracle_sess_tmp_table() const { return share::schema::ObTableType::TMP_TABLE_ORA_SESS == table_type_; }
inline bool is_oracle_trx_tmp_table() const { return share::schema::ObTableType::TMP_TABLE_ORA_TRX == table_type_; }
virtual inline bool is_aux_vp_table() const override { return share::schema::ObTableType::AUX_VERTIAL_PARTITION_TABLE == table_type_; }
inline bool is_aux_lob_piece_table() const { return share::schema::ObTableType::AUX_LOB_PIECE == table_type_; }
inline static bool is_aux_lob_meta_table(share::schema::ObTableType table_type)
{ return schema::ObTableType::AUX_LOB_META == table_type; }
inline bool is_aux_lob_meta_table() const { return share::schema::ObTableType::AUX_LOB_META == table_type_; }
inline bool is_aux_lob_table() const { return schema::is_aux_lob_table(table_type_); }
inline bool is_aux_lob_piece_table() const { return share::schema::is_aux_lob_piece_table(table_type_); }
inline bool is_aux_lob_meta_table() const { return share::schema::is_aux_lob_meta_table(table_type_); }
inline bool is_aux_lob_table() const { return is_aux_lob_meta_table() || is_aux_lob_piece_table(); }
inline bool is_aux_table() const { return share::schema::ObTableType::USER_INDEX == table_type_ || share::schema::ObTableType::AUX_VERTIAL_PARTITION_TABLE == table_type_ || share::schema::ObTableType::AUX_LOB_PIECE == table_type_ || share::schema::ObTableType::AUX_LOB_META == table_type_; }
// Primary partition table judgment: still USER_TABLE, but data_table_id_ is the same as itself,
// the default data_table_id_ is 0
@ -816,6 +817,7 @@ public:
inline bool get_in_offline_ddl_white_list() const { return in_offline_ddl_white_list_; }
inline bool has_rowid() const { return is_user_table() || is_tmp_table(); }
inline bool gen_normal_tablet() const { return has_rowid() && !is_extended_rowid_mode(); }
DECLARE_VIRTUAL_TO_STRING;
protected:

View File

@ -489,6 +489,52 @@ int ObTableSqlService::drop_inc_all_sub_partition(common::ObISQLClient &sql_clie
return ret;
}
int ObTableSqlService::rename_inc_part_info(
ObISQLClient &sql_client,
const ObTableSchema &table_schema,
const ObTableSchema &inc_table_schema,
const int64_t new_schema_version)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_ddl_allowed(table_schema))) {
LOG_WARN("check ddl allowed failed", KR(ret), K(table_schema));
} else if (!table_schema.is_user_table()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("unsupport behavior on not user table", KR(ret), K(table_schema));
} else {
const ObPartitionSchema *table_schema_ptr = &table_schema;
const ObPartitionSchema *inc_table_schema_ptr = &inc_table_schema;
ObRenameIncPartHelper rename_part_helper(table_schema_ptr, inc_table_schema_ptr, new_schema_version, sql_client);
if (OB_FAIL(rename_part_helper.rename_partition_info())) {
LOG_WARN("fail to rename partition", KR(ret), KPC(table_schema_ptr), KPC(inc_table_schema_ptr));
}
}
return ret;
}
int ObTableSqlService::rename_inc_subpart_info(
ObISQLClient &sql_client,
const ObTableSchema &table_schema,
const ObTableSchema &inc_table_schema,
const int64_t new_schema_version)
{
int ret = OB_SUCCESS;
if (OB_FAIL(check_ddl_allowed(table_schema))) {
LOG_WARN("check ddl allowed failed", KR(ret), K(table_schema));
} else if (!table_schema.is_user_table()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("unsupport behavior on not user table", KR(ret), K(table_schema));
} else {
const ObPartitionSchema *table_schema_ptr = &table_schema;
const ObPartitionSchema *inc_table_schema_ptr = &inc_table_schema;
ObRenameIncSubpartHelper rename_subpart_helper(table_schema_ptr, inc_table_schema_ptr, new_schema_version, sql_client);
if (OB_FAIL(rename_subpart_helper.rename_subpartition_info())) {
LOG_WARN("fail to rename partition", KR(ret), KPC(table_schema_ptr), KPC(inc_table_schema_ptr));
}
}
return ret;
}
int ObTableSqlService::drop_inc_part_info(
ObISQLClient &sql_client,
const ObTableSchema &table_schema,

View File

@ -154,6 +154,15 @@ public:
const ObTableSchema &ori_table,
const ObTableSchema &inc_table,
const int64_t schema_version);
int rename_inc_part_info(common::ObISQLClient &sql_client,
const ObTableSchema &ori_table,
const ObTableSchema &inc_table,
const int64_t schema_version);
int rename_inc_subpart_info(common::ObISQLClient &sql_client,
const ObTableSchema &ori_table,
const ObTableSchema &inc_table,
const int64_t schema_version);
int drop_inc_part_info(
common::ObISQLClient &sql_client,
const ObTableSchema &ori_table,

View File

@ -54,6 +54,7 @@
#include "share/external_table/ob_external_table_file_rpc_processor.h"
#include "share/external_table/ob_external_table_utils.h"
#include "share/ob_debug_sync.h"
#include "share/schema/ob_schema_utils.h"
namespace oceanbase
{
using namespace common;
@ -499,14 +500,17 @@ int ObCreateTableExecutor::execute(ObExecContext &ctx, ObCreateTableStmt &stmt)
ObSelectStmt *select_stmt = stmt.get_sub_select();
ObTableSchema &table_schema = create_table_arg.schema_;
ObSQLSessionInfo *my_session = ctx.get_my_session();
uint64_t tenant_id = table_schema.get_tenant_id();
uint64_t data_version = 0;
if (OB_ISNULL(my_session)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session is null", K(ret));
} else if (OB_FAIL(stmt.get_first_stmt(first_stmt))) {
LOG_WARN("get first statement failed", K(ret));
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
LOG_WARN("fail to get data version", KR(ret), K(tenant_id));
} else if (table_schema.is_duplicate_table()) {
bool is_compatible = false;
uint64_t tenant_id = table_schema.get_tenant_id();
if (OB_FAIL(ObShareUtil::check_compat_version_for_readonly_replica(tenant_id, is_compatible))) {
LOG_WARN("fail to check data version for duplicate table", KR(ret), K(tenant_id));
} else if (!is_compatible) {
@ -526,6 +530,13 @@ int ObCreateTableExecutor::execute(ObExecContext &ctx, ObCreateTableStmt &stmt)
create_table_arg.is_inner_ = my_session->is_inner();
create_table_arg.consumer_group_id_ = THIS_WORKER.get_group_id();
const_cast<obrpc::ObCreateTableArg&>(create_table_arg).ddl_stmt_str_ = first_stmt;
bool enable_parallel_create_table = false;
{
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id));
enable_parallel_create_table = tenant_config.is_valid()
&& tenant_config->_enable_parallel_table_creation;
}
if (OB_ISNULL(task_exec_ctx = GET_TASK_EXECUTOR_CTX(ctx))) {
ret = OB_NOT_INIT;
LOG_WARN("get task executor context failed", K(ret));
@ -541,26 +552,49 @@ int ObCreateTableExecutor::execute(ObExecContext &ctx, ObCreateTableStmt &stmt)
LOG_WARN("common rpc proxy should not be null", K(ret));
} else if (OB_ISNULL(select_stmt)) { // 普通建表的处理
if (OB_FAIL(ctx.get_sql_ctx()->schema_guard_->reset())){
LOG_WARN("schema_guard reset failed", K(ret));
} else if (OB_FAIL(common_rpc_proxy->create_table(create_table_arg, res))) {
LOG_WARN("rpc proxy create table failed", K(ret), "dst", common_rpc_proxy->get_server());
} else {
if (table_schema.is_external_table()) {
//auto refresh after create external table
OZ (ObAlterTableExecutor::update_external_file_list(
table_schema.get_tenant_id(), res.table_id_,
table_schema.get_external_file_location(),
table_schema.get_external_file_location_access_info(),
table_schema.get_external_file_pattern(),
ctx));
LOG_WARN("schema_guard reset failed", KR(ret));
} else if (table_schema.is_view_table()
|| data_version < DATA_VERSION_4_2_1_0
|| !enable_parallel_create_table) {
if (OB_FAIL(common_rpc_proxy->create_table(create_table_arg, res))) {
LOG_WARN("rpc proxy create table failed", KR(ret), "dst", common_rpc_proxy->get_server());
}
} else {
int64_t start_time = ObTimeUtility::current_time();
ObTimeoutCtx ctx;
if (OB_FAIL(ObShareUtil::set_default_timeout_ctx(ctx, GCONF._ob_ddl_timeout))) {
LOG_WARN("fail to set timeout ctx", KR(ret));
} else if (OB_FAIL(common_rpc_proxy->parallel_create_table(create_table_arg, res))) {
LOG_WARN("rpc proxy create table failed", KR(ret), "dst", common_rpc_proxy->get_server());
} else {
int64_t refresh_time = ObTimeUtility::current_time();
if (OB_FAIL(ObSchemaUtils::try_check_parallel_ddl_schema_in_sync(ctx,
tenant_id, res.schema_version_, GCONF._wait_interval_after_truncate))) {
LOG_WARN("fail to check paralleld ddl schema in sync", KR(ret), K(res));
}
int64_t end_time = ObTimeUtility::current_time();
LOG_INFO("[parallel_create_table]", KR(ret),
"cost", end_time - start_time,
"execute_time", refresh_time - start_time,
"wait_schema", end_time - refresh_time,
"table_name", create_table_arg.schema_.get_table_name());
}
}
if (OB_SUCC(ret) && table_schema.is_external_table()) {
//auto refresh after create external table
OZ (ObAlterTableExecutor::update_external_file_list(
table_schema.get_tenant_id(), res.table_id_,
table_schema.get_external_file_location(),
table_schema.get_external_file_location_access_info(),
table_schema.get_external_file_pattern(),
ctx));
}
} else {
if (table_schema.is_external_table()) {
ret = OB_NOT_SUPPORTED;
LOG_USER_ERROR(OB_NOT_SUPPORTED, "create external table as select");
} else if (OB_FAIL(execute_ctas(ctx, stmt, common_rpc_proxy))){ // 查询建表的处理
LOG_WARN("execute create table as select failed", K(ret));
LOG_WARN("execute create table as select failed", KR(ret));
}
}
@ -1902,6 +1936,8 @@ int ObAlterTableExecutor::check_alter_partition(ObExecContext &ctx,
}
} else if (obrpc::ObAlterTableArg::DROP_PARTITION == arg.alter_part_type_
|| obrpc::ObAlterTableArg::DROP_SUB_PARTITION == arg.alter_part_type_
|| obrpc::ObAlterTableArg::RENAME_PARTITION == arg.alter_part_type_
|| obrpc::ObAlterTableArg::RENAME_SUB_PARTITION == arg.alter_part_type_
|| obrpc::ObAlterTableArg::TRUNCATE_PARTITION == arg.alter_part_type_
|| obrpc::ObAlterTableArg::TRUNCATE_SUB_PARTITION == arg.alter_part_type_) {
// do-nothing
@ -2173,25 +2209,9 @@ int ObTruncateTableExecutor::execute(ObExecContext &ctx, ObTruncateTableStmt &st
} else if (!res.is_valid()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("truncate invalid ddl_res", KR(ret), K(res));
} else {
// wait schema_version refreshed on this server
while (OB_SUCC(ret) && ctx.get_timeout() > 0) {
int64_t refreshed_schema_version = OB_INVALID_VERSION;
int64_t consensus_schema_version = OB_INVALID_VERSION;
if (OB_FAIL(GCTX.schema_service_->get_tenant_refreshed_schema_version(res.tenant_id_, refreshed_schema_version))) {
LOG_WARN("get refreshed schema_version fail", KR(ret), K(res.tenant_id_));
} else if (OB_FAIL(GCTX.schema_service_->get_tenant_broadcast_consensus_version(res.tenant_id_, consensus_schema_version))) {
LOG_WARN("get consensus schema_version fail", KR(ret), K(res.tenant_id_));
} else if (refreshed_schema_version >= res.task_id_
&& consensus_schema_version >= res.task_id_) {
break;
} else if (refreshed_schema_version >= res.task_id_
&& ObTimeUtility::current_time() - step_time >= GCONF._wait_interval_after_truncate) {
break;
} else {
ob_usleep(10 * 1000);
}
}
} else if (OB_FAIL(ObSchemaUtils::try_check_parallel_ddl_schema_in_sync(ctx,
tenant_id, res.task_id_, GCONF._wait_interval_after_truncate))) {
LOG_WARN("fail to check parallel ddl schema in sync", KR(ret), K(res));
}
int64_t end_time = ObTimeUtility::current_time();
LOG_INFO("truncate_table_v2", KR(ret), "cost", end_time-start_time,

View File

@ -2391,6 +2391,157 @@ int ObAlterTableResolver::check_subpart_name(const ObPartition &partition,
}
return ret;
}
int ObAlterTableResolver::resolve_rename_partition(const ParseNode &node,
const share::schema::ObTableSchema &orig_table_schema)
{
int ret = OB_SUCCESS;
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
uint64_t tenant_data_version = 0;
ObAlterTableStmt *alter_table_stmt = get_alter_table_stmt();
if (T_ALTER_PARTITION_RENAME != node.type_
|| OB_ISNULL(node.children_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid parse tree", KR(ret));
} else if (OB_UNLIKELY(2 != node.num_child_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid parse tree, num child != 2", KR(ret), K(node.num_child_));
} else if (OB_ISNULL(session_info_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session_info is null", KR(ret), KP(this));
} else if (OB_FAIL(GET_MIN_DATA_VERSION(session_info_->get_effective_tenant_id(), tenant_data_version))) {
LOG_WARN("get data version failed", KR(ret), K(session_info_->get_effective_tenant_id()));
} else if (tenant_data_version < DATA_VERSION_4_2_1_0) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("cluster version and feature mismatch", KR(ret));
} else if (!orig_table_schema.is_user_table()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("unsupport behavior on not user table", KR(ret), K(orig_table_schema));
} else if (PARTITION_LEVEL_ZERO == part_level) {
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
LOG_USER_ERROR(OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED);
LOG_WARN("unsupport management on non partitioned table", KR(ret), K(orig_table_schema));
} else if (OB_ISNULL(alter_table_stmt)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("alter table stmt should not be null", KR(ret));
} else {
AlterTableSchema &alter_table_schema =
alter_table_stmt->get_alter_table_arg().alter_table_schema_;
const ObPartition *part = nullptr;
ObPartition inc_part;
ObString origin_partition_name(static_cast<int32_t>(node.children_[0]->str_len_),
node.children_[0]->str_value_);
ObString new_partition_name(static_cast<int32_t>(node.children_[1]->str_len_),
node.children_[1]->str_value_);
const ObPartitionOption &ori_part_option = orig_table_schema.get_part_option();
if (OB_UNLIKELY(ObCharset::case_insensitive_equal(origin_partition_name, new_partition_name))) {
ret = OB_ERR_RENAME_PARTITION_NAME_DUPLICATE;
LOG_USER_ERROR(OB_ERR_RENAME_PARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
LOG_WARN("origin part name equal to new part name", KR(ret), K(origin_partition_name), K(new_partition_name));
} else if (OB_FAIL(orig_table_schema.check_partition_duplicate_with_name(new_partition_name))) {
if (OB_DUPLICATE_OBJECT_NAME_EXIST == ret) {
ret = OB_ERR_RENAME_PARTITION_NAME_DUPLICATE;
LOG_USER_ERROR(OB_ERR_RENAME_PARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
LOG_WARN("new part name duplicate with existed partition", KR(ret), K(new_partition_name));
} else {
LOG_WARN("check new part name duplicate failed", KR(ret), K(new_partition_name));
}
} else if (OB_FAIL(orig_table_schema.get_partition_by_name(origin_partition_name, part))) {
LOG_WARN("get part by name failed", KR(ret), K(origin_partition_name), K(orig_table_schema));
} else if (OB_FAIL(alter_table_schema.set_new_part_name(new_partition_name))) {
LOG_WARN("table set new partition name failed", KR(ret), K(alter_table_schema), K(new_partition_name));
} else if (OB_FAIL(inc_part.set_part_name(origin_partition_name))) {
LOG_WARN("inc part set name failed", KR(ret), K(inc_part), K(origin_partition_name));
} else if (OB_FAIL(alter_table_schema.add_partition(inc_part))) {
LOG_WARN("alter table add inc part failed", KR(ret), K(alter_table_schema), K(inc_part));
} else if (OB_FAIL(alter_table_schema.get_part_option().assign(ori_part_option))) {
LOG_WARN("alter table set part option failed", KR(ret), K(alter_table_schema), K(ori_part_option));
} else {
alter_table_schema.get_part_option().set_part_num(alter_table_schema.get_partition_num());
alter_table_schema.set_part_level(part_level);
}
}
return ret;
}
int ObAlterTableResolver::resolve_rename_subpartition(const ParseNode &node,
const share::schema::ObTableSchema &orig_table_schema)
{
int ret = OB_SUCCESS;
const ObPartitionLevel part_level = orig_table_schema.get_part_level();
uint64_t tenant_data_version = 0;
ObAlterTableStmt *alter_table_stmt = get_alter_table_stmt();
if (T_ALTER_SUBPARTITION_RENAME != node.type_
|| OB_ISNULL(node.children_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid parse tree", KR(ret));
} else if (OB_UNLIKELY(2 != node.num_child_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid parse tree, num child != 2",KR(ret), K(node.num_child_));
} else if (OB_ISNULL(session_info_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("session_info is null", KR(ret), KP(this));
} else if (OB_FAIL(GET_MIN_DATA_VERSION(session_info_->get_effective_tenant_id(), tenant_data_version))) {
LOG_WARN("get data version failed", KR(ret), K(session_info_->get_effective_tenant_id()));
} else if (tenant_data_version < DATA_VERSION_4_2_1_0) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("cluster version and feature mismatch", KR(ret));
} else if (!orig_table_schema.is_user_table()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("unsupport behavior on not user table", KR(ret), K(orig_table_schema));
} else if (PARTITION_LEVEL_ZERO == part_level) {
ret = OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED;
LOG_USER_ERROR(OB_ERR_PARTITION_MGMT_ON_NONPARTITIONED);
LOG_WARN("unsupport management on not partition table", KR(ret));
} else if (PARTITION_LEVEL_ONE == part_level) {
ret = OB_ERR_NOT_COMPOSITE_PARTITION;
LOG_USER_ERROR(OB_ERR_NOT_COMPOSITE_PARTITION);
LOG_WARN("unsupport management on not composite partition table", KR(ret));
} else if (OB_ISNULL(alter_table_stmt)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("alter table stmt should not be null", KR(ret));
} else {
AlterTableSchema &alter_table_schema =
alter_table_stmt->get_alter_table_arg().alter_table_schema_;
const ObPartition *part = nullptr;
const ObSubPartition *subpart = nullptr;
ObPartition inc_part;
ObSubPartition inc_subpart;
ObString origin_partition_name(static_cast<int32_t>(node.children_[0]->str_len_),
node.children_[0]->str_value_);
ObString new_partition_name(static_cast<int32_t>(node.children_[1]->str_len_),
node.children_[1]->str_value_);
const ObPartitionOption &ori_part_option = orig_table_schema.get_part_option();
if (OB_UNLIKELY(ObCharset::case_insensitive_equal(origin_partition_name, new_partition_name))) {
ret = OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE;
LOG_USER_ERROR(OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
LOG_WARN("origin subpart name equal to new subpart name", KR(ret), K(origin_partition_name), K(new_partition_name));
} else if (OB_FAIL(orig_table_schema.check_partition_duplicate_with_name(new_partition_name))) {
if (OB_DUPLICATE_OBJECT_NAME_EXIST == ret) {
ret = OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE;
LOG_USER_ERROR(OB_ERR_RENAME_SUBPARTITION_NAME_DUPLICATE, new_partition_name.length(), new_partition_name.ptr());
LOG_WARN("new subpart name duplicate with existed partition", KR(ret), K(new_partition_name));
} else {
LOG_WARN("check new subpart name duplicate failed", KR(ret), K(new_partition_name));
}
} else if (OB_FAIL(orig_table_schema.get_subpartition_by_name(origin_partition_name, part, subpart))) {
LOG_WARN("get part by name failed", KR(ret), K(origin_partition_name), K(orig_table_schema));
} else if (OB_FAIL(inc_subpart.set_part_name(origin_partition_name))) {
LOG_WARN("inc subpart set name failed", KR(ret), K(inc_subpart), K(origin_partition_name));
} else if (OB_FAIL(inc_part.add_partition(inc_subpart))) {
LOG_WARN("inc part add inc subpart failed", KR(ret), K(inc_part), K(inc_subpart));
} else if (OB_FAIL(alter_table_schema.add_partition(inc_part))) {
LOG_WARN("alter table add inc part failed", KR(ret), K(alter_table_schema), K(inc_part));
} else if (OB_FAIL(alter_table_schema.set_new_part_name(new_partition_name))) {
LOG_WARN("alter table schema set new part name failed", KR(ret), K(alter_table_schema), K(new_partition_name));
} else if (OB_FAIL(alter_table_schema.get_part_option().assign(ori_part_option))) {
LOG_WARN("alter table set part option failed", KR(ret), K(alter_table_schema), K(ori_part_option));
} else {
alter_table_schema.set_part_level(orig_table_schema.get_part_level());
alter_table_schema.get_part_option().set_part_num(alter_table_schema.get_partition_num());
}
}
return ret;
}
int ObAlterTableResolver::resolve_alter_index(const ParseNode &node)
{
@ -3802,6 +3953,22 @@ int ObAlterTableResolver::resolve_partition_options(const ParseNode &node)
}
break;
}
case T_ALTER_PARTITION_RENAME:{
if (OB_FAIL(resolve_rename_partition(*partition_node, *table_schema_))) {
LOG_WARN("Resolve rename partition error!", KR(ret));
} else {
alter_table_stmt->get_alter_table_arg().alter_part_type_=ObAlterTableArg::RENAME_PARTITION;
}
break;
}
case T_ALTER_SUBPARTITION_RENAME:{
if (OB_FAIL(resolve_rename_subpartition(*partition_node, *table_schema_))) {
LOG_WARN("Resolve rename subpartition error!", KR(ret));
} else {
alter_table_stmt->get_alter_table_arg().alter_part_type_=ObAlterTableArg::RENAME_SUB_PARTITION;
}
break;
}
case T_ALTER_PARTITION_DROP: {
if (OB_FAIL(resolve_drop_partition(*partition_node, *table_schema_))) {
SQL_RESV_LOG(WARN, "Resolve drop partition error!", K(ret));

View File

@ -135,6 +135,10 @@ private:
const share::schema::ObTableSchema &orig_table_schema);
int resolve_drop_subpartition(const ParseNode &node,
const share::schema::ObTableSchema &orig_table_schema);
int resolve_rename_partition(const ParseNode &node,
const share::schema::ObTableSchema &orig_table_schema);
int resolve_rename_subpartition(const ParseNode &node,
const share::schema::ObTableSchema &orig_table_schema);
int check_subpart_name(const share::schema::ObPartition &partition,
const share::schema::ObSubPartition &subpartition);
int resolve_add_constraint(const ParseNode &node);

View File

@ -5362,7 +5362,7 @@ int ObDMLResolver::resolve_fk_table_partition_expr(const TableItem &table_item,
if (child_table_id == table_schema.get_table_id() && !foreign_key_info.is_parent_table_mock_) {
const ObTableSchema *parent_table_schema = nullptr;
bool parent_key_is_pkey = false;
const common::ObSEArray<uint64_t, 8> &parent_column_ids = foreign_key_info.parent_column_ids_;
const common::ObSEArray<uint64_t, 4> &parent_column_ids = foreign_key_info.parent_column_ids_;
const ObTableSchema *resolve_table_schema = nullptr;
uint64_t fk_scan_tid = OB_INVALID_ID;
if (OB_FAIL(schema_checker_->get_table_schema(session_info_->get_effective_tenant_id(), parent_table_id, parent_table_schema))) { //NOTE: Can we use this function to get schema here

View File

@ -150,10 +150,10 @@ public:
virtual inline int64_t get_progressive_merge_num() const override { return progressive_merge_num_; }
virtual inline uint64_t get_master_key_id() const override { return master_key_id_; }
virtual inline bool is_use_bloomfilter() const override { return is_use_bloomfilter_; }
virtual inline bool is_index_table() const override { return share::schema::ObTableSchema::is_index_table(table_type_); }
virtual inline bool is_index_table() const override { return share::schema::is_index_table(table_type_); }
virtual inline bool is_storage_index_table() const override
{
return share::schema::ObTableSchema::is_index_table(table_type_) || is_materialized_view();
return share::schema::is_index_table(table_type_) || is_materialized_view();
}
inline bool is_materialized_view() const { return share::schema::ObTableSchema::is_materialized_view(table_type_); }
virtual inline bool is_global_index_table() const override { return share::schema::ObSimpleTableSchemaV2::is_global_index_table(index_type_); }

View File

@ -223,6 +223,8 @@ enum class ObLockOBJType : char
OBJ_TYPE_EXTERNAL_TABLE_REFRESH = 6, // for external table
OBJ_TYPE_ONLINE_DDL_TABLE = 7, // online ddl table
OBJ_TYPE_ONLINE_DDL_TABLET = 8, // online ddl tablets
OBJ_TYPE_DATABASE_NAME = 9, // for database related ddl
OBJ_TYPE_OBJECT_NAME = 10, // for obj related ddl
OBJ_TYPE_MAX
};
@ -253,6 +255,18 @@ int lock_obj_type_to_string(const ObLockOBJType obj_type,
strncpy(str, "TENANT", str_len);
break;
}
case ObLockOBJType::OBJ_TYPE_EXTERNAL_TABLE_REFRESH: {
strncpy(str, "EXTERNAL_TABLE", str_len);
break;
}
case ObLockOBJType::OBJ_TYPE_DATABASE_NAME: {
strncpy(str, "DATABASE_NAME", str_len);
break;
}
case ObLockOBJType::OBJ_TYPE_OBJECT_NAME: {
strncpy(str, "OBJECT_NAME", str_len);
break;
}
default: {
strncpy(str, "UNKNOWN", str_len);
break;

View File

@ -226,7 +226,10 @@ bool ObLockParam::is_valid() const
|| ObLockOBJType::OBJ_TYPE_LS == lock_id_.obj_type_
|| ObLockOBJType::OBJ_TYPE_EXTERNAL_TABLE_REFRESH == lock_id_.obj_type_
|| ObLockOBJType::OBJ_TYPE_ONLINE_DDL_TABLE == lock_id_.obj_type_
|| ObLockOBJType::OBJ_TYPE_ONLINE_DDL_TABLET == lock_id_.obj_type_)));
|| ObLockOBJType::OBJ_TYPE_ONLINE_DDL_TABLET == lock_id_.obj_type_
|| ObLockOBJType::OBJ_TYPE_DATABASE_NAME == lock_id_.obj_type_
|| ObLockOBJType::OBJ_TYPE_OBJECT_NAME == lock_id_.obj_type_
)));
}
void ObLockRequest::reset()

Some files were not shown because too many files have changed in this diff Show More