[FEAT MERGE]view of GV$OB_THREAD

This commit is contained in:
nroskill 2023-04-17 03:11:18 +00:00 committed by ob-robot
parent 077efec4cd
commit b873dfb706
36 changed files with 1426 additions and 119 deletions

View File

@ -25,6 +25,9 @@ static void easy_baseth_pool_invoke_debug(struct ev_loop *loop);
static int easy_monitor_interval = 100;
static const int64_t easy_monitor_signal = 34;
int ob_pthread_create(pthread_t *thread, const pthread_attr_t *attr,
void *(*start_routine) (void *), void *arg);
void ob_set_thread_name(const char* type);
/**
* start
*/
@ -239,7 +242,7 @@ static void *easy_baseth_pool_monitor_func(void *args)
int64_t loopcnts[tp->thread_count];
int64_t slowcnts[tp->thread_count];
prctl(PR_SET_NAME, "EasyBasethPoolMonitor");
ob_set_thread_name("EasyBasethPoolMonitor");
memset(loopcnts, 0, sizeof(loopcnts));
memset(slowcnts, 0, sizeof(slowcnts));
@ -319,7 +322,7 @@ void easy_baseth_pool_monitor(easy_thread_pool_t *tp)
sigemptyset(&sa.sa_mask);
rc = sigaction(easy_monitor_signal, &sa, NULL);
err = pthread_create(&tp->monitor_tid, NULL, easy_baseth_pool_monitor_func, tp);
err = ob_pthread_create(&tp->monitor_tid, NULL, easy_baseth_pool_monitor_func, tp);
if (err != 0) {
tp->monitor_tid = 0;
easy_error_log("sigaction: %d, monitor_thread: 0x%lx, err:%d, errno:%d", rc, tp->monitor_tid, err, errno);

View File

@ -47,6 +47,8 @@ static void easy_io_print_status(easy_io_t *eio);
static void easy_signal_handler(int sig);
static void easy_listen_close(easy_listen_t *l);
int ob_pthread_create(pthread_t *thread, const pthread_attr_t *attr,
void *(*start_routine) (void *), void *arg);
/**
* easy_io
*/
@ -273,7 +275,7 @@ int easy_eio_start(easy_io_t *eio)
easy_list_for_each_entry(tp, &eio->thread_pool_list, list_node) {
easy_thread_pool_for_each(th, tp, 0) {
int err = 0;
if ((err = pthread_create(&(th->tid), NULL, th->on_start, (void *)th))) {
if ((err = ob_pthread_create(&(th->tid), NULL, th->on_start, (void *)th))) {
ret = EASY_ERROR;
th->tid = 0;
easy_error_log("easy_io_start, pthread_create error: %d(%d), idx: %d", err, errno, th->idx);

View File

@ -39,6 +39,8 @@ typedef struct easy_ma_t {
} easy_ma_t;
easy_ma_t g_ma;
int ob_pthread_create(pthread_t *thread, const pthread_attr_t *attr,
void *(*start_routine) (void *), void *arg);
void easy_ma_init(int port)
{
int i;
@ -215,7 +217,7 @@ int easy_ma_start()
g_ma.efd = efd;
g_ma.lfd = lfd;
g_ma.stop = 0;
if (0 != pthread_create(&g_ma.th, NULL, (void*)easy_ma_thread_func, (void*)&g_ma)) {
if (0 != ob_pthread_create(&g_ma.th, NULL, (void*)easy_ma_thread_func, (void*)&g_ma)) {
goto error_exit;
}

View File

@ -22,8 +22,9 @@ namespace oceanbase
namespace common
{
bool USE_CO_LATCH = false;
thread_local uint32_t* ObLatch::current_lock = nullptr;
thread_local uint32_t* ObLatch::current_locks[16];
thread_local uint32_t* ObLatch::current_wait = nullptr;
thread_local int8_t ObLatch::max_lock_slot_idx = 0;
class ObLatchWaitEventGuard : public ObWaitEventGuard
{
@ -72,7 +73,7 @@ int ObLatchMutex::try_lock(
if (!ATOMIC_BCAS(&lock_.val(), 0, (WRITE_MASK | uid))) {
ret = OB_EAGAIN;
} else {
ObLatch::current_lock = (uint32_t*)&lock_.val();
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&lock_.val());
}
if (need_record_stat()) {
TRY_LOCK_RECORD_STAT(latch_id, 1, ret);
@ -183,7 +184,7 @@ int ObLatchMutex::unlock()
{
int ret = OB_SUCCESS;
uint32_t lock = ATOMIC_SET(&lock_.val(), 0);
ObLatch::current_lock = nullptr;
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&lock_.val());
if (OB_UNLIKELY(0 == lock)) {
ret = OB_ERR_UNEXPECTED;
COMMON_LOG(ERROR, "invalid lock,", K(lock), K(ret));
@ -609,7 +610,7 @@ int ObLatch::try_rdlock(const uint32_t latch_id)
++i;
if (ATOMIC_BCAS(&lock_, lock, lock + 1)) {
ret = OB_SUCCESS;
ObLatch::current_lock = (uint32_t*)&lock_;
IGNORE_RETURN unreg_lock((uint32_t*)&lock_);
break;
}
}
@ -639,7 +640,7 @@ int ObLatch::try_wrlock(const uint32_t latch_id, const uint32_t *puid)
if (!ATOMIC_BCAS(&lock_, 0, (WRITE_MASK | uid))) {
ret = OB_EAGAIN;
} else {
ObLatch::current_lock = (uint32_t*)&lock_;
IGNORE_RETURN unreg_lock((uint32_t*)&lock_);
}
if (need_record_stat()) {
TRY_LOCK_RECORD_STAT(latch_id, 1, ret);
@ -710,7 +711,7 @@ int ObLatch::wr2rdlock(const uint32_t *puid)
lock = lock_;
PAUSE();
}
ObLatch::current_lock = (uint32_t*)&lock_;
IGNORE_RETURN unreg_lock((uint32_t*)&lock_);
bool only_rd_wait = true;
if (OB_FAIL(ObLatchWaitQueue::get_instance().wake_up(*this, only_rd_wait))) {
COMMON_LOG(ERROR, "Fail to wake up latch wait queue, ", K(this), K(ret));
@ -732,11 +733,11 @@ int ObLatch::unlock(const uint32_t *puid)
COMMON_LOG(ERROR, "The latch is not write locked by the uid, ", K(uid), K(wid), KCSTRING(lbt()), K(ret));
} else {
lock = ATOMIC_ANDF(&lock_, WAIT_MASK);
ObLatch::current_lock = nullptr;
IGNORE_RETURN unreg_lock((uint32_t*)&lock_);
}
} else if ((lock & (~WAIT_MASK)) > 0) {
lock = ATOMIC_AAF(&lock_, -1);
ObLatch::current_lock = nullptr;
IGNORE_RETURN unreg_lock((uint32_t*)&lock_);
} else {
ret = OB_ERR_UNEXPECTED;
COMMON_LOG(ERROR, "invalid lock,", K(lock), K(ret));

View File

@ -227,8 +227,44 @@ public:
void enable_record_stat(bool enable) { record_stat_ = enable; }
bool need_record_stat() const { return record_stat_; }
uint32_t val() const { return lock_; }
static thread_local uint32_t* current_lock;
OB_INLINE static int reg_lock(uint32_t* latch_addr)
{
int ret = -1;
if (max_lock_slot_idx < sizeof(current_locks) / sizeof(uint32_t*)) {
ret = max_lock_slot_idx++;
current_locks[ret] = latch_addr;
} else {
for (auto i = 0; -1 == ret && i < sizeof(current_locks) / sizeof(uint32_t*); ++i) {
if (OB_ISNULL(current_locks[i]) || 0 == *current_locks[i]) {
ret = i;
current_locks[i] = latch_addr;
}
}
}
return ret;
}
OB_INLINE static int unreg_lock(uint32_t* latch_addr)
{
int ret = -1;
for (int8_t i = max_lock_slot_idx - 1; -1 == ret && i >= 0; --i) {
if (latch_addr == current_locks[i]) {
ret = i;
current_locks[i] = nullptr;
if (ret == max_lock_slot_idx - 1) {
--max_lock_slot_idx;
}
}
}
return ret;
}
OB_INLINE static void clear_lock()
{
MEMSET(current_locks, 0, max_lock_slot_idx * sizeof(current_locks[0]));
max_lock_slot_idx = 0;
}
static thread_local uint32_t* current_locks[16];
static thread_local uint32_t* current_wait;
static thread_local int8_t max_lock_slot_idx;
private:
template<typename LowTryLock>
OB_INLINE int low_lock(
@ -459,7 +495,7 @@ OB_INLINE uint64_t ObLatchMutex::low_try_lock(const int64_t max_spin_cnt, const
for (; spin_cnt < max_spin_cnt; ++spin_cnt) {
if (0 == lock_.val()) {
if (ATOMIC_BCAS(&lock_.val(), 0, lock_value)) {
ObLatch::current_lock = (uint32_t*)(&lock_.val());
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)(&lock_.val()));
break;
}
}
@ -521,7 +557,7 @@ inline int ObLatch::LowTryRDLock::operator()(volatile uint32_t *latch,
conflict = false;
if (ATOMIC_BCAS(latch, lock, lock + 1)) {
ret = OB_SUCCESS;
ObLatch::current_lock = (uint32_t*)latch;
IGNORE_RETURN unreg_lock((uint32_t*)latch);
}
} else {
conflict = true;
@ -544,7 +580,7 @@ inline int ObLatch::LowTryWRLock::operator()(volatile uint32_t *latch,
conflict = false;
if (ATOMIC_BCAS(latch, lock, (lock | (WRITE_MASK | uid)))) {
ret = OB_SUCCESS;
ObLatch::current_lock = (uint32_t*)latch;
IGNORE_RETURN unreg_lock((uint32_t*)latch);
}
} else {
conflict = true;

View File

@ -150,7 +150,7 @@ public:
}
if (OB_SUCC(ret)) {
// record in try_rdlock will be overwrited by latch_.rdlock, so record again.
ObLatch::current_lock = (uint32_t*)&(latch_.lock_);
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&(latch_.lock_));
}
return ret;
}
@ -161,7 +161,7 @@ public:
get_tcref().inc_ref(&read_ref_);
if (OB_LIKELY(0 == ATOMIC_LOAD(&write_id_))) {
locked = true;
ObLatch::current_lock = (uint32_t*)&(latch_.lock_);
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&(latch_.lock_));
} else {
get_tcref().dec_ref(&read_ref_);
lcond_.signal();
@ -173,7 +173,7 @@ public:
{
int ret = OB_SUCCESS;
get_tcref().dec_ref(&read_ref_);
ObLatch::current_lock = nullptr;
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&(latch_.lock_));
lcond_.signal();
return ret;
}
@ -260,7 +260,7 @@ public:
get_tcref().inc_ref(&read_ref_, slot_id);
if (OB_LIKELY(0 == ATOMIC_LOAD(&write_id_))) {
locked = true;
ObLatch::current_lock = (uint32_t*)&(latch_.lock_);
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&(latch_.lock_));
} else {
get_tcref().dec_ref(&read_ref_, slot_id);
lcond_.signal();
@ -278,7 +278,7 @@ public:
}
if (OB_SUCC(ret)) {
// record in try_rdlock will be overwrited by latch_.rdlock, so record again.
ObLatch::current_lock = (uint32_t*)&(latch_.lock_);
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&(latch_.lock_));
}
return ret;
}
@ -286,7 +286,7 @@ public:
{
int ret = OB_SUCCESS;
get_tcref().dec_ref(&read_ref_, slot_id);
ObLatch::current_lock = nullptr;
IGNORE_RETURN ObLatch::unreg_lock((uint32_t*)&(latch_.lock_));
lcond_.signal();
return ret;
}

View File

@ -27,6 +27,7 @@
#include "lib/worker.h"
#include "lib/thread/ob_thread_name.h"
#include "lib/thread/thread.h"
#include "lib/thread/protected_stack_allocator.h"
using namespace oceanbase::lib;
@ -246,6 +247,7 @@ void *ObBaseLogWriter::flush_log_thread(void *arg)
if (OB_ISNULL(arg)) {
LOG_STDERR("invalid argument, arg = %p\n", arg);
} else {
ObStackHeaderGuard stack_header_guard;
pthread_cleanup_push(cleanup_log_thread, arg);
ObBaseLogWriter *log_writer = reinterpret_cast<ObBaseLogWriter*> (arg);
lib::set_thread_name(log_writer->thread_name_);

View File

@ -19,6 +19,7 @@
#include "lib/stat/ob_latch_define.h"
#include "lib/utility/ob_print_utils.h"
#include "lib/ob_lib_config.h"
#include "lib/thread/thread.h"
namespace oceanbase
{
@ -183,6 +184,23 @@ private:
bool need_record_;
};
class ObSleepEventGuard : public ObWaitEventGuard
{
public:
explicit ObSleepEventGuard(
const int64_t event_no,
const uint64_t timeout_ms = 0,
const int64_t sleep_us = 0
) : ObWaitEventGuard(event_no, timeout_ms, sleep_us, 0, 0, false)
{
lib::Thread::sleep_us_ = sleep_us;
}
~ObSleepEventGuard()
{
lib::Thread::sleep_us_ = 0;
}
};
class ObMaxWaitGuard
{
public:

View File

@ -2,21 +2,23 @@
#define _OCEABASE_TENANT_PRELOAD_H_
#define _GNU_SOURCE 1
#include "lib/worker.h"
#include "lib/thread/thread.h"
#include "lib/thread/ob_thread_name.h"
#include "lib/thread/protected_stack_allocator.h"
#include <dlfcn.h>
#define SYS_HOOK(func_name, ...) \
({ \
int ret = 0; \
if (!in_sys_hook++ && OB_NOT_NULL(oceanbase::lib::Worker::self_)) { \
oceanbase::lib::Worker::self_->set_is_blocking(true); \
ret = real_##func_name(__VA_ARGS__); \
oceanbase::lib::Worker::self_->set_is_blocking(false); \
} else { \
ret = real_##func_name(__VA_ARGS__); \
} \
in_sys_hook--; \
ret; \
#define SYS_HOOK(func_name, ...) \
({ \
int ret = 0; \
if (!in_sys_hook++) { \
oceanbase::lib::Thread::is_blocking_ = true; \
ret = real_##func_name(__VA_ARGS__); \
oceanbase::lib::Thread::is_blocking_ = false; \
} else { \
ret = real_##func_name(__VA_ARGS__); \
} \
in_sys_hook--; \
ret; \
})
namespace oceanbase {
@ -88,6 +90,17 @@ int pthread_rwlock_wrlock(pthread_rwlock_t *__rwlock)
return ret;
}
int pthread_join(pthread_t _thread, void **__retval)
{
static int (*real_pthread_join)(pthread_t _thread, void **__retval) =
(typeof(real_pthread_join))dlsym(RTLD_NEXT, "pthread_join");
int ret = 0;
::oceanbase::lib::Thread::thread_joined_ = _thread;
ret = SYS_HOOK(pthread_join, _thread, __retval);
::oceanbase::lib::Thread::thread_joined_ = 0;
return ret;
}
#ifdef __USE_XOPEN2K
int pthread_rwlock_timedwrlock(pthread_rwlock_t *__restrict __rwlock,
const struct timespec *__restrict __abstime)
@ -137,6 +150,45 @@ int futex_hook(uint32_t *uaddr, int futex_op, uint32_t val, const struct timespe
return ret;
}
struct PthreadCreateArgument
{
PthreadCreateArgument(void *(*start_routine)(void *), void *arg)
{
start_routine_ = start_routine;
arg_ = arg;
in_use_ = 1;
}
void *(*start_routine_)(void *);
void *arg_;
int in_use_; // TO avoid memory alloc, there is a sync wait for pthread_create.
};
void* run_func(void* arg)
{
struct PthreadCreateArgument* parg = (struct PthreadCreateArgument*)arg;
void *(*start_routine)(void *) = parg->start_routine_;
void *real_arg = parg->arg_;
ATOMIC_STORE(&(parg->in_use_), 0);
::oceanbase::lib::ObStackHeaderGuard stack_header_guard;
return start_routine(real_arg);
}
int ob_pthread_create(pthread_t *thread, const pthread_attr_t *attr,
void *(*start_routine) (void *), void *arg)
{
struct PthreadCreateArgument parg(start_routine, arg);
int ret = pthread_create(thread, attr, run_func, &parg);
while (ATOMIC_LOAD(&(parg.in_use_)) != 0) {
sched_yield();
}
return ret;
}
void ob_set_thread_name(const char* type)
{
::oceanbase::lib::set_thread_name(type);
}
} /* extern "C" */
#endif /* _OCEABASE_TENANT_PRELOAD_H_ */

View File

@ -156,5 +156,16 @@ void StackMgr::erase(ObStackHeader *header)
}
}
ObStackHeaderGuard::ObStackHeaderGuard()
{
header_.pth_ = (uint64_t)pthread_self();
g_stack_mgr.insert(&header_);
}
ObStackHeaderGuard::~ObStackHeaderGuard()
{
g_stack_mgr.erase(&header_);
}
} // lib
} // oceanbase

View File

@ -52,12 +52,32 @@ class StackMgr
{
friend class ObMemoryCutter;
public:
class Guard
{
public:
Guard(StackMgr& mgr) : mgr_(mgr), cur_(nullptr)
{
mgr_.mutex_.lock();
cur_ = mgr_.dummy_.next_;
}
~Guard() { mgr_.mutex_.unlock(); }
ObStackHeader* operator*() { return (cur_ == &(mgr_.dummy_)) ? nullptr : cur_; }
ObStackHeader* next()
{
cur_ = cur_->next_;
return (cur_ == &(mgr_.dummy_)) ? nullptr : cur_;
}
private:
StackMgr& mgr_;
ObStackHeader* cur_;
};
StackMgr()
{
dummy_.prev_ = dummy_.next_ = &dummy_;
}
void insert(ObStackHeader *);
void erase(ObStackHeader *);
private:
ObStackHeader *begin() { return dummy_.next_; }
ObStackHeader *end() { return &dummy_; }
private:
@ -65,6 +85,15 @@ private:
ObStackHeader dummy_;
};
class ObStackHeaderGuard
{
public:
ObStackHeaderGuard();
~ObStackHeaderGuard();
private:
ObStackHeader header_;
};
extern ProtectedStackAllocator g_stack_allocer;
extern StackMgr g_stack_mgr;

View File

@ -30,8 +30,10 @@
using namespace oceanbase;
using namespace oceanbase::common;
using namespace oceanbase::lib;
thread_local int64_t Thread::loop_ts_ = 0;
thread_local pthread_t Thread::thread_joined_ = 0;
thread_local int64_t Thread::sleep_us_ = 0;
thread_local bool Thread::is_blocking_ = false;
thread_local Thread* Thread::current_thread_ = nullptr;
int64_t Thread::total_thread_count_ = 0;

View File

@ -16,6 +16,7 @@
#include <functional>
#include "lib/time/ob_time_utility.h"
#include "lib/utility/ob_macro_utils.h"
#include "lib/lock/ob_latch.h"
namespace oceanbase {
namespace lib {
@ -50,6 +51,7 @@ public:
{
int64_t ret = loop_ts_;
loop_ts_ = t;
ObLatch::clear_lock();
return ret;
}
@ -59,7 +61,9 @@ public:
}
public:
static thread_local int64_t loop_ts_;
static thread_local pthread_t thread_joined_;
static thread_local int64_t sleep_us_;
static thread_local bool is_blocking_;
private:
static void* __th_start(void *th);
void destroy_stack();

View File

@ -182,7 +182,7 @@ void max(T, T) = delete;
template<oceanbase::common::ObWaitEventIds::ObWaitEventIdEnum event_id = oceanbase::common::ObWaitEventIds::DEFAULT_SLEEP>
inline void ob_usleep(const useconds_t v)
{
oceanbase::common::ObWaitEventGuard wait_guard(event_id, 0, (int64_t)v);
oceanbase::common::ObSleepEventGuard wait_guard(event_id, 0, (int64_t)v);
::usleep(v);
}

View File

@ -47,7 +47,6 @@ Worker::Worker()
st_current_priority_(0),
session_(nullptr),
cur_request_(nullptr),
is_blocking_(false),
worker_level_(INT32_MAX),
curr_request_level_(0),
group_id_(0),

View File

@ -73,10 +73,6 @@ public:
void set_req_flag(const rpc::ObRequest *cur_request) { cur_request_ = cur_request; }
bool has_req_flag() { return OB_NOT_NULL(cur_request_); }
const rpc::ObRequest *get_cur_request() { return cur_request_; }
OB_INLINE void set_is_blocking(bool v) { is_blocking_ = v; }
OB_INLINE bool is_blocking() { return is_blocking_; }
OB_INLINE void set_worker_level(const int32_t level) { worker_level_ = level; }
OB_INLINE int32_t get_worker_level() const { return worker_level_; }
@ -133,8 +129,6 @@ protected:
private:
const rpc::ObRequest *cur_request_;
// whether worker is in blocking
bool is_blocking_;
int32_t worker_level_;
int32_t curr_request_level_;
int32_t group_id_;

View File

@ -1,4 +1,3 @@
#include <sys/prctl.h>
#define MAX_PN_LISTEN 256
#define MAX_PN_GRP (1<<17)
#define MAX_PN_PER_GRP 64
@ -60,7 +59,7 @@ static pn_listen_t* locate_listen(int idx)
static void* listen_thread_func(void* arg)
{
thread_counter_reg();
prctl(PR_SET_NAME, "pnlisten");
ob_set_thread_name("pnlisten");
pn_listen_t* l = (typeof(l))arg;
eloop_run(&l->l.ep);
return NULL;
@ -69,7 +68,7 @@ static void* listen_thread_func(void* arg)
static void* pn_thread_func(void* arg)
{
thread_counter_reg();
prctl(PR_SET_NAME, "pnio");
ob_set_thread_name("pnio");
pn_t* pn = (typeof(pn))arg;
eloop_run(&pn->ep);
return NULL;
@ -86,7 +85,7 @@ PN_API int pn_listen(int port, serve_cb_t cb)
idx = -1;
} else {
pnl->serve_cb = cb;
pthread_create(&pnl->pd, NULL, listen_thread_func, pnl);
ob_pthread_create(&pnl->pd, NULL, listen_thread_func, pnl);
}
return idx;
}
@ -232,7 +231,7 @@ PN_API int pn_provision(int listen_id, int gid, int thread_count)
pn_t* pn = pn_create(listen_id, gid, count);
if (NULL == pn) {
err = ENOMEM;
} else if (0 != (err = pthread_create(&pn->pd, NULL, pn_thread_func, pn))) {
} else if (0 != (err = ob_pthread_create(&pn->pd, NULL, pn_thread_func, pn))) {
pn_destroy(pn);
} else {
pn_grp->pn_array[count++] = pn;

View File

@ -1,10 +1,14 @@
#include <stdio.h>
#include <unistd.h>
#include <sys/syscall.h>
#include <sys/prctl.h>
#include <pthread.h>
typedef void (*log_func_t)(int level, const char *file, int line, const char *function, const char *fmt, va_list ap);
extern void do_log(int level, const char* file, int line, const char* func, const char* format, ...) __attribute__((format(printf, 5, 6)));
extern void ob_set_thread_name(const char* type);
extern int ob_pthread_create(pthread_t *thread, const pthread_attr_t *attr,
void *(*start_routine) (void *), void *arg);
extern log_func_t g_log_func;
extern int g_log_level;
enum { LOG_LEVEL_ERROR = 0, LOG_LEVEL_USER_LEVEL = 1, LOG_LEVEL_WARN = 2, LOG_LEVEL_INFO = 3, LOG_LEVEL_TRACE = 4, LOG_LEVEL_DEBUG = 5 };
@ -13,6 +17,15 @@ enum { LOG_LEVEL_ERROR = 0, LOG_LEVEL_USER_LEVEL = 1, LOG_LEVEL_WARN = 2, LOG_LE
extern __thread format_t g_log_fbuf;
#ifndef rk_log_macro
#define rk_log_macro(level, ret, ...) { if (LOG_LEVEL_ ## level <= g_log_level) do_log(LOG_LEVEL_ ## level, __FILE__, __LINE__, __func__, ##__VA_ARGS__); }
void ob_set_thread_name(const char* type)
{
prctl(PR_SET_NAME, type);
}
int ob_pthread_create(pthread_t *thread, const pthread_attr_t *attr,
void *(*start_routine) (void *), void *arg)
{
return pthread_create(thread, attr, start_routine, arg);
}
#endif
#define do_rk_log_macro(...) { format_reset(&g_log_fbuf); rk_log_macro(__VA_ARGS__); }
#define rk_error(...) do_rk_log_macro(ERROR, oceanbase::common::OB_ERR_SYS, ##__VA_ARGS__)

View File

@ -1846,6 +1846,9 @@ int select_schema_slot(lua_State *L)
return 1;
}
#define GET_OTHER_TSI_ADDR(type, var_name, addr) \
const int64_t var_name##_offset = ((int64_t)addr - (int64_t)pthread_self()); \
type var_name = *(type*)(thread_base + var_name##_offset);
// list{list, list...} = dump_threads_info()
int dump_thread_info(lua_State *L)
{
@ -1858,71 +1861,112 @@ int dump_thread_info(lua_State *L)
"tid",
"thread_base",
"loop_ts",
"lock_addr",
"lock_val",
"wait_addr",
"wait_val",
"is_blocking",
"has_req"
"latch_hold",
"latch_wait",
"trace_id",
"status",
"wait_event"
};
LuaVtableGenerator gen(L, columns);
int64_t tname_offset = (int64_t)ob_get_tname() - (int64_t)pthread_self();
int64_t tid_offset = (int64_t)(&get_tid_cache()) - (int64_t)pthread_self();
int64_t loop_ts_offset = (int64_t)(&oceanbase::lib::Thread::loop_ts_) - (int64_t)pthread_self();
int64_t lock_offset = (int64_t)(&ObLatch::current_lock) - (int64_t)pthread_self();
int64_t wait_offset = (int64_t)(&ObLatch::current_wait) - (int64_t)pthread_self();
int64_t worker_offset = (int64_t)(&oceanbase::lib::Worker::self_) - (int64_t)pthread_self();
for(auto* header = g_stack_mgr.begin(); header != g_stack_mgr.end() && !gen.is_end(); header = header->next_) {
StackMgr::Guard guard(g_stack_mgr);
for(auto* header = *guard; OB_NOT_NULL(header) && !gen.is_end(); header = guard.next()) {
auto* thread_base = (char*)(header->pth_);
if (OB_NOT_NULL(thread_base)) {
// avoid SMART_CALL stack
char* tname = thread_base + tname_offset;
int64_t tid = *(int64_t*)(thread_base + tid_offset);
int64_t loop_ts = *(int64_t*)(thread_base + loop_ts_offset);
uint32_t* lock_addr = *(uint32_t**)(thread_base + lock_offset);
uint32_t* wait_addr = *(uint32_t**)(thread_base + wait_offset);
auto* worker_self = *(Worker**)(thread_base + worker_offset);
char addr[32];
gen.next_row();
// tname
GET_OTHER_TSI_ADDR(char*, tname, ob_get_tname());
// PAY ATTENTION HERE
tname = thread_base + tname_offset;
gen.next_column(tname);
// tid
GET_OTHER_TSI_ADDR(int64_t, tid, &get_tid_cache());
gen.next_column(tid);
// thread_base
snprintf(addr, 32, "%p", thread_base);
gen.next_column(addr);
// loop_ts
gen.next_column(loop_ts);
// lock_addr
// lock_val
if (OB_NOT_NULL(lock_addr)) {
snprintf(addr, 32, "%p", lock_addr);
{
char addr[32];
snprintf(addr, 32, "%p", thread_base);
gen.next_column(addr);
gen.next_column(*lock_addr);
} else {
gen.next_column("NULL");
gen.next_column("NULL");
}
// wait_addr
// wait_val
// loop_ts
GET_OTHER_TSI_ADDR(int64_t, loop_ts, &oceanbase::lib::Thread::loop_ts_);
gen.next_column(loop_ts);
// latch_hold
{
GET_OTHER_TSI_ADDR(uint32_t**, locks_addr, &ObLatch::current_locks);
locks_addr = (uint32_t**)(thread_base + locks_addr_offset);
char addrs[256];
addrs[0] = 0;
for (auto i = 0, offset1 = 0; i < sizeof(ObLatch::current_locks) / sizeof(uint32_t*); ++i) {
if (OB_NOT_NULL(locks_addr[i])) {
offset1 = snprintf(addrs + offset1, 256 - offset1, "%p ", locks_addr[i]);
}
}
if (0 == addrs[0]) {
gen.next_column("NULL");
} else {
gen.next_column(addrs);
}
}
// latch_wait
GET_OTHER_TSI_ADDR(uint32_t*, wait_addr, &ObLatch::current_wait);
if (OB_NOT_NULL(wait_addr)) {
char addr[32];
snprintf(addr, 32, "%p", wait_addr);
gen.next_column(addr);
gen.next_column(*wait_addr);
} else {
gen.next_column("NULL");
gen.next_column("NULL");
}
// is_blocking
// has_req
if (OB_NOT_NULL(worker_self)) {
gen.next_column(worker_self->is_blocking());
gen.next_column(worker_self->has_req_flag());
} else {
gen.next_column("NULL");
gen.next_column("NULL");
// trace_id
{
GET_OTHER_TSI_ADDR(ObCurTraceId::TraceId, trace_id, ObCurTraceId::get_trace_id());
char trace_id_buf[40];
IGNORE_RETURN trace_id.to_string(trace_id_buf, 40);
gen.next_column(trace_id_buf);
}
// status
{
GET_OTHER_TSI_ADDR(pthread_t, join_addr, &Thread::thread_joined_);
GET_OTHER_TSI_ADDR(int64_t, sleep_us, &Thread::sleep_us_);
GET_OTHER_TSI_ADDR(bool, is_blocking, &Thread::is_blocking_);
const char* status_str = nullptr;
if (0 != join_addr) {
status_str = "Join";
} else if (0 != sleep_us) {
status_str = "Sleep";
} else if (is_blocking) {
status_str = "Wait";
} else {
status_str = "Run";
}
gen.next_column(status_str);
}
// wait_event
{
GET_OTHER_TSI_ADDR(uint32_t*, wait_addr, &ObLatch::current_wait);
GET_OTHER_TSI_ADDR(pthread_t, join_addr, &Thread::thread_joined_);
GET_OTHER_TSI_ADDR(int64_t, sleep_us, &Thread::sleep_us_);
char wait_event[32];
wait_event[0] = '\0';
if (0 != join_addr) {
IGNORE_RETURN snprintf(wait_event, 32, "thread %u", *(uint32_t*)(thread_base + tid_offset));
} else if (0 != sleep_us) {
IGNORE_RETURN snprintf(wait_event, 32, "%ld us", sleep_us);
} else if (OB_NOT_NULL(wait_addr)) {
bool has_segv = false;
uint32_t val = 0;
do_with_crash_restore([&] {
val = *wait_addr;
}, has_segv);
if (has_segv) {
} else if (0 != (val & (1<<30))) {
IGNORE_RETURN snprintf(wait_event, 32, "wrlock on %u", val & 0x3fffffff);
} else {
IGNORE_RETURN snprintf(wait_event, 32, "%u rdlocks", val & 0x3fffffff);
}
}
gen.next_column(wait_event);
}
gen.row_end();
}
}

View File

@ -26,7 +26,9 @@
#include "lib/atomic/ob_atomic.h"
#include "lib/signal/ob_signal_utils.h"
#include "lib/thread/ob_thread_name.h"
#include "lib/thread/protected_stack_allocator.h"
#include "lib/utility/utility.h"
#include "lib/thread/thread.h"
extern "C" {
#include <lua.h>
@ -166,6 +168,7 @@ int ObUnixDomainListener::run()
ATOMIC_STORE(&stop_, false);
worker_ = std::thread([=]() {
lib::set_thread_name("LuaHandler");
lib::ObStackHeaderGuard stack_header_guard;
constexpr int64_t EPOLL_EVENT_BUFFER_SIZE = 32;
constexpr int64_t TIMEOUT = 1000;
struct epoll_event events[EPOLL_EVENT_BUFFER_SIZE];
@ -174,6 +177,7 @@ int ObUnixDomainListener::run()
while (OB_LIKELY(!ATOMIC_LOAD(&stop_))) {
int conn_fd = -1;
int ret = OB_SUCCESS;
lib::Thread::update_loop_ts();
int64_t event_cnt = epoll_wait(epoll_fd, events, EPOLL_EVENT_BUFFER_SIZE, TIMEOUT);
if (event_cnt < 0) {
if (EINTR == errno) {

View File

@ -307,19 +307,18 @@ print_to_client("select_schema_slot")
select_schema_slot(para)
para = {}
para["limit"] = {10}
para["limit"] = {}
para["dump"] = true
para["select"] = {
"tname",
"tid",
"thread_base",
"loop_ts",
"lock_addr",
"lock_val",
"wait_addr",
"wait_val",
"is_blocking",
"has_req"
"latch_hold",
"latch_wait",
"trace_id",
"status",
"wait_event"
}
print_to_client("dump_thread_info")
dump_thread_info(para)

View File

@ -362,6 +362,7 @@ ob_set_subtarget(ob_server virtual_table
virtual_table/ob_virtual_show_trace.cpp
virtual_table/ob_all_virtual_sql_plan.cpp
virtual_table/ob_all_virtual_plan_table.cpp
virtual_table/ob_all_virtual_thread.cpp
)
ob_server_add_target(ob_server)

View File

@ -394,6 +394,10 @@ void reasy_pool_set_allocator(reasy_pool_realloc_pt alloc);
}
int main(int argc, char *argv[])
{
if (0 != pthread_getname_np(pthread_self(), ob_get_tname(), OB_THREAD_NAME_BUF_LEN)) {
snprintf(ob_get_tname(), OB_THREAD_NAME_BUF_LEN, "observer");
}
ObStackHeaderGuard stack_header_guard;
#ifndef OB_USE_ASAN
init_malloc_hook();
#endif

View File

@ -51,6 +51,7 @@
#include "share/rc/ob_tenant_module_init_ctx.h"
#include "share/resource_manager/ob_cgroup_ctrl.h"
#include "sql/engine/px/ob_px_worker.h"
#include "lib/thread/protected_stack_allocator.h"
using namespace oceanbase::lib;
using namespace oceanbase::common;
@ -373,7 +374,9 @@ void ObResourceGroup::check_worker_count()
if (w->has_set_stop()) {
workers_.remove(wnode);
destroy_worker(w);
} else if (w->has_req_flag() && w->is_blocking() && w->is_default_worker()) {
} else if (w->has_req_flag()
&& Thread::is_blocking_
&& w->is_default_worker()) {
++token;
}
}
@ -832,10 +835,12 @@ int ObTenant::create_tenant_module()
void* ObTenant::wait(void* t)
{
ObStackHeaderGuard stack_header_guard;
int ret = OB_SUCCESS;
ObTenant* tenant = (ObTenant*)t;
ob_get_tenant_id() = tenant->id_;
lib::set_thread_name("UnitGC");
lib::Thread::update_loop_ts();
tenant->handle_retry_req(true);
while (tenant->req_queue_.size() > 0) {
ob_usleep(10L * 1000L);
@ -1351,7 +1356,9 @@ void ObTenant::check_worker_count()
if (w->has_set_stop()) {
workers_.remove(wnode);
destroy_worker(w);
} else if (w->has_req_flag() && w->is_blocking() && w->is_default_worker()) {
} else if (w->has_req_flag()
&& Thread::is_blocking_
&& w->is_default_worker()) {
++token;
}
}

View File

@ -0,0 +1,222 @@
/**
* 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_all_virtual_thread.h"
#include "lib/signal/ob_signal_utils.h"
#include "lib/thread/protected_stack_allocator.h"
#define GET_OTHER_TSI_ADDR(type, var_name, addr) \
const int64_t var_name##_offset = ((int64_t)addr - (int64_t)pthread_self()); \
type var_name = *(type*)(thread_base + var_name##_offset);
namespace oceanbase
{
using namespace lib;
namespace observer
{
ObAllVirtualThread::ObAllVirtualThread() : is_inited_(false)
{
}
ObAllVirtualThread::~ObAllVirtualThread()
{
reset();
}
int ObAllVirtualThread::inner_open()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(ObServerConfig::get_instance().self_addr_.ip_to_string(ip_buf_, sizeof(ip_buf_))
== false)) {
ret = OB_ERR_UNEXPECTED;
SERVER_LOG(WARN, "ip_to_string() fail", K(ret));
}
return ret;
}
void ObAllVirtualThread::reset()
{
is_inited_ = false;
}
int ObAllVirtualThread::inner_get_next_row(common::ObNewRow *&row)
{
int ret = OB_SUCCESS;
if (!is_inited_) {
const int64_t col_count = output_column_ids_.count();
StackMgr::Guard guard(g_stack_mgr);
for (auto* header = *guard; OB_NOT_NULL(header); header = guard.next()) {
auto* thread_base = (char*)(header->pth_);
if (OB_NOT_NULL(thread_base)) {
GET_OTHER_TSI_ADDR(uint64_t, tenant_id, &ob_get_tenant_id());
if (!is_sys_tenant(effective_tenant_id_)
&& tenant_id != effective_tenant_id_) {
continue;
}
GET_OTHER_TSI_ADDR(int64_t, tid, &get_tid_cache());
GET_OTHER_TSI_ADDR(uint32_t*, wait_addr, &ObLatch::current_wait);
GET_OTHER_TSI_ADDR(pthread_t, join_addr, &Thread::thread_joined_);
GET_OTHER_TSI_ADDR(int64_t, sleep_us, &Thread::sleep_us_);
for (int64_t i = 0; i < col_count && OB_SUCC(ret); ++i) {
const uint64_t col_id = output_column_ids_.at(i);
ObObj *cells = cur_row_.cells_;
switch (col_id) {
case SVR_IP: {
cells[i].set_varchar(ip_buf_);
cells[i].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case SVR_PORT: {
cells[i].set_int(GCONF.self_addr_.get_port());
break;
}
case TENANT_ID: {
cells[i].set_int(tenant_id);
break;
}
case TID: {
cells[i].set_int(tid);
break;
}
case TNAME: {
GET_OTHER_TSI_ADDR(char*, tname, ob_get_tname());
// PAY ATTENTION HERE
MEMCPY(tname_, thread_base + tname_offset, sizeof(tname_));
cells[i].set_varchar(tname_);
cells[i].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case STATUS: {
GET_OTHER_TSI_ADDR(bool, is_blocking, &Thread::is_blocking_);
const char* status_str = nullptr;
if (0 != join_addr) {
status_str = "Join";
} else if (0 != sleep_us) {
status_str = "Sleep";
} else if (is_blocking) {
status_str = "Wait";
} else {
status_str = "Run";
}
cells[i].set_varchar(status_str);
cells[i].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case WAIT_EVENT: {
wait_event_[0] = '\0';
if (0 != join_addr) {
IGNORE_RETURN snprintf(wait_event_, 64, "thread %u", *(uint32_t*)(thread_base + tid_offset));
} else if (0 != sleep_us) {
IGNORE_RETURN snprintf(wait_event_, 64, "%ld us", sleep_us);
} else if (OB_NOT_NULL(wait_addr)) {
bool has_segv = false;
uint32_t val = 0;
do_with_crash_restore([&] {
val = *wait_addr;
}, has_segv);
if (has_segv) {
} else if (0 != (val & (1<<30))) {
IGNORE_RETURN snprintf(wait_event_, 64, "wrlock on %u", val & 0x3fffffff);
} else {
IGNORE_RETURN snprintf(wait_event_, 64, "%u rdlocks", val & 0x3fffffff);
}
}
cells[i].set_varchar(wait_event_);
cells[i].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case LATCH_WAIT_ADDR: {
if (OB_ISNULL(wait_addr)) {
cells[i].set_varchar("");
} else {
IGNORE_RETURN snprintf(wait_addr_, 16, "%p", wait_addr);
cells[i].set_varchar(wait_addr_);
}
cells[i].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case LATCH_HOLD_ADDR: {
GET_OTHER_TSI_ADDR(uint32_t**, locks_addr, &ObLatch::current_locks);
locks_addr = (uint32_t**)(thread_base + locks_addr_offset);
locks_addr_[0] = 0;
for (auto i = 0, j = 0; i < sizeof(ObLatch::current_locks) / sizeof(uint32_t*); ++i) {
if (OB_NOT_NULL(locks_addr[i])) {
bool has_segv = false;
uint32_t val = 0;
do_with_crash_restore([&] {
val = *locks_addr[i];
}, has_segv);
if (!has_segv && 0 != val) {
j = snprintf(locks_addr_ + j, 256 - j, "%p ", locks_addr[i]);
}
}
}
cells[i].set_varchar(locks_addr_);
cells[i].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case TRACE_ID: {
GET_OTHER_TSI_ADDR(ObCurTraceId::TraceId, trace_id, ObCurTraceId::get_trace_id());
IGNORE_RETURN trace_id.to_string(trace_id_buf_, sizeof(trace_id_buf_));
cells[i].set_varchar(trace_id_buf_);
cells[i].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case LOOP_TS: {
GET_OTHER_TSI_ADDR(int64_t, loop_ts, &oceanbase::lib::Thread::loop_ts_);
cells[i].set_timestamp(loop_ts);
break;
}
default: {
ret = OB_ERR_UNEXPECTED;
SERVER_LOG(WARN, "unexpected column id", K(col_id), K(i), K(ret));
break;
}
}
}
if (OB_SUCC(ret)) {
// scanner最大支持64M,因此暂不考虑溢出的情况
if (OB_FAIL(scanner_.add_row(cur_row_))) {
SERVER_LOG(WARN, "fail to add row", K(ret), K(cur_row_));
if (OB_SIZE_OVERFLOW == ret) {
ret = OB_SUCCESS;
}
}
}
}
}
if (OB_SUCC(ret)) {
scanner_it_ = scanner_.begin();
is_inited_ = true;
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(scanner_it_.get_next_row(cur_row_))) {
if (OB_ITER_END != ret) {
SERVER_LOG(WARN, "fail to get next row", K(ret));
}
} else {
row = &cur_row_;
}
}
return ret;
}
} // namespace observer
} // namespace oceanbase

View File

@ -0,0 +1,60 @@
/**
* 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 SRC_OBSERVER_VIRTUAL_TABLE_OB_ALL_VIRTUAL_THREAD_H_
#define SRC_OBSERVER_VIRTUAL_TABLE_OB_ALL_VIRTUAL_THREAD_H_
#include "share/ob_virtual_table_scanner_iterator.h"
namespace oceanbase
{
namespace observer
{
class ObAllVirtualThread : public common::ObVirtualTableScannerIterator
{
enum COLUMN_ID_LIST
{
SVR_IP = common::OB_APP_MIN_COLUMN_ID,
SVR_PORT,
TENANT_ID,
TID,
TNAME,
STATUS,
WAIT_EVENT,
LATCH_WAIT_ADDR,
LATCH_HOLD_ADDR,
TRACE_ID,
LOOP_TS
};
public:
ObAllVirtualThread();
virtual ~ObAllVirtualThread() override;
virtual int inner_open() override;
virtual void reset() override;
virtual int inner_get_next_row(common::ObNewRow *&row) override;
private:
bool is_inited_;
char ip_buf_[common::OB_IP_STR_BUFF];
char tname_[16];
char wait_event_[64];
char wait_addr_[16];
char locks_addr_[256];
char trace_id_buf_[40];
private:
DISALLOW_COPY_AND_ASSIGN(ObAllVirtualThread);
};
}
}
#endif /* SRC_OBSERVER_VIRTUAL_TABLE_OB_ALL_VIRTUAL_THREAD_H_ */

View File

@ -196,6 +196,7 @@
#include "observer/virtual_table/ob_virtual_show_trace.h"
#include "observer/virtual_table/ob_all_virtual_sql_plan.h"
#include "observer/virtual_table/ob_all_virtual_plan_table.h"
#include "observer/virtual_table/ob_all_virtual_thread.h"
namespace oceanbase
{
@ -2317,6 +2318,13 @@ int ObVTIterCreator::create_vt_iter(ObVTableScanParam &params,
// }
// break;
// }
case OB_ALL_VIRTUAL_THREAD_TID: {
ObAllVirtualThread *all_virtual_thread = NULL;
if (OB_SUCC(NEW_VIRTUAL_TABLE(ObAllVirtualThread, all_virtual_thread))) {
vt_iter = static_cast<ObVirtualTableIterator *>(all_virtual_thread);
}
break;
}
END_CREATE_VT_ITER_SWITCH_LAMBDA
#define AGENT_VIRTUAL_TABLE_CREATE_ITER

View File

@ -2754,6 +2754,226 @@ int ObInnerTableSchema::all_virtual_io_scheduler_schema(ObTableSchema &table_sch
return ret;
}
int ObInnerTableSchema::all_virtual_thread_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_THREAD_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(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_THREAD_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("svr_ip", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
1, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
MAX_IP_ADDR_LENGTH, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("svr_port", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
2, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("tenant_id", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("tid", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("tname", //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
16, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("status", //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
32, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("wait_event", //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
64, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("latch_wait", //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
16, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("latch_hold", //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
256, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("trace_id", //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
40, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("loop_ts", //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)) {
table_schema.get_part_option().set_part_num(1);
table_schema.set_part_level(PARTITION_LEVEL_ONE);
table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_LIST_COLUMNS);
if (OB_FAIL(table_schema.get_part_option().set_part_expr("svr_ip, svr_port"))) {
LOG_WARN("set_part_expr failed", K(ret));
} else if (OB_FAIL(table_schema.mock_list_partition_array())) {
LOG_WARN("mock list partition array failed", K(ret));
}
}
table_schema.set_index_using_type(USING_HASH);
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_virtual_virtual_long_ops_status_mysql_sys_agent_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;

View File

@ -8741,6 +8741,225 @@ int ObInnerTableSchema::all_virtual_long_ops_status_sys_agent_schema(ObTableSche
return ret;
}
int ObInnerTableSchema::all_virtual_thread_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_THREAD_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(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_THREAD_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("SVR_IP", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
1, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_UTF8MB4_BIN, //column_collation_type
MAX_IP_ADDR_LENGTH, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("SVR_PORT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
2, //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("TENANT_ID", //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("TID", //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("TNAME", //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
16, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("STATUS", //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
32, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("WAIT_EVENT", //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
64, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("LATCH_WAIT", //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
16, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("LATCH_HOLD", //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
256, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TRACE_ID", //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
40, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("LOOP_TS", //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)) {
table_schema.get_part_option().set_part_num(1);
table_schema.set_part_level(PARTITION_LEVEL_ONE);
table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_LIST);
if (OB_FAIL(table_schema.get_part_option().set_part_expr("SVR_IP, SVR_PORT"))) {
LOG_WARN("set_part_expr failed", K(ret));
} else if (OB_FAIL(table_schema.mock_list_partition_array())) {
LOG_WARN("mock list partition array failed", K(ret));
}
}
table_schema.set_index_using_type(USING_HASH);
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,106 @@ int ObInnerTableSchema::dba_ob_rsrc_io_directives_schema(ObTableSchema &table_sc
return ret;
}
int ObInnerTableSchema::gv_ob_thread_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_GV_OB_THREAD_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_GV_OB_THREAD_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 svr_ip AS SVR_IP, svr_port AS SVR_PORT, tenant_id AS TENANT_ID, tid AS TID, tname AS TNAME, status AS STATUS, latch_wait AS LATCH_WAIT, latch_hold AS LATCH_HOLD, trace_id AS TRACE_ID FROM oceanbase.__all_virtual_thread )__"))) {
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::v_ob_thread_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_V_OB_THREAD_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_V_OB_THREAD_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 * FROM oceanbase.GV$OB_THREAD WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) {
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

@ -675,6 +675,106 @@ int ObInnerTableSchema::v_session_longops_ora_schema(ObTableSchema &table_schema
return ret;
}
int ObInnerTableSchema::gv_ob_thread_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_GV_OB_THREAD_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_GV_OB_THREAD_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 svr_ip AS SVR_IP, svr_port AS SVR_PORT, tenant_id AS TENANT_ID, tid AS TID, tname AS TNAME, status AS STATUS, latch_wait AS LATCH_WAIT, latch_hold AS LATCH_HOLD, trace_id AS TRACE_ID FROM SYS.ALL_VIRTUAL_THREAD )__"))) {
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::v_ob_thread_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_V_OB_THREAD_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_V_OB_THREAD_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 * FROM SYS.GV$OB_THREAD WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) {
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

@ -870,6 +870,7 @@ public:
static int all_virtual_ls_arb_replica_task_history_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_archive_dest_status_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_io_scheduler_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_thread_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_virtual_long_ops_status_mysql_sys_agent_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);
@ -1079,6 +1080,7 @@ public:
static int all_virtual_ls_arb_replica_task_history_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_archive_dest_status_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_long_ops_status_sys_agent_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_thread_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);
@ -1366,6 +1368,8 @@ public:
static int dba_ob_ls_log_archive_progress_schema(share::schema::ObTableSchema &table_schema);
static int cdb_ob_ls_log_archive_progress_schema(share::schema::ObTableSchema &table_schema);
static int dba_ob_rsrc_io_directives_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_thread_schema(share::schema::ObTableSchema &table_schema);
static int v_ob_thread_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);
@ -1738,6 +1742,8 @@ public:
static int dba_ob_ls_log_archive_progress_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_session_longops_ora_schema(share::schema::ObTableSchema &table_schema);
static int v_session_longops_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_thread_ora_schema(share::schema::ObTableSchema &table_schema);
static int v_ob_thread_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);
@ -2917,6 +2923,7 @@ const schema_create_func virtual_table_schema_creators [] = {
ObInnerTableSchema::all_virtual_ls_arb_replica_task_history_schema,
ObInnerTableSchema::all_virtual_archive_dest_status_schema,
ObInnerTableSchema::all_virtual_io_scheduler_schema,
ObInnerTableSchema::all_virtual_thread_schema,
ObInnerTableSchema::all_virtual_virtual_long_ops_status_mysql_sys_agent_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,
@ -3135,6 +3142,7 @@ const schema_create_func virtual_table_schema_creators [] = {
ObInnerTableSchema::all_virtual_ls_arb_replica_task_history_ora_schema,
ObInnerTableSchema::all_virtual_archive_dest_status_ora_schema,
ObInnerTableSchema::all_virtual_long_ops_status_sys_agent_schema,
ObInnerTableSchema::all_virtual_thread_ora_schema,
ObInnerTableSchema::all_virtual_sql_plan_monitor_ora_all_virtual_sql_plan_monitor_i1_schema,
ObInnerTableSchema::all_virtual_system_event_ora_all_virtual_system_event_i1_schema,
ObInnerTableSchema::all_virtual_sysstat_ora_all_virtual_sysstat_i1_schema,
@ -3433,6 +3441,8 @@ const schema_create_func sys_view_schema_creators [] = {
ObInnerTableSchema::dba_ob_ls_log_archive_progress_schema,
ObInnerTableSchema::cdb_ob_ls_log_archive_progress_schema,
ObInnerTableSchema::dba_ob_rsrc_io_directives_schema,
ObInnerTableSchema::gv_ob_thread_schema,
ObInnerTableSchema::v_ob_thread_schema,
ObInnerTableSchema::dba_synonyms_schema,
ObInnerTableSchema::dba_objects_ora_schema,
ObInnerTableSchema::all_objects_schema,
@ -3805,6 +3815,8 @@ const schema_create_func sys_view_schema_creators [] = {
ObInnerTableSchema::dba_ob_ls_log_archive_progress_ora_schema,
ObInnerTableSchema::gv_session_longops_ora_schema,
ObInnerTableSchema::v_session_longops_ora_schema,
ObInnerTableSchema::gv_ob_thread_ora_schema,
ObInnerTableSchema::v_ob_thread_ora_schema,
NULL,};
const schema_create_func core_index_table_schema_creators [] = {
@ -4283,6 +4295,7 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_TID,
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_TID,
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_TID,
OB_ALL_VIRTUAL_THREAD_TID,
OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TID,
@ -4500,6 +4513,7 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_ORA_TID,
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA_TID,
OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TID,
OB_ALL_VIRTUAL_THREAD_ORA_TID,
OB_GV_OB_PLAN_CACHE_STAT_TID,
OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID,
OB_SCHEMATA_TID,
@ -4710,6 +4724,8 @@ const uint64_t tenant_space_tables [] = {
OB_V_OB_ARCHIVE_DEST_STATUS_TID,
OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_TID,
OB_DBA_OB_RSRC_IO_DIRECTIVES_TID,
OB_GV_OB_THREAD_TID,
OB_V_OB_THREAD_TID,
OB_DBA_SYNONYMS_TID,
OB_DBA_OBJECTS_ORA_TID,
OB_ALL_OBJECTS_TID,
@ -5082,6 +5098,8 @@ const uint64_t tenant_space_tables [] = {
OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_ORA_TID,
OB_GV_SESSION_LONGOPS_ORA_TID,
OB_V_SESSION_LONGOPS_ORA_TID,
OB_GV_OB_THREAD_ORA_TID,
OB_V_OB_THREAD_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,
@ -5705,7 +5723,8 @@ const uint64_t all_ora_mapping_virtual_table_org_tables [] = {
OB_ALL_VIRTUAL_TRANS_SCHEDULER_TID,
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_TID,
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_TID,
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_TID, };
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_TID,
OB_ALL_VIRTUAL_THREAD_TID, };
const uint64_t all_ora_mapping_virtual_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID
, OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID
@ -5817,6 +5836,7 @@ const uint64_t all_ora_mapping_virtual_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_O
, OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_ORA_TID
, OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_ORA_TID
, OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA_TID
, OB_ALL_VIRTUAL_THREAD_ORA_TID
, };
/* start/end_pos is start/end postition for column with tenant id */
@ -6200,6 +6220,7 @@ const char* const tenant_space_table_names [] = {
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_TNAME,
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_TNAME,
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_TNAME,
OB_ALL_VIRTUAL_THREAD_TNAME,
OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TNAME,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TNAME,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TNAME,
@ -6417,6 +6438,7 @@ const char* const tenant_space_table_names [] = {
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_ORA_TNAME,
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA_TNAME,
OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TNAME,
OB_ALL_VIRTUAL_THREAD_ORA_TNAME,
OB_GV_OB_PLAN_CACHE_STAT_TNAME,
OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME,
OB_SCHEMATA_TNAME,
@ -6627,6 +6649,8 @@ const char* const tenant_space_table_names [] = {
OB_V_OB_ARCHIVE_DEST_STATUS_TNAME,
OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_TNAME,
OB_DBA_OB_RSRC_IO_DIRECTIVES_TNAME,
OB_GV_OB_THREAD_TNAME,
OB_V_OB_THREAD_TNAME,
OB_DBA_SYNONYMS_TNAME,
OB_DBA_OBJECTS_ORA_TNAME,
OB_ALL_OBJECTS_TNAME,
@ -6999,6 +7023,8 @@ const char* const tenant_space_table_names [] = {
OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_ORA_TNAME,
OB_GV_SESSION_LONGOPS_ORA_TNAME,
OB_V_SESSION_LONGOPS_ORA_TNAME,
OB_GV_OB_THREAD_ORA_TNAME,
OB_V_OB_THREAD_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,
@ -7638,6 +7664,7 @@ const uint64_t tenant_distributed_vtables [] = {
OB_ALL_VIRTUAL_QUERY_RESPONSE_TIME_TID,
OB_ALL_VIRTUAL_TABLET_COMPACTION_INFO_TID,
OB_ALL_VIRTUAL_MALLOC_SAMPLE_INFO_TID,
OB_ALL_VIRTUAL_THREAD_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TID,
OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID,
@ -7693,7 +7720,8 @@ const uint64_t tenant_distributed_vtables [] = {
OB_ALL_VIRTUAL_REPLAY_STAT_ORA_TID,
OB_ALL_VIRTUAL_APPLY_STAT_ORA_TID,
OB_ALL_VIRTUAL_ARCHIVE_STAT_ORA_TID,
OB_ALL_VIRTUAL_TRANS_SCHEDULER_ORA_TID, };
OB_ALL_VIRTUAL_TRANS_SCHEDULER_ORA_TID,
OB_ALL_VIRTUAL_THREAD_ORA_TID, };
const uint64_t restrict_access_virtual_tables[] = {
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID,
@ -7779,7 +7807,8 @@ const uint64_t restrict_access_virtual_tables[] = {
OB_ALL_VIRTUAL_TRANS_SCHEDULER_ORA_TID,
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_ORA_TID,
OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_ORA_TID,
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA_TID };
OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA_TID,
OB_ALL_VIRTUAL_THREAD_ORA_TID };
static inline bool is_restrict_access_virtual_table(const uint64_t tid)
@ -9828,11 +9857,11 @@ 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 = 230;
const int64_t OB_VIRTUAL_TABLE_COUNT = 577;
const int64_t OB_SYS_VIEW_COUNT = 659;
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1471;
const int64_t OB_VIRTUAL_TABLE_COUNT = 579;
const int64_t OB_SYS_VIEW_COUNT = 663;
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1477;
const int64_t OB_CORE_SCHEMA_VERSION = 1;
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1474;
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1480;
} // end namespace share
} // end namespace oceanbase

View File

@ -612,6 +612,7 @@ const uint64_t OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_TID = 12364; // "__all_virtual
const uint64_t OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_TID = 12365; // "__all_virtual_ls_arb_replica_task_history"
const uint64_t OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_TID = 12366; // "__all_virtual_archive_dest_status"
const uint64_t OB_ALL_VIRTUAL_IO_SCHEDULER_TID = 12369; // "__all_virtual_io_scheduler"
const uint64_t OB_ALL_VIRTUAL_THREAD_TID = 12384; // "__all_virtual_thread"
const uint64_t OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TID = 12393; // "__all_virtual_virtual_long_ops_status_mysql_sys_agent"
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"
@ -821,6 +822,7 @@ const uint64_t OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_ORA_TID = 15288; // "ALL_VIRTU
const uint64_t OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_ORA_TID = 15289; // "ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_ORA"
const uint64_t OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA_TID = 15290; // "ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA"
const uint64_t OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TID = 15297; // "ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT"
const uint64_t OB_ALL_VIRTUAL_THREAD_ORA_TID = 15298; // "ALL_VIRTUAL_THREAD_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"
@ -1108,6 +1110,8 @@ const uint64_t OB_V_OB_ARCHIVE_DEST_STATUS_TID = 21362; // "V$OB_ARCHIVE_DEST_ST
const uint64_t OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_TID = 21363; // "DBA_OB_LS_LOG_ARCHIVE_PROGRESS"
const uint64_t OB_CDB_OB_LS_LOG_ARCHIVE_PROGRESS_TID = 21364; // "CDB_OB_LS_LOG_ARCHIVE_PROGRESS"
const uint64_t OB_DBA_OB_RSRC_IO_DIRECTIVES_TID = 21369; // "DBA_OB_RSRC_IO_DIRECTIVES"
const uint64_t OB_GV_OB_THREAD_TID = 21380; // "GV$OB_THREAD"
const uint64_t OB_V_OB_THREAD_TID = 21381; // "V$OB_THREAD"
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"
@ -1480,6 +1484,8 @@ const uint64_t OB_V_OB_ARCHIVE_DEST_STATUS_ORA_TID = 28176; // "V$OB_ARCHIVE_DES
const uint64_t OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_ORA_TID = 28177; // "DBA_OB_LS_LOG_ARCHIVE_PROGRESS_ORA"
const uint64_t OB_GV_SESSION_LONGOPS_ORA_TID = 28185; // "GV$SESSION_LONGOPS_ORA"
const uint64_t OB_V_SESSION_LONGOPS_ORA_TID = 28186; // "V$SESSION_LONGOPS_ORA"
const uint64_t OB_GV_OB_THREAD_ORA_TID = 28187; // "GV$OB_THREAD_ORA"
const uint64_t OB_V_OB_THREAD_ORA_TID = 28188; // "V$OB_THREAD_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"
@ -2643,6 +2649,7 @@ const char *const OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_TNAME = "__all_virtual_ls_a
const char *const OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_TNAME = "__all_virtual_ls_arb_replica_task_history";
const char *const OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_TNAME = "__all_virtual_archive_dest_status";
const char *const OB_ALL_VIRTUAL_IO_SCHEDULER_TNAME = "__all_virtual_io_scheduler";
const char *const OB_ALL_VIRTUAL_THREAD_TNAME = "__all_virtual_thread";
const char *const OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TNAME = "__all_virtual_virtual_long_ops_status_mysql_sys_agent";
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";
@ -2852,6 +2859,7 @@ const char *const OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_ORA_TNAME = "ALL_VIRTUAL_LS
const char *const OB_ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY_ORA_TNAME = "ALL_VIRTUAL_LS_ARB_REPLICA_TASK_HISTORY";
const char *const OB_ALL_VIRTUAL_ARCHIVE_DEST_STATUS_ORA_TNAME = "ALL_VIRTUAL_ARCHIVE_DEST_STATUS";
const char *const OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TNAME = "ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT";
const char *const OB_ALL_VIRTUAL_THREAD_ORA_TNAME = "ALL_VIRTUAL_THREAD";
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";
@ -3139,6 +3147,8 @@ const char *const OB_V_OB_ARCHIVE_DEST_STATUS_TNAME = "V$OB_ARCHIVE_DEST_STATUS"
const char *const OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_TNAME = "DBA_OB_LS_LOG_ARCHIVE_PROGRESS";
const char *const OB_CDB_OB_LS_LOG_ARCHIVE_PROGRESS_TNAME = "CDB_OB_LS_LOG_ARCHIVE_PROGRESS";
const char *const OB_DBA_OB_RSRC_IO_DIRECTIVES_TNAME = "DBA_OB_RSRC_IO_DIRECTIVES";
const char *const OB_GV_OB_THREAD_TNAME = "GV$OB_THREAD";
const char *const OB_V_OB_THREAD_TNAME = "V$OB_THREAD";
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";
@ -3511,6 +3521,8 @@ const char *const OB_V_OB_ARCHIVE_DEST_STATUS_ORA_TNAME = "V$OB_ARCHIVE_DEST_STA
const char *const OB_DBA_OB_LS_LOG_ARCHIVE_PROGRESS_ORA_TNAME = "DBA_OB_LS_LOG_ARCHIVE_PROGRESS";
const char *const OB_GV_SESSION_LONGOPS_ORA_TNAME = "GV$SESSION_LONGOPS";
const char *const OB_V_SESSION_LONGOPS_ORA_TNAME = "V$SESSION_LONGOPS";
const char *const OB_GV_OB_THREAD_ORA_TNAME = "GV$OB_THREAD";
const char *const OB_V_OB_THREAD_ORA_TNAME = "V$OB_THREAD";
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";

View File

@ -11436,7 +11436,32 @@ def_table_schema(
# 12381: __all_virtual_task_opt_stat_gather_history
# 12382: __all_virtual_table_opt_stat_gather_history
# 12383: __all_virtual_opt_stat_gather_monitor
# 12384: __all_virtual_thread
def_table_schema(
owner = 'fengshuo.fs',
table_name = '__all_virtual_thread',
table_id = '12384',
table_type = 'VIRTUAL_TABLE',
in_tenant_space = True,
gm_columns = [],
rowkey_columns = [],
normal_columns = [
('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'),
('svr_port', 'int'),
('tenant_id', 'int'),
('tid', 'int'),
('tname', 'varchar:16'),
('status', 'varchar:32'),
('wait_event', 'varchar:64'),
('latch_wait', 'varchar:16'),
('latch_hold', 'varchar:256'),
('trace_id', 'varchar:40'),
('loop_ts', 'timestamp')
],
partition_columns = ['svr_ip', 'svr_port'],
vtable_route_policy = 'distributed',
)
# 12385: __all_virtual_arbitration_member_info
# 12386: __all_virtual_server_storage
# 12387: __all_virtual_arbitration_service_status
@ -11735,8 +11760,10 @@ def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15290'
# 15294: __all_task_opt_stat_gather_history
# 15295: __all_table_opt_stat_gather_history
# 15296: __all_virtual_opt_stat_gather_monitor
def_table_schema(**gen_sys_agent_virtual_table_def('15297', all_def_keywords['__all_virtual_long_ops_status']))
# 15298: __all_virtual_thread
def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15298', all_def_keywords['__all_virtual_thread'])))
# 15299: __all_virtual_wr_active_session_history
# 15300: __all_virtual_wr_snapshot
# 15301: __all_virtual_wr_statname
@ -25671,8 +25698,47 @@ def_table_schema(
# 21377: V$OB_OPT_STAT_GATHER_MONITOR
# 21378: DBA_OB_TASK_OPT_STAT_GATHER_HISTORY
# 21379: DBA_OB_TABLE_OPT_STAT_GATHER_HISTORY
# 21380: GV$OB_THREAD
# 21381: V$OB_THREAD
def_table_schema(
owner = 'fengshuo.fs',
table_name = 'GV$OB_THREAD',
table_id = '21380',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
rowkey_columns = [],
normal_columns = [],
in_tenant_space = True,
view_definition = """
SELECT svr_ip AS SVR_IP,
svr_port AS SVR_PORT,
tenant_id AS TENANT_ID,
tid AS TID,
tname AS TNAME,
status AS STATUS,
latch_wait AS LATCH_WAIT,
latch_hold AS LATCH_HOLD,
trace_id AS TRACE_ID
FROM oceanbase.__all_virtual_thread
""".replace("\n", " "),
)
def_table_schema(
owner = 'fengshuo.fs',
table_name = 'V$OB_THREAD',
table_id = '21381',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
rowkey_columns = [],
normal_columns = [],
in_tenant_space = True,
view_definition = """
SELECT
*
FROM oceanbase.GV$OB_THREAD
WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT()
""".replace("\n", " "),
)
# 21382: GV$OB_ARBITRATION_MEMBER_INFO
# 21383: V$OB_ARBITRATION_MEMBER_INFO
# 21384: DBA_OB_ZONE_STORAGE
@ -48625,8 +48691,50 @@ def_table_schema(
""".replace("\n", " ")
)
# 28187: GV$OB_THREAD
# 28188: V$OB_THREAD
def_table_schema(
owner = 'fengshuo.fs',
table_name = 'GV$OB_THREAD',
name_postfix = '_ORA',
database_id = 'OB_ORA_SYS_DATABASE_ID',
table_id = '28187',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
rowkey_columns = [],
normal_columns = [],
in_tenant_space = True,
view_definition = """
SELECT svr_ip AS SVR_IP,
svr_port AS SVR_PORT,
tenant_id AS TENANT_ID,
tid AS TID,
tname AS TNAME,
status AS STATUS,
latch_wait AS LATCH_WAIT,
latch_hold AS LATCH_HOLD,
trace_id AS TRACE_ID
FROM SYS.ALL_VIRTUAL_THREAD
""".replace("\n", " "),
)
def_table_schema(
owner = 'fengshuo.fs',
table_name = 'V$OB_THREAD',
name_postfix = '_ORA',
database_id = 'OB_ORA_SYS_DATABASE_ID',
table_id = '28188',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
rowkey_columns = [],
normal_columns = [],
in_tenant_space = True,
view_definition = """
SELECT
*
FROM SYS.GV$OB_THREAD
WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT()
""".replace("\n", " "),
)
# 28189: GV$OB_ARBITRATION_MEMBER_INFO
# 28190: V$OB_ARBITRATION_MEMBER_INFO
# 28191: GV$OB_ARBITRATION_SERVICE_STATUS

View File

@ -79,7 +79,7 @@ int ObVTableLocationService::init(
//TODO move vtable location cache from kvcache to inner cache
} else if (OB_FAIL(vtable_cache_.init(OB_VTABLE_CACHE_NAME, 1000))) {
LOG_WARN("vtable_cache init failed", KR(ret));
} else if (OB_FAIL(update_queue_.init(this, 1/*thread_cnt*/, 100/*queue_size*/))) {
} else if (OB_FAIL(update_queue_.init(this, 1/*thread_cnt*/, 100/*queue_size*/, "VTblLocAsyncUp"))) {
LOG_WARN("update_queue init failed", KR(ret));
} else {
server_tracer_ = &server_tracer;

View File

@ -584,6 +584,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr
12365 __all_virtual_ls_arb_replica_task_history 2 201001 1
12366 __all_virtual_archive_dest_status 2 201001 1
12369 __all_virtual_io_scheduler 2 201001 1
12384 __all_virtual_thread 2 201001 1
12393 __all_virtual_virtual_long_ops_status_mysql_sys_agent 2 201001 1
20001 GV$OB_PLAN_CACHE_STAT 1 201001 1
20002 GV$OB_PLAN_CACHE_PLAN_STAT 1 201001 1
@ -872,6 +873,8 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr
21363 DBA_OB_LS_LOG_ARCHIVE_PROGRESS 1 201001 1
21364 CDB_OB_LS_LOG_ARCHIVE_PROGRESS 1 201001 1
21369 DBA_OB_RSRC_IO_DIRECTIVES 1 201001 1
21380 GV$OB_THREAD 1 201001 1
21381 V$OB_THREAD 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;