[FEAT MERGE] Support index usage monitoring

This commit is contained in:
obdev 2024-02-07 16:12:34 +00:00 committed by ob-robot
parent f7a91cb0b1
commit 112b0dfbd4
35 changed files with 3119 additions and 25 deletions

View File

@ -90,6 +90,7 @@
#include "storage/high_availability/ob_rebuild_service.h"
#include "observer/table/ob_htable_lock_mgr.h"
#include "observer/table/ob_table_session_pool.h"
#include "share/index_usage/ob_index_usage_info_mgr.h"
namespace oceanbase
{
@ -709,6 +710,7 @@ int MockTenantModuleEnv::init()
MTL_BIND2(mtl_new_default, table::ObHTableLockMgr::mtl_init, nullptr, nullptr, nullptr, table::ObHTableLockMgr::mtl_destroy);
MTL_BIND2(mtl_new_default, omt::ObTenantSrs::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, table::ObTableApiSessPoolMgr::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObIndexUsageInfoMgr::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, storage::ObTabletMemtableMgrPool::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default);
}
if (OB_FAIL(ret)) {

View File

@ -151,6 +151,7 @@
#include "observer/table/ob_htable_lock_mgr.h"
#include "observer/table/ob_table_session_pool.h"
#include "observer/ob_server_event_history_table_operator.h"
#include "share/index_usage/ob_index_usage_info_mgr.h"
using namespace oceanbase;
using namespace oceanbase::lib;
@ -556,6 +557,7 @@ int ObMultiTenant::init(ObAddr myaddr,
MTL_BIND2(mtl_new_default, ObOptStatMonitorManager::mtl_init, ObOptStatMonitorManager::mtl_start, ObOptStatMonitorManager::mtl_stop, ObOptStatMonitorManager::mtl_wait, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObTenantSrs::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, table::ObTableApiSessPoolMgr::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObIndexUsageInfoMgr::mtl_init, mtl_start_default, mtl_stop_default, mtl_wait_default, mtl_destroy_default);
MTL_BIND2(mtl_new_default, storage::ObTabletMemtableMgrPool::mtl_init, nullptr, nullptr, nullptr, mtl_destroy_default);
}

View File

@ -14,6 +14,7 @@
#include "ob_table_scan_executor.h"
#include "ob_table_context.h"
#include "sql/das/ob_das_utils.h"
#include "share/index_usage/ob_index_usage_info_mgr.h"
namespace oceanbase
{
@ -276,6 +277,15 @@ int ObTableApiScanExecutor::close()
}
}
if (OB_SUCC(ret)) {
oceanbase::share::ObIndexUsageInfoMgr *mgr = MTL(oceanbase::share::ObIndexUsageInfoMgr *);
if (tb_ctx_.get_table_id() == tb_ctx_.get_ref_table_id()) {
// skip // use primary key, do nothing
} else if (OB_NOT_NULL(mgr)) {
mgr->update(tb_ctx_.get_tenant_id(), tb_ctx_.get_index_table_id());
}
}
return ret;
}

View File

@ -270,6 +270,8 @@ ob_set_subtarget(ob_share common_mixed
table/ob_table_load_sql_statistics.cpp
table/ob_ttl_util.cpp
table/ob_table_config_util.cpp
index_usage/ob_index_usage_info_mgr.cpp
index_usage/ob_index_usage_report_task.cpp
)
ob_set_subtarget(ob_share tablet

View File

@ -1025,5 +1025,10 @@ bool ObKvFeatureModeParser::parse(const char *str, uint8_t *arr, int64_t len)
return bret;
}
bool ObConfigIndexStatsModeChecker::check(const ObConfigItem &t) const {
const ObString tmp_str(t.str());
return 0 == tmp_str.case_compare("SAMPLE") || 0 == tmp_str.case_compare("ALL");
}
} // end of namepace common
} // end of namespace oceanbase

View File

@ -680,6 +680,15 @@ public:
DISALLOW_COPY_AND_ASSIGN(ObKvFeatureModeParser);
};
class ObConfigIndexStatsModeChecker : public ObConfigChecker {
public:
ObConfigIndexStatsModeChecker(){}
virtual ~ObConfigIndexStatsModeChecker(){}
bool check(const ObConfigItem &t) const;
private:
DISALLOW_COPY_AND_ASSIGN(ObConfigIndexStatsModeChecker);
};
typedef __ObConfigContainer<ObConfigStringKey,
ObConfigItem, OB_MAX_CONFIG_NUMBER> ObConfigContainer;
} // namespace common

View File

@ -0,0 +1,277 @@
/**
* 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.
*/
#include "ob_index_usage_info_mgr.h"
#include "observer/ob_server_struct.h"
#include "observer/omt/ob_multi_tenant.h"
#include "share/rc/ob_tenant_base.h"
#include "share/schema/ob_tenant_schema_service.h"
#define USING_LOG_PREFIX SERVER
using namespace oceanbase::common;
namespace oceanbase
{
namespace share
{
const char *OB_INDEX_USAGE_MANAGER = "IndexUsageMgr";
void ObIndexUsageOp::operator()(common::hash::HashMapPair<ObIndexUsageKey, ObIndexUsageInfo> &data)
{
if (ObIndexUsageOpMode::UPDATE == op_mode_) {
ATOMIC_INC(&data.second.total_exec_count_);
data.second.last_used_time_ = current_time_;
} else if (ObIndexUsageOpMode::RESET == op_mode_) {
old_info_ = data.second;
data.second.reset();
}
}
ObIndexUsageInfoMgr::ObIndexUsageInfoMgr()
: is_inited_(false), is_enabled_(false), is_sample_mode_(true), max_entries_(30000),
current_time_(common::ObClockGenerator::getClock()),
min_tenant_data_version_(0),
tenant_id_(OB_INVALID_TENANT_ID),
hashmap_count_(0),
index_usage_map_(nullptr), report_task_(), refresh_conf_task_(), allocator_() {}
ObIndexUsageInfoMgr::~ObIndexUsageInfoMgr()
{
destroy();
}
int ObIndexUsageInfoMgr::mtl_init(ObIndexUsageInfoMgr *&index_usage_mgr)
{
int ret = OB_SUCCESS;
const uint64_t tenant_id = MTL_ID();
if (OB_FAIL(index_usage_mgr->init(tenant_id))) {
LOG_WARN("ObIndexUsageInfoMgr init failed", K(ret), K(tenant_id));
}
return ret;
}
uint64_t ObIndexUsageInfoMgr::calc_hashmap_count(const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
int64_t hashmap_count = 0;
ObTenantBase * tenant = MTL_CTX();
int64_t tenant_min_cpu = tenant != NULL ? std::max((int)tenant->unit_min_cpu(), 1) : 1;
int64_t tenant_min_thread_cnt = tenant_min_cpu * DEFAULT_CPU_QUATO_CONCURRENCY;
int64_t tenant_memory_limit = lib::get_tenant_memory_limit(tenant_id);
int64_t hashmap_memory = tenant_min_thread_cnt * ONE_HASHMAP_MEMORY;
int64_t hashmap_tenant_memory_limit = tenant_memory_limit * 2 / 1000;
int64_t hashmap_memory_limit = hashmap_memory < hashmap_tenant_memory_limit ? hashmap_memory : hashmap_tenant_memory_limit;
hashmap_count = hashmap_memory_limit / ONE_HASHMAP_MEMORY + 1;
LOG_TRACE("success to get hash map count",
K(tenant_min_cpu), K(tenant_min_thread_cnt), K(tenant_memory_limit),
K(hashmap_memory), K(hashmap_tenant_memory_limit), K(hashmap_memory_limit), K(hashmap_count));
return hashmap_count;
}
int ObIndexUsageInfoMgr::create_hash_map(const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
hashmap_count_ = calc_hashmap_count(tenant_id);
void *tmp_ptr = allocator_.alloc(sizeof(ObIndexUsageHashMap) * hashmap_count_) ;
if (NULL == tmp_ptr) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc memory for hashmap", K(ret), K(hashmap_count_));
} else {
index_usage_map_ = new (tmp_ptr) ObIndexUsageHashMap[hashmap_count_];
const ObMemAttr attr(tenant_id, OB_INDEX_USAGE_MANAGER);
for (int64_t i = 0 ; OB_SUCC(ret) && i < hashmap_count_; ++i) {
ObIndexUsageHashMap *hashmap = index_usage_map_ + i;
if (OB_FAIL(hashmap->create(DEFAULT_MAX_HASH_BUCKET_CNT, attr))) {
LOG_WARN("create hash map failed", K(ret));
}
}
}
return ret;
}
void ObIndexUsageInfoMgr::destroy_hash_map()
{
if (OB_NOT_NULL(index_usage_map_)) {
for (int i = 0 ; i < hashmap_count_; ++i) {
ObIndexUsageHashMap *hashmap = index_usage_map_ + i;
hashmap->~ObIndexUsageHashMap();
}
allocator_.free(index_usage_map_);
index_usage_map_ = nullptr;
}
}
int ObIndexUsageInfoMgr::init(const uint64_t tenant_id)
{
int ret = OB_SUCCESS;
const ObMemAttr attr(tenant_id, OB_INDEX_USAGE_MANAGER);
if (OB_UNLIKELY(is_inited_)) {
ret = OB_INIT_TWICE;
LOG_WARN("init twice", K(ret));
} else if (OB_UNLIKELY(tenant_id == OB_INVALID_TENANT_ID)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id));
} else if (!is_user_tenant(tenant_id)) {
LOG_INFO("index monitoring only for user tenant", K(tenant_id));
} else if (OB_FAIL(allocator_.init(ObMallocAllocator::get_instance(), OB_MALLOC_NORMAL_BLOCK_SIZE, attr))) {
LOG_WARN("init allocator failed", K(ret));
} else if (OB_FAIL(create_hash_map(tenant_id))) {
LOG_WARN("create hash map failed", K(ret));
} else if (OB_FALSE_IT(refresh_config())) {
} else {
is_inited_ = true;
tenant_id_ = tenant_id;
LOG_TRACE("success to init ObIndexUsageInfoMgr", K(tenant_id_));
}
return ret;
}
void ObIndexUsageInfoMgr::destroy()
{
if (is_inited_) {
// cancel report task
if (report_task_.get_is_inited()) {
bool is_exist = true;
if (TG_TASK_EXIST(MTL(omt::ObSharedTimer *)->get_tg_id(), report_task_, is_exist) == OB_SUCCESS && is_exist) {
TG_CANCEL_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), report_task_);
TG_WAIT_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), report_task_);
report_task_.destroy();
}
}
if (refresh_conf_task_.get_is_inited()) {
bool is_exist = true;
if (TG_TASK_EXIST(MTL(omt::ObSharedTimer *)->get_tg_id(), refresh_conf_task_, is_exist) == OB_SUCCESS && is_exist) {
TG_CANCEL_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), refresh_conf_task_);
TG_WAIT_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), refresh_conf_task_);
refresh_conf_task_.destroy();
}
}
is_inited_ = false;
is_enabled_ = false;
destroy_hash_map();
allocator_.reset();
}
}
int ObIndexUsageInfoMgr::start()
{
int ret = OB_SUCCESS;
if (is_inited_) {
// report index usage
if (OB_FAIL(TG_SCHEDULE(MTL(omt::ObSharedTimer *)->get_tg_id(), report_task_, INDEX_USAGE_REPORT_INTERVAL, true))) {
LOG_WARN("failed to schedule index usage report task", K(ret));
} else if (report_task_.init(this)) {
LOG_WARN("fail to init report task", K(ret));
} else if (OB_FAIL(TG_SCHEDULE(MTL(omt::ObSharedTimer *)->get_tg_id(), refresh_conf_task_, INDEX_USAGE_REFRESH_CONF_INTERVAL, true))) {
LOG_WARN("failed to schedule index usage refresh conf task", K(ret));
} else if (refresh_conf_task_.init((this))) {
LOG_WARN("fail to init refresh conf task", K(ret));
} else {
LOG_TRACE("success to start ObIndexUsageInfoMgr", K(tenant_id_));
}
}
return ret;
}
void ObIndexUsageInfoMgr::stop()
{
if (OB_LIKELY(report_task_.get_is_inited())) {
TG_CANCEL_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), report_task_);
}
if (OB_LIKELY(refresh_conf_task_.get_is_inited())) {
TG_CANCEL_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), refresh_conf_task_);
}
}
void ObIndexUsageInfoMgr::wait()
{
if (OB_LIKELY(report_task_.get_is_inited())) {
TG_WAIT_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), report_task_);
}
if (OB_LIKELY(refresh_conf_task_.get_is_inited())) {
TG_WAIT_TASK(MTL(omt::ObSharedTimer *)->get_tg_id(), refresh_conf_task_);
}
}
bool ObIndexUsageInfoMgr::sample_filterd(const uint64_t random_num)
{
bool is_filtered = true;
if (!is_sample_mode_) {
is_filtered = false;
} else {
const int seed = random_num % 20;
const double sample_ratio = SAMPLE_RATIO * 1.0 / 100;
const int cycle = sample_ratio < 1 ? 20 * sample_ratio : 20;
for (int i = 0; i < cycle; ++i) {
if (i == seed) {
is_filtered = false;
break;
}
}
}
return is_filtered;
}
void ObIndexUsageInfoMgr::update(const uint64_t tenant_id, const uint64_t index_table_id)
{
int ret = OB_SUCCESS;
uint64_t random_num = common::ObClockGenerator::getClock();
if (!is_inited_ || !is_enabled_) {
// do nothing
} else if (OB_UNLIKELY(tenant_id == OB_INVALID_TENANT_ID || index_table_id == OB_INVALID_ID)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(tenant_id), K(index_table_id));
} else if (min_tenant_data_version_ < DATA_VERSION_4_3_0_0) {
// skip
} else if (sample_filterd(random_num)) {
// skip
} else {
ObIndexUsageKey key(index_table_id);
ObIndexUsageOp update_op(ObIndexUsageOpMode::UPDATE, current_time_);
int idx = random_num % hashmap_count_;
if (is_inner_object_id(index_table_id)) {
// do nothing
} else if (OB_SUCC(index_usage_map_[idx].read_atomic(key, update_op))) {
// key exists, update success
} else if (OB_LIKELY(ret == OB_HASH_NOT_EXIST)) {
// key not exist, insert new one
ObIndexUsageInfo new_info;
new_info.total_exec_count_ = 1;
new_info.last_used_time_ = current_time_;
if (max_entries_ <= index_usage_map_[idx].size()) {
LOG_TRACE("index usage hashmap reaches max entries");
} else if (OB_FAIL(index_usage_map_[idx].set_or_update(key, new_info, update_op))) {
LOG_WARN("failed to set or update index-usage map", K(ret));
}
} else {
LOG_WARN("failed to update index-usage map", K(ret));
}
}
}
void ObIndexUsageInfoMgr::refresh_config()
{
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id_));
if (OB_LIKELY(tenant_config.is_valid())) {
max_entries_ = tenant_config->_iut_max_entries.get();
is_enabled_ = tenant_config->_iut_enable;
is_sample_mode_ = tenant_config->_iut_stat_collection_type.get_value_string().case_compare("SAMPLE") == 0;
}
}
} // namespace share
} // namespace oceanbase

View File

@ -0,0 +1,328 @@
/**
* 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_SHARE_OB_INDEX_USAGE_INFO_MGR_H_
#define OCEANBASE_SHARE_OB_INDEX_USAGE_INFO_MGR_H_
#include "lib/allocator/ob_fifo_allocator.h"
#include "lib/function/ob_function.h"
#include "lib/hash/ob_hashmap.h"
#include "lib/list/ob_list.h"
#include "lib/mysqlclient/ob_mysql_proxy.h"
#include "lib/task/ob_timer.h"
#include "lib/time/ob_time_utility.h"
#include "share/schema/ob_schema_getter_guard.h"
namespace oceanbase
{
namespace share
{
class ObIndexUsageInfoMgr;
enum ObIndexUsageOpMode
{
UPDATE = 0, // for update haspmap
RESET // for reset hashmap
};
struct ObIndexUsageKey final
{
public:
ObIndexUsageKey(const uint64_t index_table_id) : index_table_id_(index_table_id) {}
ObIndexUsageKey() : index_table_id_(OB_INVALID_ID) {}
~ObIndexUsageKey() {}
uint64_t hash() const { return index_table_id_; }
inline int hash(uint64_t &hash_val) const {
hash_val = hash();
return OB_SUCCESS;
}
bool operator==(const ObIndexUsageKey &other) const {
return index_table_id_ == other.index_table_id_;
}
int assign(const ObIndexUsageKey &key) {
index_table_id_ = key.index_table_id_;
return OB_SUCCESS;
}
TO_STRING_KV(K_(index_table_id));
public:
uint64_t index_table_id_;
};
/* strcut stores increment stasitic data */
struct ObIndexUsageInfo final
{
public:
ObIndexUsageInfo()
: total_access_count_(0),
total_exec_count_(0),
total_rows_returned_(0),
bucket_0_access_count_(0),
bucket_1_access_count_(0),
bucket_2_10_access_count_(0),
bucket_2_10_rows_returned_(0),
bucket_11_100_access_count_(0),
bucket_11_100_rows_returned_(0),
bucket_101_1000_access_count_(0),
bucket_101_1000_rows_returned_(0),
bucket_1000_plus_access_count_(0),
bucket_1000_plus_rows_returned_(0),
last_used_time_(ObTimeUtility::current_time()) {}
~ObIndexUsageInfo() {}
void reset() {
total_access_count_ = 0;
total_exec_count_ = 0;
total_rows_returned_ = 0;
bucket_0_access_count_ = 0;
bucket_1_access_count_ = 0;
bucket_2_10_access_count_ = 0;
bucket_2_10_rows_returned_ = 0;
bucket_11_100_access_count_ = 0;
bucket_11_100_rows_returned_ = 0;
bucket_101_1000_access_count_ = 0;
bucket_101_1000_rows_returned_ = 0;
bucket_1000_plus_access_count_ = 0;
bucket_1000_plus_rows_returned_ = 0;
last_used_time_ = ObTimeUtility::current_time();
}
bool has_data () {
return (total_access_count_ > 0 ||
total_exec_count_ > 0 ||
total_rows_returned_ > 0 ||
bucket_0_access_count_ > 0 ||
bucket_1_access_count_ > 0 ||
bucket_2_10_access_count_ > 0 ||
bucket_2_10_rows_returned_ > 0 ||
bucket_11_100_access_count_ > 0 ||
bucket_11_100_rows_returned_ > 0 ||
bucket_101_1000_access_count_ > 0 ||
bucket_101_1000_rows_returned_ > 0 ||
bucket_1000_plus_access_count_ > 0 ||
bucket_1000_plus_rows_returned_ > 0);
}
int assign (const ObIndexUsageInfo &info) {
total_access_count_ = info.total_access_count_;
total_exec_count_ = info.total_exec_count_;
total_rows_returned_ = info.total_rows_returned_;
bucket_0_access_count_ = info.bucket_0_access_count_;
bucket_1_access_count_ = info.bucket_1_access_count_;
bucket_2_10_access_count_ = info.bucket_2_10_access_count_;
bucket_2_10_rows_returned_ = info.bucket_2_10_rows_returned_;
bucket_11_100_access_count_ = info.bucket_11_100_access_count_;
bucket_11_100_rows_returned_ = info.bucket_11_100_rows_returned_;
bucket_101_1000_access_count_ = info.bucket_101_1000_access_count_;
bucket_101_1000_rows_returned_ = info.bucket_101_1000_rows_returned_;
bucket_1000_plus_access_count_ = info.bucket_1000_plus_access_count_;
bucket_1000_plus_rows_returned_ = info.bucket_1000_plus_rows_returned_;
last_used_time_ = info.last_used_time_;
return OB_SUCCESS;
}
TO_STRING_KV(K_(total_access_count), K_(total_exec_count), K_(total_rows_returned),
K_(bucket_0_access_count), K_(bucket_1_access_count),
K_(bucket_2_10_access_count), K_(bucket_2_10_rows_returned),
K_(bucket_11_100_access_count), K_(bucket_11_100_rows_returned),
K_(bucket_101_1000_access_count), K_(bucket_101_1000_rows_returned),
K_(bucket_1000_plus_access_count), K_(bucket_1000_plus_rows_returned),
K_(last_used_time));
public:
volatile int64_t total_access_count_;
volatile int64_t total_exec_count_;
volatile int64_t total_rows_returned_;
volatile int64_t bucket_0_access_count_;
volatile int64_t bucket_1_access_count_;
volatile int64_t bucket_2_10_access_count_;
volatile int64_t bucket_2_10_rows_returned_;
volatile int64_t bucket_11_100_access_count_;
volatile int64_t bucket_11_100_rows_returned_;
volatile int64_t bucket_101_1000_access_count_;
volatile int64_t bucket_101_1000_rows_returned_;
volatile int64_t bucket_1000_plus_access_count_;
volatile int64_t bucket_1000_plus_rows_returned_;
int64_t last_used_time_;
};
typedef common::hash::ObHashMap<ObIndexUsageKey, ObIndexUsageInfo, common::hash::ReadWriteDefendMode> ObIndexUsageHashMap;
typedef common::hash::ObHashMap<ObIndexUsageKey, uint64_t, common::hash::ReadWriteDefendMode> IndexUsageDeletedMap;
typedef common::hash::HashMapPair<ObIndexUsageKey, ObIndexUsageInfo> ObIndexUsagePair;
typedef common::ObList<ObIndexUsagePair, common::ObIAllocator> ObIndexUsagePairList;
class ObIndexUsageReportTask : public common::ObTimerTask
{
static const int64_t MAX_DUMP_ITEM_COUNT = 6000;
static const int64_t DUMP_BATCH_SIZE = 100;
static const int64_t MAX_CHECK_NOT_EXIST_CNT = 8; // 2h
static const int64_t MAX_DELETE_HASHMAP_SIZE = 3000;
public:
ObIndexUsageReportTask();
virtual ~ObIndexUsageReportTask() {};
int init(ObIndexUsageInfoMgr *mgr);
void destroy();
void set_sql_proxy(common::ObMySQLProxy *sql_proxy) { sql_proxy_ = sql_proxy; }
void set_mgr(ObIndexUsageInfoMgr *mgr) { mgr_ = mgr; }
void set_is_inited(const bool is_inited) { is_inited_ = is_inited; }
common::ObMySQLProxy *get_sql_proxy() { return sql_proxy_; }
ObIndexUsageInfoMgr *get_mgr() { return mgr_; }
bool get_is_inited() { return is_inited_; }
private:
virtual void runTimerTask();
int storage_index_usage(const ObIndexUsagePairList &info_list);
int del_index_usage(const ObIndexUsageKey &key);
int check_and_delete(const ObIArray<ObIndexUsageKey> &candidate_deleted_item, ObIndexUsageHashMap *hashmap);
int dump();
private:
struct GetIndexUsageItemsFn final
{
public:
GetIndexUsageItemsFn(IndexUsageDeletedMap &deleted_map, const uint64_t tenant_id, common::ObIAllocator &allocator) :
deleted_map_(deleted_map),
schema_guard_(nullptr),
tenant_id_(tenant_id),
dump_items_(allocator),
remove_items_(),
total_dump_count_(0)
{}
public:
~GetIndexUsageItemsFn() = default;
int operator() (common::hash::HashMapPair<ObIndexUsageKey, ObIndexUsageInfo> &entry);
void set_schema_guard(schema::ObSchemaGetterGuard *schema_guard) { schema_guard_ = schema_guard; }
public:
IndexUsageDeletedMap &deleted_map_;
schema::ObSchemaGetterGuard *schema_guard_;
uint64_t tenant_id_;
ObIndexUsagePairList dump_items_;
ObArray<ObIndexUsageKey> remove_items_;
uint64_t total_dump_count_;
};
private:
bool is_inited_;
ObIndexUsageInfoMgr *mgr_;
common::ObMySQLProxy *sql_proxy_; // 写入内部表需要 sql proxy
IndexUsageDeletedMap deleted_map_;
};
class ObIndexUsageRefreshConfTask : public common::ObTimerTask
{
//friend ObIndexUsageInfoMgr;
public:
ObIndexUsageRefreshConfTask();
virtual ~ObIndexUsageRefreshConfTask() {};
int init(ObIndexUsageInfoMgr *mgr);
void destroy();
void set_mgr(ObIndexUsageInfoMgr *mgr) { mgr_ = mgr; }
void set_is_inited(const bool is_inited) { is_inited_ = is_inited; }
ObIndexUsageInfoMgr *get_mgr() { return mgr_; }
bool get_is_inited() { return is_inited_; }
private:
virtual void runTimerTask();
private:
bool is_inited_;
ObIndexUsageInfoMgr *mgr_;
};
// callback for update or reset map value
class ObIndexUsageOp final
{
public:
explicit ObIndexUsageOp(ObIndexUsageOpMode mode, const uint64_t time = 0) :
op_mode_(mode), old_info_(), current_time_(time) {}
virtual ~ObIndexUsageOp() {}
void operator() (common::hash::HashMapPair<ObIndexUsageKey, ObIndexUsageInfo> &data);
const ObIndexUsageInfo &retrive_info() { return old_info_; }
private:
ObIndexUsageOpMode op_mode_;
ObIndexUsageInfo old_info_;
uint64_t current_time_;
DISALLOW_COPY_AND_ASSIGN(ObIndexUsageOp);
};
#define INDEX_USAGE_INFO_MGR (MTL(ObIndexUsageInfoMgr*))
class ObIndexUsageInfoMgr final
{
static const int64_t SAMPLE_RATIO = 10; // 采样模式下的采样比例 10%
static const int64_t DEFAULT_MAX_HASH_BUCKET_CNT = 3000;
static const int64_t INDEX_USAGE_REFRESH_CONF_INTERVAL = 2 * 1000 * 1000L; // 2s
static const int64_t ONE_HASHMAP_MEMORY = 4 << 20; // 4M
static const int64_t DEFAULT_CPU_QUATO_CONCURRENCY = 2;
#ifdef ERRSIM
static const int64_t INDEX_USAGE_REPORT_INTERVAL = 2 * 1000L * 1000L; // 2s
#else
static const int64_t INDEX_USAGE_REPORT_INTERVAL = 15 * 60 * 1000L * 1000L; // 15min
#endif
public:
static int mtl_init(ObIndexUsageInfoMgr *&index_usage_mgr);
ObIndexUsageInfoMgr();
~ObIndexUsageInfoMgr();
public:
int start(); // start timer task
int init(const uint64_t tenant_id);
void stop();
void wait();
void destroy();
void update(const uint64_t tenant_id, const uint64_t index_table_id);
void refresh_config();
void set_is_enabled(const bool is_enable) { is_enabled_ = is_enable; }
void set_is_sample_mode(const bool mode) { is_sample_mode_ = mode; }
void set_max_entries(const uint64_t entries) { max_entries_ = entries; }
void set_current_time(const uint64_t time) { current_time_ = time; }
void set_min_tenant_data_version(const uint64_t version) { min_tenant_data_version_ = version; }
bool get_is_enabled() { return is_enabled_; }
bool get_is_sample_mode() { return is_sample_mode_; }
uint64_t get_max_entries() { return max_entries_; }
uint64_t get_current_time() { return current_time_; }
uint64_t get_min_tenant_data_version() { return min_tenant_data_version_; }
uint64_t get_tenant_id() { return tenant_id_; }
uint64_t calc_hashmap_count(const uint64_t tenant_id);
uint64_t get_hashmap_count() { return hashmap_count_; }
ObIndexUsageHashMap *get_index_usage_map() { return index_usage_map_; }
common::ObIAllocator &get_allocator() { return allocator_; }
private:
bool sample_filterd(const uint64_t random_num);
void destroy_hash_map();
int create_hash_map(const uint64_t tenant_id);
private:
bool is_inited_;
bool is_enabled_;
bool is_sample_mode_;
uint64_t max_entries_;
uint64_t current_time_;
uint64_t min_tenant_data_version_;
uint64_t tenant_id_;
uint64_t hashmap_count_;
ObIndexUsageHashMap *index_usage_map_;
ObIndexUsageReportTask report_task_;
ObIndexUsageRefreshConfTask refresh_conf_task_;
common::ObFIFOAllocator allocator_;
};
} // namespace share
} // namespace oceanbase
#endif

View File

@ -0,0 +1,403 @@
/**
* 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.
*/
#include "ob_index_usage_info_mgr.h"
#include "share/inner_table/ob_inner_table_schema.h"
#include "share/inner_table/ob_inner_table_schema_constants.h"
#include "share/ob_dml_sql_splicer.h"
#include "share/rc/ob_tenant_base.h"
#include "observer/omt/ob_tenant_config_mgr.h"
#define USING_LOG_PREFIX SERVER
using namespace oceanbase::common;
namespace oceanbase
{
namespace share
{
const char *OB_INDEX_USAGE_REPORT_TASK = "IndexUsageReportTask";
#define INSERT_INDEX_USAGE_HEAD_SQL \
"INSERT INTO %s" \
" (tenant_id, object_id, name, owner," \
" total_access_count, total_exec_count, total_rows_returned," \
" bucket_0_access_count, bucket_1_access_count," \
" bucket_2_10_access_count, bucket_2_10_rows_returned," \
" bucket_11_100_access_count, bucket_11_100_rows_returned," \
" bucket_101_1000_access_count, bucket_101_1000_rows_returned," \
" bucket_1000_plus_access_count, bucket_1000_plus_rows_returned," \
" last_used, last_flush_time) VALUES "
#define INSERT_INDEX_USAGE_ON_DUPLICATE_END_SQL \
" ON DUPLICATE KEY UPDATE" \
" total_access_count = total_access_count + VALUES(total_access_count)," \
" total_exec_count = total_exec_count + VALUES(total_exec_count)," \
" total_rows_returned = total_rows_returned + VALUES(total_rows_returned)," \
" bucket_0_access_count = bucket_0_access_count + VALUES(bucket_0_access_count)," \
" bucket_1_access_count = bucket_1_access_count + VALUES(bucket_1_access_count)," \
" bucket_2_10_access_count = bucket_2_10_access_count + VALUES(bucket_2_10_access_count)," \
" bucket_2_10_rows_returned = bucket_2_10_rows_returned + VALUES(bucket_2_10_rows_returned)," \
" bucket_11_100_access_count = bucket_11_100_access_count + VALUES(bucket_11_100_access_count)," \
" bucket_11_100_rows_returned = bucket_11_100_rows_returned + VALUES(bucket_11_100_rows_returned)," \
" bucket_101_1000_access_count = bucket_101_1000_access_count + VALUES(bucket_101_1000_access_count)," \
" bucket_101_1000_rows_returned = bucket_101_1000_rows_returned + VALUES(bucket_101_1000_rows_returned)," \
" bucket_1000_plus_access_count = bucket_1000_plus_access_count + VALUES(bucket_1000_plus_access_count)," \
" bucket_1000_plus_rows_returned = bucket_1000_plus_rows_returned + VALUES(bucket_1000_plus_rows_returned)," \
" last_used = VALUES(last_used)," \
" last_flush_time = VALUES(last_flush_time) "
ObIndexUsageReportTask::ObIndexUsageReportTask() :
is_inited_(false),
mgr_(nullptr),
sql_proxy_(nullptr) {}
int ObIndexUsageReportTask::GetIndexUsageItemsFn::operator()(common::hash::HashMapPair<ObIndexUsageKey, ObIndexUsageInfo> &entry)
{
int ret = OB_SUCCESS;
bool exist = true;
const uint64_t index_table_id = entry.first.index_table_id_;
if (OB_ISNULL(schema_guard_) || OB_INVALID_TENANT_ID == tenant_id_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid argument", K(ret), KP(schema_guard_), K(tenant_id_));
} else if (OB_FAIL(schema_guard_->check_table_exist(tenant_id_, index_table_id, exist))) {
LOG_WARN("fail to check table exists", K(ret), K(tenant_id_), K(index_table_id));
} else if (!exist) {
if (OB_FAIL(remove_items_.push_back(entry.first))) {
LOG_WARN("fail to push back remove key", K(ret), K(entry.first));
}
} else {
if (entry.second.has_data()) { // has new data
ObIndexUsagePair pair;
pair.init(entry.first, entry.second); // clear data
if (OB_FAIL(dump_items_.push_back(pair))) {
LOG_WARN("fail to push back to list", K(ret), K(tenant_id_));
} else if (++total_dump_count_ >= MAX_DUMP_ITEM_COUNT) {
ret = OB_ITER_END;
LOG_INFO("Reach index usage info dump limit", K(ret), K(total_dump_count_));
}
entry.second.reset(); // reset record
}
if (OB_SUCC(ret)) { // remove exist item from deleted map
if (OB_FAIL(deleted_map_.erase_refactored(entry.first))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to remove exist item from deleted map", K(ret));
}
}
}
}
return ret;
}
int ObIndexUsageReportTask::init(ObIndexUsageInfoMgr *mgr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(mgr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null pointer", K(ret));
} else {
const uint64_t tenant_id = MTL_ID();
const ObMemAttr attr(tenant_id, OB_INDEX_USAGE_REPORT_TASK);
if (OB_FAIL(deleted_map_.create(MAX_DELETE_HASHMAP_SIZE, attr))) {
LOG_WARN("fail to create deleted map", K(ret), K(tenant_id));
} else {
set_sql_proxy(GCTX.sql_proxy_);
set_mgr(mgr);
set_is_inited(true);
}
}
return ret;
}
void ObIndexUsageReportTask::destroy()
{
set_is_inited(false);
set_sql_proxy(nullptr);
set_mgr(nullptr);
deleted_map_.clear();
}
void ObIndexUsageReportTask::runTimerTask()
{
int ret = OB_SUCCESS;
if (!is_inited_) {
} else if (OB_FAIL(dump())) {
LOG_WARN("dump index usage info failed", K(ret), K(MTL_ID()));
}
}
int ObIndexUsageReportTask::storage_index_usage(const ObIndexUsagePairList &info_list)
{
int ret = OB_SUCCESS;
uint64_t data_version = 0;
uint64_t tenant_id = MTL_ID();
uint64_t extract_tenant_id = ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id);
if (OB_ISNULL(sql_proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_proxy is null", K(ret));
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
LOG_WARN("failed to GET_MIN_DATA_VERSION", K(ret));
} else if (data_version < DATA_VERSION_4_3_0_0) {
// skip // only for version > 4.3.0.0
LOG_INFO("data version less than 430 is not support", K(tenant_id));
} else if (!info_list.empty()) {
ObSqlString insert_update_sql;
if (OB_FAIL(insert_update_sql.append_fmt(INSERT_INDEX_USAGE_HEAD_SQL, OB_ALL_INDEX_USAGE_INFO_TNAME))) {
LOG_WARN("fail to append sql string", K(ret));
}
// append sql string
for (ObIndexUsagePairList::const_iterator it = info_list.begin(); OB_SUCC(ret) && it != info_list.end(); it++) {
if (OB_FAIL(insert_update_sql.append_fmt(
"(%lu,%lu,'','', %lu,%lu,%lu,%lu,%lu,%lu,%lu,%lu,%lu,%lu,%lu,%lu,%lu,usec_to_time(%lu),now(6)),",
extract_tenant_id,
it->first.index_table_id_,
it->second.total_exec_count_, // total_access_count_
it->second.total_exec_count_,
it->second.total_rows_returned_,
it->second.bucket_0_access_count_,
it->second.bucket_1_access_count_,
it->second.bucket_2_10_access_count_,
it->second.bucket_2_10_rows_returned_,
it->second.bucket_11_100_access_count_,
it->second.bucket_11_100_rows_returned_,
it->second.bucket_101_1000_access_count_,
it->second.bucket_101_1000_rows_returned_,
it->second.bucket_1000_plus_access_count_,
it->second.bucket_1000_plus_rows_returned_,
it->second.last_used_time_))) {
LOG_WARN("fail to append sql string", K(ret));
}
}
if (OB_SUCC(ret)) {
int64_t affected_rows = 0;
insert_update_sql.set_length(insert_update_sql.length() - 1);
if (OB_FAIL(insert_update_sql.append(INSERT_INDEX_USAGE_ON_DUPLICATE_END_SQL))) {
LOG_WARN("fail to append sql string", K(ret));
} else if (OB_FAIL(sql_proxy_->write(tenant_id, insert_update_sql.ptr(), affected_rows))) {
LOG_WARN("insert update sql error", K(ret), K(tenant_id), K(insert_update_sql));
} else if (affected_rows < 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected affected rows", K(ret), K(tenant_id), K(affected_rows));
}
}
}
return ret;
}
int ObIndexUsageReportTask::del_index_usage(const ObIndexUsageKey &key)
{
int ret = OB_SUCCESS;
int64_t affected_rows = 0;
uint64_t data_version = 0;
uint64_t tenant_id = MTL_ID();
uint64_t extract_tenant_id = ObSchemaUtils::get_extract_tenant_id(tenant_id, tenant_id);
ObDMLSqlSplicer dml;
ObDMLExecHelper exec(*sql_proxy_, tenant_id);
if (OB_ISNULL(sql_proxy_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sql_proxy is null", K(ret));
} else if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) {
LOG_WARN("failed to GET_MIN_DATA_VERSION", K(ret));
} else if (data_version < DATA_VERSION_4_3_0_0) {
// skip // only for version > 4.3.0.0
LOG_INFO("data version less than 430 is not support", K(tenant_id));
} else if (OB_FAIL(dml.add_pk_column("tenant_id", extract_tenant_id)) ||
OB_FAIL(dml.add_pk_column("object_id", key.index_table_id_))) {
LOG_WARN("dml add column failed", K(ret), K(tenant_id), K(extract_tenant_id));
} else if (OB_FAIL(exec.exec_delete(OB_ALL_INDEX_USAGE_INFO_TNAME, dml, affected_rows))) {
LOG_WARN("del sql exec error", K(ret), K(tenant_id), K(extract_tenant_id), K(key));
}
return ret;
}
int ObIndexUsageReportTask::check_and_delete(const ObIArray<ObIndexUsageKey> &candidate_deleted_item, ObIndexUsageHashMap *hashmap)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(hashmap)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null pointer", K(ret));
}
for (int64_t i = 0; OB_SUCC(ret) && i < candidate_deleted_item.count(); ++i) {
const ObIndexUsageKey &key = candidate_deleted_item.at(i);
uint64_t value = 0;
if (OB_FAIL(deleted_map_.get_refactored(key, value))) {
if (OB_HASH_NOT_EXIST == ret) { // insert new
ret = OB_SUCCESS;
if (deleted_map_.size() >= MAX_DELETE_HASHMAP_SIZE) {
LOG_INFO("reach max deleted map upper limited", K(deleted_map_.size()));
break;
} else if (OB_FAIL(deleted_map_.set_refactored(key, 1))) {
LOG_WARN("fail to set delete map", K(ret), K(key));
}
} else {
LOG_WARN("fail to get from deleted map", K(ret));
}
} else if (++value < MAX_CHECK_NOT_EXIST_CNT) { // update
if (OB_FAIL(deleted_map_.set_refactored(key, value))) {
LOG_WARN("fail to set deleted map", K(ret), K(key), K(value));
}
} else { // delete
if (OB_FAIL(deleted_map_.erase_refactored(key))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to del deleted map record", K(ret), K(key));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(del_index_usage(key))) {
LOG_WARN("fail to del inner table record", K(ret));
} else if (OB_FAIL(hashmap->erase_refactored(key))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to del index usage hashmap record", K(ret), K(key));
}
}
}
}
}
return ret;
}
/*
1. dump ObIndexUsageInfo by batch size DUMP_BATCH_SIZE to dump limit MAX_DUMP_ITEM_COUNT (1 cycle)
2. check deleted index to del record in hashmap or in inner table
*/
int ObIndexUsageReportTask::dump()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(mgr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("index usage mgr not init", K(ret));
} else {
const uint64_t tenant_id = MTL_ID();
mgr_->refresh_config();
GetIndexUsageItemsFn index_usage_items_fn(deleted_map_, tenant_id, mgr_->get_allocator());
// dump batch
for (int64_t i = 0; OB_SUCC(ret) && i < mgr_->get_hashmap_count() &&
index_usage_items_fn.total_dump_count_ < MAX_DUMP_ITEM_COUNT; ++i) {
ObIndexUsageHashMap *hashmap = mgr_->get_index_usage_map() + i;
{
ObSchemaGetterGuard schema_guard;
int64_t schema_version = 0;
if (OB_ISNULL(GCTX.schema_service_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("schema service null pointer", K(ret));
} else if (OB_FAIL(GCTX.schema_service_->get_tenant_schema_guard(tenant_id, schema_guard))) {
LOG_WARN("failed to get schema_guard", K(ret), K(tenant_id));
} else if (OB_FAIL(schema_guard.get_schema_version(tenant_id, schema_version))) {
LOG_WARN("failed to get schema version", K(ret), K(tenant_id));
} else if (!ObSchemaService::is_formal_version(schema_version)) {
ret = OB_EAGAIN;
LOG_INFO("is not a formal_schema_version", K(ret), K(tenant_id), K(schema_version));
} else if (OB_FALSE_IT(index_usage_items_fn.set_schema_guard(&schema_guard))) {
} else if (OB_FAIL(hashmap->foreach_refactored(index_usage_items_fn))) {
if (OB_ITER_END != ret) {
LOG_WARN("foreach refactored failed", K(ret), K(tenant_id));
} else {
ret = OB_SUCCESS; // reach max dump count
}
}
}
if (OB_SUCC(ret)) {
// dump
const uint64_t dump_item_count = index_usage_items_fn.dump_items_.size();
uint64_t index = 0;
ObIndexUsagePairList tmp_list(mgr_->get_allocator());
for (ObIndexUsagePairList::const_iterator it = index_usage_items_fn.dump_items_.begin();
OB_SUCC(ret) && index < dump_item_count; it++, index++) {
ObIndexUsagePair tmp_pair;
tmp_pair.init(it->first, it->second); // clear data
if (OB_FAIL(tmp_list.push_back(tmp_pair))) {
LOG_WARN("fail to push back to list", K(ret), K(tenant_id));
} else if (tmp_list.size() < DUMP_BATCH_SIZE && index < dump_item_count - 1) {
// continue
} else if (OB_FAIL(storage_index_usage(tmp_list))) {
LOG_WARN("flush index usage batch failed", K(ret), K(tenant_id));
} else {
tmp_list.reset();
}
}
if (OB_SUCC(ret)) {
index_usage_items_fn.dump_items_.reset();
if (OB_FAIL(check_and_delete(index_usage_items_fn.remove_items_, hashmap))) {
LOG_WARN("fail to check and delete index usage record", K(ret));
} else {
index_usage_items_fn.remove_items_.reuse();
}
}
}
}
}
return ret;
}
// ========================= refresh tenant config ===========================//
ObIndexUsageRefreshConfTask::ObIndexUsageRefreshConfTask() :
is_inited_(false), mgr_(nullptr)
{}
int ObIndexUsageRefreshConfTask::init(ObIndexUsageInfoMgr *mgr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(mgr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null pointer", K(ret));
} else {
set_mgr(mgr);
set_is_inited(true);
}
return ret;
}
void ObIndexUsageRefreshConfTask::destroy()
{
set_is_inited(false);
set_mgr(nullptr);
}
void ObIndexUsageRefreshConfTask::runTimerTask()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(mgr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("index usage mgr not init", K(ret));
} else if (!is_inited_) {
// skip
} else {
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(mgr_->get_tenant_id()));
if (OB_LIKELY(tenant_config.is_valid())) {
mgr_->set_max_entries(tenant_config->_iut_max_entries.get());
mgr_->set_is_enabled(tenant_config->_iut_enable);
mgr_->set_is_sample_mode(tenant_config->_iut_stat_collection_type.get_value_string().case_compare("SAMPLE") == 0);
LOG_TRACE("success to refresh index usage config.",
K(mgr_->get_max_entries()), K(mgr_->get_is_enabled()), K(mgr_->get_is_sample_mode()));
}
// get data version
uint64_t data_version = 0;
if (OB_FAIL(GET_MIN_DATA_VERSION(mgr_->get_tenant_id(), data_version))) {
LOG_WARN("failed to GET_MIN_DATA_VERSION", K(ret));
} else {
mgr_->set_min_tenant_data_version(data_version);
}
// get time
mgr_->set_current_time(common::ObClockGenerator::getClock());
}
}
} // namespace share
} // namespace oceanbase
#undef INSERT_INDEX_USAGE_HEAD_SQL
#undef INSERT_INDEX_USAGE_ON_DUPLICATE_END_SQL

View File

@ -0,0 +1,393 @@
/**
* 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 "ob_inner_table_schema.h"
#include "share/schema/ob_schema_macro_define.h"
#include "share/schema/ob_schema_service_sql_impl.h"
#include "share/schema/ob_table_schema.h"
#include "share/scn.h"
namespace oceanbase
{
using namespace share::schema;
using namespace common;
namespace share
{
int ObInnerTableSchema::all_virtual_index_usage_info_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_INVALID_ID);
table_schema.set_database_id(OB_SYS_DATABASE_ID);
table_schema.set_table_id(OB_ALL_VIRTUAL_INDEX_USAGE_INFO_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(2);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(VIRTUAL_TABLE);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_INDEX_USAGE_INFO_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("tenant_id", //column_name
++column_id, //column_id
1, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("object_id", //column_name
++column_id, //column_id
2, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("gmt_create", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("gmt_modified", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("name", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
128, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("owner", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
128, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("total_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("total_exec_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("total_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_0_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_1_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_2_10_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_2_10_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_11_100_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_11_100_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_101_1000_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_101_1000_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_1000_plus_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_1000_plus_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("last_used", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("last_flush_time", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(0);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -4823,6 +4823,365 @@ int ObInnerTableSchema::all_virtual_aux_stat_real_agent_ora_schema(ObTableSchema
return ret;
}
int ObInnerTableSchema::all_virtual_index_usage_info_real_agent_ora_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_INVALID_ID);
table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID);
table_schema.set_table_id(OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(2);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(VIRTUAL_TABLE);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN);
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TENANT_ID", //column_name
++column_id, //column_id
1, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("OBJECT_ID", //column_name
++column_id, //column_id
2, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("NAME", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_UTF8MB4_BIN, //column_collation_type
128, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("OWNER", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_UTF8MB4_BIN, //column_collation_type
128, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TOTAL_ACCESS_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TOTAL_EXEC_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TOTAL_ROWS_RETURNED", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_0_ACCESS_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_1_ACCESS_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_2_10_ACCESS_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_2_10_ROWS_RETURNED", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_11_100_ACCESS_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_11_100_ROWS_RETURNED", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_101_1000_ACCESS_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_101_1000_ROWS_RETURNED", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_1000_PLUS_ACCESS_COUNT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("BUCKET_1000_PLUS_ROWS_RETURNED", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("LAST_USED", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampLTZType, //column_type
CS_TYPE_INVALID, //column_collation_type
0, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("LAST_FLUSH_TIME", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampLTZType, //column_type
CS_TYPE_INVALID, //column_collation_type
0, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("GMT_CREATE", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampLTZType, //column_type
CS_TYPE_INVALID, //column_collation_type
0, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("GMT_MODIFIED", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampLTZType, //column_type
CS_TYPE_INVALID, //column_collation_type
0, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(0);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -825,6 +825,56 @@ int ObInnerTableSchema::cdb_ob_aux_statistics_schema(ObTableSchema &table_schema
return ret;
}
int ObInnerTableSchema::dba_index_usage_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_INVALID_ID);
table_schema.set_database_id(OB_SYS_DATABASE_ID);
table_schema.set_table_id(OB_DBA_INDEX_USAGE_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(0);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(SYSTEM_VIEW);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_DBA_INDEX_USAGE_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT CAST(IUT.OBJECT_ID AS SIGNED) AS OBJECT_ID, CAST(T.TABLE_NAME AS CHAR(128)) AS NAME, CAST(DB.DATABASE_NAME AS CHAR(128)) AS OWNER, CAST(IUT.TOTAL_ACCESS_COUNT AS SIGNED) AS TOTAL_ACCESS_COUNT, CAST(IUT.TOTAL_EXEC_COUNT AS SIGNED) AS TOTAL_EXEC_COUNT, CAST(IUT.TOTAL_ROWS_RETURNED AS SIGNED) AS TOTAL_ROWS_RETURNED, CAST(IUT.BUCKET_0_ACCESS_COUNT AS SIGNED) AS BUCKET_0_ACCESS_COUNT, CAST(IUT.BUCKET_1_ACCESS_COUNT AS SIGNED) AS BUCKET_1_ACCESS_COUNT, CAST(IUT.BUCKET_2_10_ACCESS_COUNT AS SIGNED) AS BUCKET_2_10_ACCESS_COUNT, CAST(IUT.BUCKET_2_10_ROWS_RETURNED AS SIGNED) AS BUCKET_2_10_ROWS_RETURNED, CAST(IUT.BUCKET_11_100_ACCESS_COUNT AS SIGNED) AS BUCKET_11_100_ACCESS_COUNT, CAST(IUT.BUCKET_11_100_ROWS_RETURNED AS SIGNED) AS BUCKET_11_100_ROWS_RETURNED, CAST(IUT.BUCKET_101_1000_ACCESS_COUNT AS SIGNED) AS BUCKET_101_1000_ACCESS_COUNT, CAST(IUT.BUCKET_101_1000_ROWS_RETURNED AS SIGNED) AS BUCKET_101_1000_ROWS_RETURNED, CAST(IUT.BUCKET_1000_PLUS_ACCESS_COUNT AS SIGNED) AS BUCKET_1000_PLUS_ACCESS_COUNT, CAST(IUT.BUCKET_1000_PLUS_ROWS_RETURNED AS SIGNED) AS BUCKET_1000_PLUS_ROWS_RETURNED, CAST(IUT.LAST_USED AS CHAR(128)) AS LAST_USED FROM oceanbase.__all_index_usage_info IUT JOIN oceanbase.__all_table T ON IUT.OBJECT_ID = T.TABLE_ID JOIN oceanbase.__all_database DB ON T.DATABASE_ID = DB.DATABASE_ID WHERE T.TABLE_ID = IUT.OBJECT_ID )__"))) {
LOG_ERROR("fail to set view_definition", K(ret));
}
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(0);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -0,0 +1,80 @@
/**
* 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 "ob_inner_table_schema.h"
#include "share/schema/ob_schema_macro_define.h"
#include "share/schema/ob_schema_service_sql_impl.h"
#include "share/schema/ob_table_schema.h"
#include "share/scn.h"
namespace oceanbase
{
using namespace share::schema;
using namespace common;
namespace share
{
int ObInnerTableSchema::cdb_index_usage_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_INVALID_ID);
table_schema.set_database_id(OB_SYS_DATABASE_ID);
table_schema.set_table_id(OB_CDB_INDEX_USAGE_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(0);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(SYSTEM_VIEW);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_CDB_INDEX_USAGE_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT IUT.TENANT_ID AS CON_ID, CAST(IUT.OBJECT_ID AS SIGNED) AS OBJECT_ID, CAST(T.TABLE_NAME AS CHAR(128)) AS NAME, CAST(DB.DATABASE_NAME AS CHAR(128)) AS OWNER, CAST(IUT.TOTAL_ACCESS_COUNT AS SIGNED) AS TOTAL_ACCESS_COUNT, CAST(IUT.TOTAL_EXEC_COUNT AS SIGNED) AS TOTAL_EXEC_COUNT, CAST(IUT.TOTAL_ROWS_RETURNED AS SIGNED) AS TOTAL_ROWS_RETURNED, CAST(IUT.BUCKET_0_ACCESS_COUNT AS SIGNED) AS BUCKET_0_ACCESS_COUNT, CAST(IUT.BUCKET_1_ACCESS_COUNT AS SIGNED) AS BUCKET_1_ACCESS_COUNT, CAST(IUT.BUCKET_2_10_ACCESS_COUNT AS SIGNED) AS BUCKET_2_10_ACCESS_COUNT, CAST(IUT.BUCKET_2_10_ROWS_RETURNED AS SIGNED) AS BUCKET_2_10_ROWS_RETURNED, CAST(IUT.BUCKET_11_100_ACCESS_COUNT AS SIGNED) AS BUCKET_11_100_ACCESS_COUNT, CAST(IUT.BUCKET_11_100_ROWS_RETURNED AS SIGNED) AS BUCKET_11_100_ROWS_RETURNED, CAST(IUT.BUCKET_101_1000_ACCESS_COUNT AS SIGNED) AS BUCKET_101_1000_ACCESS_COUNT, CAST(IUT.BUCKET_101_1000_ROWS_RETURNED AS SIGNED) AS BUCKET_101_1000_ROWS_RETURNED, CAST(IUT.BUCKET_1000_PLUS_ACCESS_COUNT AS SIGNED) AS BUCKET_1000_PLUS_ACCESS_COUNT, CAST(IUT.BUCKET_1000_PLUS_ROWS_RETURNED AS SIGNED) AS BUCKET_1000_PLUS_ROWS_RETURNED, CAST(IUT.LAST_USED AS CHAR(128)) AS LAST_USED FROM oceanbase.__all_virtual_index_usage_info IUT JOIN oceanbase.__all_virtual_table T ON IUT.TENANT_ID = T.TENANT_ID AND IUT.OBJECT_ID = T.TABLE_ID JOIN oceanbase.__all_virtual_database DB ON IUT.TENANT_ID = DB.TENANT_ID AND t.DATABASE_ID = DB.DATABASE_ID WHERE T.TABLE_ID = IUT.OBJECT_ID )__"))) {
LOG_ERROR("fail to set view_definition", K(ret));
}
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(0);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -75,6 +75,56 @@ int ObInnerTableSchema::dba_ob_aux_statistics_ora_schema(ObTableSchema &table_sc
return ret;
}
int ObInnerTableSchema::dba_index_usage_ora_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_INVALID_ID);
table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID);
table_schema.set_table_id(OB_DBA_INDEX_USAGE_ORA_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(0);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(SYSTEM_VIEW);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_DBA_INDEX_USAGE_ORA_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT CAST(IUT.OBJECT_ID AS NUMBER) AS OBJECT_ID, CAST(T.TABLE_NAME AS VARCHAR2(128)) AS NAME, CAST(DB.DATABASE_NAME AS VARCHAR2(128)) AS OWNER, CAST(IUT.TOTAL_ACCESS_COUNT AS NUMBER) AS TOTAL_ACCESS_COUNT, CAST(IUT.TOTAL_EXEC_COUNT AS NUMBER) AS TOTAL_EXEC_COUNT, CAST(IUT.TOTAL_ROWS_RETURNED AS NUMBER) AS TOTAL_ROWS_RETURNED, CAST(IUT.BUCKET_0_ACCESS_COUNT AS NUMBER) AS BUCKET_0_ACCESS_COUNT, CAST(IUT.BUCKET_1_ACCESS_COUNT AS NUMBER) AS BUCKET_1_ACCESS_COUNT, CAST(IUT.BUCKET_2_10_ACCESS_COUNT AS NUMBER) AS BUCKET_2_10_ACCESS_COUNT, CAST(IUT.BUCKET_2_10_ROWS_RETURNED AS NUMBER) AS BUCKET_2_10_ROWS_RETURNED, CAST(IUT.BUCKET_11_100_ACCESS_COUNT AS NUMBER) AS BUCKET_11_100_ACCESS_COUNT, CAST(IUT.BUCKET_11_100_ROWS_RETURNED AS NUMBER) AS BUCKET_11_100_ROWS_RETURNED, CAST(IUT.BUCKET_101_1000_ACCESS_COUNT AS NUMBER) AS BUCKET_101_1000_ACCESS_COUNT, CAST(IUT.BUCKET_101_1000_ROWS_RETURNED AS NUMBER) AS BUCKET_101_1000_ROWS_RETURNED, CAST(IUT.BUCKET_1000_PLUS_ACCESS_COUNT AS NUMBER) AS BUCKET_1000_PLUS_ACCESS_COUNT, CAST(IUT.BUCKET_1000_PLUS_ROWS_RETURNED AS NUMBER) AS BUCKET_1000_PLUS_ROWS_RETURNED, CAST(IUT.LAST_USED AS VARCHAR2(128)) AS LAST_USED FROM SYS.ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT IUT JOIN SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T ON IUT.OBJECT_ID = T.TABLE_ID JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON T.DATABASE_ID = DB.DATABASE_ID WHERE T.TABLE_ID = IUT.OBJECT_ID )__"))) {
LOG_ERROR("fail to set view_definition", K(ret));
}
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(0);
table_schema.set_max_used_column_id(column_id);
return ret;
}
int ObInnerTableSchema::all_table_idx_data_table_id_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;

View File

@ -7141,6 +7141,385 @@ int ObInnerTableSchema::all_aux_stat_schema(ObTableSchema &table_schema)
return ret;
}
int ObInnerTableSchema::all_index_usage_info_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID);
table_schema.set_database_id(OB_SYS_DATABASE_ID);
table_schema.set_table_id(OB_ALL_INDEX_USAGE_INFO_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(2);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(SYSTEM_TABLE);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_ALL_INDEX_USAGE_INFO_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
ObObj gmt_create_default;
ObObj gmt_create_default_null;
gmt_create_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG);
gmt_create_default_null.set_null();
ADD_COLUMN_SCHEMA_TS_T("gmt_create", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_BINARY,//collation_type
0, //column length
-1, //column_precision
6, //column_scale
true,//is nullable
false, //is_autoincrement
false, //is_on_update_for_timestamp
gmt_create_default_null,
gmt_create_default)
}
if (OB_SUCC(ret)) {
ObObj gmt_modified_default;
ObObj gmt_modified_default_null;
gmt_modified_default.set_ext(ObActionFlag::OP_DEFAULT_NOW_FLAG);
gmt_modified_default_null.set_null();
ADD_COLUMN_SCHEMA_TS_T("gmt_modified", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_BINARY,//collation_type
0, //column length
-1, //column_precision
6, //column_scale
true,//is nullable
false, //is_autoincrement
true, //is_on_update_for_timestamp
gmt_modified_default_null,
gmt_modified_default)
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("tenant_id", //column_name
++column_id, //column_id
1, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("object_id", //column_name
++column_id, //column_id
2, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("name", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
128, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("owner", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
128, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("total_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("total_exec_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("total_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_0_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_1_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_2_10_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_2_10_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_11_100_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_11_100_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_101_1000_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_101_1000_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_1000_plus_access_count", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("bucket_1000_plus_rows_returned", //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
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("last_used", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("last_flush_time", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(OB_ALL_INDEX_USAGE_INFO_TID);
table_schema.set_aux_lob_meta_tid(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TID);
table_schema.set_aux_lob_piece_tid(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TID);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -2590,6 +2590,141 @@ int ObInnerTableSchema::all_aux_stat_aux_lob_meta_schema(ObTableSchema &table_sc
return ret;
}
int ObInnerTableSchema::all_index_usage_info_aux_lob_meta_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID);
table_schema.set_database_id(OB_SYS_DATABASE_ID);
table_schema.set_table_id(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(2);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(AUX_LOB_META);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("lob_id", //column_name
++column_id, //column_id
1, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_BINARY, //column_collation_type
16, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("seq_id", //column_name
++column_id, //column_id
2, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_BINARY, //column_collation_type
8192, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("binary_len", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObUInt32Type, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(uint32_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("char_len", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObUInt32Type, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(uint32_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("piece_id", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObUInt64Type, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(uint64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("lob_data", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_BINARY, //column_collation_type
262144, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TID);
table_schema.set_data_table_id(OB_ALL_INDEX_USAGE_INFO_TID);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -1735,6 +1735,96 @@ int ObInnerTableSchema::all_aux_stat_aux_lob_piece_schema(ObTableSchema &table_s
return ret;
}
int ObInnerTableSchema::all_index_usage_info_aux_lob_piece_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_SYS_TABLEGROUP_ID);
table_schema.set_database_id(OB_SYS_DATABASE_ID);
table_schema.set_table_id(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(1);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(AUX_LOB_PIECE);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("piece_id", //column_name
++column_id, //column_id
1, //rowkey_id
0, //index_id
0, //part_key_pos
ObUInt64Type, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(uint64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("data_len", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObUInt32Type, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(uint32_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("lob_data", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_BINARY, //column_collation_type
32, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
table_schema.set_index_using_type(USING_BTREE);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TID);
table_schema.set_data_table_id(OB_ALL_INDEX_USAGE_INFO_TID);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -557,6 +557,7 @@ public:
static int all_import_table_task_schema(share::schema::ObTableSchema &table_schema);
static int all_import_table_task_history_schema(share::schema::ObTableSchema &table_schema);
static int all_aux_stat_schema(share::schema::ObTableSchema &table_schema);
static int all_index_usage_info_schema(share::schema::ObTableSchema &table_schema);
static int tenant_virtual_all_table_schema(share::schema::ObTableSchema &table_schema);
static int tenant_virtual_table_column_schema(share::schema::ObTableSchema &table_schema);
static int tenant_virtual_table_index_schema(share::schema::ObTableSchema &table_schema);
@ -961,6 +962,7 @@ public:
static int all_virtual_import_table_task_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_import_table_task_history_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_aux_stat_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_index_usage_info_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_sql_audit_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_plan_stat_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_plan_cache_plan_explain_ora_schema(share::schema::ObTableSchema &table_schema);
@ -1210,6 +1212,7 @@ public:
static int all_virtual_import_table_task_history_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_ls_info_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_aux_stat_real_agent_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_index_usage_info_real_agent_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_plan_cache_stat_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_plan_cache_plan_stat_schema(share::schema::ObTableSchema &table_schema);
static int schemata_schema(share::schema::ObTableSchema &table_schema);
@ -1579,6 +1582,8 @@ public:
static int v_ob_tenant_runtime_info_schema(share::schema::ObTableSchema &table_schema);
static int dba_ob_aux_statistics_schema(share::schema::ObTableSchema &table_schema);
static int cdb_ob_aux_statistics_schema(share::schema::ObTableSchema &table_schema);
static int dba_index_usage_schema(share::schema::ObTableSchema &table_schema);
static int cdb_index_usage_schema(share::schema::ObTableSchema &table_schema);
static int dba_synonyms_schema(share::schema::ObTableSchema &table_schema);
static int dba_objects_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_objects_schema(share::schema::ObTableSchema &table_schema);
@ -2002,6 +2007,7 @@ public:
static int v_ob_ls_log_restore_status_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_flt_trace_config_ora_schema(share::schema::ObTableSchema &table_schema);
static int dba_ob_aux_statistics_ora_schema(share::schema::ObTableSchema &table_schema);
static int dba_index_usage_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_table_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema);
static int all_column_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema);
static int all_ddl_operation_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema);
@ -2267,6 +2273,7 @@ public:
static int all_import_table_task_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema);
static int all_import_table_task_history_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema);
static int all_aux_stat_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema);
static int all_index_usage_info_aux_lob_meta_schema(share::schema::ObTableSchema &table_schema);
static int all_table_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema);
static int all_column_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema);
static int all_ddl_operation_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema);
@ -2532,6 +2539,7 @@ public:
static int all_import_table_task_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema);
static int all_import_table_task_history_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema);
static int all_aux_stat_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema);
static int all_index_usage_info_aux_lob_piece_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_ash_all_virtual_ash_i1_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_sql_plan_monitor_all_virtual_sql_plan_monitor_i1_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_sql_audit_all_virtual_sql_audit_i1_schema(share::schema::ObTableSchema &table_schema);
@ -3001,6 +3009,7 @@ const schema_create_func sys_table_schema_creators [] = {
ObInnerTableSchema::all_import_table_task_schema,
ObInnerTableSchema::all_import_table_task_history_schema,
ObInnerTableSchema::all_aux_stat_schema,
ObInnerTableSchema::all_index_usage_info_schema,
NULL,};
const schema_create_func virtual_table_schema_creators [] = {
@ -3408,6 +3417,7 @@ const schema_create_func virtual_table_schema_creators [] = {
ObInnerTableSchema::all_virtual_import_table_task_schema,
ObInnerTableSchema::all_virtual_import_table_task_history_schema,
ObInnerTableSchema::all_virtual_aux_stat_schema,
ObInnerTableSchema::all_virtual_index_usage_info_schema,
ObInnerTableSchema::all_virtual_ash_all_virtual_ash_i1_schema,
ObInnerTableSchema::all_virtual_sql_plan_monitor_all_virtual_sql_plan_monitor_i1_schema,
ObInnerTableSchema::all_virtual_sql_audit_all_virtual_sql_audit_i1_schema,
@ -3667,6 +3677,7 @@ const schema_create_func virtual_table_schema_creators [] = {
ObInnerTableSchema::all_virtual_import_table_task_history_ora_schema,
ObInnerTableSchema::all_virtual_ls_info_ora_schema,
ObInnerTableSchema::all_virtual_aux_stat_real_agent_ora_schema,
ObInnerTableSchema::all_virtual_index_usage_info_real_agent_ora_schema,
ObInnerTableSchema::all_virtual_table_real_agent_ora_idx_data_table_id_real_agent_schema,
ObInnerTableSchema::all_virtual_table_real_agent_ora_idx_db_tb_name_real_agent_schema,
ObInnerTableSchema::all_virtual_table_real_agent_ora_idx_tb_name_real_agent_schema,
@ -4120,6 +4131,8 @@ const schema_create_func sys_view_schema_creators [] = {
ObInnerTableSchema::v_ob_tenant_runtime_info_schema,
ObInnerTableSchema::dba_ob_aux_statistics_schema,
ObInnerTableSchema::cdb_ob_aux_statistics_schema,
ObInnerTableSchema::dba_index_usage_schema,
ObInnerTableSchema::cdb_index_usage_schema,
ObInnerTableSchema::dba_synonyms_schema,
ObInnerTableSchema::dba_objects_ora_schema,
ObInnerTableSchema::all_objects_schema,
@ -4543,6 +4556,7 @@ const schema_create_func sys_view_schema_creators [] = {
ObInnerTableSchema::v_ob_ls_log_restore_status_ora_schema,
ObInnerTableSchema::gv_ob_flt_trace_config_ora_schema,
ObInnerTableSchema::dba_ob_aux_statistics_ora_schema,
ObInnerTableSchema::dba_index_usage_ora_schema,
NULL,};
const schema_create_func core_index_table_schema_creators [] = {
@ -4901,6 +4915,7 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_IMPORT_TABLE_TASK_TID,
OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID,
OB_ALL_AUX_STAT_TID,
OB_ALL_INDEX_USAGE_INFO_TID,
OB_TENANT_VIRTUAL_ALL_TABLE_TID,
OB_TENANT_VIRTUAL_TABLE_COLUMN_TID,
OB_TENANT_VIRTUAL_TABLE_INDEX_TID,
@ -5357,6 +5372,7 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID,
OB_ALL_VIRTUAL_LS_INFO_ORA_TID,
OB_ALL_VIRTUAL_AUX_STAT_REAL_AGENT_ORA_TID,
OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TID,
OB_GV_OB_PLAN_CACHE_STAT_TID,
OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID,
OB_SCHEMATA_TID,
@ -5621,6 +5637,7 @@ const uint64_t tenant_space_tables [] = {
OB_DBA_OB_IMPORT_TABLE_TASKS_TID,
OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TID,
OB_DBA_OB_AUX_STATISTICS_TID,
OB_DBA_INDEX_USAGE_TID,
OB_DBA_SYNONYMS_TID,
OB_DBA_OBJECTS_ORA_TID,
OB_ALL_OBJECTS_TID,
@ -6044,6 +6061,7 @@ const uint64_t tenant_space_tables [] = {
OB_V_OB_LS_LOG_RESTORE_STATUS_ORA_TID,
OB_GV_OB_FLT_TRACE_CONFIG_ORA_TID,
OB_DBA_OB_AUX_STATISTICS_ORA_TID,
OB_DBA_INDEX_USAGE_ORA_TID,
OB_ALL_TABLE_IDX_DATA_TABLE_ID_TID,
OB_ALL_TABLE_IDX_DB_TB_NAME_TID,
OB_ALL_TABLE_IDX_TB_NAME_TID,
@ -6452,6 +6470,7 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID,
OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID,
OB_ALL_AUX_STAT_AUX_LOB_META_TID,
OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TID,
OB_ALL_TABLE_AUX_LOB_PIECE_TID,
OB_ALL_COLUMN_AUX_LOB_PIECE_TID,
OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TID,
@ -6695,7 +6714,8 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID,
OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID,
OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID,
OB_ALL_AUX_STAT_AUX_LOB_PIECE_TID, };
OB_ALL_AUX_STAT_AUX_LOB_PIECE_TID,
OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TID, };
const uint64_t all_ora_mapping_virtual_table_org_tables [] = {
OB_ALL_VIRTUAL_SQL_AUDIT_TID,
@ -7232,6 +7252,7 @@ const char* const tenant_space_table_names [] = {
OB_ALL_IMPORT_TABLE_TASK_TNAME,
OB_ALL_IMPORT_TABLE_TASK_HISTORY_TNAME,
OB_ALL_AUX_STAT_TNAME,
OB_ALL_INDEX_USAGE_INFO_TNAME,
OB_TENANT_VIRTUAL_ALL_TABLE_TNAME,
OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME,
OB_TENANT_VIRTUAL_TABLE_INDEX_TNAME,
@ -7688,6 +7709,7 @@ const char* const tenant_space_table_names [] = {
OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME,
OB_ALL_VIRTUAL_LS_INFO_ORA_TNAME,
OB_ALL_VIRTUAL_AUX_STAT_REAL_AGENT_ORA_TNAME,
OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TNAME,
OB_GV_OB_PLAN_CACHE_STAT_TNAME,
OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME,
OB_SCHEMATA_TNAME,
@ -7952,6 +7974,7 @@ const char* const tenant_space_table_names [] = {
OB_DBA_OB_IMPORT_TABLE_TASKS_TNAME,
OB_DBA_OB_IMPORT_TABLE_TASK_HISTORY_TNAME,
OB_DBA_OB_AUX_STATISTICS_TNAME,
OB_DBA_INDEX_USAGE_TNAME,
OB_DBA_SYNONYMS_TNAME,
OB_DBA_OBJECTS_ORA_TNAME,
OB_ALL_OBJECTS_TNAME,
@ -8375,6 +8398,7 @@ const char* const tenant_space_table_names [] = {
OB_V_OB_LS_LOG_RESTORE_STATUS_ORA_TNAME,
OB_GV_OB_FLT_TRACE_CONFIG_ORA_TNAME,
OB_DBA_OB_AUX_STATISTICS_ORA_TNAME,
OB_DBA_INDEX_USAGE_ORA_TNAME,
OB_ALL_TABLE_IDX_DATA_TABLE_ID_TNAME,
OB_ALL_TABLE_IDX_DB_TB_NAME_TNAME,
OB_ALL_TABLE_IDX_TB_NAME_TNAME,
@ -8783,6 +8807,7 @@ const char* const tenant_space_table_names [] = {
OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TNAME,
OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TNAME,
OB_ALL_AUX_STAT_AUX_LOB_META_TNAME,
OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TNAME,
OB_ALL_TABLE_AUX_LOB_PIECE_TNAME,
OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME,
OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TNAME,
@ -9026,7 +9051,8 @@ const char* const tenant_space_table_names [] = {
OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME,
OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TNAME,
OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TNAME,
OB_ALL_AUX_STAT_AUX_LOB_PIECE_TNAME, };
OB_ALL_AUX_STAT_AUX_LOB_PIECE_TNAME,
OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TNAME, };
const uint64_t only_rs_vtables [] = {
OB_ALL_VIRTUAL_CORE_META_TABLE_TID,
@ -9356,7 +9382,8 @@ const uint64_t restrict_access_virtual_tables[] = {
OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_ORA_TID,
OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_ORA_TID,
OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TID,
OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID };
OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID,
OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TID };
static inline bool is_restrict_access_virtual_table(const uint64_t tid)
@ -11623,6 +11650,14 @@ LOBMapping const lob_aux_table_mappings [] = {
ObInnerTableSchema::all_aux_stat_aux_lob_piece_schema
},
{
OB_ALL_INDEX_USAGE_INFO_TID,
OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TID,
OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TID,
ObInnerTableSchema::all_index_usage_info_aux_lob_meta_schema,
ObInnerTableSchema::all_index_usage_info_aux_lob_piece_schema
},
};
static inline bool get_sys_table_lob_aux_table_id(const uint64_t tid, uint64_t& meta_tid, uint64_t& piece_tid)
@ -11660,12 +11695,12 @@ static inline int get_sys_table_lob_aux_schema(const uint64_t tid,
}
const int64_t OB_CORE_TABLE_COUNT = 4;
const int64_t OB_SYS_TABLE_COUNT = 262;
const int64_t OB_VIRTUAL_TABLE_COUNT = 744;
const int64_t OB_SYS_VIEW_COUNT = 792;
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1803;
const int64_t OB_SYS_TABLE_COUNT = 263;
const int64_t OB_VIRTUAL_TABLE_COUNT = 746;
const int64_t OB_SYS_VIEW_COUNT = 795;
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1809;
const int64_t OB_CORE_SCHEMA_VERSION = 1;
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1806;
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1812;
} // end namespace share
} // end namespace oceanbase

View File

@ -21,7 +21,7 @@ inner_lob_map_t inner_lob_map;
bool lob_mapping_init()
{
int ret = OB_SUCCESS;
if (OB_FAIL(inner_lob_map.create(265, ObModIds::OB_INNER_LOB_HASH_SET))) {
if (OB_FAIL(inner_lob_map.create(266, ObModIds::OB_INNER_LOB_HASH_SET))) {
SERVER_LOG(WARN, "fail to create inner lob map", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < ARRAYSIZEOF(lob_aux_table_mappings); ++i) {

View File

@ -329,6 +329,13 @@ bool vt_mapping_init()
tmp_vt_mapping.is_real_vt_ = true;
}
{
int64_t idx = OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TID - start_idx;
VTMapping &tmp_vt_mapping = vt_mappings[idx];
tmp_vt_mapping.mapping_tid_ = OB_ALL_INDEX_USAGE_INFO_TID;
tmp_vt_mapping.is_real_vt_ = true;
}
{
int64_t idx = OB_ALL_VIRTUAL_JOB_LOG_REAL_AGENT_ORA_TID - start_idx;
VTMapping &tmp_vt_mapping = vt_mappings[idx];

View File

@ -293,6 +293,7 @@ const uint64_t OB_ALL_IMPORT_TABLE_JOB_HISTORY_TID = 478; // "__all_import_table
const uint64_t OB_ALL_IMPORT_TABLE_TASK_TID = 479; // "__all_import_table_task"
const uint64_t OB_ALL_IMPORT_TABLE_TASK_HISTORY_TID = 480; // "__all_import_table_task_history"
const uint64_t OB_ALL_AUX_STAT_TID = 494; // "__all_aux_stat"
const uint64_t OB_ALL_INDEX_USAGE_INFO_TID = 495; // "__all_index_usage_info"
const uint64_t OB_TENANT_VIRTUAL_ALL_TABLE_TID = 10001; // "__tenant_virtual_all_table"
const uint64_t OB_TENANT_VIRTUAL_TABLE_COLUMN_TID = 10002; // "__tenant_virtual_table_column"
const uint64_t OB_TENANT_VIRTUAL_TABLE_INDEX_TID = 10003; // "__tenant_virtual_table_index"
@ -697,6 +698,7 @@ const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TID = 12425; // "__all_vi
const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TID = 12426; // "__all_virtual_import_table_task"
const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TID = 12427; // "__all_virtual_import_table_task_history"
const uint64_t OB_ALL_VIRTUAL_AUX_STAT_TID = 12447; // "__all_virtual_aux_stat"
const uint64_t OB_ALL_VIRTUAL_INDEX_USAGE_INFO_TID = 12459; // "__all_virtual_index_usage_info"
const uint64_t OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID = 15009; // "ALL_VIRTUAL_SQL_AUDIT_ORA"
const uint64_t OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID = 15010; // "ALL_VIRTUAL_PLAN_STAT_ORA"
const uint64_t OB_ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA_TID = 15012; // "ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA"
@ -946,6 +948,7 @@ const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TID = 15411; // "ALL_VIRTUAL
const uint64_t OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TID = 15412; // "ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA"
const uint64_t OB_ALL_VIRTUAL_LS_INFO_ORA_TID = 15414; // "ALL_VIRTUAL_LS_INFO_ORA"
const uint64_t OB_ALL_VIRTUAL_AUX_STAT_REAL_AGENT_ORA_TID = 15427; // "ALL_VIRTUAL_AUX_STAT_REAL_AGENT_ORA"
const uint64_t OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TID = 15440; // "ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA"
const uint64_t OB_GV_OB_PLAN_CACHE_STAT_TID = 20001; // "GV$OB_PLAN_CACHE_STAT"
const uint64_t OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID = 20002; // "GV$OB_PLAN_CACHE_PLAN_STAT"
const uint64_t OB_SCHEMATA_TID = 20003; // "SCHEMATA"
@ -1315,6 +1318,8 @@ const uint64_t OB_GV_OB_TENANT_RUNTIME_INFO_TID = 21477; // "GV$OB_TENANT_RUNTIM
const uint64_t OB_V_OB_TENANT_RUNTIME_INFO_TID = 21478; // "V$OB_TENANT_RUNTIME_INFO"
const uint64_t OB_DBA_OB_AUX_STATISTICS_TID = 21497; // "DBA_OB_AUX_STATISTICS"
const uint64_t OB_CDB_OB_AUX_STATISTICS_TID = 21498; // "CDB_OB_AUX_STATISTICS"
const uint64_t OB_DBA_INDEX_USAGE_TID = 21499; // "DBA_INDEX_USAGE"
const uint64_t OB_CDB_INDEX_USAGE_TID = 21513; // "CDB_INDEX_USAGE"
const uint64_t OB_DBA_SYNONYMS_TID = 25001; // "DBA_SYNONYMS"
const uint64_t OB_DBA_OBJECTS_ORA_TID = 25002; // "DBA_OBJECTS_ORA"
const uint64_t OB_ALL_OBJECTS_TID = 25003; // "ALL_OBJECTS"
@ -1738,6 +1743,7 @@ const uint64_t OB_V_OB_TIMESTAMP_SERVICE_ORA_TID = 28193; // "V$OB_TIMESTAMP_SER
const uint64_t OB_V_OB_LS_LOG_RESTORE_STATUS_ORA_TID = 28194; // "V$OB_LS_LOG_RESTORE_STATUS_ORA"
const uint64_t OB_GV_OB_FLT_TRACE_CONFIG_ORA_TID = 28195; // "GV$OB_FLT_TRACE_CONFIG_ORA"
const uint64_t OB_DBA_OB_AUX_STATISTICS_ORA_TID = 28210; // "DBA_OB_AUX_STATISTICS_ORA"
const uint64_t OB_DBA_INDEX_USAGE_ORA_TID = 28214; // "DBA_INDEX_USAGE_ORA"
const uint64_t OB_ALL_TABLE_AUX_LOB_META_TID = 50003; // "__all_table_aux_lob_meta"
const uint64_t OB_ALL_COLUMN_AUX_LOB_META_TID = 50004; // "__all_column_aux_lob_meta"
const uint64_t OB_ALL_DDL_OPERATION_AUX_LOB_META_TID = 50005; // "__all_ddl_operation_aux_lob_meta"
@ -2003,6 +2009,7 @@ const uint64_t OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TID = 50478; // "__a
const uint64_t OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TID = 50479; // "__all_import_table_task_aux_lob_meta"
const uint64_t OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TID = 50480; // "__all_import_table_task_history_aux_lob_meta"
const uint64_t OB_ALL_AUX_STAT_AUX_LOB_META_TID = 50494; // "__all_aux_stat_aux_lob_meta"
const uint64_t OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TID = 50495; // "__all_index_usage_info_aux_lob_meta"
const uint64_t OB_ALL_TABLE_AUX_LOB_PIECE_TID = 60003; // "__all_table_aux_lob_piece"
const uint64_t OB_ALL_COLUMN_AUX_LOB_PIECE_TID = 60004; // "__all_column_aux_lob_piece"
const uint64_t OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TID = 60005; // "__all_ddl_operation_aux_lob_piece"
@ -2268,6 +2275,7 @@ const uint64_t OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TID = 60478; // "__
const uint64_t OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TID = 60479; // "__all_import_table_task_aux_lob_piece"
const uint64_t OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TID = 60480; // "__all_import_table_task_history_aux_lob_piece"
const uint64_t OB_ALL_AUX_STAT_AUX_LOB_PIECE_TID = 60494; // "__all_aux_stat_aux_lob_piece"
const uint64_t OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TID = 60495; // "__all_index_usage_info_aux_lob_piece"
const uint64_t OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TID = 14999; // "__all_virtual_plan_cache_stat"
const uint64_t OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TID = 14998; // "__all_virtual_session_event"
const uint64_t OB_ALL_VIRTUAL_SESSION_WAIT_ALL_VIRTUAL_SESSION_WAIT_I1_TID = 14997; // "__all_virtual_session_wait"
@ -2724,6 +2732,7 @@ const char *const OB_ALL_IMPORT_TABLE_JOB_HISTORY_TNAME = "__all_import_table_jo
const char *const OB_ALL_IMPORT_TABLE_TASK_TNAME = "__all_import_table_task";
const char *const OB_ALL_IMPORT_TABLE_TASK_HISTORY_TNAME = "__all_import_table_task_history";
const char *const OB_ALL_AUX_STAT_TNAME = "__all_aux_stat";
const char *const OB_ALL_INDEX_USAGE_INFO_TNAME = "__all_index_usage_info";
const char *const OB_TENANT_VIRTUAL_ALL_TABLE_TNAME = "__tenant_virtual_all_table";
const char *const OB_TENANT_VIRTUAL_TABLE_COLUMN_TNAME = "__tenant_virtual_table_column";
const char *const OB_TENANT_VIRTUAL_TABLE_INDEX_TNAME = "__tenant_virtual_table_index";
@ -3128,6 +3137,7 @@ const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_JOB_HISTORY_TNAME = "__all_virtual
const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_TNAME = "__all_virtual_import_table_task";
const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_TNAME = "__all_virtual_import_table_task_history";
const char *const OB_ALL_VIRTUAL_AUX_STAT_TNAME = "__all_virtual_aux_stat";
const char *const OB_ALL_VIRTUAL_INDEX_USAGE_INFO_TNAME = "__all_virtual_index_usage_info";
const char *const OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TNAME = "ALL_VIRTUAL_SQL_AUDIT";
const char *const OB_ALL_VIRTUAL_PLAN_STAT_ORA_TNAME = "ALL_VIRTUAL_PLAN_STAT";
const char *const OB_ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA_TNAME = "ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN";
@ -3377,6 +3387,7 @@ const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_ORA_TNAME = "ALL_VIRTUAL_IMPO
const char *const OB_ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY_ORA_TNAME = "ALL_VIRTUAL_IMPORT_TABLE_TASK_HISTORY";
const char *const OB_ALL_VIRTUAL_LS_INFO_ORA_TNAME = "ALL_VIRTUAL_LS_INFO";
const char *const OB_ALL_VIRTUAL_AUX_STAT_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL_AUX_STAT_REAL_AGENT";
const char *const OB_ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT_ORA_TNAME = "ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT";
const char *const OB_GV_OB_PLAN_CACHE_STAT_TNAME = "GV$OB_PLAN_CACHE_STAT";
const char *const OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME = "GV$OB_PLAN_CACHE_PLAN_STAT";
const char *const OB_SCHEMATA_TNAME = "SCHEMATA";
@ -3746,6 +3757,8 @@ const char *const OB_GV_OB_TENANT_RUNTIME_INFO_TNAME = "GV$OB_TENANT_RUNTIME_INF
const char *const OB_V_OB_TENANT_RUNTIME_INFO_TNAME = "V$OB_TENANT_RUNTIME_INFO";
const char *const OB_DBA_OB_AUX_STATISTICS_TNAME = "DBA_OB_AUX_STATISTICS";
const char *const OB_CDB_OB_AUX_STATISTICS_TNAME = "CDB_OB_AUX_STATISTICS";
const char *const OB_DBA_INDEX_USAGE_TNAME = "DBA_INDEX_USAGE";
const char *const OB_CDB_INDEX_USAGE_TNAME = "CDB_INDEX_USAGE";
const char *const OB_DBA_SYNONYMS_TNAME = "DBA_SYNONYMS";
const char *const OB_DBA_OBJECTS_ORA_TNAME = "DBA_OBJECTS";
const char *const OB_ALL_OBJECTS_TNAME = "ALL_OBJECTS";
@ -4169,6 +4182,7 @@ const char *const OB_V_OB_TIMESTAMP_SERVICE_ORA_TNAME = "V$OB_TIMESTAMP_SERVICE"
const char *const OB_V_OB_LS_LOG_RESTORE_STATUS_ORA_TNAME = "V$OB_LS_LOG_RESTORE_STATUS";
const char *const OB_GV_OB_FLT_TRACE_CONFIG_ORA_TNAME = "GV$OB_FLT_TRACE_CONFIG";
const char *const OB_DBA_OB_AUX_STATISTICS_ORA_TNAME = "DBA_OB_AUX_STATISTICS";
const char *const OB_DBA_INDEX_USAGE_ORA_TNAME = "DBA_INDEX_USAGE";
const char *const OB_ALL_TABLE_AUX_LOB_META_TNAME = "__all_table_aux_lob_meta";
const char *const OB_ALL_COLUMN_AUX_LOB_META_TNAME = "__all_column_aux_lob_meta";
const char *const OB_ALL_DDL_OPERATION_AUX_LOB_META_TNAME = "__all_ddl_operation_aux_lob_meta";
@ -4434,6 +4448,7 @@ const char *const OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_META_TNAME = "__all_im
const char *const OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_META_TNAME = "__all_import_table_task_aux_lob_meta";
const char *const OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_META_TNAME = "__all_import_table_task_history_aux_lob_meta";
const char *const OB_ALL_AUX_STAT_AUX_LOB_META_TNAME = "__all_aux_stat_aux_lob_meta";
const char *const OB_ALL_INDEX_USAGE_INFO_AUX_LOB_META_TNAME = "__all_index_usage_info_aux_lob_meta";
const char *const OB_ALL_TABLE_AUX_LOB_PIECE_TNAME = "__all_table_aux_lob_piece";
const char *const OB_ALL_COLUMN_AUX_LOB_PIECE_TNAME = "__all_column_aux_lob_piece";
const char *const OB_ALL_DDL_OPERATION_AUX_LOB_PIECE_TNAME = "__all_ddl_operation_aux_lob_piece";
@ -4699,6 +4714,7 @@ const char *const OB_ALL_IMPORT_TABLE_JOB_HISTORY_AUX_LOB_PIECE_TNAME = "__all_i
const char *const OB_ALL_IMPORT_TABLE_TASK_AUX_LOB_PIECE_TNAME = "__all_import_table_task_aux_lob_piece";
const char *const OB_ALL_IMPORT_TABLE_TASK_HISTORY_AUX_LOB_PIECE_TNAME = "__all_import_table_task_history_aux_lob_piece";
const char *const OB_ALL_AUX_STAT_AUX_LOB_PIECE_TNAME = "__all_aux_stat_aux_lob_piece";
const char *const OB_ALL_INDEX_USAGE_INFO_AUX_LOB_PIECE_TNAME = "__all_index_usage_info_aux_lob_piece";
const char *const OB_ALL_VIRTUAL_PLAN_CACHE_STAT_ALL_VIRTUAL_PLAN_CACHE_STAT_I1_TNAME = "__idx_11003_all_virtual_plan_cache_stat_i1";
const char *const OB_ALL_VIRTUAL_SESSION_EVENT_ALL_VIRTUAL_SESSION_EVENT_I1_TNAME = "__idx_11013_all_virtual_session_event_i1";
const char *const OB_ALL_VIRTUAL_SESSION_WAIT_ALL_VIRTUAL_SESSION_WAIT_I1_TNAME = "__idx_11014_all_virtual_session_wait_i1";

View File

@ -6390,7 +6390,39 @@ def_table_schema(
('network_speed', 'bigint', '1000')
],
)
# 495 : __all_index_usage_info
def_table_schema(
owner = 'yangjiali.yjl',
table_name = '__all_index_usage_info',
table_id = '495',
table_type = 'SYSTEM_TABLE',
gm_columns = ['gmt_create', 'gmt_modified'],
rowkey_columns = [
('tenant_id', 'bigint'),
('object_id', 'bigint')
],
in_tenant_space = True,
normal_columns = [
('name', 'varchar:128'),
('owner', 'varchar:128'),
('total_access_count', 'bigint'),
('total_exec_count', 'bigint'),
('total_rows_returned', 'bigint'),
('bucket_0_access_count', 'bigint'),
('bucket_1_access_count', 'bigint'),
('bucket_2_10_access_count', 'bigint'),
('bucket_2_10_rows_returned', 'bigint'),
('bucket_11_100_access_count', 'bigint'),
('bucket_11_100_rows_returned', 'bigint'),
('bucket_101_1000_access_count', 'bigint'),
('bucket_101_1000_rows_returned', 'bigint'),
('bucket_1000_plus_access_count', 'bigint'),
('bucket_1000_plus_rows_returned', 'bigint'),
('last_used','timestamp'),
('last_flush_time', 'timestamp'),
],
)
# 496 : __all_detect_lock_info
# 497 : __all_client_to_server_session_info
# 498 :__all_transfer_partition_task
@ -13221,7 +13253,12 @@ def_table_schema(**gen_iterate_virtual_table_def(
# 12456: __all_virtual_dbms_lock_allocated
# 12457: __all_virtual_sharing_storage_compaction_info
# 12458: __all_virtual_ls_snapshot_in_storage_node
# 12459: __all_virtual_index_usage_info
def_table_schema(**gen_iterate_virtual_table_def(
table_id = '12459',
table_name = '__all_virtual_index_usage_info',
keywords = all_def_keywords['__all_index_usage_info']))
# 12460: __all_virtual_audit_log_filter
# 12461: __all_virtual_audit_log_user
# 12462: __all_virtual_column_privilege
@ -13651,7 +13688,9 @@ def_table_schema(**gen_oracle_mapping_real_virtual_table_def('15427', all_def_ke
# 15437: __all_virtual_clone_job_history
# 15438: __all_virtual_tenant_snapshot_create_job
# 15439: __all_virtual_ls_snapshot_in_storage_node
# 15440: __all_virtual_index_usage_info
def_table_schema(**no_direct_access(gen_oracle_mapping_real_virtual_table_def('15440', all_def_keywords['__all_index_usage_info'])))
# 15441: __all_virtual_share_storage_quota_assignment
# 余留位置
@ -30263,6 +30302,7 @@ def_table_schema(
#21479 GV$OB_CGROUP_CONFIG
#21480 V$OB_CGROUP_CONFIG
#21481 DBA_WR_SYSTEM_EVENT
#21482 CDB_WR_SYSTEM_EVENT
#21483 DBA_WR_EVENT_NAME
@ -30321,7 +30361,42 @@ def_table_schema(
""".replace("\n", " ")
)
#21499 DBA_OB_INDEX_USAGE
def_table_schema(
owner = 'yangjiali.yjl',
table_name = 'DBA_INDEX_USAGE',
table_id = '21499',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
normal_columns = [],
view_definition = """
SELECT
CAST(IUT.OBJECT_ID AS SIGNED) AS OBJECT_ID,
CAST(T.TABLE_NAME AS CHAR(128)) AS NAME,
CAST(DB.DATABASE_NAME AS CHAR(128)) AS OWNER,
CAST(IUT.TOTAL_ACCESS_COUNT AS SIGNED) AS TOTAL_ACCESS_COUNT,
CAST(IUT.TOTAL_EXEC_COUNT AS SIGNED) AS TOTAL_EXEC_COUNT,
CAST(IUT.TOTAL_ROWS_RETURNED AS SIGNED) AS TOTAL_ROWS_RETURNED,
CAST(IUT.BUCKET_0_ACCESS_COUNT AS SIGNED) AS BUCKET_0_ACCESS_COUNT,
CAST(IUT.BUCKET_1_ACCESS_COUNT AS SIGNED) AS BUCKET_1_ACCESS_COUNT,
CAST(IUT.BUCKET_2_10_ACCESS_COUNT AS SIGNED) AS BUCKET_2_10_ACCESS_COUNT,
CAST(IUT.BUCKET_2_10_ROWS_RETURNED AS SIGNED) AS BUCKET_2_10_ROWS_RETURNED,
CAST(IUT.BUCKET_11_100_ACCESS_COUNT AS SIGNED) AS BUCKET_11_100_ACCESS_COUNT,
CAST(IUT.BUCKET_11_100_ROWS_RETURNED AS SIGNED) AS BUCKET_11_100_ROWS_RETURNED,
CAST(IUT.BUCKET_101_1000_ACCESS_COUNT AS SIGNED) AS BUCKET_101_1000_ACCESS_COUNT,
CAST(IUT.BUCKET_101_1000_ROWS_RETURNED AS SIGNED) AS BUCKET_101_1000_ROWS_RETURNED,
CAST(IUT.BUCKET_1000_PLUS_ACCESS_COUNT AS SIGNED) AS BUCKET_1000_PLUS_ACCESS_COUNT,
CAST(IUT.BUCKET_1000_PLUS_ROWS_RETURNED AS SIGNED) AS BUCKET_1000_PLUS_ROWS_RETURNED,
CAST(IUT.LAST_USED AS CHAR(128)) AS LAST_USED
FROM
oceanbase.__all_index_usage_info IUT
JOIN oceanbase.__all_table T ON IUT.OBJECT_ID = T.TABLE_ID
JOIN oceanbase.__all_database DB ON T.DATABASE_ID = DB.DATABASE_ID
WHERE T.TABLE_ID = IUT.OBJECT_ID
""".replace("\n", " "),
)
#21500 DBA_OB_SYS_VARIABLES
#21501 DBA_OB_TRANSFER_PARTITION_TASKS
#21502 CDB_OB_TRANSFER_PARTITION_TASKS
@ -30335,7 +30410,45 @@ def_table_schema(
#21510 DBA_OB_CLONE_PROGRESS
#21511 mysql.role_edges
#21512 mysql.default_roles
#21513 CDB_INDEX_USAGE
def_table_schema(
owner = 'yangjiali.yjl',
table_name = 'CDB_INDEX_USAGE',
table_id = '21513',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
rowkey_columns = [],
normal_columns = [],
view_definition = """
SELECT
IUT.TENANT_ID AS CON_ID,
CAST(IUT.OBJECT_ID AS SIGNED) AS OBJECT_ID,
CAST(T.TABLE_NAME AS CHAR(128)) AS NAME,
CAST(DB.DATABASE_NAME AS CHAR(128)) AS OWNER,
CAST(IUT.TOTAL_ACCESS_COUNT AS SIGNED) AS TOTAL_ACCESS_COUNT,
CAST(IUT.TOTAL_EXEC_COUNT AS SIGNED) AS TOTAL_EXEC_COUNT,
CAST(IUT.TOTAL_ROWS_RETURNED AS SIGNED) AS TOTAL_ROWS_RETURNED,
CAST(IUT.BUCKET_0_ACCESS_COUNT AS SIGNED) AS BUCKET_0_ACCESS_COUNT,
CAST(IUT.BUCKET_1_ACCESS_COUNT AS SIGNED) AS BUCKET_1_ACCESS_COUNT,
CAST(IUT.BUCKET_2_10_ACCESS_COUNT AS SIGNED) AS BUCKET_2_10_ACCESS_COUNT,
CAST(IUT.BUCKET_2_10_ROWS_RETURNED AS SIGNED) AS BUCKET_2_10_ROWS_RETURNED,
CAST(IUT.BUCKET_11_100_ACCESS_COUNT AS SIGNED) AS BUCKET_11_100_ACCESS_COUNT,
CAST(IUT.BUCKET_11_100_ROWS_RETURNED AS SIGNED) AS BUCKET_11_100_ROWS_RETURNED,
CAST(IUT.BUCKET_101_1000_ACCESS_COUNT AS SIGNED) AS BUCKET_101_1000_ACCESS_COUNT,
CAST(IUT.BUCKET_101_1000_ROWS_RETURNED AS SIGNED) AS BUCKET_101_1000_ROWS_RETURNED,
CAST(IUT.BUCKET_1000_PLUS_ACCESS_COUNT AS SIGNED) AS BUCKET_1000_PLUS_ACCESS_COUNT,
CAST(IUT.BUCKET_1000_PLUS_ROWS_RETURNED AS SIGNED) AS BUCKET_1000_PLUS_ROWS_RETURNED,
CAST(IUT.LAST_USED AS CHAR(128)) AS LAST_USED
FROM
oceanbase.__all_virtual_index_usage_info IUT
JOIN oceanbase.__all_virtual_table T
ON IUT.TENANT_ID = T.TENANT_ID AND IUT.OBJECT_ID = T.TABLE_ID
JOIN oceanbase.__all_virtual_database DB
ON IUT.TENANT_ID = DB.TENANT_ID AND t.DATABASE_ID = DB.DATABASE_ID
WHERE T.TABLE_ID = IUT.OBJECT_ID
""".replace("\n", " "),
)
#21514 mysql.audit_log_filter
#21515 mysql.audit_log_user
#21516 mysql.columns_priv
@ -55027,7 +55140,45 @@ def_table_schema(
# 28211: DBA_OB_SYS_VARIABLES
# 28212: GV$OB_ACTIVE_SESSION_HISTORY
# 28213: V$OB_ACTIVE_SESSION_HISTORY
# 28214: DBA_INDEX_USAGE
def_table_schema(
owner = 'yangjiali.yjl',
table_name = 'DBA_INDEX_USAGE',
name_postfix = '_ORA',
database_id = 'OB_ORA_SYS_DATABASE_ID',
table_id = '28214',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
rowkey_columns = [],
normal_columns = [],
in_tenant_space = True,
view_definition = """
SELECT
CAST(IUT.OBJECT_ID AS NUMBER) AS OBJECT_ID,
CAST(T.TABLE_NAME AS VARCHAR2(128)) AS NAME,
CAST(DB.DATABASE_NAME AS VARCHAR2(128)) AS OWNER,
CAST(IUT.TOTAL_ACCESS_COUNT AS NUMBER) AS TOTAL_ACCESS_COUNT,
CAST(IUT.TOTAL_EXEC_COUNT AS NUMBER) AS TOTAL_EXEC_COUNT,
CAST(IUT.TOTAL_ROWS_RETURNED AS NUMBER) AS TOTAL_ROWS_RETURNED,
CAST(IUT.BUCKET_0_ACCESS_COUNT AS NUMBER) AS BUCKET_0_ACCESS_COUNT,
CAST(IUT.BUCKET_1_ACCESS_COUNT AS NUMBER) AS BUCKET_1_ACCESS_COUNT,
CAST(IUT.BUCKET_2_10_ACCESS_COUNT AS NUMBER) AS BUCKET_2_10_ACCESS_COUNT,
CAST(IUT.BUCKET_2_10_ROWS_RETURNED AS NUMBER) AS BUCKET_2_10_ROWS_RETURNED,
CAST(IUT.BUCKET_11_100_ACCESS_COUNT AS NUMBER) AS BUCKET_11_100_ACCESS_COUNT,
CAST(IUT.BUCKET_11_100_ROWS_RETURNED AS NUMBER) AS BUCKET_11_100_ROWS_RETURNED,
CAST(IUT.BUCKET_101_1000_ACCESS_COUNT AS NUMBER) AS BUCKET_101_1000_ACCESS_COUNT,
CAST(IUT.BUCKET_101_1000_ROWS_RETURNED AS NUMBER) AS BUCKET_101_1000_ROWS_RETURNED,
CAST(IUT.BUCKET_1000_PLUS_ACCESS_COUNT AS NUMBER) AS BUCKET_1000_PLUS_ACCESS_COUNT,
CAST(IUT.BUCKET_1000_PLUS_ROWS_RETURNED AS NUMBER) AS BUCKET_1000_PLUS_ROWS_RETURNED,
CAST(IUT.LAST_USED AS VARCHAR2(128)) AS LAST_USED
FROM
SYS.ALL_VIRTUAL_INDEX_USAGE_INFO_REAL_AGENT IUT
JOIN SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T ON IUT.OBJECT_ID = T.TABLE_ID
JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON T.DATABASE_ID = DB.DATABASE_ID
WHERE T.TABLE_ID = IUT.OBJECT_ID
""".replace("\n", " ")
)
# 28215: GV$OB_LS_SNAPSHOTS
# 28216: V$OB_LS_SNAPSHOTS
# 28217: GV$OB_SHARE_STORAGE_QUOTA_ASSIGNMENT

View File

@ -1682,6 +1682,7 @@ case OB_ALL_VIRTUAL_FUNC_TID:
case OB_ALL_VIRTUAL_FUNC_HISTORY_TID:
case OB_ALL_VIRTUAL_HISTOGRAM_STAT_TID:
case OB_ALL_VIRTUAL_HISTOGRAM_STAT_HISTORY_TID:
case OB_ALL_VIRTUAL_INDEX_USAGE_INFO_TID:
case OB_ALL_VIRTUAL_JOB_TID:
case OB_ALL_VIRTUAL_JOB_LOG_TID:
case OB_ALL_VIRTUAL_LS_TID:
@ -2489,6 +2490,21 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
break;
}
case OB_ALL_VIRTUAL_INDEX_USAGE_INFO_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
SERVER_LOG(WARN, "create virtual table iterator failed", K(ret));
} else if (OB_FAIL(iter->init(OB_ALL_INDEX_USAGE_INFO_TID, index_schema, params))) {
SERVER_LOG(WARN, "virtual table iter init failed", K(ret));
iter->~ObIterateVirtualTable();
allocator.free(iter);
iter = NULL;
} else {
vt_iter = iter;
}
break;
}
case OB_ALL_VIRTUAL_JOB_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -2683,7 +2699,9 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_ORI_SCHEMA_VERSION_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -2698,9 +2716,7 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_OUTLINE_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -2985,7 +3001,9 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_RLS_POLICY_HISTORY_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -3000,9 +3018,7 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_RLS_SECURITY_COLUMN_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -3287,7 +3303,9 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_SYS_VARIABLE_HISTORY_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -3302,9 +3320,7 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_SYSAUTH_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -3589,7 +3605,9 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_TENANT_OLS_COMPONENT_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -3604,9 +3622,7 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_TENANT_OLS_COMPONENT_HISTORY_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -3891,7 +3907,9 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_TIME_ZONE_NAME_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {
@ -3906,9 +3924,7 @@ case OB_ALL_VIRTUAL_USER_HISTORY_TID:
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
BEGIN_CREATE_VT_ITER_SWITCH_LAMBDA
case OB_ALL_VIRTUAL_TIME_ZONE_TRANSITION_TID: {
ObIterateVirtualTable *iter = NULL;
if (OB_FAIL(NEW_VIRTUAL_TABLE(ObIterateVirtualTable, iter))) {

15
src/share/parameter/ob_parameter_seed.ipp Executable file → Normal file
View File

@ -1765,6 +1765,21 @@ ERRSIM_DEF_STR(palf_inject_receive_log_error_zone, OB_CLUSTER_PARAMETER, "", "sp
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
ERRSIM_DEF_STR(migrate_check_member_list_error_zone, OB_CLUSTER_PARAMETER, "", "specifies the zone name that migrate want to inject error when change member list",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
// index usage
DEF_BOOL(_iut_enable, OB_TENANT_PARAMETER, "True",
"specifies whether allow the index table usage start monitoring.",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_INT(_iut_max_entries, OB_TENANT_PARAMETER, "30000", "[0,]",
"maximum of index entries to be monitoring.",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE))
DEF_STR_WITH_CHECKER(_iut_stat_collection_type, OB_TENANT_PARAMETER, "SAMPLE",
common::ObConfigIndexStatsModeChecker,
"specify index table usage stat collection type, values: SAMPLE, ALL",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_INT(optimizer_index_cost_adj, OB_TENANT_PARAMETER, "0", "[0,100]",
"adjust costing of index scan",
ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));

View File

@ -203,6 +203,7 @@ class ObGlobalAutoIncService;
class ObDagWarningHistoryManager;
class ObTenantErrsimModuleMgr;
class ObTenantErrsimEventMgr;
class ObIndexUsageInfoMgr;
namespace schema
{
class ObTenantSchemaService;
@ -337,6 +338,7 @@ using ObTableScanIteratorObjPool = common::ObServerObjectPool<oceanbase::storage
table::ObHTableLockMgr*, \
table::ObTTLService*, \
table::ObTableApiSessPoolMgr*, \
share::ObIndexUsageInfoMgr*, \
storage::ObTabletMemtableMgrPool* \
)

View File

@ -32,6 +32,7 @@
#include "share/external_table/ob_external_table_file_mgr.h"
#include "share/external_table/ob_external_table_utils.h"
#include "lib/container/ob_array_wrap.h"
#include "share/index_usage/ob_index_usage_info_mgr.h"
namespace oceanbase
{
@ -1362,6 +1363,19 @@ int ObTableScanOp::inner_close()
if (OB_SUCC(ret)) {
fill_sql_plan_monitor_info();
}
if (OB_SUCC(ret) && MY_SPEC.should_scan_index()) {
ObSQLSessionInfo *session = GET_MY_SESSION(ctx_);
if (OB_NOT_NULL(session)) {
uint64_t tenant_id = session->get_effective_tenant_id();
uint64_t index_id = MY_CTDEF.scan_ctdef_.ref_table_id_;
oceanbase::share::ObIndexUsageInfoMgr* mgr = MTL(oceanbase::share::ObIndexUsageInfoMgr*);
if (OB_NOT_NULL(mgr)) {
mgr->update(tenant_id, index_id);
}
}
}
if (OB_SUCC(ret)) {
iter_end_ = false;
need_init_before_get_row_ = true;

View File

@ -230,6 +230,7 @@ select * from information_schema.tables where table_schema in ('oceanbase', 'mys
| def | mysql | user | SYSTEM TABLE | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | ALL_OB_EXTERNAL_TABLE_FILES | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_DB_LINKS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_INDEX_USAGE | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_IND_PARTITIONS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_IND_STATISTICS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_IND_SUBPARTITIONS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
@ -511,6 +512,7 @@ select * from information_schema.tables where table_schema in ('oceanbase', 'mys
| def | oceanbase | __all_func_history | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_histogram_stat | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_histogram_stat_history | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_index_usage_info | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_job | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_job_log | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_ls | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
@ -1017,6 +1019,8 @@ select * from information_schema.statistics where table_schema in ('oceanbase',
| def | oceanbase | __all_histogram_stat_history | 0 | oceanbase | PRIMARY | 5 | endpoint_num | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_histogram_stat_history | 0 | oceanbase | PRIMARY | 6 | savtime | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_histogram_stat_history | 1 | oceanbase | idx_histogram_stat_his_savtime | 1 | savtime | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_index_usage_info | 0 | oceanbase | PRIMARY | 1 | tenant_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_index_usage_info | 0 | oceanbase | PRIMARY | 2 | object_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 0 | oceanbase | PRIMARY | 1 | tenant_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 0 | oceanbase | PRIMARY | 2 | job | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 1 | oceanbase | idx_job_powner | 1 | powner | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
@ -1516,6 +1520,7 @@ select * from information_schema.tables where table_schema in ('oceanbase', 'mys
| def | mysql | user | SYSTEM TABLE | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | ALL_OB_EXTERNAL_TABLE_FILES | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_DB_LINKS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_INDEX_USAGE | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_IND_PARTITIONS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_IND_STATISTICS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | DBA_IND_SUBPARTITIONS | SYSTEM VIEW | MEMORY | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
@ -1796,6 +1801,7 @@ select * from information_schema.tables where table_schema in ('oceanbase', 'mys
| def | oceanbase | __all_func_history | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_histogram_stat | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_histogram_stat_history | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_index_usage_info | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_job | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_job_log | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
| def | oceanbase | __all_ls | SYSTEM TABLE | InnoDB | NULL | DYNAMIC | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | NULL | utf8mb4_general_ci | NULL | NULL | |
@ -2437,6 +2443,8 @@ select * from information_schema.statistics where table_schema in ('oceanbase',
| def | oceanbase | __all_import_table_task | 0 | oceanbase | PRIMARY | 2 | task_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_import_table_task_history | 0 | oceanbase | PRIMARY | 1 | tenant_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_import_table_task_history | 0 | oceanbase | PRIMARY | 2 | task_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_index_usage_info | 0 | oceanbase | PRIMARY | 1 | tenant_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_index_usage_info | 0 | oceanbase | PRIMARY | 2 | object_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 0 | oceanbase | PRIMARY | 1 | tenant_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 0 | oceanbase | PRIMARY | 2 | job | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 1 | oceanbase | idx_job_powner | 1 | powner | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
@ -3251,6 +3259,8 @@ select * from information_schema.statistics where table_schema in ('oceanbase',
| def | oceanbase | __all_histogram_stat_history | 0 | oceanbase | PRIMARY | 5 | endpoint_num | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_histogram_stat_history | 0 | oceanbase | PRIMARY | 6 | savtime | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_histogram_stat_history | 1 | oceanbase | idx_histogram_stat_his_savtime | 1 | savtime | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_index_usage_info | 0 | oceanbase | PRIMARY | 1 | tenant_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_index_usage_info | 0 | oceanbase | PRIMARY | 2 | object_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 0 | oceanbase | PRIMARY | 1 | tenant_id | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 0 | oceanbase | PRIMARY | 2 | job | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |
| def | oceanbase | __all_job | 1 | oceanbase | idx_job_powner | 1 | powner | A | NULL | NULL | NULL | | BTREE | | | YES | NULL |

View File

@ -324,6 +324,9 @@ _ha_tablet_info_batch_count
_hidden_sys_tenant_memory
_ignore_system_memory_over_limit_error
_io_callback_thread_count
_iut_enable
_iut_max_entries
_iut_stat_collection_type
_lcl_op_interval
_load_tde_encrypt_engine
_log_writer_parallelism

View File

@ -5568,3 +5568,25 @@ NETWORK_SPEED(MB/S) bigint(20) YES NULL
select /*+QUERY_TIMEOUT(60000000)*/ count(*) as cnt from (select * from oceanbase.DBA_OB_AUX_STATISTICS limit 1);
cnt
1
desc oceanbase.DBA_INDEX_USAGE;
Field Type Null Key Default Extra
OBJECT_ID bigint(20) NO NULL
NAME varchar(128) NO
OWNER varchar(128) NO
TOTAL_ACCESS_COUNT bigint(20) NO NULL
TOTAL_EXEC_COUNT bigint(20) NO NULL
TOTAL_ROWS_RETURNED bigint(20) NO NULL
BUCKET_0_ACCESS_COUNT bigint(20) NO NULL
BUCKET_1_ACCESS_COUNT bigint(20) NO NULL
BUCKET_2_10_ACCESS_COUNT bigint(20) NO NULL
BUCKET_2_10_ROWS_RETURNED bigint(20) NO NULL
BUCKET_11_100_ACCESS_COUNT bigint(20) NO NULL
BUCKET_11_100_ROWS_RETURNED bigint(20) NO NULL
BUCKET_101_1000_ACCESS_COUNT bigint(20) NO NULL
BUCKET_101_1000_ROWS_RETURNED bigint(20) NO NULL
BUCKET_1000_PLUS_ACCESS_COUNT bigint(20) NO NULL
BUCKET_1000_PLUS_ROWS_RETURNED bigint(20) NO NULL
LAST_USED varchar(128) NO NULL
select /*+QUERY_TIMEOUT(60000000)*/ count(*) as cnt from (select * from oceanbase.DBA_INDEX_USAGE limit 1);
cnt
1

View File

@ -8009,3 +8009,48 @@ NETWORK_SPEED(MB/S) bigint(20) YES NULL
select /*+QUERY_TIMEOUT(60000000)*/ count(*) as cnt from (select * from oceanbase.CDB_OB_AUX_STATISTICS limit 1);
cnt
1
desc oceanbase.DBA_INDEX_USAGE;
Field Type Null Key Default Extra
OBJECT_ID bigint(20) NO NULL
NAME varchar(128) NO
OWNER varchar(128) NO
TOTAL_ACCESS_COUNT bigint(20) NO NULL
TOTAL_EXEC_COUNT bigint(20) NO NULL
TOTAL_ROWS_RETURNED bigint(20) NO NULL
BUCKET_0_ACCESS_COUNT bigint(20) NO NULL
BUCKET_1_ACCESS_COUNT bigint(20) NO NULL
BUCKET_2_10_ACCESS_COUNT bigint(20) NO NULL
BUCKET_2_10_ROWS_RETURNED bigint(20) NO NULL
BUCKET_11_100_ACCESS_COUNT bigint(20) NO NULL
BUCKET_11_100_ROWS_RETURNED bigint(20) NO NULL
BUCKET_101_1000_ACCESS_COUNT bigint(20) NO NULL
BUCKET_101_1000_ROWS_RETURNED bigint(20) NO NULL
BUCKET_1000_PLUS_ACCESS_COUNT bigint(20) NO NULL
BUCKET_1000_PLUS_ROWS_RETURNED bigint(20) NO NULL
LAST_USED varchar(128) NO NULL
select /*+QUERY_TIMEOUT(60000000)*/ count(*) as cnt from (select * from oceanbase.DBA_INDEX_USAGE limit 1);
cnt
1
desc oceanbase.CDB_INDEX_USAGE;
Field Type Null Key Default Extra
CON_ID bigint(20) NO NULL
OBJECT_ID bigint(20) NO NULL
NAME varchar(128) NO
OWNER varchar(128) NO
TOTAL_ACCESS_COUNT bigint(20) NO NULL
TOTAL_EXEC_COUNT bigint(20) NO NULL
TOTAL_ROWS_RETURNED bigint(20) NO NULL
BUCKET_0_ACCESS_COUNT bigint(20) NO NULL
BUCKET_1_ACCESS_COUNT bigint(20) NO NULL
BUCKET_2_10_ACCESS_COUNT bigint(20) NO NULL
BUCKET_2_10_ROWS_RETURNED bigint(20) NO NULL
BUCKET_11_100_ACCESS_COUNT bigint(20) NO NULL
BUCKET_11_100_ROWS_RETURNED bigint(20) NO NULL
BUCKET_101_1000_ACCESS_COUNT bigint(20) NO NULL
BUCKET_101_1000_ROWS_RETURNED bigint(20) NO NULL
BUCKET_1000_PLUS_ACCESS_COUNT bigint(20) NO NULL
BUCKET_1000_PLUS_ROWS_RETURNED bigint(20) NO NULL
LAST_USED varchar(128) NO NULL
select /*+QUERY_TIMEOUT(60000000)*/ count(*) as cnt from (select * from oceanbase.CDB_INDEX_USAGE limit 1);
cnt
1

View File

@ -8661,3 +8661,29 @@ network_speed bigint(20) YES NULL
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_aux_stat;
IF(count(*) >= 0, 1, 0)
1
desc oceanbase.__all_virtual_index_usage_info;
Field Type Null Key Default Extra
tenant_id bigint(20) NO PRI NULL
object_id bigint(20) NO PRI NULL
gmt_create timestamp(6) NO NULL
gmt_modified timestamp(6) NO NULL
name varchar(128) NO NULL
owner varchar(128) NO NULL
total_access_count bigint(20) NO NULL
total_exec_count bigint(20) NO NULL
total_rows_returned bigint(20) NO NULL
bucket_0_access_count bigint(20) NO NULL
bucket_1_access_count bigint(20) NO NULL
bucket_2_10_access_count bigint(20) NO NULL
bucket_2_10_rows_returned bigint(20) NO NULL
bucket_11_100_access_count bigint(20) NO NULL
bucket_11_100_rows_returned bigint(20) NO NULL
bucket_101_1000_access_count bigint(20) NO NULL
bucket_101_1000_rows_returned bigint(20) NO NULL
bucket_1000_plus_access_count bigint(20) NO NULL
bucket_1000_plus_rows_returned bigint(20) NO NULL
last_used timestamp(6) NO NULL
last_flush_time timestamp(6) NO NULL
select /*+QUERY_TIMEOUT(60000000)*/ IF(count(*) >= 0, 1, 0) from oceanbase.__all_virtual_index_usage_info;
IF(count(*) >= 0, 1, 0)
1

View File

@ -265,6 +265,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr
479 __all_import_table_task 0 201001 1
480 __all_import_table_task_history 0 201001 1
494 __all_aux_stat 0 201001 1
495 __all_index_usage_info 0 201001 1
10001 __tenant_virtual_all_table 2 201001 1
10002 __tenant_virtual_table_column 2 201001 1
10003 __tenant_virtual_table_index 2 201001 1
@ -669,6 +670,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr
12426 __all_virtual_import_table_task 2 201001 1
12427 __all_virtual_import_table_task_history 2 201001 1
12447 __all_virtual_aux_stat 2 201001 1
12459 __all_virtual_index_usage_info 2 201001 1
20001 GV$OB_PLAN_CACHE_STAT 1 201001 1
20002 GV$OB_PLAN_CACHE_PLAN_STAT 1 201001 1
20003 SCHEMATA 1 201002 1
@ -1038,6 +1040,8 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr
21478 V$OB_TENANT_RUNTIME_INFO 1 201001 1
21497 DBA_OB_AUX_STATISTICS 1 201001 1
21498 CDB_OB_AUX_STATISTICS 1 201001 1
21499 DBA_INDEX_USAGE 1 201001 1
21513 CDB_INDEX_USAGE 1 201001 1
check sys table count and table_id range success
check count and table_id range for virtual table success
select * from information_schema.CHARACTER_SETS limit 1;

View File

@ -97,3 +97,4 @@ add_subdirectory(interrupt)
add_subdirectory(tablet)
add_subdirectory(location_cache)
add_subdirectory(detect)
add_subdirectory(index_usage)

View File

@ -0,0 +1 @@
ob_unittest(test_index_usage)

View File

@ -0,0 +1,152 @@
/**
* 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.
*/
#include <gtest/gtest.h>
#define private public // 获取私有成员
#include "share/index_usage/ob_index_usage_info_mgr.h"
#include "lib/utility/ob_test_util.h"
#include "mtlenv/mock_tenant_module_env.h"
#include "share/rc/ob_tenant_base.h"
using namespace oceanbase;
using namespace oceanbase::common;
using namespace oceanbase::table;
using namespace oceanbase::sql;
using namespace oceanbase::share;
using namespace oceanbase::storage;
class TestIndexUsageInfo: public ::testing::Test
{
public:
TestIndexUsageInfo(): mgr_() {}
virtual ~TestIndexUsageInfo() {}
static void SetUpTestCase()
{
EXPECT_EQ(OB_SUCCESS, MockTenantModuleEnv::get_instance().init());
oceanbase::omt::ObTenantMeta tenant_meta;
uint64_t tenant_id = 1888;
ASSERT_EQ(OB_SUCCESS, MockTenantModuleEnv::construct_default_tenant_meta(tenant_id, tenant_meta));
ASSERT_EQ(OB_SUCCESS, GCTX.omt_->create_tenant(tenant_meta, false));
}
static void TearDownTestCase()
{
uint64_t tenant_id = 1888;
bool lock_succ = false;
// ASSERT_EQ(OB_SUCCESS, GCTX.omt_->remove_tenant(tenant_id, lock_succ));
while (OB_EAGAIN == GCTX.omt_->remove_tenant(tenant_id, lock_succ));
MockTenantModuleEnv::get_instance().destroy();
}
void SetUp()
{
mgr_.is_enabled_ = true;
mgr_.is_sample_mode_=false;
ASSERT_EQ(OB_SUCCESS, mgr_.init(1888)); // mock user tenant id
ASSERT_EQ(OB_SUCCESS, mgr_.start());
}
void TearDown()
{
mgr_.stop();
mgr_.destroy();
}
int64_t check_size() {
int64_t i = 0;
int64_t total_size = 0;
for (i = 0; i < mgr_.hashmap_count_; ++i) {
total_size += mgr_.index_usage_map_[i].size();
}
return total_size;
}
private:
ObIndexUsageInfoMgr mgr_;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(TestIndexUsageInfo);
};
TEST_F(TestIndexUsageInfo, test_init)
{
ObIndexUsageInfoMgr mgr;
ASSERT_FALSE(mgr.is_inited_);
ASSERT_FALSE(mgr.report_task_.get_is_inited());
ASSERT_FALSE(mgr.refresh_conf_task_.get_is_inited());
ASSERT_EQ(nullptr, mgr.report_task_.get_mgr());
ASSERT_EQ(nullptr, mgr.report_task_.get_sql_proxy());
ASSERT_TRUE(mgr_.is_inited_);
ASSERT_TRUE(mgr_.report_task_.get_is_inited());
ASSERT_TRUE(mgr_.refresh_conf_task_.get_is_inited());
ASSERT_EQ(&mgr_, mgr_.report_task_.get_mgr());
ASSERT_EQ(&mgr_, mgr_.refresh_conf_task_.get_mgr());
ASSERT_EQ(GCTX.sql_proxy_, mgr_.report_task_.get_sql_proxy());
}
TEST_F(TestIndexUsageInfo, test_update)
{
int64_t mock_tenant_id = 1888;
int64_t mock_index_id = 500002;
ASSERT_TRUE(mgr_.is_inited_);
ASSERT_EQ(0, check_size());
mgr_.is_sample_mode_=false;
mgr_.is_enabled_=true;
mgr_.min_tenant_data_version_=DATA_VERSION_4_3_0_0;
mgr_.update(mock_tenant_id, mock_index_id);
ASSERT_EQ(1, check_size());
}
TEST_F(TestIndexUsageInfo, test_sample_filter)
{
// about 10% sample ratio
bool is_filter = true;
int64_t count = 0;
for (int64_t i = 0; i < 20; i++) {
int64_t random_num = common::ObClockGenerator::getClock();
is_filter = mgr_.sample_filterd(random_num);
if (!is_filter) {
count++;
}
}
ASSERT_TRUE(count < 20);
}
TEST_F(TestIndexUsageInfo, test_refresh_config)
{
mgr_.is_enabled_ = false;
mgr_.is_sample_mode_=false;
mgr_.max_entries_=1000;
mgr_.refresh_config();
ASSERT_EQ(true, mgr_.is_enabled_);
ASSERT_EQ(true, mgr_.is_sample_mode_);
ASSERT_EQ(30000, mgr_.max_entries_);
}
TEST_F(TestIndexUsageInfo, test_destory)
{
mgr_.destroy();
ASSERT_FALSE(mgr_.is_inited_);
ASSERT_FALSE(mgr_.report_task_.get_is_inited());
ASSERT_FALSE(mgr_.refresh_conf_task_.get_is_inited());
ASSERT_EQ(nullptr, mgr_.index_usage_map_);
ASSERT_EQ(nullptr, mgr_.report_task_.get_sql_proxy());
ASSERT_EQ(nullptr, mgr_.report_task_.get_mgr());
ASSERT_EQ(nullptr, mgr_.refresh_conf_task_.get_mgr());
}
int main(int argc, char **argv)
{
OB_LOGGER.set_log_level("INFO");
OB_LOGGER.set_file_name("TestIndexUsageInfo.log", true);
::testing::InitGoogleTest(&argc,argv);
return RUN_ALL_TESTS();
}