[FEAT MERGE]vos merge into master
Co-authored-by: ZenoWang <wzybuaasoft@163.com> Co-authored-by: zhjc1124 <zhjc1124@gmail.com> Co-authored-by: JiahuaChen <garfieldjia@qq.com>
This commit is contained in:
parent
11dc664903
commit
a4c97d03f7
43
deps/oblib/src/common/ob_local_store.h
vendored
43
deps/oblib/src/common/ob_local_store.h
vendored
@ -1,43 +0,0 @@
|
||||
/**
|
||||
* 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_COMMON_LOCAL_STORE_
|
||||
#define OCEANBASE_COMMON_LOCAL_STORE_
|
||||
|
||||
#include "lib/oblog/ob_log_level.h"
|
||||
|
||||
#include <cstdint>
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace common
|
||||
{
|
||||
struct ObLocalStore
|
||||
{
|
||||
ObLocalStore()
|
||||
: stack_addr_(nullptr), stack_size_(0)
|
||||
{}
|
||||
ObThreadLogLevel log_level_;
|
||||
void *stack_addr_;
|
||||
size_t stack_size_;
|
||||
};
|
||||
|
||||
inline ObLocalStore *get_local_store()
|
||||
{
|
||||
// use thread local for raw thread.
|
||||
RLOCAL_INLINE(ObLocalStore, default_local_store);
|
||||
return &default_local_store;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
#endif
|
2
deps/oblib/src/lib/CMakeLists.txt
vendored
2
deps/oblib/src/lib/CMakeLists.txt
vendored
@ -134,7 +134,6 @@ ob_set_subtarget(oblib_lib common_mixed
|
||||
string/ob_string_buffer.cpp
|
||||
task/ob_timer.cpp
|
||||
task/ob_timer_monitor.cpp
|
||||
thread_local/ob_tsi_factory.cpp
|
||||
thread_local/ob_tsi_utils.cpp
|
||||
thread_local/thread_buffer.cpp
|
||||
time/Time.cpp
|
||||
@ -248,6 +247,7 @@ ob_set_subtarget(ob_malloc_object_list common_alloc
|
||||
alloc/memory_dump.cpp
|
||||
alloc/ob_malloc_allocator.cpp
|
||||
alloc/ob_malloc_callback.cpp
|
||||
alloc/ob_malloc_sample_struct.cpp
|
||||
alloc/ob_tenant_ctx_allocator.cpp
|
||||
alloc/object_mgr.cpp
|
||||
alloc/object_set.cpp
|
||||
|
7
deps/oblib/src/lib/alloc/alloc_func.cpp
vendored
7
deps/oblib/src/lib/alloc/alloc_func.cpp
vendored
@ -171,5 +171,12 @@ int set_meta_obj_limit(uint64_t tenant_id, int64_t meta_obj_pct_lmt)
|
||||
return set_ctx_limit(tenant_id, common::ObCtxIds::META_OBJ_CTX_ID, ctx_limit);
|
||||
}
|
||||
|
||||
int set_rpc_limit(uint64_t tenant_id, int64_t rpc_pct_lmt)
|
||||
{
|
||||
const int64_t tenant_limit = get_tenant_memory_limit(tenant_id);
|
||||
const int64_t rpc_lmt = (tenant_limit / 100) * rpc_pct_lmt;
|
||||
return set_ctx_limit(tenant_id, common::ObCtxIds::RPC_CTX_ID, rpc_lmt);
|
||||
}
|
||||
|
||||
} // end of namespace lib
|
||||
} // end of namespace oceanbase
|
||||
|
4
deps/oblib/src/lib/alloc/alloc_func.h
vendored
4
deps/oblib/src/lib/alloc/alloc_func.h
vendored
@ -53,6 +53,10 @@ int set_wa_limit(uint64_t tenand_id, int64_t wa_pctg);
|
||||
// set meta object memory limit for specified tenant.
|
||||
// - meta_obj_pct_lmt: percentage limitation of tenant memory can be used for meta object.
|
||||
int set_meta_obj_limit(uint64_t tenant_id, int64_t meta_obj_pct_lmt);
|
||||
|
||||
// set rpc memory limit for specified tenant.
|
||||
// - rpc_pct_lmt: percentage limitation of tenant rpc memory.
|
||||
int set_rpc_limit(uint64_t tenant_id, int64_t rpc_pct_lmt);
|
||||
} // end of namespace lib
|
||||
} // end of namespace oceanbase
|
||||
|
||||
|
4
deps/oblib/src/lib/alloc/alloc_struct.h
vendored
4
deps/oblib/src/lib/alloc/alloc_struct.h
vendored
@ -237,6 +237,7 @@ struct AObject {
|
||||
struct {
|
||||
struct {
|
||||
uint8_t on_leak_check_ : 1;
|
||||
uint8_t on_malloc_sample_ : 1;
|
||||
};
|
||||
};
|
||||
};
|
||||
@ -475,7 +476,8 @@ char *ABlock::data() const
|
||||
AObject::AObject()
|
||||
: MAGIC_CODE_(FREE_AOBJECT_MAGIC_CODE),
|
||||
nobjs_(0), nobjs_prev_(0), obj_offset_(0),
|
||||
alloc_bytes_(0), tenant_id_(0)
|
||||
alloc_bytes_(0), tenant_id_(0),
|
||||
on_leak_check_(false), on_malloc_sample_(false)
|
||||
{
|
||||
}
|
||||
|
||||
|
2
deps/oblib/src/lib/alloc/block_set.h
vendored
2
deps/oblib/src/lib/alloc/block_set.h
vendored
@ -106,11 +106,13 @@ private:
|
||||
|
||||
void BlockSet::lock()
|
||||
{
|
||||
ObDisableDiagnoseGuard diagnose_disable_guard;
|
||||
locker_->lock();
|
||||
}
|
||||
|
||||
void BlockSet::unlock()
|
||||
{
|
||||
ObDisableDiagnoseGuard diagnose_disable_guard;
|
||||
locker_->unlock();
|
||||
}
|
||||
|
||||
|
31
deps/oblib/src/lib/alloc/malloc_hook.cpp
vendored
31
deps/oblib/src/lib/alloc/malloc_hook.cpp
vendored
@ -60,22 +60,21 @@ void *ob_malloc_retry(size_t size)
|
||||
return ptr;
|
||||
}
|
||||
|
||||
static __thread bool in_hook = false;
|
||||
void *ob_malloc_hook(size_t size, const void *)
|
||||
{
|
||||
void *ptr = nullptr;
|
||||
size_t real_size = size + Header::SIZE;
|
||||
void *tmp_ptr = nullptr;
|
||||
bool from_mmap = false;
|
||||
if (OB_UNLIKELY(in_hook)) {
|
||||
if (OB_UNLIKELY(in_hook())) {
|
||||
if (MAP_FAILED == (tmp_ptr = ::mmap(nullptr, real_size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0))) {
|
||||
tmp_ptr = nullptr;
|
||||
}
|
||||
from_mmap = true;
|
||||
} else {
|
||||
bool in_hook_bak = in_hook;
|
||||
in_hook = true;
|
||||
DEFER(in_hook = in_hook_bak);
|
||||
bool in_hook_bak = in_hook();
|
||||
in_hook()= true;
|
||||
DEFER(in_hook()= in_hook_bak);
|
||||
tmp_ptr = ob_malloc_retry(real_size);
|
||||
}
|
||||
if (OB_LIKELY(tmp_ptr != nullptr)) {
|
||||
@ -96,9 +95,9 @@ void ob_free_hook(void *ptr, const void *)
|
||||
if (OB_UNLIKELY(header->from_mmap_)) {
|
||||
::munmap(orig_ptr, header->data_size_ + Header::SIZE + header->offset_);
|
||||
} else {
|
||||
bool in_hook_bak = in_hook;
|
||||
in_hook = true;
|
||||
DEFER(in_hook = in_hook_bak);
|
||||
bool in_hook_bak = in_hook();
|
||||
in_hook()= true;
|
||||
DEFER(in_hook()= in_hook_bak);
|
||||
ob_free(orig_ptr);
|
||||
}
|
||||
}
|
||||
@ -110,15 +109,15 @@ void *ob_realloc_hook(void *ptr, size_t size, const void *caller)
|
||||
size_t real_size = size + Header::SIZE;
|
||||
void *tmp_ptr = nullptr;
|
||||
bool from_mmap = false;
|
||||
if (OB_UNLIKELY(in_hook)) {
|
||||
if (OB_UNLIKELY(in_hook())) {
|
||||
if (MAP_FAILED == (tmp_ptr = ::mmap(nullptr, real_size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0))) {
|
||||
tmp_ptr = nullptr;
|
||||
}
|
||||
from_mmap = true;
|
||||
} else {
|
||||
bool in_hook_bak = in_hook;
|
||||
in_hook = true;
|
||||
DEFER(in_hook = in_hook_bak);
|
||||
bool in_hook_bak = in_hook();
|
||||
in_hook()= true;
|
||||
DEFER(in_hook()= in_hook_bak);
|
||||
tmp_ptr = ob_malloc_retry(real_size);
|
||||
}
|
||||
if (OB_LIKELY(tmp_ptr != nullptr)) {
|
||||
@ -149,15 +148,15 @@ void *ob_memalign_hook(size_t alignment, size_t size, const void *)
|
||||
size_t real_size = 2 * MAX(alignment, Header::SIZE) + size;
|
||||
void *tmp_ptr = nullptr;
|
||||
bool from_mmap = false;
|
||||
if (OB_UNLIKELY(in_hook)) {
|
||||
if (OB_UNLIKELY(in_hook())) {
|
||||
if (MAP_FAILED == (tmp_ptr = ::mmap(nullptr, real_size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0))) {
|
||||
tmp_ptr = nullptr;
|
||||
}
|
||||
from_mmap = true;
|
||||
} else {
|
||||
bool in_hook_bak = in_hook;
|
||||
in_hook = true;
|
||||
DEFER(in_hook = in_hook_bak);
|
||||
bool in_hook_bak = in_hook();
|
||||
in_hook()= true;
|
||||
DEFER(in_hook()= in_hook_bak);
|
||||
tmp_ptr = ob_malloc_retry(real_size);
|
||||
}
|
||||
if (OB_LIKELY(tmp_ptr != nullptr)) {
|
||||
|
6
deps/oblib/src/lib/alloc/malloc_hook.h
vendored
6
deps/oblib/src/lib/alloc/malloc_hook.h
vendored
@ -15,4 +15,10 @@
|
||||
|
||||
extern void init_malloc_hook();
|
||||
|
||||
inline bool& in_hook()
|
||||
{
|
||||
thread_local bool in_hook = false;
|
||||
return in_hook;
|
||||
}
|
||||
|
||||
#endif /* MALLOC_HOOK_H */
|
||||
|
63
deps/oblib/src/lib/alloc/memory_dump.cpp
vendored
63
deps/oblib/src/lib/alloc/memory_dump.cpp
vendored
@ -128,6 +128,13 @@ int ObMemoryDump::init()
|
||||
w_stat_ = new (r_stat_ + 1) Stat();
|
||||
dump_context_ = context;
|
||||
is_inited_ = true;
|
||||
if (OB_FAIL(r_stat_->malloc_sample_map_.create(1000, "MallocInfoMap",
|
||||
"MallocInfoMap"))) {
|
||||
LOG_WARN("create memory info map for reading failed", K(ret));
|
||||
} else if (OB_FAIL(w_stat_->malloc_sample_map_.create(1000, "MallocInfoMap",
|
||||
"MallocInfoMap"))) {
|
||||
LOG_WARN("create memory info map for writing failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(TG_SET_RUNNABLE_AND_START(TGDefIDs::MEMORY_DUMP, *this))) {
|
||||
@ -179,6 +186,17 @@ int ObMemoryDump::push(void *task)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMemoryDump::load_malloc_sample_map(ObMallocSampleMap &malloc_sample_map)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLatchRGuard guard(iter_lock_, ObLatchIds::MEM_DUMP_ITER_LOCK);
|
||||
auto &map = r_stat_->malloc_sample_map_;
|
||||
for (auto it = map.begin(); OB_SUCC(ret) && it != map.end(); ++it) {
|
||||
ret = malloc_sample_map.set_refactored(it->first, it->second);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMemoryDump::run1()
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
@ -365,11 +383,12 @@ int label_stat(AChunk *chunk, ABlock *block, AObject *object,
|
||||
ObString str(len, label);
|
||||
LabelItem *litem = nullptr;
|
||||
LabelInfoItem *linfoitem = lmap.get(str);
|
||||
int64_t bt_size = object->on_malloc_sample_ ? AOBJECT_BACKTRACE_SIZE : 0;
|
||||
if (NULL != linfoitem) {
|
||||
// exist
|
||||
litem = linfoitem->litem_;
|
||||
litem->hold_ += hold;
|
||||
litem->used_ += object->alloc_bytes_;
|
||||
litem->used_ += (object->alloc_bytes_ - bt_size);
|
||||
litem->count_++;
|
||||
if (chunk != linfoitem->chunk_) {
|
||||
litem->chunk_cnt_ += 1;
|
||||
@ -389,7 +408,7 @@ int label_stat(AChunk *chunk, ABlock *block, AObject *object,
|
||||
litem->str_[len] = '\0';
|
||||
litem->str_len_ = len;
|
||||
litem->hold_ = hold;
|
||||
litem->used_ = object->alloc_bytes_;
|
||||
litem->used_ = (object->alloc_bytes_ - bt_size);
|
||||
litem->count_ = 1;
|
||||
litem->block_cnt_ = 1;
|
||||
litem->chunk_cnt_ = 1;
|
||||
@ -400,6 +419,35 @@ int label_stat(AChunk *chunk, ABlock *block, AObject *object,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int malloc_sample_stat(uint64_t tenant_id, uint64_t ctx_id,
|
||||
AObject *object, ObMallocSampleMap &malloc_sample_map)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (object->in_use_ && object->on_malloc_sample_) {
|
||||
int64_t offset = object->alloc_bytes_ - AOBJECT_BACKTRACE_SIZE;
|
||||
ObMallocSampleKey key;
|
||||
key.tenant_id_ = tenant_id;
|
||||
key.ctx_id_ = ctx_id;
|
||||
void **backtrace = reinterpret_cast<void**>(&object->data_[offset]);
|
||||
int32_t bt_size = 0;
|
||||
while (bt_size < AOBJECT_BACKTRACE_COUNT && nullptr != backtrace[bt_size]) {
|
||||
key.bt_[bt_size] = backtrace[bt_size];
|
||||
++bt_size;
|
||||
}
|
||||
key.bt_size_ = bt_size;
|
||||
STRNCPY(key.label_, object->label_, sizeof(key.label_));
|
||||
key.label_[sizeof(key.label_) - 1] = '\0';
|
||||
ObMallocSampleValue *item = malloc_sample_map.get(key);
|
||||
if (NULL != item) {
|
||||
item->alloc_count_ += 1;
|
||||
item->alloc_bytes_ += offset;
|
||||
} else {
|
||||
ret = malloc_sample_map.set_refactored(key, ObMallocSampleValue(1, offset));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMemoryDump::handle(void *task)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -415,6 +463,7 @@ void ObMemoryDump::handle(void *task)
|
||||
get_tenant_ids(tenant_ids_, MAX_TENANT_CNT, tenant_cnt);
|
||||
std::sort(tenant_ids_, tenant_ids_ + tenant_cnt);
|
||||
w_stat_->tcr_cnt_ = 0;
|
||||
w_stat_->malloc_sample_map_.clear();
|
||||
int64_t item_used = 0;
|
||||
int64_t log_pos = 0;
|
||||
IGNORE_RETURN databuff_printf(log_buf_, LOG_BUF_LEN, log_pos,
|
||||
@ -456,8 +505,9 @@ void ObMemoryDump::handle(void *task)
|
||||
UNUSEDx(chunk, block);
|
||||
return OB_SUCCESS;
|
||||
},
|
||||
[tenant_id, &lmap, w_stat, &item_used]
|
||||
[tenant_id, ctx_id, &lmap, w_stat, &item_used]
|
||||
(AChunk *chunk, ABlock *block, AObject *object) {
|
||||
int ret = OB_SUCCESS;
|
||||
if (object->in_use_) {
|
||||
bool expect = AOBJECT_TAIL_MAGIC_CODE ==
|
||||
reinterpret_cast<uint64_t&>(object->data_[object->alloc_bytes_]);
|
||||
@ -470,9 +520,14 @@ void ObMemoryDump::handle(void *task)
|
||||
K(length), K(label));
|
||||
}
|
||||
}
|
||||
return label_stat(chunk, block, object, lmap,
|
||||
ret = label_stat(chunk, block, object, lmap,
|
||||
w_stat->up2date_items_, ARRAYSIZEOF(w_stat->up2date_items_),
|
||||
item_used);
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = malloc_sample_stat(tenant_id, ctx_id,
|
||||
object, w_stat->malloc_sample_map_);
|
||||
}
|
||||
return ret;
|
||||
});
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("parse_chunk_meta failed", K(ret), KP(chunk));
|
||||
|
4
deps/oblib/src/lib/alloc/memory_dump.h
vendored
4
deps/oblib/src/lib/alloc/memory_dump.h
vendored
@ -13,6 +13,7 @@
|
||||
#ifndef OCEANBASE_DUMP_MEMORY_H_
|
||||
#define OCEANBASE_DUMP_MEMORY_H_
|
||||
|
||||
#include "lib/alloc/ob_malloc_sample_struct.h"
|
||||
#include "lib/queue/ob_lighty_queue.h"
|
||||
#include "lib/hash/ob_hashmap.h"
|
||||
#include "lib/rc/context.h"
|
||||
@ -26,6 +27,7 @@ namespace oceanbase
|
||||
namespace observer
|
||||
{
|
||||
class ObAllVirtualMemoryInfo;
|
||||
class ObMallocSampleInfo;
|
||||
}
|
||||
namespace lib
|
||||
{
|
||||
@ -142,6 +144,7 @@ struct TenantCtxRange
|
||||
struct Stat {
|
||||
LabelItem up2date_items_[MAX_LABEL_ITEM_CNT];
|
||||
TenantCtxRange tcrs_[MAX_TENANT_CNT * ObCtxIds::MAX_CTX_ID];
|
||||
lib::ObMallocSampleMap malloc_sample_map_;
|
||||
int tcr_cnt_ = 0;
|
||||
};
|
||||
|
||||
@ -182,6 +185,7 @@ public:
|
||||
lib::ObMutexGuard guard(task_mutex_);
|
||||
avaliable_task_set_ |= (1 << pos);
|
||||
}
|
||||
int load_malloc_sample_map(lib::ObMallocSampleMap& malloc_sample_map);
|
||||
private:
|
||||
void run1() override;
|
||||
void handle(void *task);
|
||||
|
5
deps/oblib/src/lib/alloc/ob_futex_v2.cpp
vendored
5
deps/oblib/src/lib/alloc/ob_futex_v2.cpp
vendored
@ -16,16 +16,15 @@
|
||||
#include <linux/futex.h>
|
||||
#include "lib/ob_abort.h"
|
||||
|
||||
#define futex(...) syscall(SYS_futex,__VA_ARGS__)
|
||||
inline int futex_v2_wake(volatile int *p, int val)
|
||||
{
|
||||
return static_cast<int>(futex((int *)p, FUTEX_WAKE_PRIVATE, val, NULL, NULL, 0));
|
||||
return static_cast<int>(futex((uint *)p, FUTEX_WAKE_PRIVATE, val, NULL));
|
||||
}
|
||||
|
||||
inline int futex_v2_wait(volatile int *p, int val, const timespec *timeout)
|
||||
{
|
||||
int ret = 0;
|
||||
if (0 != futex((int *)p, FUTEX_WAIT_PRIVATE, val, timeout, NULL, 0)) {
|
||||
if (0 != futex((uint *)p, FUTEX_WAIT_PRIVATE, val, timeout)) {
|
||||
ret = errno;
|
||||
}
|
||||
return ret;
|
||||
|
41
deps/oblib/src/lib/alloc/ob_malloc_allocator.cpp
vendored
41
deps/oblib/src/lib/alloc/ob_malloc_allocator.cpp
vendored
@ -97,6 +97,7 @@ void *ObMallocAllocator::alloc(const int64_t size, const oceanbase::lib::ObMemAt
|
||||
return ::malloc(size);
|
||||
#else
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
int ret = OB_E(EventTable::EN_4) OB_SUCCESS;
|
||||
void *ptr = NULL;
|
||||
ObTenantCtxAllocatorGuard allocator = NULL;
|
||||
@ -129,13 +130,6 @@ void *ObMallocAllocator::alloc(const int64_t size, const oceanbase::lib::ObMemAt
|
||||
ptr = allocator->alloc(size, inner_attr);
|
||||
}
|
||||
|
||||
if (OB_NOT_NULL(ptr)) {
|
||||
AObject *obj = reinterpret_cast<AObject*>((char*)ptr - AOBJECT_HEADER_SIZE);
|
||||
abort_unless(obj->MAGIC_CODE_ == AOBJECT_MAGIC_CODE
|
||||
|| obj->MAGIC_CODE_ == BIG_AOBJECT_MAGIC_CODE);
|
||||
get_mem_leak_checker().on_alloc(*obj, inner_attr);
|
||||
}
|
||||
|
||||
return ptr;
|
||||
#endif
|
||||
}
|
||||
@ -148,6 +142,7 @@ void *ObMallocAllocator::realloc(
|
||||
return ::realloc(const_cast<void *>(ptr), size);
|
||||
#else
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
// Won't create tenant allocator!!
|
||||
void *nptr = NULL;
|
||||
int ret = OB_E(EventTable::EN_4) OB_SUCCESS;
|
||||
@ -172,11 +167,6 @@ void *ObMallocAllocator::realloc(
|
||||
// do nothing
|
||||
} else if (OB_ISNULL(nptr = allocator->realloc(ptr, size, inner_attr))) {
|
||||
// do nothing
|
||||
} else {
|
||||
AObject *obj = reinterpret_cast<AObject*>((char*)nptr - AOBJECT_HEADER_SIZE);
|
||||
abort_unless(obj->MAGIC_CODE_ == AOBJECT_MAGIC_CODE
|
||||
|| obj->MAGIC_CODE_ == BIG_AOBJECT_MAGIC_CODE);
|
||||
get_mem_leak_checker().on_alloc(*obj, inner_attr);
|
||||
}
|
||||
return nptr;;
|
||||
#endif
|
||||
@ -189,24 +179,7 @@ void ObMallocAllocator::free(void *ptr)
|
||||
#else
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
// directly free object instead of using tenant allocator.
|
||||
if (NULL != ptr) {
|
||||
AObject *obj = reinterpret_cast<AObject*>((char*)ptr - AOBJECT_HEADER_SIZE);
|
||||
abort_unless(NULL != obj);
|
||||
abort_unless(obj->MAGIC_CODE_ == AOBJECT_MAGIC_CODE
|
||||
|| obj->MAGIC_CODE_ == BIG_AOBJECT_MAGIC_CODE);
|
||||
abort_unless(obj->in_use_);
|
||||
SANITY_POISON(obj->data_, obj->alloc_bytes_);
|
||||
|
||||
get_mem_leak_checker().on_free(*obj);
|
||||
ABlock *block = obj->block();
|
||||
abort_unless(block);
|
||||
abort_unless(block->is_valid());
|
||||
abort_unless(block->in_use_);
|
||||
abort_unless(block->obj_set_ != NULL);
|
||||
|
||||
ObjectSet *set = block->obj_set_;
|
||||
set->free_object(obj);
|
||||
}
|
||||
ObTenantCtxAllocator::common_free(ptr);
|
||||
#endif // PERF_MODE
|
||||
}
|
||||
|
||||
@ -223,6 +196,7 @@ ObTenantCtxAllocatorGuard ObMallocAllocator::get_tenant_ctx_allocator_without_tl
|
||||
}
|
||||
}
|
||||
const int64_t slot = tenant_id % PRESERVED_TENANT_COUNT;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
BucketRLockGuard guard(const_cast<BucketLock&>(locks_[slot]),
|
||||
GETTID() % BucketLock::BUCKET_COUNT);
|
||||
if (OB_LIKELY(tenant_id < PRESERVED_TENANT_COUNT)) {
|
||||
@ -342,6 +316,7 @@ int ObMallocAllocator::add_tenant_allocator(ObTenantCtxAllocator *allocator)
|
||||
uint64_t tenant_id = allocator->get_tenant_id();
|
||||
const int64_t slot = tenant_id % PRESERVED_TENANT_COUNT;
|
||||
// critical area is extremely small, just wait without trylock
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
BucketWLockGuard guard(locks_[slot]);
|
||||
ObTenantCtxAllocator **cur = &allocators_[slot];
|
||||
while ((NULL != *cur) && (*cur)->get_tenant_id() < tenant_id) {
|
||||
@ -361,6 +336,7 @@ ObTenantCtxAllocator *ObMallocAllocator::take_off_tenant_allocator(uint64_t tena
|
||||
{
|
||||
ObTenantCtxAllocator *ta = NULL;
|
||||
const int64_t slot = tenant_id % PRESERVED_TENANT_COUNT;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
BucketWLockGuard guard(locks_[slot]);
|
||||
ObTenantCtxAllocator **cur = &allocators_[slot];
|
||||
while (*cur && (*cur)->get_tenant_id() < tenant_id) {
|
||||
@ -566,6 +542,7 @@ int ObMallocAllocator::set_tenant_ctx_idle(const uint64_t tenant_id,
|
||||
int ObMallocAllocator::get_chunks(AChunk **chunks, int cap, int &cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
for (int64_t slot = 0; OB_SUCC(ret) && slot < PRESERVED_TENANT_COUNT; ++slot) {
|
||||
ObTenantCtxAllocatorGuard tas[16]; // TODO: should be dynamic array, but enough so far
|
||||
int tas_cnt = 0;
|
||||
@ -624,6 +601,7 @@ ObTenantCtxAllocatorGuard ObMallocAllocator::get_tenant_ctx_allocator_unrecycled
|
||||
uint64_t tenant_id, uint64_t ctx_id) const
|
||||
{
|
||||
ObTenantCtxAllocatorGuard ta;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
ObLatchRGuard guard(const_cast<ObLatch&>(unrecycled_lock_), ObLatchIds::OB_ALLOCATOR_LOCK);
|
||||
ObTenantCtxAllocator * const *cur = &unrecycled_allocators_;
|
||||
while (*cur) {
|
||||
@ -643,6 +621,7 @@ void ObMallocAllocator::add_tenant_allocator_unrecycled(ObTenantCtxAllocator *al
|
||||
modify_tenant_memory_access_permission(allocator, false);
|
||||
}
|
||||
#endif
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
ObLatchWGuard guard(unrecycled_lock_, ObLatchIds::OB_ALLOCATOR_LOCK);
|
||||
allocator->get_next() = unrecycled_allocators_;
|
||||
unrecycled_allocators_ = allocator;
|
||||
@ -652,6 +631,7 @@ ObTenantCtxAllocator *ObMallocAllocator::take_off_tenant_allocator_unrecycled(ui
|
||||
{
|
||||
ObTenantCtxAllocator *ta = NULL;
|
||||
{
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
ObLatchWGuard guard(unrecycled_lock_, ObLatchIds::OB_ALLOCATOR_LOCK);
|
||||
ObTenantCtxAllocator **cur = &unrecycled_allocators_;
|
||||
while (*cur) {
|
||||
@ -760,6 +740,7 @@ int ObMallocAllocator::recycle_tenant_allocator(uint64_t tenant_id)
|
||||
void ObMallocAllocator::get_unrecycled_tenant_ids(uint64_t *ids, int cap, int &cnt) const
|
||||
{
|
||||
cnt = 0;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
ObLatchRGuard guard(const_cast<ObLatch&>(unrecycled_lock_), ObLatchIds::OB_ALLOCATOR_LOCK);
|
||||
ObTenantCtxAllocator * const *cur = &unrecycled_allocators_;
|
||||
while (*cur && cnt < cap) {
|
||||
|
@ -10,16 +10,19 @@
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#include "lib/thread_local/ob_tsi_factory.h"
|
||||
#include "lib/alloc/ob_malloc_sample_struct.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace common
|
||||
namespace lib
|
||||
{
|
||||
TSIFactory &get_tsi_fatcory()
|
||||
{
|
||||
static TSIFactory instance;
|
||||
return instance;
|
||||
}
|
||||
} // namespace common
|
||||
} // namespace oceanbase
|
||||
#if defined(__x86_64__)
|
||||
int32_t ObMallocSampleLimiter::min_malloc_sample_interval = 16;
|
||||
int32_t ObMallocSampleLimiter::max_malloc_sample_interval = 256;
|
||||
#else
|
||||
int32_t ObMallocSampleLimiter::min_malloc_sample_interval = 10000;
|
||||
int32_t ObMallocSampleLimiter::max_malloc_sample_interval = 10000;
|
||||
#endif
|
||||
|
||||
} // end of namespace lib
|
||||
} // end of namespace oceanbase
|
182
deps/oblib/src/lib/alloc/ob_malloc_sample_struct.h
vendored
Normal file
182
deps/oblib/src/lib/alloc/ob_malloc_sample_struct.h
vendored
Normal file
@ -0,0 +1,182 @@
|
||||
/**
|
||||
* 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_MALLOC_SAMPLE_STRUCT_H_
|
||||
#define OCEANBASE_MALLOC_SAMPLE_STRUCT_H_
|
||||
|
||||
#include "lib/alloc/alloc_struct.h"
|
||||
#include "lib/hash/ob_hashmap.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace lib
|
||||
{
|
||||
static const int32_t AOBJECT_BACKTRACE_COUNT = 16;
|
||||
static const int32_t AOBJECT_BACKTRACE_SIZE = sizeof(void*) * AOBJECT_BACKTRACE_COUNT;
|
||||
static const int32_t MAX_MALLOC_SAMPLER_NUM = (1<<15) - 1;
|
||||
|
||||
class ObMallocSampleLimiter
|
||||
{
|
||||
public:
|
||||
ObMallocSampleLimiter();
|
||||
bool try_acquire(int64_t alloc_bytes);
|
||||
static bool malloc_sample_allowed(const int64_t size, const ObMemAttr &attr);
|
||||
static void set_interval(int32_t max_ratio, int32_t min_ratio);
|
||||
private:
|
||||
static int32_t min_malloc_sample_interval;
|
||||
static int32_t max_malloc_sample_interval;
|
||||
static const int32_t INTERVAL_UPPER_LIMIT = 10000;
|
||||
static const int32_t MUST_SAMPLE_SIZE = 16<<20;
|
||||
static const int32_t CUMULATIVE_SAMPLE_SIZE = 4<<20;
|
||||
int64_t count_;
|
||||
int64_t hold_;
|
||||
};
|
||||
|
||||
static ObMallocSampleLimiter rate_limiters[MAX_MALLOC_SAMPLER_NUM + 1];
|
||||
|
||||
struct ObMallocSampleKey
|
||||
{
|
||||
ObMallocSampleKey()
|
||||
{}
|
||||
int64_t hash() const;
|
||||
bool operator==(const ObMallocSampleKey &other) const;
|
||||
int64_t tenant_id_;
|
||||
int64_t ctx_id_;
|
||||
int32_t bt_size_;
|
||||
char label_[lib::AOBJECT_LABEL_SIZE + 1];
|
||||
void *bt_[AOBJECT_BACKTRACE_COUNT];
|
||||
};
|
||||
|
||||
struct ObMallocSampleValue
|
||||
{
|
||||
ObMallocSampleValue()
|
||||
{}
|
||||
ObMallocSampleValue(int64_t alloc_count, int64_t alloc_bytes)
|
||||
: alloc_count_(alloc_count), alloc_bytes_(alloc_bytes)
|
||||
{}
|
||||
int64_t alloc_count_;
|
||||
int64_t alloc_bytes_;
|
||||
};
|
||||
|
||||
typedef hash::ObHashMap<ObMallocSampleKey, ObMallocSampleValue,
|
||||
hash::NoPthreadDefendMode> ObMallocSampleMap;
|
||||
|
||||
|
||||
inline uint64_t ob_malloc_sample_hash(const char* data)
|
||||
{
|
||||
return (uint64_t)data * 0xdeece66d + 0xb;
|
||||
}
|
||||
|
||||
inline ObMallocSampleLimiter::ObMallocSampleLimiter()
|
||||
: count_(0), hold_(0)
|
||||
{}
|
||||
|
||||
inline bool ObMallocSampleLimiter::try_acquire(int64_t alloc_bytes)
|
||||
{
|
||||
bool ret = false;
|
||||
// Condition sample: controlled by sampler interval and Cumulative hold.
|
||||
hold_ += alloc_bytes;
|
||||
count_ += 1;
|
||||
if (min_malloc_sample_interval <= count_) {
|
||||
if (hold_ >= CUMULATIVE_SAMPLE_SIZE || max_malloc_sample_interval <= count_) {
|
||||
count_ = 0;
|
||||
hold_ = 0;
|
||||
ret = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
#ifndef PERF_MODE
|
||||
inline bool ObMallocSampleLimiter::malloc_sample_allowed(const int64_t size, const ObMemAttr &attr)
|
||||
{
|
||||
bool ret = false;
|
||||
if (OB_UNLIKELY(INTERVAL_UPPER_LIMIT == min_malloc_sample_interval)) {
|
||||
// Zero sample mode.
|
||||
} else if (OB_UNLIKELY(MUST_SAMPLE_SIZE <= size)) {
|
||||
// Full sample when size is bigger than 16M.
|
||||
ret = true;
|
||||
} else {
|
||||
uint64_t hash_val = ob_malloc_sample_hash(attr.label_.str_);
|
||||
if (rate_limiters[hash_val & MAX_MALLOC_SAMPLER_NUM].try_acquire(size)) {
|
||||
ret = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
#else
|
||||
inline bool ObMallocSampleLimiter::malloc_sample_allowed(const int64_t size, const ObMemAttr &attr)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
#endif
|
||||
|
||||
inline void ObMallocSampleLimiter::set_interval(int32_t max_interval, int32_t min_interval)
|
||||
{
|
||||
if (min_interval < 1 || max_interval > INTERVAL_UPPER_LIMIT
|
||||
|| max_interval < min_interval) {
|
||||
_OB_LOG_RET(WARN, common::OB_INVALID_ARGUMENT, "set the min or max malloc times between two samples unexpected,"
|
||||
"max_interval=%d, min_interval=%d", max_interval, min_interval);
|
||||
} else {
|
||||
min_malloc_sample_interval = min_interval;
|
||||
max_malloc_sample_interval = max_interval;
|
||||
_OB_LOG_RET(INFO, common::OB_SUCCESS, "set the min or max malloc times between two samples succeed,"
|
||||
"max_interval=%d, min_interval=%d", max_interval, min_interval);
|
||||
}
|
||||
}
|
||||
|
||||
inline int64_t ObMallocSampleKey::hash() const
|
||||
{
|
||||
int64_t hash_val = 0;
|
||||
hash_val = murmurhash(&tenant_id_, sizeof(tenant_id_), hash_val);
|
||||
hash_val = murmurhash(&ctx_id_, sizeof(ctx_id_), hash_val);
|
||||
hash_val = murmurhash(label_, sizeof(label_), hash_val);
|
||||
hash_val = murmurhash(bt_, bt_size_ * sizeof(void*), hash_val);
|
||||
return hash_val;
|
||||
}
|
||||
|
||||
inline bool ObMallocSampleKey::operator==(const ObMallocSampleKey &other) const
|
||||
{
|
||||
bool ret = true;
|
||||
if (tenant_id_ != other.tenant_id_ || ctx_id_ != other.ctx_id_
|
||||
|| 0 != STRNCMP(label_, other.label_, sizeof(label_))) {
|
||||
ret = false;
|
||||
}
|
||||
if (ret) {
|
||||
if (other.bt_size_ != bt_size_) {
|
||||
ret = false;
|
||||
} else {
|
||||
for (int i = 0; i < bt_size_; ++i) {
|
||||
if ((int64_t)bt_[i] != (int64_t)other.bt_[i]) {
|
||||
ret = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
#define ob_malloc_sample_backtrace(obj, size) \
|
||||
{ \
|
||||
if (OB_UNLIKELY(obj->on_malloc_sample_)) { \
|
||||
int bt_len = backtrace(reinterpret_cast<void**>(&obj->data_[size]), AOBJECT_BACKTRACE_COUNT); \
|
||||
if (AOBJECT_BACKTRACE_COUNT > bt_len) { \
|
||||
reinterpret_cast<void*&>(obj->data_[size + bt_len * sizeof(void*)]) = nullptr; \
|
||||
} \
|
||||
} \
|
||||
}
|
||||
|
||||
} // end of namespace lib
|
||||
} // end of namespace oceanbase
|
||||
|
||||
#endif
|
185
deps/oblib/src/lib/alloc/ob_tenant_ctx_allocator.cpp
vendored
185
deps/oblib/src/lib/alloc/ob_tenant_ctx_allocator.cpp
vendored
@ -13,6 +13,8 @@
|
||||
#define USING_LOG_PREFIX LIB
|
||||
|
||||
#include "lib/alloc/ob_tenant_ctx_allocator.h"
|
||||
#include "lib/alloc/ob_malloc_sample_struct.h"
|
||||
#include "lib/allocator/ob_mem_leak_checker.h"
|
||||
#include "lib/allocator/ob_tc_malloc.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/alloc/memory_dump.h"
|
||||
@ -25,33 +27,10 @@ using namespace oceanbase::lib;
|
||||
using namespace oceanbase::common;
|
||||
void *ObTenantCtxAllocator::alloc(const int64_t size, const ObMemAttr &attr)
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
abort_unless(attr.tenant_id_ == tenant_id_);
|
||||
abort_unless(attr.ctx_id_ == ctx_id_);
|
||||
BACKTRACE_RET(WARN, OB_INVALID_ARGUMENT, !attr.label_.is_valid(), "[OB_MOD_DO_NOT_USE_ME ALLOC]size:%ld", size);
|
||||
void *ptr = NULL;
|
||||
AObject *obj = obj_mgr_.alloc_object(size, attr);
|
||||
if(OB_ISNULL(obj) && g_alloc_failed_ctx().need_wash()) {
|
||||
int64_t total_size = sync_wash();
|
||||
obj = obj_mgr_.alloc_object(size, attr);
|
||||
}
|
||||
if (NULL != obj) {
|
||||
ptr = obj->data_;
|
||||
SANITY_POISON(obj, AOBJECT_HEADER_SIZE);
|
||||
SANITY_UNPOISON(obj->data_, obj->alloc_bytes_);
|
||||
SANITY_POISON((void*)upper_align((int64_t)obj->data_ + obj->alloc_bytes_, 8), sizeof(AOBJECT_TAIL_MAGIC_CODE));
|
||||
}
|
||||
if (NULL == ptr && REACH_TIME_INTERVAL(1 * 1000 * 1000)) {
|
||||
const char *msg = alloc_failed_msg();
|
||||
LOG_DBA_WARN(OB_ALLOCATE_MEMORY_FAILED, "[OOPS]", "alloc failed reason", KCSTRING(msg));
|
||||
_OB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "oops, alloc failed, tenant_id=%ld, ctx_id=%ld, ctx_name=%s, ctx_hold=%ld, "
|
||||
"ctx_limit=%ld, tenant_hold=%ld, tenant_limit=%ld",
|
||||
tenant_id_, ctx_id_,
|
||||
common::get_global_ctx_info().get_ctx_name(ctx_id_),
|
||||
get_hold(), get_limit(), get_tenant_hold(), get_tenant_limit());
|
||||
// 49 is the user defined signal to dump memory
|
||||
raise(49);
|
||||
}
|
||||
void *ptr = common_alloc(size, attr, *this, obj_mgr_);
|
||||
return ptr;
|
||||
}
|
||||
|
||||
@ -64,54 +43,14 @@ int64_t ObTenantCtxAllocator::get_obj_hold(void *ptr)
|
||||
|
||||
void* ObTenantCtxAllocator::realloc(const void *ptr, const int64_t size, const ObMemAttr &attr)
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
void *nptr = NULL;
|
||||
AObject *obj = NULL;
|
||||
BACKTRACE_RET(WARN, OB_INVALID_ARGUMENT, !attr.label_.is_valid(), "[OB_MOD_DO_NOT_USE_ME REALLOC]size:%ld", size);
|
||||
if (NULL != ptr) {
|
||||
obj = reinterpret_cast<AObject*>((char*)ptr - AOBJECT_HEADER_SIZE);
|
||||
abort_unless(obj->is_valid());
|
||||
abort_unless(obj->in_use_);
|
||||
abort_unless(obj->block()->is_valid());
|
||||
abort_unless(obj->block()->in_use_);
|
||||
SANITY_POISON(obj->data_, obj->alloc_bytes_);
|
||||
}
|
||||
obj = obj_mgr_.realloc_object(obj, size, attr);
|
||||
if(OB_ISNULL(obj) && g_alloc_failed_ctx().need_wash()) {
|
||||
int64_t total_size = sync_wash();
|
||||
obj = obj_mgr_.realloc_object(obj, size, attr);
|
||||
}
|
||||
if (obj != NULL) {
|
||||
nptr = obj->data_;
|
||||
SANITY_POISON(obj, AOBJECT_HEADER_SIZE);
|
||||
SANITY_UNPOISON(obj->data_, obj->alloc_bytes_);
|
||||
SANITY_POISON((void*)upper_align((int64_t)obj->data_ + obj->alloc_bytes_, 8), sizeof(AOBJECT_TAIL_MAGIC_CODE));
|
||||
} else if (REACH_TIME_INTERVAL(1 * 1000 * 1000)) {
|
||||
const char *msg = alloc_failed_msg();
|
||||
LOG_DBA_WARN(OB_ALLOCATE_MEMORY_FAILED, "[OOPS]", "alloc failed reason", KCSTRING(msg));
|
||||
_OB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "oops, alloc failed, tenant_id=%ld, ctx_id=%ld, ctx_name=%s, ctx_hold=%ld, "
|
||||
"ctx_limit=%ld, tenant_hold=%ld, tenant_limit=%ld",
|
||||
tenant_id_, ctx_id_,
|
||||
common::get_global_ctx_info().get_ctx_name(ctx_id_),
|
||||
get_hold(), get_limit(), get_tenant_hold(), get_tenant_limit());
|
||||
// 49 is the user defined signal to dump memory
|
||||
raise(49);
|
||||
}
|
||||
BACKTRACE_RET(WARN, OB_INVALID_ARGUMENT, !attr.label_.is_valid(), "[OB_MOD_DO_NOT_USE_ME REALLOC]size:%ld", size);
|
||||
void *nptr = common_realloc(ptr, size, attr, *this, obj_mgr_);
|
||||
return nptr;
|
||||
}
|
||||
|
||||
void ObTenantCtxAllocator::free(void *ptr)
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
if (NULL != ptr) {
|
||||
AObject *obj = reinterpret_cast<AObject*>((char*)ptr - AOBJECT_HEADER_SIZE);
|
||||
abort_unless(NULL != obj);
|
||||
abort_unless(obj->MAGIC_CODE_ == AOBJECT_MAGIC_CODE
|
||||
|| obj->MAGIC_CODE_ == BIG_AOBJECT_MAGIC_CODE);
|
||||
abort_unless(obj->in_use_);
|
||||
SANITY_POISON(obj->data_, obj->alloc_bytes_);
|
||||
obj_mgr_.free_object(obj);
|
||||
}
|
||||
common_free(ptr);
|
||||
}
|
||||
int ObTenantCtxAllocator::iter_label(VisitFunc func) const
|
||||
{
|
||||
@ -234,6 +173,7 @@ void ObTenantCtxAllocator::print_usage() const
|
||||
|
||||
AChunk *ObTenantCtxAllocator::pop_chunk()
|
||||
{
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(chunk_freelist_mutex_);
|
||||
AChunk *chunk = head_chunk_.next_;
|
||||
AChunk *next_chunk = nullptr == chunk ? nullptr : chunk->next_;
|
||||
@ -246,6 +186,7 @@ AChunk *ObTenantCtxAllocator::pop_chunk()
|
||||
|
||||
void ObTenantCtxAllocator::push_chunk(AChunk *chunk)
|
||||
{
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(chunk_freelist_mutex_);
|
||||
chunk->next_ = head_chunk_.next_;
|
||||
head_chunk_.next_ = chunk;
|
||||
@ -277,6 +218,7 @@ AChunk *ObTenantCtxAllocator::alloc_chunk(const int64_t size, const ObMemAttr &a
|
||||
chunk = ObPageManagerCenter::get_instance().alloc_from_thread_local_cache(tenant_id_, ctx_id_);
|
||||
}
|
||||
} else {
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(using_list_mutex_);
|
||||
chunk->prev2_ = &using_list_head_;
|
||||
chunk->next2_ = using_list_head_.next2_;
|
||||
@ -290,6 +232,7 @@ AChunk *ObTenantCtxAllocator::alloc_chunk(const int64_t size, const ObMemAttr &a
|
||||
void ObTenantCtxAllocator::free_chunk(AChunk *chunk, const ObMemAttr &attr)
|
||||
{
|
||||
if (chunk != nullptr) {
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(using_list_mutex_);
|
||||
chunk->prev2_->next2_ = chunk->next2_;
|
||||
chunk->next2_->prev2_ = chunk->prev2_;
|
||||
@ -388,6 +331,7 @@ int ObTenantCtxAllocator::set_idle(const int64_t set_size, const bool reserve/*=
|
||||
|
||||
void ObTenantCtxAllocator::get_chunks(AChunk **chunks, int cap, int &cnt)
|
||||
{
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(using_list_mutex_);
|
||||
AChunk *cur = using_list_head_.next2_;
|
||||
while (cur != &using_list_head_ && cnt < cap) {
|
||||
@ -449,3 +393,110 @@ void ObTenantCtxAllocator::update_wash_stat(int64_t related_chunks, int64_t bloc
|
||||
(void)ATOMIC_FAA(&washed_blocks_, blocks);
|
||||
(void)ATOMIC_FAA(&washed_size_, size);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void* ObTenantCtxAllocator::common_alloc(const int64_t size, const ObMemAttr &attr,
|
||||
ObTenantCtxAllocator& ta, T &allocator)
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
void *ret = nullptr;
|
||||
bool sample_allowed = ObMallocSampleLimiter::malloc_sample_allowed(size, attr);
|
||||
const int64_t alloc_size = sample_allowed ? (size + AOBJECT_BACKTRACE_SIZE) : size;
|
||||
AObject *obj = allocator.alloc_object(alloc_size, attr);
|
||||
if (OB_ISNULL(obj) && g_alloc_failed_ctx().need_wash()) {
|
||||
int64_t total_size = ta.sync_wash();
|
||||
obj = allocator.alloc_object(alloc_size, attr);
|
||||
}
|
||||
if (NULL != obj) {
|
||||
obj->on_malloc_sample_ = sample_allowed;
|
||||
ob_malloc_sample_backtrace(obj, size);
|
||||
ret = obj->data_;
|
||||
get_mem_leak_checker().on_alloc(*obj, attr);
|
||||
SANITY_POISON(obj, AOBJECT_HEADER_SIZE);
|
||||
SANITY_UNPOISON(obj->data_, size);
|
||||
SANITY_POISON((void*)upper_align((int64_t)obj->data_ + size, 8),
|
||||
alloc_size - size + sizeof(AOBJECT_TAIL_MAGIC_CODE));
|
||||
}
|
||||
if (OB_UNLIKELY(nullptr == obj) && REACH_TIME_INTERVAL(1 * 1000 * 1000)) {
|
||||
const char *msg = alloc_failed_msg();
|
||||
LOG_DBA_WARN(OB_ALLOCATE_MEMORY_FAILED, "[OOPS]", "alloc failed reason", KCSTRING(msg));
|
||||
_OB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "oops, alloc failed, tenant_id=%ld, ctx_id=%ld, ctx_name=%s, ctx_hold=%ld, "
|
||||
"ctx_limit=%ld, tenant_hold=%ld, tenant_limit=%ld",
|
||||
attr.tenant_id_, attr.ctx_id_,
|
||||
get_global_ctx_info().get_ctx_name(attr.ctx_id_),
|
||||
ta.get_hold(), ta.get_limit(), ta.get_tenant_hold(), ta.get_tenant_limit());
|
||||
// 49 is the user defined signal to dump memory
|
||||
raise(49);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void* ObTenantCtxAllocator::common_realloc(const void *ptr, const int64_t size,
|
||||
const ObMemAttr &attr, ObTenantCtxAllocator& ta,
|
||||
T &allocator)
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
void *nptr = NULL;
|
||||
AObject *obj = NULL;
|
||||
if (NULL != ptr) {
|
||||
obj = reinterpret_cast<AObject*>((char*)ptr - AOBJECT_HEADER_SIZE);
|
||||
abort_unless(obj->is_valid());
|
||||
abort_unless(obj->in_use_);
|
||||
abort_unless(obj->block()->is_valid());
|
||||
abort_unless(obj->block()->in_use_);
|
||||
SANITY_POISON(obj->data_, obj->alloc_bytes_);
|
||||
get_mem_leak_checker().on_free(*obj);
|
||||
}
|
||||
bool sample_allowed = ObMallocSampleLimiter::malloc_sample_allowed(size, attr);
|
||||
const int64_t alloc_size = sample_allowed ? (size + AOBJECT_BACKTRACE_SIZE) : size;
|
||||
obj = allocator.realloc_object(obj, alloc_size, attr);
|
||||
if(OB_ISNULL(obj) && g_alloc_failed_ctx().need_wash()) {
|
||||
int64_t total_size = ta.sync_wash();
|
||||
obj = allocator.realloc_object(obj, alloc_size, attr);
|
||||
}
|
||||
if (obj != NULL) {
|
||||
obj->on_malloc_sample_ = sample_allowed;
|
||||
ob_malloc_sample_backtrace(obj, size);
|
||||
nptr = obj->data_;
|
||||
get_mem_leak_checker().on_alloc(*obj, attr);
|
||||
SANITY_POISON(obj, AOBJECT_HEADER_SIZE);
|
||||
SANITY_UNPOISON(obj->data_, size);
|
||||
SANITY_POISON((void*)upper_align((int64_t)obj->data_ + size, 8),
|
||||
alloc_size - size + sizeof(AOBJECT_TAIL_MAGIC_CODE));
|
||||
} else if (REACH_TIME_INTERVAL(1 * 1000 * 1000)) {
|
||||
const char *msg = alloc_failed_msg();
|
||||
LOG_DBA_WARN(OB_ALLOCATE_MEMORY_FAILED, "[OOPS]", "alloc failed reason", KCSTRING(msg));
|
||||
_OB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "oops, alloc failed, tenant_id=%ld, ctx_id=%ld, ctx_name=%s, ctx_hold=%ld, "
|
||||
"ctx_limit=%ld, tenant_hold=%ld, tenant_limit=%ld",
|
||||
attr.tenant_id_, attr.ctx_id_,
|
||||
get_global_ctx_info().get_ctx_name(attr.ctx_id_),
|
||||
ta.get_hold(), ta.get_limit(), ta.get_tenant_hold(), ta.get_tenant_limit());
|
||||
// 49 is the user defined signal to dump memory
|
||||
raise(49);
|
||||
}
|
||||
return nptr;
|
||||
}
|
||||
|
||||
void ObTenantCtxAllocator::common_free(void *ptr)
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
if (NULL != ptr) {
|
||||
AObject *obj = reinterpret_cast<AObject*>((char*)ptr - AOBJECT_HEADER_SIZE);
|
||||
abort_unless(NULL != obj);
|
||||
abort_unless(obj->MAGIC_CODE_ == AOBJECT_MAGIC_CODE
|
||||
|| obj->MAGIC_CODE_ == BIG_AOBJECT_MAGIC_CODE);
|
||||
abort_unless(obj->in_use_);
|
||||
SANITY_POISON(obj->data_, obj->alloc_bytes_);
|
||||
|
||||
get_mem_leak_checker().on_free(*obj);
|
||||
ABlock *block = obj->block();
|
||||
abort_unless(block);
|
||||
abort_unless(block->is_valid());
|
||||
abort_unless(block->in_use_);
|
||||
abort_unless(block->obj_set_ != NULL);
|
||||
|
||||
ObjectSet *os = block->obj_set_;
|
||||
os->free_object(obj);
|
||||
}
|
||||
}
|
||||
|
@ -184,6 +184,19 @@ private:
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
public:
|
||||
template <typename T>
|
||||
static void* common_alloc(const int64_t size, const ObMemAttr &attr,
|
||||
ObTenantCtxAllocator& ta, T &allocator);
|
||||
|
||||
template <typename T>
|
||||
static void* common_realloc(const void *ptr, const int64_t size,
|
||||
const ObMemAttr &attr, ObTenantCtxAllocator& ta,
|
||||
T &allocator);
|
||||
|
||||
static void common_free(void *ptr);
|
||||
|
||||
private:
|
||||
ObTenantResourceMgrHandle resource_handle_;
|
||||
int64_t ref_cnt_;
|
||||
|
2
deps/oblib/src/lib/alloc/object_set.cpp
vendored
2
deps/oblib/src/lib/alloc/object_set.cpp
vendored
@ -377,6 +377,7 @@ void ObjectSet::free_object(AObject *obj)
|
||||
}
|
||||
#endif
|
||||
const int64_t ctx_id = blk_mgr_->ctx_id_;
|
||||
ObDisableDiagnoseGuard diagnose_disable_guard;
|
||||
if (ctx_id == common::ObCtxIds::LIBEASY) {
|
||||
if (locker_->trylock()) {
|
||||
do_free_object(obj);
|
||||
@ -417,6 +418,7 @@ void ObjectSet::do_free_object(AObject *obj)
|
||||
used_bytes_ -= hold;
|
||||
|
||||
obj->in_use_ = false;
|
||||
obj->on_malloc_sample_ = false;
|
||||
if (!obj->is_large_) {
|
||||
free_normal_object(obj);
|
||||
} else {
|
||||
|
2
deps/oblib/src/lib/alloc/object_set.h
vendored
2
deps/oblib/src/lib/alloc/object_set.h
vendored
@ -126,11 +126,13 @@ private:
|
||||
|
||||
inline void ObjectSet::lock()
|
||||
{
|
||||
ObDisableDiagnoseGuard diagnose_disable_guard;
|
||||
locker_->lock();
|
||||
}
|
||||
|
||||
inline void ObjectSet::unlock()
|
||||
{
|
||||
ObDisableDiagnoseGuard diagnose_disable_guard;
|
||||
locker_->unlock();
|
||||
}
|
||||
|
||||
|
54
deps/oblib/src/lib/allocator/ob_allocator_v2.cpp
vendored
54
deps/oblib/src/lib/allocator/ob_allocator_v2.cpp
vendored
@ -30,37 +30,13 @@ void *ObAllocator::alloc(const int64_t size, const ObMemAttr &attr)
|
||||
ret = init();
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
BACKTRACE(WARN, !attr.label_.is_valid(), "[OB_MOD_DO_NOT_USE_ME ALLOC]size:%ld", size);
|
||||
BACKTRACE_RET(WARN, OB_INVALID_ARGUMENT, !attr.label_.is_valid(), "[OB_MOD_DO_NOT_USE_ME ALLOC]size:%ld", size);
|
||||
ObMemAttr inner_attr = attr_;
|
||||
if (attr.label_.is_valid()) {
|
||||
inner_attr.label_ = attr.label_;
|
||||
}
|
||||
AObject *obj = os_.alloc_object(size, inner_attr);
|
||||
if (OB_ISNULL(obj) && g_alloc_failed_ctx().need_wash()) {
|
||||
auto ta = lib::ObMallocAllocator::get_instance()->get_tenant_ctx_allocator(attr_.tenant_id_,
|
||||
attr_.ctx_id_);
|
||||
int64_t total_size = ta->sync_wash();
|
||||
obj = os_.alloc_object(size, inner_attr);
|
||||
}
|
||||
if (NULL != obj) {
|
||||
ptr = obj->data_;
|
||||
get_mem_leak_checker().on_alloc(*obj, inner_attr);
|
||||
SANITY_POISON(obj, AOBJECT_HEADER_SIZE);
|
||||
SANITY_UNPOISON(obj->data_, obj->alloc_bytes_);
|
||||
SANITY_POISON((void*)upper_align((int64_t)obj->data_ + obj->alloc_bytes_, 8), sizeof(AOBJECT_TAIL_MAGIC_CODE));
|
||||
}
|
||||
if (OB_UNLIKELY(nullptr == obj) && REACH_TIME_INTERVAL(1 * 1000 * 1000)) {
|
||||
auto ta = lib::ObMallocAllocator::get_instance()->get_tenant_ctx_allocator(attr_.tenant_id_,
|
||||
attr_.ctx_id_);
|
||||
_OB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "[OOPS] alloc failed reason: %s", alloc_failed_msg());
|
||||
_OB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "oops, alloc failed, tenant_id=%ld, ctx_id=%ld, ctx_name=%s, ctx_hold=%ld, "
|
||||
"ctx_limit=%ld, tenant_hold=%ld, tenant_limit=%ld",
|
||||
inner_attr.tenant_id_, inner_attr.ctx_id_,
|
||||
common::get_global_ctx_info().get_ctx_name(inner_attr.ctx_id_),
|
||||
ta->get_hold(), ta->get_limit(), ta->get_tenant_hold(), ta->get_tenant_limit());
|
||||
// 49 is the user defined signal to dump memory
|
||||
raise(49);
|
||||
}
|
||||
auto ta = lib::ObMallocAllocator::get_instance()->get_tenant_ctx_allocator(attr_.tenant_id_, attr_.ctx_id_);
|
||||
ptr = ObTenantCtxAllocator::common_alloc(size, inner_attr, *(ta.ref_allocator()), os_);
|
||||
}
|
||||
return ptr;
|
||||
}
|
||||
@ -68,28 +44,8 @@ void *ObAllocator::alloc(const int64_t size, const ObMemAttr &attr)
|
||||
void ObAllocator::free(void *ptr)
|
||||
{
|
||||
SANITY_DISABLE_CHECK_RANGE(); // prevent sanity_check_range
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!is_inited_)) {
|
||||
ret = init();
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_LIKELY(nullptr != ptr)) {
|
||||
AObject *obj = reinterpret_cast<AObject*>((char*)ptr - lib::AOBJECT_HEADER_SIZE);
|
||||
abort_unless(NULL != obj);
|
||||
abort_unless(obj->MAGIC_CODE_ == lib::AOBJECT_MAGIC_CODE
|
||||
|| obj->MAGIC_CODE_ == lib::BIG_AOBJECT_MAGIC_CODE);
|
||||
abort_unless(obj->in_use_);
|
||||
SANITY_POISON(obj->data_, obj->alloc_bytes_);
|
||||
|
||||
get_mem_leak_checker().on_free(*obj);
|
||||
lib::ABlock *block = obj->block();
|
||||
abort_unless(block);
|
||||
abort_unless(block->is_valid());
|
||||
ObjectSet *os = block->obj_set_;
|
||||
abort_unless(os == &os_);
|
||||
os->free_object(obj);
|
||||
}
|
||||
}
|
||||
// directly free object instead of using tenant allocator.
|
||||
ObTenantCtxAllocator::common_free(ptr);
|
||||
}
|
||||
|
||||
void *ObParallelAllocator::alloc(const int64_t size, const ObMemAttr &attr)
|
||||
|
@ -160,7 +160,6 @@ public:
|
||||
|
||||
void on_alloc(lib::AObject &obj, const ObMemAttr &attr)
|
||||
{
|
||||
obj.on_leak_check_ = false;
|
||||
if (is_label_check() &&
|
||||
label_match(obj) &&
|
||||
(tenant_id_ == UINT64_MAX || tenant_id_ == attr.tenant_id_) &&
|
||||
@ -187,6 +186,7 @@ public:
|
||||
if (is_label_check() &&
|
||||
obj.on_leak_check_ &&
|
||||
label_match(obj)) {
|
||||
obj.on_leak_check_ = false;
|
||||
PtrKey ptr_key;
|
||||
ptr_key.ptr_ = obj.data_;
|
||||
int ret = OB_SUCCESS;
|
||||
|
2
deps/oblib/src/lib/allocator/ob_mod_define.h
vendored
2
deps/oblib/src/lib/allocator/ob_mod_define.h
vendored
@ -43,6 +43,8 @@ CTX_ITEM_DEF(META_OBJ_CTX_ID)
|
||||
CTX_ITEM_DEF(TX_CALLBACK_CTX_ID)
|
||||
CTX_ITEM_DEF(LOB_CTX_ID)
|
||||
CTX_ITEM_DEF(PS_CACHE_CTX_ID)
|
||||
CTX_ITEM_DEF(RPC_CTX_ID)
|
||||
CTX_ITEM_DEF(PKT_NIO)
|
||||
CTX_ITEM_DEF(TX_DATA_TABLE)
|
||||
CTX_ITEM_DEF(MAX_CTX_ID)
|
||||
#endif
|
||||
|
@ -32,6 +32,7 @@ ObPageManagerCenter &ObPageManagerCenter::get_instance()
|
||||
int ObPageManagerCenter::register_pm(ObPageManager &pm)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(mutex_);
|
||||
rb_tree_.insert(&pm);
|
||||
pm.has_register_ = true;
|
||||
@ -42,6 +43,7 @@ int ObPageManagerCenter::register_pm(ObPageManager &pm)
|
||||
|
||||
void ObPageManagerCenter::unregister_pm(ObPageManager &pm)
|
||||
{
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(mutex_);
|
||||
pm.has_register_ = false;
|
||||
rb_tree_.remove(&pm);
|
||||
@ -57,7 +59,7 @@ int ObPageManagerCenter::print_tenant_stat(int64_t tenant_id, char *buf,
|
||||
int64_t len, int64_t &pos)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
lib::ObMutexGuard guard(mutex_);
|
||||
int64_t sum_used = 0;
|
||||
int64_t sum_hold = 0;
|
||||
@ -74,6 +76,7 @@ AChunk *ObPageManagerCenter::alloc_from_thread_local_cache(int64_t tenant_id, in
|
||||
int tmpret = OB_SUCCESS;
|
||||
AChunk *ret = nullptr;
|
||||
const int RETRY_LIMIT = 10;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
for (int retry = 0; retry < RETRY_LIMIT && OB_EAGAIN == (tmpret = mutex_.trylock()); ++retry) {
|
||||
sched_yield();
|
||||
}
|
||||
|
@ -18,11 +18,13 @@
|
||||
#include "lib/allocator/ob_allocator.h"
|
||||
#include "lib/allocator/ob_mod_define.h"
|
||||
#include "lib/list/ob_free_list.h"
|
||||
#include "lib/utility/ob_tracepoint.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "lib/hash_func/ob_hash_func.h"
|
||||
#include "lib/allocator/ob_mem_leak_checker.h"
|
||||
#include "lib/alloc/ob_malloc_allocator.h"
|
||||
#include "lib/worker.h"
|
||||
#include "lib/alloc/malloc_hook.h"
|
||||
|
||||
using namespace oceanbase::lib;
|
||||
using namespace oceanbase::common;
|
||||
@ -131,7 +133,12 @@ const ObCtxInfo &get_global_ctx_info()
|
||||
|
||||
void __attribute__((constructor(MALLOC_INIT_PRIORITY))) init_global_memory_pool()
|
||||
{
|
||||
auto& t = EventTable::instance();
|
||||
auto& c = get_mem_leak_checker();
|
||||
auto& a = AChunkMgr::instance();
|
||||
in_hook()= true;
|
||||
global_default_allocator = ObMallocAllocator::get_instance();
|
||||
in_hook()= false;
|
||||
#ifndef OB_USE_ASAN
|
||||
abort_unless(OB_SUCCESS == install_ob_signal_handler());
|
||||
#endif
|
||||
@ -151,6 +158,5 @@ int64_t get_virtual_memory_used(int64_t *resident_size)
|
||||
return page_cnt * ps;
|
||||
}
|
||||
|
||||
|
||||
} // end namespace common
|
||||
} // end namespace oceanbase
|
||||
|
7
deps/oblib/src/lib/lock/ob_futex.cpp
vendored
7
deps/oblib/src/lib/lock/ob_futex.cpp
vendored
@ -22,7 +22,12 @@ static struct timespec make_timespec(int64_t us)
|
||||
return ts;
|
||||
}
|
||||
|
||||
|
||||
extern "C" {
|
||||
int __attribute__((weak)) futex_hook(uint32_t *uaddr, int futex_op, uint32_t val, const struct timespec* timeout)
|
||||
{
|
||||
return syscall(SYS_futex, uaddr, futex_op, val, timeout);
|
||||
}
|
||||
}
|
||||
|
||||
using namespace oceanbase::lib;
|
||||
using namespace oceanbase::common;
|
||||
|
21
deps/oblib/src/lib/lock/ob_futex.h
vendored
21
deps/oblib/src/lib/lock/ob_futex.h
vendored
@ -18,25 +18,22 @@
|
||||
#include "lib/list/ob_dlist.h"
|
||||
#include <linux/futex.h>
|
||||
#include "lib/ob_abort.h"
|
||||
#define futex(...) syscall(SYS_futex,__VA_ARGS__)
|
||||
|
||||
extern "C" {
|
||||
extern int futex_hook(uint32_t *uaddr, int futex_op, uint32_t val, const struct timespec* timeout);
|
||||
}
|
||||
|
||||
#define futex(...) futex_hook(__VA_ARGS__)
|
||||
|
||||
inline int futex_wake(volatile int *p, int val)
|
||||
{
|
||||
return static_cast<int>(futex((int *)p, FUTEX_WAKE_PRIVATE, val, NULL, NULL, 0));
|
||||
return futex((uint32_t *)p, FUTEX_WAKE_PRIVATE, val, NULL);
|
||||
}
|
||||
|
||||
inline int futex_wait(volatile int *p, int val, const timespec *timeout)
|
||||
{
|
||||
int ret = 0;
|
||||
if (0 != futex((int *)p, FUTEX_WAIT_PRIVATE, val, timeout, NULL, 0)) {
|
||||
ret = errno;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int futex_wait_until(volatile int *p, int val, const timespec *timeout, int wait_mask)
|
||||
{
|
||||
int ret = 0;
|
||||
if (0 != futex((int *)p, FUTEX_WAIT_BITSET_PRIVATE, val, timeout, NULL, wait_mask)) {
|
||||
if (0 != futex((uint32_t *)p, FUTEX_WAIT_PRIVATE, val, timeout)) {
|
||||
ret = errno;
|
||||
}
|
||||
return ret;
|
||||
|
72
deps/oblib/src/lib/lock/ob_latch.cpp
vendored
72
deps/oblib/src/lib/lock/ob_latch.cpp
vendored
@ -22,11 +22,33 @@ namespace oceanbase
|
||||
namespace common
|
||||
{
|
||||
bool USE_CO_LATCH = false;
|
||||
thread_local uint32_t* ObLatch::current_lock = nullptr;
|
||||
thread_local uint32_t* ObLatch::current_wait = nullptr;
|
||||
|
||||
class ObLatchWaitEventGuard : public ObWaitEventGuard
|
||||
{
|
||||
public:
|
||||
explicit ObLatchWaitEventGuard(
|
||||
const int64_t event_no,
|
||||
const uint64_t timeout_ms = 0,
|
||||
const int64_t p1 = 0,
|
||||
uint32_t* p2_addr = 0,
|
||||
const int64_t p3 = 0,
|
||||
const bool is_atomic = false
|
||||
) : ObWaitEventGuard(event_no, timeout_ms, p1, OB_ISNULL(p2_addr) ? 0 : *p2_addr, p3, is_atomic)
|
||||
{
|
||||
ObLatch::current_wait = p2_addr;
|
||||
}
|
||||
~ObLatchWaitEventGuard() { ObLatch::current_wait = nullptr; }
|
||||
};
|
||||
/**
|
||||
* -------------------------------------------------------ObLatchMutex---------------------------------------------------------------
|
||||
*/
|
||||
ObLatchMutex::ObLatchMutex()
|
||||
: lock_(), record_stat_(true)
|
||||
: lock_()
|
||||
#ifndef PERF_MODE
|
||||
,record_stat_(true)
|
||||
#endif
|
||||
{
|
||||
}
|
||||
|
||||
@ -51,8 +73,12 @@ int ObLatchMutex::try_lock(
|
||||
} else {
|
||||
if (!ATOMIC_BCAS(&lock_.val(), 0, (WRITE_MASK | uid))) {
|
||||
ret = OB_EAGAIN;
|
||||
} else {
|
||||
ObLatch::current_lock = (uint32_t*)&lock_.val();
|
||||
}
|
||||
if (need_record_stat()) {
|
||||
TRY_LOCK_RECORD_STAT(latch_id, 1, ret);
|
||||
}
|
||||
TRY_LOCK_RECORD_STAT(latch_id, 1, ret, record_stat_);
|
||||
}
|
||||
HOLD_LOCK_INC();
|
||||
return ret;
|
||||
@ -93,11 +119,11 @@ int ObLatchMutex::lock(
|
||||
//wait
|
||||
waited = true;
|
||||
// latch mutex wait is an atomic wait event
|
||||
ObWaitEventGuard wait_guard(
|
||||
ObLatchWaitEventGuard wait_guard(
|
||||
OB_LATCHES[latch_id].wait_event_idx_,
|
||||
abs_timeout_us / 1000,
|
||||
reinterpret_cast<uint64_t>(this),
|
||||
lock_.val(),
|
||||
(uint32_t*)&lock_.val(),
|
||||
0,
|
||||
true /*is_atomic*/);
|
||||
if (OB_FAIL(wait(abs_timeout_us, uid))) {
|
||||
@ -109,7 +135,9 @@ int ObLatchMutex::lock(
|
||||
}
|
||||
}
|
||||
}
|
||||
LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt, record_stat_);
|
||||
if (need_record_stat()) {
|
||||
LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt);
|
||||
}
|
||||
}
|
||||
HOLD_LOCK_INC();
|
||||
return ret;
|
||||
@ -157,7 +185,7 @@ int ObLatchMutex::unlock()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint32_t lock = ATOMIC_SET(&lock_.val(), 0);
|
||||
|
||||
ObLatch::current_lock = nullptr;
|
||||
if (OB_UNLIKELY(0 == lock)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
COMMON_LOG(ERROR, "invalid lock,", K(lock), K(ret));
|
||||
@ -237,11 +265,11 @@ int ObLatchWaitQueue::wait(
|
||||
}
|
||||
|
||||
{
|
||||
ObWaitEventGuard wait_guard(
|
||||
ObLatchWaitEventGuard wait_guard(
|
||||
ObWaitEventIds::LATCH_WAIT_QUEUE_LOCK_WAIT,
|
||||
abs_timeout_us / 1000,
|
||||
reinterpret_cast<uint64_t>(this),
|
||||
latch.lock_,
|
||||
(uint32_t*)&latch.lock_,
|
||||
0,
|
||||
true /*is_atomic*/);
|
||||
ts.tv_sec = timeout / 1000000;
|
||||
@ -545,6 +573,9 @@ void ObLockDiagnose::print()
|
||||
|
||||
ObLatch::ObLatch()
|
||||
: lock_(0)
|
||||
#ifndef PERF_MODE
|
||||
, record_stat_(true)
|
||||
#endif
|
||||
{
|
||||
}
|
||||
|
||||
@ -582,6 +613,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_;
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -590,8 +622,9 @@ int ObLatch::try_rdlock(const uint32_t latch_id)
|
||||
}
|
||||
PAUSE();
|
||||
} while (true);
|
||||
|
||||
TRY_LOCK_RECORD_STAT(latch_id, i, ret, true);
|
||||
if (need_record_stat()) {
|
||||
TRY_LOCK_RECORD_STAT(latch_id, i, ret);
|
||||
}
|
||||
}
|
||||
HOLD_LOCK_INC();
|
||||
return ret;
|
||||
@ -609,9 +642,12 @@ int ObLatch::try_wrlock(const uint32_t latch_id, const uint32_t *puid)
|
||||
} else {
|
||||
if (!ATOMIC_BCAS(&lock_, 0, (WRITE_MASK | uid))) {
|
||||
ret = OB_EAGAIN;
|
||||
} else {
|
||||
ObLatch::current_lock = (uint32_t*)&lock_;
|
||||
}
|
||||
if (need_record_stat()) {
|
||||
TRY_LOCK_RECORD_STAT(latch_id, 1, ret);
|
||||
}
|
||||
|
||||
TRY_LOCK_RECORD_STAT(latch_id, 1, ret, true);
|
||||
}
|
||||
HOLD_LOCK_INC();
|
||||
return ret;
|
||||
@ -678,6 +714,7 @@ int ObLatch::wr2rdlock(const uint32_t *puid)
|
||||
lock = lock_;
|
||||
PAUSE();
|
||||
}
|
||||
ObLatch::current_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));
|
||||
@ -699,9 +736,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;
|
||||
}
|
||||
} else if ((lock & (~WAIT_MASK)) > 0) {
|
||||
lock = ATOMIC_AAF(&lock_, -1);
|
||||
ObLatch::current_lock = nullptr;
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
COMMON_LOG(ERROR, "invalid lock,", K(lock), K(ret));
|
||||
@ -769,11 +808,11 @@ OB_INLINE int ObLatch::low_lock(
|
||||
} else {
|
||||
//wait
|
||||
waited = true;
|
||||
ObWaitEventGuard wait_guard(
|
||||
ObLatchWaitEventGuard wait_guard(
|
||||
OB_LATCHES[latch_id].wait_event_idx_,
|
||||
abs_timeout_us / 1000,
|
||||
reinterpret_cast<uint64_t>(this),
|
||||
lock,
|
||||
(uint32_t*)&lock_,
|
||||
0);
|
||||
ObWaitProc proc(*this, wait_mode);
|
||||
if (OB_FAIL(ObLatchWaitQueue::get_instance().wait(
|
||||
@ -791,8 +830,9 @@ OB_INLINE int ObLatch::low_lock(
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt, true);
|
||||
if (need_record_stat()) {
|
||||
LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
84
deps/oblib/src/lib/lock/ob_latch.h
vendored
84
deps/oblib/src/lib/lock/ob_latch.h
vendored
@ -37,10 +37,10 @@ extern bool USE_CO_LATCH;
|
||||
do { \
|
||||
} while(0)
|
||||
|
||||
#if !PERF_MODE
|
||||
#define TRY_LOCK_RECORD_STAT(latch_id, spin_cnt, ret, enable) \
|
||||
#ifndef PERF_MODE
|
||||
#define TRY_LOCK_RECORD_STAT(latch_id, spin_cnt, ret) \
|
||||
do { \
|
||||
if (enable) { \
|
||||
if (lib::is_diagnose_info_enabled()) { \
|
||||
ObDiagnoseTenantInfo *di = ObDiagnoseTenantInfo::get_local_diagnose_info(); \
|
||||
if (NULL != di) { \
|
||||
ObLatchStat &latch_stat = di->get_latch_stats().items_[latch_id]; \
|
||||
@ -54,35 +54,35 @@ extern bool USE_CO_LATCH;
|
||||
} \
|
||||
} while(0)
|
||||
#else
|
||||
#define TRY_LOCK_RECORD_STAT(latch_id, spin_cnt, ret, enable)
|
||||
#define TRY_LOCK_RECORD_STAT(latch_id, spin_cnt, ret)
|
||||
#endif
|
||||
|
||||
#if !PERF_MODE
|
||||
#define LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt, enable) \
|
||||
do { \
|
||||
if (enable) { \
|
||||
ObDiagnoseTenantInfo *di = ObDiagnoseTenantInfo::get_local_diagnose_info(); \
|
||||
if (NULL != di) { \
|
||||
ObLatchStat &latch_stat = di->get_latch_stats().items_[latch_id]; \
|
||||
++latch_stat.gets_; \
|
||||
latch_stat.spin_gets_ += spin_cnt; \
|
||||
latch_stat.sleeps_ += yield_cnt; \
|
||||
if (OB_UNLIKELY(waited)) { \
|
||||
++latch_stat.misses_; \
|
||||
ObDiagnoseSessionInfo *dsi = ObDiagnoseSessionInfo::get_local_diagnose_info(); \
|
||||
if (NULL != dsi) { \
|
||||
latch_stat.wait_time_ += dsi->get_curr_wait().wait_time_; \
|
||||
if (dsi->get_curr_wait().wait_time_ > 1000 * 1000) { \
|
||||
COMMON_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "The Latch wait too much time, ", \
|
||||
K(dsi->get_curr_wait()), KCSTRING(lbt())); \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
#ifndef PERF_MODE
|
||||
#define LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt) \
|
||||
do { \
|
||||
if (lib::is_diagnose_info_enabled()) { \
|
||||
ObDiagnoseTenantInfo *di = ObDiagnoseTenantInfo::get_local_diagnose_info(); \
|
||||
if (NULL != di) { \
|
||||
ObLatchStat &latch_stat = di->get_latch_stats().items_[latch_id]; \
|
||||
++latch_stat.gets_; \
|
||||
latch_stat.spin_gets_ += spin_cnt; \
|
||||
latch_stat.sleeps_ += yield_cnt; \
|
||||
if (OB_UNLIKELY(waited)) { \
|
||||
++latch_stat.misses_; \
|
||||
ObDiagnoseSessionInfo *dsi = ObDiagnoseSessionInfo::get_local_diagnose_info(); \
|
||||
if (NULL != dsi) { \
|
||||
latch_stat.wait_time_ += dsi->get_curr_wait().wait_time_; \
|
||||
if (dsi->get_curr_wait().wait_time_ > 1000 * 1000) { \
|
||||
COMMON_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "The Latch wait too much time, ", \
|
||||
K(dsi->get_curr_wait()), KCSTRING(lbt())); \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
} \
|
||||
} while(0)
|
||||
#else
|
||||
#define LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt, enable)
|
||||
#define LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt)
|
||||
#endif
|
||||
|
||||
struct ObLatchWaitMode
|
||||
@ -111,7 +111,13 @@ public:
|
||||
inline bool is_locked();
|
||||
inline uint32_t get_wid();
|
||||
int64_t to_string(char* buf, const int64_t buf_len);
|
||||
#ifndef PERF_MODE
|
||||
void enable_record_stat(bool enable) { record_stat_ = enable; }
|
||||
bool need_record_stat() const { return record_stat_; }
|
||||
#else
|
||||
void enable_record_stat(bool enable) { UNUSED(enable); }
|
||||
bool need_record_stat() const { return false; }
|
||||
#endif
|
||||
|
||||
private:
|
||||
OB_INLINE uint64_t low_try_lock(const int64_t max_spin_cnt, const uint32_t lock_value);
|
||||
@ -122,7 +128,9 @@ private:
|
||||
static const uint32_t WAIT_MASK = 1<<31;
|
||||
lib::ObFutex lock_;
|
||||
//volatile int32_t lock_;
|
||||
#ifndef PERF_MODE
|
||||
bool record_stat_;
|
||||
#endif
|
||||
};
|
||||
|
||||
class ObLatch;
|
||||
@ -206,8 +214,11 @@ private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObLatchWaitQueue);
|
||||
};
|
||||
|
||||
class TCRWLock;
|
||||
|
||||
class ObLatch
|
||||
{
|
||||
friend class TCRWLock;
|
||||
public:
|
||||
ObLatch();
|
||||
~ObLatch();
|
||||
@ -228,7 +239,16 @@ public:
|
||||
inline bool is_wrlocked_by(const uint32_t *puid = NULL) const;
|
||||
inline uint32_t get_wid() const;
|
||||
int64_t to_string(char* buf, const int64_t buf_len) const;
|
||||
|
||||
#ifndef PERF_MODE
|
||||
void enable_record_stat(bool enable) { record_stat_ = enable; }
|
||||
bool need_record_stat() const { return record_stat_; }
|
||||
#else
|
||||
void enable_record_stat(bool enable) { UNUSED(enable); }
|
||||
bool need_record_stat() const { return false; }
|
||||
#endif
|
||||
uint32_t val() const { return lock_; }
|
||||
static thread_local uint32_t* current_lock;
|
||||
static thread_local uint32_t* current_wait;
|
||||
private:
|
||||
template<typename LowTryLock>
|
||||
OB_INLINE int low_lock(
|
||||
@ -260,6 +280,9 @@ private:
|
||||
static const uint32_t WAIT_MASK = 1<<31;
|
||||
static const uint32_t MAX_READ_LOCK_CNT = 1<<24;
|
||||
volatile uint32_t lock_;
|
||||
#ifndef PERF_MODE
|
||||
bool record_stat_;
|
||||
#endif
|
||||
};
|
||||
|
||||
struct ObLDLockType
|
||||
@ -459,6 +482,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());
|
||||
break;
|
||||
}
|
||||
}
|
||||
@ -520,6 +544,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;
|
||||
}
|
||||
} else {
|
||||
conflict = true;
|
||||
@ -542,6 +567,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;
|
||||
}
|
||||
} else {
|
||||
conflict = true;
|
||||
|
1
deps/oblib/src/lib/lock/ob_spin_rwlock.h
vendored
1
deps/oblib/src/lib/lock/ob_spin_rwlock.h
vendored
@ -50,6 +50,7 @@ public:
|
||||
inline int unlock() { return latch_.unlock(); }
|
||||
inline void rdunlock() { unlock(); }
|
||||
inline void wrunlock() { unlock(); }
|
||||
inline void enable_record_stat(bool enable) { latch_.enable_record_stat(enable); }
|
||||
private:
|
||||
ObLatch latch_;
|
||||
uint32_t latch_id_;
|
||||
|
16
deps/oblib/src/lib/lock/ob_tc_rwlock.h
vendored
16
deps/oblib/src/lib/lock/ob_tc_rwlock.h
vendored
@ -148,6 +148,10 @@ public:
|
||||
ret = latch_.unlock();
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
// record in try_rdlock will be overwrited by latch_.rdlock, so record again.
|
||||
ObLatch::current_lock = (uint32_t*)&(latch_.lock_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
inline bool try_rdlock()
|
||||
@ -157,6 +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_);
|
||||
} else {
|
||||
get_tcref().dec_ref(&read_ref_);
|
||||
lcond_.signal();
|
||||
@ -168,6 +173,7 @@ public:
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
get_tcref().dec_ref(&read_ref_);
|
||||
ObLatch::current_lock = nullptr;
|
||||
lcond_.signal();
|
||||
return ret;
|
||||
}
|
||||
@ -179,6 +185,9 @@ public:
|
||||
ATOMIC_STORE(&write_id_, itid);
|
||||
get_tcref().sync(&read_ref_);
|
||||
int64_t ttl = 0;
|
||||
// although we know that waiting myself is meanless,
|
||||
// but it is helpful for us to understand the lock logic.
|
||||
ObLatch::current_wait = (uint32_t*)&(latch_.lock_);
|
||||
while(0 != ATOMIC_LOAD(&read_ref_)
|
||||
&& (ttl = abs_timeout_us - ObTimeUtility::current_time()) >= 0) {
|
||||
lcond_.wait(std::min(ttl, (int64_t)10 * 1000));
|
||||
@ -190,6 +199,7 @@ public:
|
||||
} else {
|
||||
ATOMIC_STORE(&write_id_, itid | WRITE_MASK);
|
||||
}
|
||||
ObLatch::current_wait = nullptr;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -250,6 +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_);
|
||||
} else {
|
||||
get_tcref().dec_ref(&read_ref_, slot_id);
|
||||
lcond_.signal();
|
||||
@ -265,12 +276,17 @@ public:
|
||||
ret = latch_.unlock();
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
// record in try_rdlock will be overwrited by latch_.rdlock, so record again.
|
||||
ObLatch::current_lock = (uint32_t*)&(latch_.lock_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
inline int rdunlock(int64_t slot_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
get_tcref().dec_ref(&read_ref_, slot_id);
|
||||
ObLatch::current_lock = nullptr;
|
||||
lcond_.signal();
|
||||
return ret;
|
||||
}
|
||||
|
23
deps/oblib/src/lib/ob_define.h
vendored
23
deps/oblib/src/lib/ob_define.h
vendored
@ -2166,7 +2166,6 @@ enum ObTraceGranularity
|
||||
#define DIO_ALIGN_SIZE 4096
|
||||
#define DIO_READ_ALIGN_SIZE 4096
|
||||
#define DIO_ALLOCATOR_CACHE_BLOCK_SIZE (OB_DEFAULT_MACRO_BLOCK_SIZE + DIO_READ_ALIGN_SIZE)
|
||||
#define CORO_INIT_PRIORITY 120
|
||||
#define MALLOC_INIT_PRIORITY 128
|
||||
#define NORMAL_INIT_PRIORITY (MALLOC_INIT_PRIORITY + 1)
|
||||
|
||||
@ -2343,22 +2342,22 @@ OB_INLINE int64_t ob_gettid()
|
||||
return tid;
|
||||
}
|
||||
|
||||
OB_INLINE uint64_t &ob_get_tenant_id()
|
||||
OB_INLINE uint64_t& ob_get_tenant_id()
|
||||
{
|
||||
RLOCAL_INLINE(uint64_t, tenant_id);
|
||||
thread_local uint64_t tenant_id = 0;;
|
||||
return tenant_id;
|
||||
}
|
||||
|
||||
OB_INLINE char *ob_get_tname()
|
||||
OB_INLINE char* ob_get_tname()
|
||||
{
|
||||
struct TNameBuf {
|
||||
TNameBuf() {
|
||||
snprintf(v_, oceanbase::OB_THREAD_NAME_BUF_LEN, "%s", "");
|
||||
}
|
||||
char v_[oceanbase::OB_THREAD_NAME_BUF_LEN];
|
||||
};
|
||||
RLOCAL_INLINE(TNameBuf, tname);
|
||||
return tname.v_;
|
||||
thread_local char tname[oceanbase::OB_THREAD_NAME_BUF_LEN] = {0};
|
||||
return tname;
|
||||
}
|
||||
|
||||
OB_INLINE const char*& ob_get_origin_thread_name()
|
||||
{
|
||||
thread_local const char* tname = nullptr;
|
||||
return tname;
|
||||
}
|
||||
|
||||
// There are many clusters in arbitration server, we need a field identify the different clusters.
|
||||
|
24
deps/oblib/src/lib/ob_lib_config.cpp
vendored
24
deps/oblib/src/lib/ob_lib_config.cpp
vendored
@ -16,27 +16,9 @@ namespace oceanbase
|
||||
namespace lib
|
||||
{
|
||||
|
||||
ObLibConfig::ObLibConfig()
|
||||
: enable_diagnose_info_(true),
|
||||
enable_trace_log_(true)
|
||||
{
|
||||
}
|
||||
|
||||
ObLibConfig &ObLibConfig::get_instance()
|
||||
{
|
||||
static ObLibConfig instance_;
|
||||
return instance_;
|
||||
}
|
||||
|
||||
void ObLibConfig::reload_diagnose_info_config(const bool enable_diagnose_info)
|
||||
{
|
||||
ATOMIC_SET(&enable_diagnose_info_, enable_diagnose_info);
|
||||
}
|
||||
|
||||
void ObLibConfig::reload_trace_log_config(const bool enable_trace_log)
|
||||
{
|
||||
ATOMIC_SET(&enable_trace_log_, enable_trace_log);
|
||||
}
|
||||
volatile bool ObLibConfig::enable_diagnose_info_ = true;
|
||||
volatile bool ObLibConfig::enable_trace_log_ = true;
|
||||
thread_local bool ObDisableDiagnoseGuard::in_disable_diagnose_guard_ = false;
|
||||
|
||||
} //lib
|
||||
} //oceanbase
|
||||
|
56
deps/oblib/src/lib/ob_lib_config.h
vendored
56
deps/oblib/src/lib/ob_lib_config.h
vendored
@ -13,56 +13,66 @@
|
||||
#ifndef OB_LIB_CONFIG_H_
|
||||
#define OB_LIB_CONFIG_H_
|
||||
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/atomic/ob_atomic.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace lib
|
||||
{
|
||||
bool is_diagnose_info_enabled();
|
||||
void reload_diagnose_info_config(const bool);
|
||||
bool is_trace_log_enabled();
|
||||
void reload_trace_log_config(const bool);
|
||||
|
||||
class ObLibConfig
|
||||
{
|
||||
friend bool is_diagnose_info_enabled();
|
||||
friend void reload_diagnose_info_config(const bool);
|
||||
friend bool is_trace_log_enabled();
|
||||
friend void reload_trace_log_config(const bool);
|
||||
private:
|
||||
static volatile bool enable_diagnose_info_ CACHE_ALIGNED;
|
||||
static volatile bool enable_trace_log_ CACHE_ALIGNED;
|
||||
};
|
||||
|
||||
class ObPerfModeGuard
|
||||
{
|
||||
friend bool is_diagnose_info_enabled();
|
||||
friend bool is_trace_log_enabled();
|
||||
public:
|
||||
static ObLibConfig &get_instance();
|
||||
void reload_diagnose_info_config(const bool enable_diagnose_info);
|
||||
void reload_trace_log_config(const bool enable_trace_log);
|
||||
bool is_diagnose_info_enabled() const
|
||||
explicit ObPerfModeGuard() : old_value_(in_disable_diagnose_guard_)
|
||||
{
|
||||
return enable_diagnose_info_;
|
||||
in_disable_diagnose_guard_ = true;
|
||||
}
|
||||
bool is_trace_log_enabled() const
|
||||
~ObPerfModeGuard()
|
||||
{
|
||||
return enable_trace_log_;
|
||||
in_disable_diagnose_guard_ = old_value_;
|
||||
}
|
||||
private:
|
||||
ObLibConfig();
|
||||
virtual ~ObLibConfig() = default;
|
||||
volatile bool enable_diagnose_info_ CACHE_ALIGNED;
|
||||
volatile bool enable_trace_log_ CACHE_ALIGNED;
|
||||
static thread_local bool in_disable_diagnose_guard_;
|
||||
bool old_value_;
|
||||
};
|
||||
|
||||
using ObDisableDiagnoseGuard = ObPerfModeGuard;
|
||||
|
||||
inline bool is_diagnose_info_enabled()
|
||||
{
|
||||
return ObLibConfig::get_instance().is_diagnose_info_enabled();
|
||||
return ObLibConfig::enable_diagnose_info_ && !ObPerfModeGuard::in_disable_diagnose_guard_;
|
||||
}
|
||||
|
||||
inline int reload_diagnose_info_config(const bool enable_diagnose_info)
|
||||
inline void reload_diagnose_info_config(const bool enable_diagnose_info)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
ObLibConfig::get_instance().reload_diagnose_info_config(enable_diagnose_info);
|
||||
return ret;
|
||||
ATOMIC_STORE(&ObLibConfig::enable_diagnose_info_, enable_diagnose_info);
|
||||
}
|
||||
|
||||
inline bool is_trace_log_enabled()
|
||||
{
|
||||
return ObLibConfig::get_instance().is_trace_log_enabled();
|
||||
return ObLibConfig::enable_trace_log_ && !ObPerfModeGuard::in_disable_diagnose_guard_;
|
||||
}
|
||||
|
||||
inline int reload_trace_log_config(const bool enable_trace_log)
|
||||
inline void reload_trace_log_config(const bool enable_trace_log)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
ObLibConfig::get_instance().reload_trace_log_config(enable_trace_log);
|
||||
return ret;
|
||||
ATOMIC_STORE(&ObLibConfig::enable_trace_log_, enable_trace_log);
|
||||
}
|
||||
|
||||
} //lib
|
||||
|
@ -369,7 +369,7 @@ int ObObjFreeList::init(const char *name, const int64_t obj_size,
|
||||
alignment_ = alignment;
|
||||
obj_count_base_ = (OP_GLOBAL == cache_type) ? 0 : obj_count;
|
||||
type_size_base_ = obj_size;
|
||||
only_global_ = (OP_RECLAIM != cache_type);
|
||||
only_global_ = (OP_GLOBAL == cache_type);
|
||||
name_ = name;
|
||||
|
||||
// Make sure we align *all* the objects in the allocation,
|
||||
@ -631,22 +631,6 @@ void *ObObjFreeList::reclaim_alloc(ObThreadCache *thread_cache)
|
||||
return ret;
|
||||
}
|
||||
|
||||
void *ObObjFreeList::tc_alloc(ObThreadCache *thread_cache)
|
||||
{
|
||||
void *ret = NULL;
|
||||
if (OB_LIKELY(NULL != thread_cache)){
|
||||
if (NULL != (ret = thread_cache->inner_free_list_.pop())) {
|
||||
thread_cache->nr_free_--;
|
||||
thread_cache->nr_malloc_++;
|
||||
} else if (only_global_) {
|
||||
if (NULL != (ret = global_alloc())) {
|
||||
thread_cache->nr_malloc_++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void *ObObjFreeList::alloc()
|
||||
{
|
||||
void *ret = NULL;
|
||||
@ -661,7 +645,9 @@ void *ObObjFreeList::alloc()
|
||||
}
|
||||
|
||||
if (only_global_) {
|
||||
ret = tc_alloc(thread_cache);
|
||||
if (OB_NOT_NULL(ret = global_alloc())) {
|
||||
thread_cache->nr_malloc_++;
|
||||
}
|
||||
} else {
|
||||
ret = reclaim_alloc(thread_cache);
|
||||
}
|
||||
@ -691,36 +677,6 @@ void ObObjFreeList::reclaim_free(ObThreadCache *cur_thread_cache, void *item)
|
||||
rcu_read_unlock(cur_thread_cache);
|
||||
}
|
||||
|
||||
void ObObjFreeList::tc_free(ObThreadCache *cur_thread_cache, void *item)
|
||||
{
|
||||
if (obj_count_base_ > 0) {
|
||||
// free all thread cache obj upto global free list if it's overflow
|
||||
if (OB_UNLIKELY(cur_thread_cache->nr_free_ >= obj_count_base_)) {
|
||||
void *next = NULL;
|
||||
obj_free_list_.push(item);
|
||||
// keep half of obj_count_base_
|
||||
int64_t low_watermark = obj_count_base_ / 2;
|
||||
while (cur_thread_cache->nr_free_ > low_watermark
|
||||
&& NULL != (next = cur_thread_cache->inner_free_list_.pop())) {
|
||||
obj_free_list_.push(next);
|
||||
cur_thread_cache->nr_free_--;
|
||||
}
|
||||
} else {
|
||||
cur_thread_cache->inner_free_list_.push(reinterpret_cast<ObFreeObject *>(item));
|
||||
cur_thread_cache->nr_free_++;
|
||||
}
|
||||
} else {
|
||||
global_free(item);
|
||||
}
|
||||
|
||||
/**
|
||||
* For global allocate mode, maybe thread A allocates memory and thread B frees it.
|
||||
* So when thread B frees, B's thread cache maybe NULL. The thread_cache->nr_malloc_
|
||||
* isn't the actual malloc number of this thread, maybe it's negative.
|
||||
*/
|
||||
cur_thread_cache->nr_malloc_--;
|
||||
}
|
||||
|
||||
void ObObjFreeList::free(void *item)
|
||||
{
|
||||
ObThreadCache *thread_cache = NULL;
|
||||
@ -734,7 +690,8 @@ void ObObjFreeList::free(void *item)
|
||||
|
||||
if (OB_LIKELY(NULL != thread_cache)) {
|
||||
if (only_global_) {
|
||||
tc_free(thread_cache, item);
|
||||
global_free(item);
|
||||
thread_cache->nr_malloc_--;
|
||||
} else {
|
||||
reclaim_free(thread_cache, item);
|
||||
}
|
||||
|
@ -43,7 +43,6 @@ extern void thread_shutdown_cleanup(void *);
|
||||
enum ObMemCacheType
|
||||
{
|
||||
OP_GLOBAL,
|
||||
OP_TC,
|
||||
OP_RECLAIM
|
||||
};
|
||||
|
||||
@ -177,8 +176,6 @@ private:
|
||||
void global_free(void *item);
|
||||
void *reclaim_alloc(ObThreadCache *thread_cache);
|
||||
void reclaim_free(ObThreadCache *cur_thread_cache, void *item);
|
||||
void *tc_alloc(ObThreadCache *thread_cache);
|
||||
void tc_free(ObThreadCache *cur_thread_cache, void *item);
|
||||
ObThreadCache *init_thread_cache();
|
||||
void privatize_thread_cache(ObThreadCache *cur_thread_cache, ObThreadCache *src_thread_cache);
|
||||
|
||||
@ -796,16 +793,14 @@ inline void call_destructor(T *ptr) {
|
||||
// 3. because object pool uses singleton, please only use one of global,
|
||||
// tc or reclaim interfaces for each object type in the whole procject.
|
||||
// Note:
|
||||
// op_alloc,op_tc_alloc and op_reclaim_alloc call the default constructor if it exist,
|
||||
// op_alloc and op_reclaim_alloc call the default constructor if it exist,
|
||||
// else it just reinterpret_cast ptr.
|
||||
//
|
||||
// op_alloc_args,op_tc_alloc_args and op_reclaim_args call the constructor with args.
|
||||
// op_alloc_args and op_reclaim_args call the constructor with args.
|
||||
// It uses placement new to construct instance, if args is null and there isn't public
|
||||
// default constructor, compiler isn't happy.
|
||||
//
|
||||
// op_alloc_args uses global object freelist, save memory but performance is poor.
|
||||
// op_tc_alloc_args uses thread local object free list, perfromance is better but
|
||||
// waste some memory.
|
||||
// op_reclaim_alloc_args uses thread local object free list and with memory reclaim,
|
||||
// performace is good and object waste less memory.
|
||||
|
||||
@ -887,61 +882,7 @@ inline void call_destructor(T *ptr) {
|
||||
} \
|
||||
})
|
||||
|
||||
// thread cache pool allocator interface
|
||||
#define op_tc_alloc_args(type, args...) \
|
||||
({ \
|
||||
type *ret = NULL; \
|
||||
common::ObClassAllocator<type> *instance = \
|
||||
common::ObClassAllocator<type>::get(common::OPNum<type>::LOCAL_NUM, common::OP_TC, \
|
||||
common::OPNum<type>::LABEL); \
|
||||
if (OB_LIKELY(NULL != instance)) { \
|
||||
void *tmp = instance->alloc_void(); \
|
||||
if (OB_LIKELY(NULL != tmp)) { \
|
||||
ret = new (tmp) type(args); \
|
||||
} \
|
||||
} \
|
||||
ret; \
|
||||
})
|
||||
|
||||
#define op_tc_alloc(type) \
|
||||
({ \
|
||||
type *ret = NULL; \
|
||||
common::ObClassAllocator<type> *instance = \
|
||||
common::ObClassAllocator<type>::get(common::OPNum<type>::LOCAL_NUM, common::OP_TC, \
|
||||
common::OPNum<type>::LABEL); \
|
||||
if (OB_LIKELY(NULL != instance)) { \
|
||||
ret = instance->alloc(); \
|
||||
} \
|
||||
ret; \
|
||||
})
|
||||
|
||||
#define op_tc_free(ptr) \
|
||||
({ \
|
||||
common::ObClassAllocator<__typeof__(*ptr)> *instance = \
|
||||
common::ObClassAllocator<__typeof__(*ptr)>::get(common::OPNum<__typeof__(*ptr)>::LOCAL_NUM, common::OP_TC, \
|
||||
common::OPNum<__typeof__(*ptr)>::LABEL); \
|
||||
if (OB_LIKELY(NULL != instance)) { \
|
||||
instance->free(ptr); \
|
||||
} \
|
||||
})
|
||||
|
||||
// thread cache pool and reclaim allocator interface
|
||||
#define op_reclaim_alloc_args(type, args...) \
|
||||
({ \
|
||||
type *ret = NULL; \
|
||||
common::ObClassAllocator<type> *instance = \
|
||||
common::ObClassAllocator<type>::get(common::OPNum<type>::LOCAL_NUM, common::OP_RECLAIM, \
|
||||
common::OPNum<type>::LABEL); \
|
||||
if (OB_LIKELY(NULL != instance)) { \
|
||||
void *tmp = instance->alloc_void(); \
|
||||
if (OB_LIKELY(NULL != tmp)) { \
|
||||
ret = new (tmp) type(args); \
|
||||
} \
|
||||
} \
|
||||
ret; \
|
||||
})
|
||||
|
||||
#define op_reclaim_alloc(type) \
|
||||
#define op_reclaim_alloc_old(type) \
|
||||
({ \
|
||||
OLD_STATIC_ASSERT((std::is_default_constructible<type>::value), "type is not default constructible"); \
|
||||
type *ret = NULL; \
|
||||
@ -954,7 +895,7 @@ inline void call_destructor(T *ptr) {
|
||||
ret; \
|
||||
})
|
||||
|
||||
#define op_reclaim_free(ptr) \
|
||||
#define op_reclaim_free_old(ptr) \
|
||||
({ \
|
||||
common::ObClassAllocator<__typeof__(*ptr)> *instance = \
|
||||
common::ObClassAllocator<__typeof__(*ptr)>::get(common::OPNum<__typeof__(*ptr)>::LOCAL_NUM, \
|
||||
@ -965,6 +906,14 @@ inline void call_destructor(T *ptr) {
|
||||
} \
|
||||
})
|
||||
|
||||
#ifndef PERF_MODE
|
||||
#define op_reclaim_alloc(type) op_alloc(type)
|
||||
#define op_reclaim_free(ptr) op_free(ptr)
|
||||
#else
|
||||
#define op_reclaim_alloc(type) op_reclaim_alloc_old(type)
|
||||
#define op_reclaim_free(ptr) op_reclaim_free_old(ptr)
|
||||
#endif
|
||||
|
||||
} // end of namespace common
|
||||
} // end of namespace oceanbase
|
||||
|
||||
|
3
deps/oblib/src/lib/objectpool/ob_pool.ipp
vendored
3
deps/oblib/src/lib/objectpool/ob_pool.ipp
vendored
@ -50,6 +50,9 @@ void ObPool<BlockAllocatorT, LockT>::reset()
|
||||
{
|
||||
BlockHeader *curr = blocklist_;
|
||||
BlockHeader *next = NULL;
|
||||
//if (in_use_count_ != 0) {
|
||||
// LIB_LOG(ERROR, "there was memory leak", K(in_use_count_), K(free_count_), K(total_count_));
|
||||
//}
|
||||
while (NULL != curr) {
|
||||
next = curr->next_;
|
||||
block_allocator_.free(curr);
|
||||
|
13
deps/oblib/src/lib/oblog/ob_base_log_writer.cpp
vendored
13
deps/oblib/src/lib/oblog/ob_base_log_writer.cpp
vendored
@ -26,6 +26,7 @@
|
||||
#include "lib/oblog/ob_log_print_kv.h"
|
||||
#include "lib/worker.h"
|
||||
#include "lib/thread/ob_thread_name.h"
|
||||
#include "lib/thread/thread.h"
|
||||
|
||||
using namespace oceanbase::lib;
|
||||
|
||||
@ -79,14 +80,7 @@ int ObBaseLogWriter::init(
|
||||
log_cfg_ = log_cfg;
|
||||
max_buffer_item_cnt_ = log_cfg.max_buffer_item_cnt_;
|
||||
memset((void*) log_items_, 0, sizeof(ObIBaseLogItem*) * max_buffer_item_cnt_);
|
||||
if (STRLEN(thread_name) > MAX_THREAD_NAME_LEN) {
|
||||
ret = OB_SIZE_OVERFLOW;
|
||||
LOG_STDERR("Size of thread_name exceeds the limit, thread_name_size=%lu.\n", sizeof(thread_name));
|
||||
} else if (tenant_id == 0) {
|
||||
MEMCPY(thread_name_, thread_name, STRLEN(thread_name));
|
||||
} else {
|
||||
snprintf(thread_name_, PR_SET_NAME, "T%lu_%s", tenant_id, thread_name);
|
||||
}
|
||||
thread_name_ = thread_name;
|
||||
if (OB_SUCC(ret)) {
|
||||
is_inited_ = true;
|
||||
LOG_STDOUT("successfully init ObBaseLogWriter\n");
|
||||
@ -254,7 +248,7 @@ void *ObBaseLogWriter::flush_log_thread(void *arg)
|
||||
} else {
|
||||
pthread_cleanup_push(cleanup_log_thread, arg);
|
||||
ObBaseLogWriter *log_writer = reinterpret_cast<ObBaseLogWriter*> (arg);
|
||||
prctl(PR_SET_NAME, log_writer->thread_name_, 0, 0, 0);
|
||||
lib::set_thread_name(log_writer->thread_name_);
|
||||
log_writer->flush_log();
|
||||
pthread_cleanup_pop(1);
|
||||
}
|
||||
@ -264,6 +258,7 @@ void *ObBaseLogWriter::flush_log_thread(void *arg)
|
||||
void ObBaseLogWriter::flush_log()
|
||||
{
|
||||
while (!has_stopped_) {
|
||||
IGNORE_RETURN lib::Thread::update_loop_ts(ObTimeUtility::fast_current_time());
|
||||
pthread_mutex_lock(&thread_mutex_);
|
||||
// 每个线程执行16次再重新抢占, 对cpu cache hit有利
|
||||
for (int64_t i = 0; i < 16; i++) {
|
||||
|
@ -116,7 +116,7 @@ protected:
|
||||
|
||||
SimpleCond* log_write_cond_;
|
||||
SimpleCond* log_flush_cond_;
|
||||
char thread_name_[PR_SET_NAME];
|
||||
const char* thread_name_;
|
||||
};
|
||||
|
||||
}
|
||||
|
59
deps/oblib/src/lib/oblog/ob_log.h
vendored
59
deps/oblib/src/lib/oblog/ob_log.h
vendored
@ -41,7 +41,6 @@
|
||||
#include "lib/oblog/ob_async_log_struct.h"
|
||||
#include "lib/utility/ob_defer.h"
|
||||
#include "lib/oblog/ob_syslog_rate_limiter.h"
|
||||
#include "common/ob_local_store.h"
|
||||
|
||||
#define OB_LOG_MAX_PAR_MOD_SIZE 32
|
||||
#define OB_LOG_MAX_SUB_MOD_SIZE 32
|
||||
@ -161,8 +160,9 @@ public:
|
||||
static inline const ObLogIdLevelMap *get();
|
||||
|
||||
static inline int8_t get_level();
|
||||
//@brief Get the thread-only ObThreadLogLevel.
|
||||
static inline ObThreadLogLevel *get_thread_log_level();
|
||||
private:
|
||||
static inline const ObLogIdLevelMap*& get_id_level_map_();
|
||||
static inline int8_t& get_level_();
|
||||
};
|
||||
|
||||
class ObThreadFlags
|
||||
@ -834,64 +834,47 @@ inline int8_t ObLogIdLevelMap::get_level(uint64_t par_mod_id, uint64_t sub_mod_i
|
||||
|
||||
inline void ObThreadLogLevelUtils::init()
|
||||
{
|
||||
ObThreadLogLevel *trace_log_level = get_thread_log_level();
|
||||
if (NULL != trace_log_level) {
|
||||
trace_log_level->id_level_map_ = NULL;
|
||||
}
|
||||
get_id_level_map_() = NULL;
|
||||
}
|
||||
|
||||
inline void ObThreadLogLevelUtils::init(const ObLogIdLevelMap *id_level_map)
|
||||
{
|
||||
ObThreadLogLevel *trace_log_level = get_thread_log_level();
|
||||
if (NULL != trace_log_level) {
|
||||
trace_log_level->id_level_map_ = id_level_map;
|
||||
trace_log_level->level_ = (id_level_map == NULL
|
||||
? (int8_t)OB_LOG_LEVEL_NONE : id_level_map->non_mod_level_);
|
||||
}
|
||||
get_id_level_map_() = id_level_map;
|
||||
get_level_() = (id_level_map == NULL
|
||||
? (int8_t)OB_LOG_LEVEL_NONE : id_level_map->non_mod_level_);
|
||||
}
|
||||
|
||||
inline void ObThreadLogLevelUtils::init(const int8_t level)
|
||||
{
|
||||
ObThreadLogLevel *trace_log_level = get_thread_log_level();
|
||||
if (NULL != trace_log_level) {
|
||||
trace_log_level->level_ = level;
|
||||
}
|
||||
get_level_() = level;
|
||||
}
|
||||
|
||||
inline void ObThreadLogLevelUtils::clear()
|
||||
{
|
||||
ObThreadLogLevel *trace_log_level = get_thread_log_level();
|
||||
if (NULL != trace_log_level) {
|
||||
trace_log_level->id_level_map_ = NULL;
|
||||
trace_log_level->level_ = OB_LOG_LEVEL_NONE;
|
||||
}
|
||||
get_id_level_map_() = NULL;
|
||||
get_level_() = OB_LOG_LEVEL_NONE;
|
||||
}
|
||||
|
||||
inline const ObLogIdLevelMap *ObThreadLogLevelUtils::get()
|
||||
{
|
||||
const ObLogIdLevelMap *ret = NULL;
|
||||
ObThreadLogLevel *trace_log_level = get_thread_log_level();
|
||||
if (NULL != trace_log_level) {
|
||||
ret = trace_log_level->id_level_map_;
|
||||
}
|
||||
return ret;
|
||||
return get_id_level_map_();
|
||||
}
|
||||
|
||||
inline int8_t ObThreadLogLevelUtils::get_level()
|
||||
{
|
||||
int8_t level = OB_LOG_LEVEL_NONE;
|
||||
ObThreadLogLevel *trace_log_level = get_thread_log_level();
|
||||
if (NULL != trace_log_level) {
|
||||
level = trace_log_level->level_;
|
||||
}
|
||||
return level;
|
||||
return get_level_();;
|
||||
}
|
||||
|
||||
inline ObThreadLogLevel *ObThreadLogLevelUtils::get_thread_log_level()
|
||||
inline const ObLogIdLevelMap*& ObThreadLogLevelUtils::get_id_level_map_()
|
||||
{
|
||||
ObThreadLogLevel *ret = nullptr;
|
||||
ret = &(common::get_local_store()->log_level_);
|
||||
return ret;
|
||||
thread_local const ObLogIdLevelMap* id_level_map = nullptr;
|
||||
return id_level_map;
|
||||
}
|
||||
|
||||
inline int8_t& ObThreadLogLevelUtils::get_level_()
|
||||
{
|
||||
thread_local int8_t level = OB_LOG_LEVEL_NONE;
|
||||
return level;
|
||||
}
|
||||
|
||||
inline void ObLogger::check_log_end(ObPLogItem &log_item, int64_t pos)
|
||||
|
20
deps/oblib/src/lib/queue/ob_priority_queue.h
vendored
20
deps/oblib/src/lib/queue/ob_priority_queue.h
vendored
@ -101,11 +101,11 @@ private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObPriorityQueue);
|
||||
};
|
||||
|
||||
template <int HIGH_HIGH_PRIOS, int HIGH_PRIOS=0, int LOW_PRIOS=0>
|
||||
template <int HIGH_PRIOS, int NORMAL_PRIOS=0, int LOW_PRIOS=0>
|
||||
class ObPriorityQueue2
|
||||
{
|
||||
public:
|
||||
enum { PRIO_CNT = HIGH_HIGH_PRIOS + HIGH_PRIOS + LOW_PRIOS };
|
||||
enum { PRIO_CNT = HIGH_PRIOS + NORMAL_PRIOS + LOW_PRIOS };
|
||||
|
||||
ObPriorityQueue2() : queue_(), size_(0), limit_(INT64_MAX) {}
|
||||
~ObPriorityQueue2() {}
|
||||
@ -137,9 +137,9 @@ public:
|
||||
} else if (OB_FAIL(queue_[priority].push(data))) {
|
||||
// do nothing
|
||||
} else {
|
||||
if (priority < HIGH_HIGH_PRIOS) {
|
||||
if (priority < HIGH_PRIOS) {
|
||||
cond_.signal(1, 0);
|
||||
} else if (priority < HIGH_PRIOS + HIGH_HIGH_PRIOS) {
|
||||
} else if (priority < NORMAL_PRIOS + HIGH_PRIOS) {
|
||||
cond_.signal(1, 1);
|
||||
} else {
|
||||
cond_.signal(1, 2);
|
||||
@ -157,14 +157,14 @@ public:
|
||||
return do_pop(data, PRIO_CNT, timeout_us);
|
||||
}
|
||||
|
||||
int pop_high(ObLink*& data, int64_t timeout_us)
|
||||
int pop_normal(ObLink*& data, int64_t timeout_us)
|
||||
{
|
||||
return do_pop(data, HIGH_HIGH_PRIOS + HIGH_PRIOS, timeout_us);
|
||||
return do_pop(data, HIGH_PRIOS + NORMAL_PRIOS, timeout_us);
|
||||
}
|
||||
|
||||
int pop_high_high(ObLink*& data, int64_t timeout_us)
|
||||
int pop_high(ObLink*& data, int64_t timeout_us)
|
||||
{
|
||||
return do_pop(data, HIGH_HIGH_PRIOS, timeout_us);
|
||||
return do_pop(data, HIGH_PRIOS, timeout_us);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -175,9 +175,9 @@ private:
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
COMMON_LOG(ERROR, "timeout is invalid", K(ret), K(timeout_us));
|
||||
} else {
|
||||
if (plimit <= HIGH_HIGH_PRIOS) {
|
||||
if (plimit <= HIGH_PRIOS) {
|
||||
cond_.prepare(0);
|
||||
} else if (plimit <= HIGH_PRIOS + HIGH_HIGH_PRIOS) {
|
||||
} else if (plimit <= NORMAL_PRIOS + HIGH_PRIOS) {
|
||||
cond_.prepare(1);
|
||||
} else {
|
||||
cond_.prepare(2);
|
||||
|
6
deps/oblib/src/lib/resource/achunk_mgr.cpp
vendored
6
deps/oblib/src/lib/resource/achunk_mgr.cpp
vendored
@ -23,7 +23,6 @@
|
||||
#include "lib/alloc/alloc_struct.h"
|
||||
#include "lib/alloc/alloc_failed_reason.h"
|
||||
#include "lib/alloc/memory_sanity.h"
|
||||
#include "lib/stat/ob_diagnose_info.h"
|
||||
|
||||
using namespace oceanbase::lib;
|
||||
|
||||
@ -68,8 +67,6 @@ void *AChunkMgr::direct_alloc(const uint64_t size, const bool can_use_huge_page,
|
||||
{
|
||||
common::ObTimeGuard time_guard(__func__, 1000 * 1000);
|
||||
int orig_errno = errno;
|
||||
EVENT_INC(MMAP_COUNT);
|
||||
EVENT_ADD(MMAP_SIZE, size);
|
||||
|
||||
void *ptr = nullptr;
|
||||
ptr = low_alloc(size, can_use_huge_page, huge_page_used, alloc_shadow);
|
||||
@ -114,8 +111,7 @@ void *AChunkMgr::direct_alloc(const uint64_t size, const bool can_use_huge_page,
|
||||
void AChunkMgr::direct_free(const void *ptr, const uint64_t size)
|
||||
{
|
||||
common::ObTimeGuard time_guard(__func__, 1000 * 1000);
|
||||
EVENT_INC(MUNMAP_COUNT);
|
||||
EVENT_ADD(MUNMAP_SIZE, size);
|
||||
|
||||
ATOMIC_FAA(&unmaps_, 1);
|
||||
if (size > INTACT_ACHUNK_SIZE) {
|
||||
ATOMIC_FAA(&large_unmaps_, 1);
|
||||
|
3
deps/oblib/src/lib/resource/achunk_mgr.h
vendored
3
deps/oblib/src/lib/resource/achunk_mgr.h
vendored
@ -22,6 +22,7 @@
|
||||
#include "lib/atomic/ob_atomic.h"
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/lock/ob_mutex.h"
|
||||
#include "lib/ob_lib_config.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -70,6 +71,7 @@ public:
|
||||
{
|
||||
bool bret = false;
|
||||
if (count() < max_chunk_cache_cnt_) {
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
if (with_mutex_) {
|
||||
mutex_.lock();
|
||||
}
|
||||
@ -97,6 +99,7 @@ public:
|
||||
{
|
||||
AChunk *chunk = NULL;
|
||||
if (!OB_ISNULL(header_)) {
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
if (with_mutex_) {
|
||||
mutex_.lock();
|
||||
}
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <new>
|
||||
#include <stdlib.h>
|
||||
|
||||
#include "lib/alloc/memory_sanity.h"
|
||||
#include "lib/oblog/ob_log.h"
|
||||
#include "lib/stat/ob_diagnose_info.h"
|
||||
#include "lib/utility/utility.h"
|
||||
@ -427,6 +428,7 @@ ObResourceMgr::ObResourceMgr()
|
||||
: inited_(false), cache_washer_(NULL), locks_(), tenant_resource_mgrs_()
|
||||
{
|
||||
for (int64_t i = 0; i < MAX_TENANT_COUNT; ++i) {
|
||||
locks_[i].enable_record_stat(false);
|
||||
locks_[i].set_latch_id(common::ObLatchIds::TENANT_RES_MGR_LIST_LOCK);
|
||||
}
|
||||
}
|
||||
@ -461,6 +463,7 @@ ObResourceMgr &ObResourceMgr::get_instance()
|
||||
static ObResourceMgr resource_mgr;
|
||||
if (!resource_mgr.inited_) {
|
||||
// use first lock to avoid concurrent init of resource mgr
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
SpinWLockGuard guard(resource_mgr.locks_[0]);
|
||||
if (!resource_mgr.inited_) {
|
||||
int ret = OB_SUCCESS;
|
||||
@ -481,6 +484,7 @@ int ObResourceMgr::set_cache_washer(ObICacheWasher &cache_washer)
|
||||
} else {
|
||||
cache_washer_ = &cache_washer;
|
||||
for (int64_t pos = 0; pos < MAX_TENANT_COUNT; ++pos) {
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
SpinWLockGuard guard(locks_[pos]);
|
||||
ObTenantResourceMgr *tenant_resource_mgr = tenant_resource_mgrs_[pos];
|
||||
while (NULL != tenant_resource_mgr) {
|
||||
@ -505,6 +509,7 @@ int ObResourceMgr::get_tenant_resource_mgr(const uint64_t tenant_id,
|
||||
LOG_WARN("invalid argument", K(ret), K(tenant_id));
|
||||
} else {
|
||||
const int64_t pos = tenant_id % MAX_TENANT_COUNT;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
ObTenantResourceMgr *tenant_resource_mgr = NULL;
|
||||
{
|
||||
SpinRLockGuard guard(locks_[pos]);
|
||||
@ -555,6 +560,7 @@ void ObResourceMgr::dec_ref(ObTenantResourceMgr *tenant_resource_mgr)
|
||||
int64_t ref_cnt = 0;
|
||||
if (0 == (ref_cnt = ATOMIC_SAF(&tenant_resource_mgr->ref_cnt_, 1))) {
|
||||
const int64_t pos = tenant_resource_mgr->tenant_id_ % MAX_TENANT_COUNT;
|
||||
ObDisableDiagnoseGuard disable_diagnose_guard;
|
||||
SpinWLockGuard guard(locks_[pos]);
|
||||
if (0 == ATOMIC_LOAD(&tenant_resource_mgr->ref_cnt_)) {
|
||||
int ret = OB_SUCCESS;
|
||||
|
4
deps/oblib/src/lib/stat/ob_di_cache.cpp
vendored
4
deps/oblib/src/lib/stat/ob_di_cache.cpp
vendored
@ -76,7 +76,7 @@ ObDISessionCache &ObDISessionCache::get_instance()
|
||||
int ObDISessionCache::get_node(uint64_t session_id, ObDISessionCollect *&session_collect)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObRandom *random = ObDITls<ObRandom>::get_instance();
|
||||
ObRandom *random = GET_TSI(ObRandom);
|
||||
ObSessionBucket &bucket = di_map_[session_id % OB_MAX_SERVER_SESSION_CNT];
|
||||
while (1) {
|
||||
bucket.lock_.rdlock();
|
||||
@ -227,7 +227,7 @@ int ObDIThreadTenantCache::get_node(uint64_t tenant_id, ObDITenantCollect *&tena
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(tenant_collect = tenant_cache_.get_node(tenant_id, tenant_collect))) {
|
||||
if (nullptr == extend_tenant_cache_) {
|
||||
extend_tenant_cache_ = ObDITls<ObDIBaseTenantCache<MAX_TENANT_NUM_PER_SERVER>>::get_instance();
|
||||
extend_tenant_cache_ = GET_TSI(ObDIBaseTenantCache<MAX_TENANT_NUM_PER_SERVER>);
|
||||
}
|
||||
if (nullptr != extend_tenant_cache_) {
|
||||
tenant_collect = extend_tenant_cache_->get_node(tenant_id, tenant_collect);
|
||||
|
202
deps/oblib/src/lib/stat/ob_di_tls.h
vendored
202
deps/oblib/src/lib/stat/ob_di_tls.h
vendored
@ -14,102 +14,154 @@
|
||||
#define OB_DI_TLS_H_
|
||||
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/allocator/ob_malloc.h"
|
||||
|
||||
#include <cxxabi.h>
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace common
|
||||
{
|
||||
|
||||
template <class T>
|
||||
template <class T, size_t tag>
|
||||
class ObDITls
|
||||
{
|
||||
// avoid reconstruct during construction.
|
||||
static constexpr uint64_t PLACE_HOLDER = 0x1;
|
||||
static constexpr uint64_t MAX_TNAME_LENGTH = 128;
|
||||
public:
|
||||
static ObDITls &get_di_tls();
|
||||
void destroy();
|
||||
T *new_instance();
|
||||
static T *get_instance();
|
||||
static T* get_instance();
|
||||
OB_INLINE bool is_valid() { return OB_NOT_NULL(instance_) && PLACE_HOLDER != (uint64_t)instance_; }
|
||||
private:
|
||||
ObDITls() : key_(INT32_MAX)
|
||||
{
|
||||
if (0 != pthread_key_create(&key_, destroy_thread_data_)) {
|
||||
}
|
||||
}
|
||||
~ObDITls() { destroy(); }
|
||||
static void destroy_thread_data_(void *ptr);
|
||||
ObDITls() : instance_(nullptr) {}
|
||||
~ObDITls();
|
||||
static const char* get_label();
|
||||
private:
|
||||
pthread_key_t key_;
|
||||
static TLOCAL(T *, instance_);
|
||||
static TLOCAL(bool, disable_);
|
||||
T* instance_;
|
||||
};
|
||||
// NOTE: thread local diagnose information
|
||||
// TODO: check if multi-query execute within one thread.
|
||||
template <class T>
|
||||
TLOCAL(T *, ObDITls<T>::instance_);
|
||||
template <class T>
|
||||
TLOCAL(bool, ObDITls<T>::disable_);
|
||||
|
||||
template <class T>
|
||||
void ObDITls<T>::destroy_thread_data_(void *ptr)
|
||||
{
|
||||
if (NULL != ptr) {
|
||||
T *tls = (T *)ptr;
|
||||
instance_ = NULL;
|
||||
disable_ = true;
|
||||
delete tls;
|
||||
}
|
||||
}
|
||||
|
||||
template <class T>
|
||||
ObDITls<T> &ObDITls<T>::get_di_tls()
|
||||
{
|
||||
static ObDITls<T> di_tls;
|
||||
return di_tls;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
void ObDITls<T>::destroy()
|
||||
{
|
||||
if (INT32_MAX != key_) {
|
||||
void *ptr = pthread_getspecific(key_);
|
||||
destroy_thread_data_(ptr);
|
||||
if (0 != pthread_key_delete(key_)) {
|
||||
template <class T, size_t tag>
|
||||
const char* ObDITls<T, tag>::get_label() {
|
||||
const char* cxxname = typeid(T).name();
|
||||
const char* ret = "DITls";
|
||||
static char buf[MAX_TNAME_LENGTH];
|
||||
if (nullptr == cxxname || strlen(cxxname) > MAX_TNAME_LENGTH - 5) {
|
||||
// do nothing, avoid realloc in __cxa_demangle
|
||||
} else {
|
||||
int status = 0;
|
||||
int length = MAX_TNAME_LENGTH - 3;
|
||||
ret = abi::__cxa_demangle(cxxname, buf + 3, (size_t*)&length, &status);
|
||||
if (0 != status) {
|
||||
ret = "DITls";
|
||||
} else {
|
||||
key_ = INT32_MAX;
|
||||
// remove namespace
|
||||
length = MAX_TNAME_LENGTH - 1;
|
||||
while (length >= 3 && buf[length] != ':') {
|
||||
--length;
|
||||
}
|
||||
length -= 2;
|
||||
buf[length] = '[';
|
||||
buf[length + 1] = 'T';
|
||||
buf[length + 2] = ']';
|
||||
ret = buf + length;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
T *ObDITls<T>::new_instance()
|
||||
template <class T, size_t tag>
|
||||
ObDITls<T, tag>::~ObDITls()
|
||||
{
|
||||
T *instance = NULL;
|
||||
if (INT32_MAX != key_) {
|
||||
T *tls = (T *)pthread_getspecific(key_);
|
||||
if (NULL == tls) {
|
||||
tls = new (std::nothrow) T();
|
||||
if (NULL != tls && 0 != pthread_setspecific(key_, tls)) {
|
||||
delete tls;
|
||||
tls = NULL;
|
||||
if (is_valid()) {
|
||||
ob_delete(instance_);
|
||||
instance_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
template <class T, size_t tag>
|
||||
T* ObDITls<T, tag>::get_instance()
|
||||
{
|
||||
static thread_local ObDITls<T, tag> di_tls;
|
||||
if (!di_tls.is_valid()) {
|
||||
static const char* label = get_label();
|
||||
di_tls.instance_ = (T*)PLACE_HOLDER;
|
||||
// add tenant
|
||||
di_tls.instance_ = OB_NEW(T, label);
|
||||
}
|
||||
return di_tls.instance_;
|
||||
}
|
||||
|
||||
template <class T, int N, size_t tag>
|
||||
class ObDITls<T[N], tag>
|
||||
{
|
||||
// avoid reconstruct during construction.
|
||||
static constexpr uint64_t PLACE_HOLDER = 0x1;
|
||||
static constexpr uint64_t MAX_TNAME_LENGTH = 128;
|
||||
public:
|
||||
static T* get_instance();
|
||||
OB_INLINE bool is_valid() { return OB_NOT_NULL(instance_) && PLACE_HOLDER != (uint64_t)instance_; }
|
||||
private:
|
||||
ObDITls() : instance_(nullptr) {}
|
||||
~ObDITls();
|
||||
static const char* get_label();
|
||||
private:
|
||||
T* instance_;
|
||||
};
|
||||
|
||||
template <class T, int N, size_t tag>
|
||||
const char* ObDITls<T[N], tag>::get_label() {
|
||||
const char* cxxname = typeid(T).name();
|
||||
const char* ret = "DITls";
|
||||
static char buf[MAX_TNAME_LENGTH];
|
||||
if (nullptr == cxxname || strlen(cxxname) > MAX_TNAME_LENGTH - 5) {
|
||||
// do nothing, avoid realloc in __cxa_demangle
|
||||
} else {
|
||||
int status = 0;
|
||||
int length = MAX_TNAME_LENGTH - 3;
|
||||
ret = abi::__cxa_demangle(cxxname, buf + 3, (size_t*)&length, &status);
|
||||
if (0 != status) {
|
||||
ret = "DITls";
|
||||
} else {
|
||||
// remove namespace
|
||||
length = MAX_TNAME_LENGTH - 1;
|
||||
while (length >= 3 && buf[length] != ':') {
|
||||
--length;
|
||||
}
|
||||
length -= 2;
|
||||
buf[length] = '[';
|
||||
buf[length + 1] = 'T';
|
||||
buf[length + 2] = ']';
|
||||
ret = buf + length;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <class T, int N, size_t tag>
|
||||
ObDITls<T[N], tag>::~ObDITls()
|
||||
{
|
||||
if (is_valid()) {
|
||||
for (auto i = 0; i < N; ++i) {
|
||||
instance_[i].~T();
|
||||
}
|
||||
ob_free(instance_);
|
||||
}
|
||||
}
|
||||
|
||||
template <class T, int N, size_t tag>
|
||||
T* ObDITls<T[N], tag>::get_instance()
|
||||
{
|
||||
static thread_local ObDITls<T[N], tag> di_tls;
|
||||
if (!di_tls.is_valid()) {
|
||||
static const char* label = get_label();
|
||||
di_tls.instance_ = (T*)PLACE_HOLDER;
|
||||
// add tenant
|
||||
if (OB_NOT_NULL(di_tls.instance_ = (T*)ob_malloc(sizeof(T) * N, label))) {
|
||||
for (auto i = 0; i < N; ++i) {
|
||||
new (di_tls.instance_ + i) T;
|
||||
}
|
||||
}
|
||||
if (NULL != tls) {
|
||||
instance = tls;
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
template <class T>
|
||||
T *ObDITls<T>::get_instance()
|
||||
{
|
||||
if (OB_UNLIKELY(NULL == instance_)) {
|
||||
if (OB_LIKELY(!disable_)) {
|
||||
disable_ = true;
|
||||
instance_ = get_di_tls().new_instance();
|
||||
disable_ = false;
|
||||
}
|
||||
}
|
||||
return instance_;
|
||||
return di_tls.instance_;
|
||||
}
|
||||
|
||||
}
|
||||
|
4
deps/oblib/src/lib/stat/ob_diagnose_info.cpp
vendored
4
deps/oblib/src/lib/stat/ob_diagnose_info.cpp
vendored
@ -555,7 +555,7 @@ ObDiagnoseSessionInfo *ObDiagnoseSessionInfo::get_local_diagnose_info()
|
||||
if (lib::is_diagnose_info_enabled()) {
|
||||
ObDISessionCollect *collect = NULL;
|
||||
ObSessionDIBuffer *buffer = NULL;
|
||||
buffer = ObDITls<ObSessionDIBuffer>::get_instance();
|
||||
buffer = GET_TSI(ObSessionDIBuffer);
|
||||
if (NULL != buffer) {
|
||||
collect = buffer->get_curr_session();
|
||||
if (NULL != collect) {
|
||||
@ -656,7 +656,7 @@ ObDiagnoseTenantInfo *ObDiagnoseTenantInfo::get_local_diagnose_info()
|
||||
if (lib::is_diagnose_info_enabled()) {
|
||||
ObDITenantCollect *collect = NULL;
|
||||
ObSessionDIBuffer *buffer = NULL;
|
||||
buffer = ObDITls<ObSessionDIBuffer>::get_instance();
|
||||
buffer = GET_TSI(ObSessionDIBuffer);
|
||||
if (NULL != buffer) {
|
||||
if (NULL == (collect = buffer->get_curr_tenant())) {
|
||||
if (OB_FAIL(buffer->switch_tenant(OB_SYS_TENANT_ID))) {
|
||||
|
6
deps/oblib/src/lib/stat/ob_session_stat.cpp
vendored
6
deps/oblib/src/lib/stat/ob_session_stat.cpp
vendored
@ -40,7 +40,7 @@ ObSessionStatEstGuard::ObSessionStatEstGuard(const uint64_t tenant_id, const uin
|
||||
prev_session_id_(0)
|
||||
{
|
||||
if (oceanbase::lib::is_diagnose_info_enabled()) {
|
||||
buffer_ = ObDITls<ObSessionDIBuffer>::get_instance();
|
||||
buffer_ = GET_TSI(ObSessionDIBuffer);
|
||||
if (NULL != buffer_) {
|
||||
prev_tenant_id_ = buffer_->get_tenant_id();
|
||||
if (NULL != (buffer_->get_curr_session())) {
|
||||
@ -67,9 +67,5 @@ ObSessionStatEstGuard::~ObSessionStatEstGuard()
|
||||
}
|
||||
}
|
||||
|
||||
void __attribute__((constructor(101))) init_SessionDIBuffer()
|
||||
{
|
||||
oceanbase::common::ObDITls<ObSessionDIBuffer>::get_instance();
|
||||
}
|
||||
} /* namespace common */
|
||||
} /* namespace oceanbase */
|
||||
|
4
deps/oblib/src/lib/stat/ob_session_stat.h
vendored
4
deps/oblib/src/lib/stat/ob_session_stat.h
vendored
@ -14,7 +14,7 @@
|
||||
#define OB_SESSION_STAT_H_
|
||||
|
||||
#include "lib/stat/ob_di_cache.h"
|
||||
#include "lib/stat/ob_di_tls.h"
|
||||
#include "lib/thread_local/ob_tsi_factory.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -77,7 +77,7 @@ public:
|
||||
: prev_tenant_id_(OB_SYS_TENANT_ID)
|
||||
{
|
||||
if (oceanbase::lib::is_diagnose_info_enabled()) {
|
||||
buffer_ = ObDITls<ObSessionDIBuffer>::get_instance();
|
||||
buffer_ = GET_TSI(ObSessionDIBuffer);
|
||||
if (NULL != buffer_) {
|
||||
prev_tenant_id_ = buffer_->get_tenant_id();
|
||||
if (0 < tenant_id) {
|
||||
|
@ -197,10 +197,10 @@ STAT_EVENT_ADD_DEF(LOCATION_CACHE_NONBLOCK_MISS, "location nonblock get miss", O
|
||||
STAT_EVENT_ADD_DEF(LOCATION_CACHE_RPC_CHECK, "location cache rpc renew count", ObStatClassIds::CACHE, "location cache rpc renew count", 50021, true, true)
|
||||
STAT_EVENT_ADD_DEF(LOCATION_CACHE_RENEW, "location cache renew", ObStatClassIds::CACHE, "location cache renew", 50022, true, true)
|
||||
STAT_EVENT_ADD_DEF(LOCATION_CACHE_RENEW_IGNORED, "location cache renew ignored", ObStatClassIds::CACHE, "location cache renew ignored", 50023, true, true)
|
||||
STAT_EVENT_ADD_DEF(MMAP_COUNT, "mmap count", ObStatClassIds::CACHE, "mmap count", 50024, true, true)
|
||||
STAT_EVENT_ADD_DEF(MUNMAP_COUNT, "munmap count", ObStatClassIds::CACHE, "munmap count", 50025, true, true)
|
||||
STAT_EVENT_ADD_DEF(MMAP_SIZE, "mmap size", ObStatClassIds::CACHE, "mmap size", 50026, true, true)
|
||||
STAT_EVENT_ADD_DEF(MUNMAP_SIZE, "munmap size", ObStatClassIds::CACHE, "munmap size", 50027, true, true)
|
||||
//STAT_EVENT_ADD_DEF(MMAP_COUNT, "mmap count", ObStatClassIds::CACHE, "mmap count", 50024, true, true)
|
||||
//STAT_EVENT_ADD_DEF(MUNMAP_COUNT, "munmap count", ObStatClassIds::CACHE, "munmap count", 50025, true, true)
|
||||
//STAT_EVENT_ADD_DEF(MMAP_SIZE, "mmap size", ObStatClassIds::CACHE, "mmap size", 50026, true, true)
|
||||
//STAT_EVENT_ADD_DEF(MUNMAP_SIZE, "munmap size", ObStatClassIds::CACHE, "munmap size", 50027, true, true)
|
||||
STAT_EVENT_ADD_DEF(KVCACHE_SYNC_WASH_TIME, "kvcache sync wash time", ObStatClassIds::CACHE, "kvcache sync wash time", 50028, true, true)
|
||||
STAT_EVENT_ADD_DEF(KVCACHE_SYNC_WASH_COUNT, "kvcache sync wash count", ObStatClassIds::CACHE, "kvcache sync wash count", 50029, true, true)
|
||||
STAT_EVENT_ADD_DEF(LOCATION_CACHE_RPC_RENEW_FAIL, "location cache rpc renew fail count", ObStatClassIds::CACHE, "location cache rpc renew fail count", 50030, true, true)
|
||||
|
1
deps/oblib/src/lib/task/ob_timer.cpp
vendored
1
deps/oblib/src/lib/task/ob_timer.cpp
vendored
@ -294,6 +294,7 @@ void ObTimer::run1()
|
||||
set_thread_name("ObTimer");
|
||||
}
|
||||
while (true) {
|
||||
IGNORE_RETURN lib::Thread::update_loop_ts();
|
||||
{
|
||||
ObMonitor<Mutex>::Lock guard(monitor_);
|
||||
static const int64_t STATISTICS_INTERVAL_US = 600L * 1000 * 1000; // 10m
|
||||
|
@ -112,6 +112,7 @@ void ObAsyncTaskQueue::run2()
|
||||
} else {
|
||||
ObAddr zero_addr;
|
||||
while (!stop_) {
|
||||
IGNORE_RETURN lib::Thread::update_loop_ts(ObTimeUtility::fast_current_time());
|
||||
if (REACH_TIME_INTERVAL(600 * 1000 * 1000)) {
|
||||
//每隔一段时间,打印队列的大小
|
||||
LOG_INFO("[ASYNC TASK QUEUE]", "queue_size", queue_.size());
|
||||
|
@ -40,7 +40,11 @@ public:
|
||||
void logical_stop();
|
||||
void logical_wait();
|
||||
void wait() override; // wait running task stoped
|
||||
bool has_set_stop() const override { return ATOMIC_LOAD(&stop_); };
|
||||
bool has_set_stop() const override
|
||||
{
|
||||
IGNORE_RETURN lib::Thread::update_loop_ts();
|
||||
return ATOMIC_LOAD(&stop_);
|
||||
}
|
||||
|
||||
// destroy thread
|
||||
int destroy();
|
||||
|
2
deps/oblib/src/lib/thread/ob_thread_name.h
vendored
2
deps/oblib/src/lib/thread/ob_thread_name.h
vendored
@ -32,6 +32,7 @@ inline void set_thread_name(const char* type, uint64_t idx)
|
||||
{
|
||||
char *name = ob_get_tname();
|
||||
uint64_t tenant_id = ob_get_tenant_id();
|
||||
ob_get_origin_thread_name() = type;
|
||||
if (tenant_id == 0) {
|
||||
snprintf(name, OB_THREAD_NAME_BUF_LEN, "%s%ld", type, idx);
|
||||
} else {
|
||||
@ -44,6 +45,7 @@ inline void set_thread_name(const char* type)
|
||||
{
|
||||
char *name = ob_get_tname();
|
||||
uint64_t tenant_id = ob_get_tenant_id();
|
||||
ob_get_origin_thread_name() = type;
|
||||
if (tenant_id == 0) {
|
||||
snprintf(name, OB_THREAD_NAME_BUF_LEN, "%s", type);
|
||||
} else {
|
||||
|
@ -43,8 +43,8 @@ public:
|
||||
void dealloc(void *ptr);
|
||||
static ssize_t adjust_size(const ssize_t size);
|
||||
static ObStackHeader *stack_header(void *ptr);
|
||||
private:
|
||||
static ssize_t page_size();
|
||||
private:
|
||||
void *__alloc(const uint64_t tenant_id, const ssize_t size);
|
||||
};
|
||||
|
||||
|
3
deps/oblib/src/lib/thread/thread.cpp
vendored
3
deps/oblib/src/lib/thread/thread.cpp
vendored
@ -30,7 +30,8 @@ using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::lib;
|
||||
|
||||
TLOCAL(Thread *, Thread::current_thread_) = nullptr;
|
||||
thread_local int64_t Thread::loop_ts_ = 0;
|
||||
thread_local Thread* Thread::current_thread_ = nullptr;
|
||||
int64_t Thread::total_thread_count_ = 0;
|
||||
|
||||
Thread &Thread::current()
|
||||
|
18
deps/oblib/src/lib/thread/thread.h
vendored
18
deps/oblib/src/lib/thread/thread.h
vendored
@ -14,6 +14,7 @@
|
||||
#define CORO_THREAD_H
|
||||
|
||||
#include <functional>
|
||||
#include "lib/time/ob_time_utility.h"
|
||||
#include "lib/utility/ob_macro_utils.h"
|
||||
|
||||
namespace oceanbase {
|
||||
@ -48,10 +49,24 @@ public:
|
||||
|
||||
bool has_set_stop() const;
|
||||
|
||||
OB_INLINE static int64_t update_loop_ts(int64_t t)
|
||||
{
|
||||
int64_t ret = loop_ts_;
|
||||
loop_ts_ = t;
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_INLINE static int64_t update_loop_ts()
|
||||
{
|
||||
return update_loop_ts(common::ObTimeUtility::fast_current_time());
|
||||
}
|
||||
public:
|
||||
static thread_local int64_t loop_ts_;
|
||||
|
||||
private:
|
||||
static void* __th_start(void *th);
|
||||
void destroy_stack();
|
||||
static TLOCAL(Thread *, current_thread_);
|
||||
static thread_local Thread* current_thread_;
|
||||
|
||||
private:
|
||||
static int64_t total_thread_count_;
|
||||
@ -80,6 +95,7 @@ OB_INLINE pid_t Thread::get_tid() const
|
||||
|
||||
OB_INLINE bool Thread::has_set_stop() const
|
||||
{
|
||||
IGNORE_RETURN update_loop_ts();
|
||||
return stop_;
|
||||
}
|
||||
|
||||
|
@ -26,6 +26,7 @@ public:
|
||||
virtual void run1() = 0;
|
||||
bool has_set_stop() const
|
||||
{
|
||||
IGNORE_RETURN lib::Thread::update_loop_ts();
|
||||
return ATOMIC_LOAD(&stop_);
|
||||
}
|
||||
void set_stop(bool stop)
|
||||
|
5
deps/oblib/src/lib/thread/threads.cpp
vendored
5
deps/oblib/src/lib/thread/threads.cpp
vendored
@ -278,8 +278,3 @@ void Threads::destroy()
|
||||
threads_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
void Threads::set_thread_max_tasks(uint64_t cnt)
|
||||
{
|
||||
thread_max_tasks_ = cnt;
|
||||
}
|
||||
|
16
deps/oblib/src/lib/thread/threads.h
vendored
16
deps/oblib/src/lib/thread/threads.h
vendored
@ -58,7 +58,6 @@ public:
|
||||
threads_(nullptr),
|
||||
stack_size_(global_thread_stack_size),
|
||||
stop_(true),
|
||||
thread_max_tasks_(INT64_MAX),
|
||||
run_wrapper_(nullptr),
|
||||
cgroup_(INVALID_CGROUP)
|
||||
{}
|
||||
@ -103,8 +102,6 @@ public:
|
||||
return threads_[0]->get_tid();
|
||||
}
|
||||
public:
|
||||
void set_thread_max_tasks(uint64_t cnt);
|
||||
|
||||
template <class Functor>
|
||||
int submit(const Functor &func)
|
||||
{
|
||||
@ -113,9 +110,17 @@ public:
|
||||
return ret;
|
||||
}
|
||||
ThreadCGroup get_cgroup() { return cgroup_; }
|
||||
virtual bool has_set_stop() const
|
||||
{
|
||||
IGNORE_RETURN lib::Thread::update_loop_ts();
|
||||
return ATOMIC_LOAD(&stop_);
|
||||
}
|
||||
bool &has_set_stop()
|
||||
{
|
||||
IGNORE_RETURN lib::Thread::update_loop_ts();
|
||||
return stop_;
|
||||
}
|
||||
protected:
|
||||
virtual bool has_set_stop() const { return ATOMIC_LOAD(&stop_); }
|
||||
bool &has_set_stop() { return stop_; }
|
||||
int64_t get_thread_count() const { return n_threads_; }
|
||||
uint64_t get_thread_idx() const { return thread_idx_; }
|
||||
void set_thread_idx(int64_t idx) { thread_idx_ = idx; }
|
||||
@ -140,7 +145,6 @@ private:
|
||||
bool stop_;
|
||||
// protect for thread count changing.
|
||||
common::SpinRWLock lock_;
|
||||
int64_t thread_max_tasks_;
|
||||
// tenant ctx
|
||||
IRunWrapper *run_wrapper_;
|
||||
// thread cgroups
|
||||
|
452
deps/oblib/src/lib/thread_local/ob_tsi_factory.h
vendored
452
deps/oblib/src/lib/thread_local/ob_tsi_factory.h
vendored
@ -12,457 +12,17 @@
|
||||
|
||||
#ifndef OCEANBASE_COMMON_OB_TSI_FACTORY_
|
||||
#define OCEANBASE_COMMON_OB_TSI_FACTORY_
|
||||
#include <stdlib.h>
|
||||
#include <stdint.h>
|
||||
#include <stdio.h>
|
||||
#include <pthread.h>
|
||||
#include <new>
|
||||
#include <typeinfo>
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "lib/allocator/ob_malloc.h"
|
||||
#include "lib/allocator/ob_mod_define.h"
|
||||
|
||||
#include "lib/stat/ob_di_tls.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace common
|
||||
{
|
||||
enum TSICommonType
|
||||
{
|
||||
TSI_COMMON_OBJPOOL_1 = 1001,
|
||||
TSI_COMMON_SCAN_PARAM_1,
|
||||
TSI_COMMON_SCANNER_1,
|
||||
TSI_COMMON_MUTATOR_1,
|
||||
TSI_COMMON_THE_META_1,
|
||||
TSI_COMMON_GET_PARAM_1,
|
||||
TSI_COMMON_MULTI_WAKEUP_1,
|
||||
TSI_COMMON_PACKET_TRACE_ID_1,
|
||||
TSI_COMMON_SEQ_ID_1,
|
||||
TSI_COMMON_OBSERVER_1,
|
||||
TSI_COMMON_TO_CSTRING_BUFFER_OBJ_1,
|
||||
TSI_COMMON_TO_CSTRING_BUFFER_OBJ_2,
|
||||
TSI_COMMON_TO_CSTRING_BUFFER_OBJ_3,
|
||||
TSI_COMMON_DEBUG_SYNC_ARRAY,
|
||||
};
|
||||
|
||||
enum TSISSTableType
|
||||
{
|
||||
TSI_SSTABLE_FILE_BUFFER_1 = 2001,
|
||||
TSI_SSTABLE_THREAD_AIO_BUFFER_MGR_ARRAY_1,
|
||||
TSI_SSTABLE_MODULE_ARENA_1,
|
||||
TSI_SSTABLE_COMPRESS_BUFFER_MODULE_ARENA_1,
|
||||
TSI_SSTABLE_COLUMN_CHECKSUM_MODULE_ARENA_1
|
||||
};
|
||||
|
||||
enum TSIBlockSSTableType
|
||||
{
|
||||
TSI_BLOCKSSTABLE_FILE_BUFFER_1 = 11001,
|
||||
TSI_BLOCKSSTABLE_FILE_BUFFER_2 = 11002,
|
||||
TSI_BLOCKSSTABLE_ROW_CHECKSUM_GENERATOR_1,
|
||||
TSI_BLOCKSSTABLE_BLOCK_INDEX_MGR,
|
||||
TSI_BLOCKSSTABLE_BLOCK_INDEX_TRANSFORMER,
|
||||
TSI_BLOCKSSTABLE_TABLET_ARRAY_1,
|
||||
TSI_BLOCKSSTABLE_GET_READER_ARRAY_1,
|
||||
TSI_BLOCKSSTABLE_SCAN_READER_ARRAY_1,
|
||||
TSI_BLOCKSSTABLE_CHUNK_LOG_WRITER_1,
|
||||
TSI_BLOCKSSTABLE_MACRO_BLOCK_META_LOG_1,
|
||||
TSI_BLOCKSSTABLE_SSTABLE_SCHEMA_1,
|
||||
TSI_BLOCKSSTABLE_FETCH_DATA_1,
|
||||
TSI_BLOCKSSTABLE_SEND_DATA_1,
|
||||
};
|
||||
|
||||
enum TSIChunkserverType
|
||||
{
|
||||
TSI_CS_SCANNER_1 = 3001,
|
||||
TSI_CS_NEW_SCANNER_1,
|
||||
TSI_CS_NEW_SCANNER_2,
|
||||
TSI_CS_GET_PARAM_1,
|
||||
TSI_CS_SCAN_PARAM_1,
|
||||
TSI_CS_SQL_SCAN_PARAM_1,
|
||||
TSI_CS_SQL_GET_PARAM_1,
|
||||
TSI_CS_TABLET_REPORT_INFO_LIST_1,
|
||||
TSI_CS_TABLET_REPORT_INFO_LIST_2,
|
||||
TSI_CS_SSTABLE_GETTER_1,
|
||||
TSI_CS_GET_THREAD_CONTEXT_1,
|
||||
TSI_CS_SSTABLE_SCANNER_1,
|
||||
TSI_CS_SCHEMA_DECODER_ASSIS_1,
|
||||
TSI_CS_THEEAD_META_WRITER_1,
|
||||
TSI_CS_COMPACTSSTABLE_ITERATOR_1,
|
||||
TSI_CS_COMPACTSSTABLE_GET_SCANEER_1,
|
||||
TSI_CS_COLUMNFILTER_1,
|
||||
TSI_CS_QUERY_SERVICE_1,
|
||||
TSI_CS_TABLET_SERVICE_1,
|
||||
TSI_CS_STATIC_DATA_SERVICE_1,
|
||||
TSI_CS_MULTI_TABLET_MERGER_1,
|
||||
TSI_CS_TABLE_IMPORT_INFO_1,
|
||||
TSI_CS_FETCH_DATA_1,
|
||||
TSI_CS_FETCH_DATA_2,
|
||||
TSI_CS_SSTABLE_SCAN_PARAM_1,
|
||||
TSI_CS_MEDIATING_ALLOCATOR_1,
|
||||
TSI_CS_TABLE_LOCAL_INDEX_BUILDER_1,
|
||||
TSI_CS_TABLE_LOCAL_INDEX_SAMPLER_1,
|
||||
TSI_CS_TABLE_GLOBAL_INDEX_BUILDER_1,
|
||||
TSI_CS_MIGRATE_SCAN_1,
|
||||
TSI_CS_MIGRATE_SCAN_2,
|
||||
TSI_CS_TABLET_DATA_CORRECTION_REPORT_INFO_LIST,
|
||||
};
|
||||
|
||||
enum TSIUpdateserverType
|
||||
{
|
||||
TSI_UPS_SCANNER_1 = 4001,
|
||||
TSI_UPS_NEW_SCANNER_1,
|
||||
TSI_UPS_NEW_SCANNER_2,
|
||||
TSI_UPS_GET_PARAM_1,
|
||||
TSI_UPS_SCAN_PARAM_1,
|
||||
TSI_UPS_INC_SCAN_1,
|
||||
TSI_UPS_INC_GET_1,
|
||||
TSI_UPS_MUTATOR_1,
|
||||
TSI_UPS_SCANNER_ARRAY_1,
|
||||
TSI_UPS_UPS_MUTATOR_1,
|
||||
TSI_UPS_TABLE_UTILS_SET_1,
|
||||
TSI_UPS_COLUMN_FILTER_1,
|
||||
TSI_UPS_COLUMN_MAP_1,
|
||||
TSI_UPS_TABLE_LIST_1,
|
||||
TSI_UPS_ROW_COMPACTION_1,
|
||||
TSI_UPS_ROW_COMPACTION_2,
|
||||
TSI_UPS_CLIENT_WRAPPER_TSI_1,
|
||||
TSI_UPS_FIXED_SIZE_BUFFER_1,
|
||||
TSI_UPS_FIXED_SIZE_BUFFER_2,
|
||||
TSI_UPS_SCAN_PARAM_2,
|
||||
TSI_UPS_SQL_SCAN_PARAM_1,
|
||||
TSI_UPS_ARENA_ALLOC_1,
|
||||
TSI_UPS_SCAN_MERGE_1,
|
||||
TSI_UPS_GET_MERGE_1,
|
||||
TSI_UPS_SQL_MULTI_SCAN_MERGE_1,
|
||||
TSI_UPS_SCHEMA_MGR_1,
|
||||
TSI_UPS_SINGLE_GET_PARAM_1,
|
||||
TSI_UPS_EMPTY_SCAN_1,
|
||||
TSI_UPS_EMPTY_GET_1,
|
||||
TSI_UPS_TABLE_DUMP_1,
|
||||
};
|
||||
|
||||
enum TSIMergeserverType
|
||||
{
|
||||
TSI_MS_SCANNER_1 = 5001,
|
||||
TSI_MS_ORG_GET_PARAM_1,
|
||||
TSI_MS_DECODED_GET_PARAM_1,
|
||||
TSI_MS_GET_PARAM_WITH_NAME_1,
|
||||
TSI_MS_ORG_SCAN_PARAM_1,
|
||||
TSI_MS_DECODED_SCAN_PARAM_1,
|
||||
TSI_MS_SCHEMA_DECODER_ASSIS_1,
|
||||
TSI_MS_GET_EVENT_1,
|
||||
TSI_MS_SCAN_EVENT_1,
|
||||
TSI_MS_MS_SCAN_PARAM_1,
|
||||
TSI_MS_ORG_MUTATOR_1,
|
||||
TSI_MS_DECODED_MUTATOR_1,
|
||||
TSI_MS_UPS_SCANNER_1,
|
||||
TSI_MS_NEW_SCANNER_1,
|
||||
TSI_MS_NEW_SCANNER_2,
|
||||
TSI_MS_SQL_SCAN_PARAM_1,
|
||||
TSI_MS_SERVER_COUNTER_ID,
|
||||
TSI_MS_MGET_STRING_BUF_1,
|
||||
TSI_MS_SCAN_STRING_BUF_1,
|
||||
};
|
||||
|
||||
enum TSIOlapDrive
|
||||
{
|
||||
TSI_OLAP_SCAN_EXTRA_INFO_1 = 6001,
|
||||
TSI_OLAP_THREAD_ROW_KEY_1,
|
||||
TSI_OLAP_GET_PARAM_1,
|
||||
TSI_OLAP_SCAN_PARAM_1,
|
||||
TSI_OLAP_SCANNER_1,
|
||||
TSI_OLAP_MUTATOR_1,
|
||||
};
|
||||
|
||||
enum TSISqlType
|
||||
{
|
||||
TSI_SQL_GET_PARAM_1 = 7001,
|
||||
TSI_SQL_GET_PARAM_2 = 7002,
|
||||
TSI_SQL_EXPR_STACK_1 = 7003,
|
||||
TSI_SQL_EXPR_EXTRA_PARAMS_1 = 7005,
|
||||
TSI_SQL_TP_ARENA_1 = 7006,
|
||||
TSI_SQL_ROW_1 = 7007,
|
||||
TSI_SQL_PLAN_EXECUTOR_1 = 7008,
|
||||
TSI_SQL_EXPLAIN_FORMATOR_1 = 7009,
|
||||
TSI_SQL_CALC_BUF_1 = 7010,
|
||||
};
|
||||
|
||||
enum TSIMySQLType
|
||||
{
|
||||
TSI_MYSQL_CLIENT_WAIT_1 = 8001,
|
||||
TSI_MYSQL_RESULT_SET_1,
|
||||
TSI_MYSQL_PREPARE_RESULT_1,
|
||||
TSI_MYSQL_SESSION_KEY_1,
|
||||
};
|
||||
|
||||
enum TSIRootserverType
|
||||
{
|
||||
TSI_RS_SCANNER_1 = 9001,
|
||||
TSI_RS_GET_PARAM_1,
|
||||
TSI_RS_MS_PROVIDER_1,
|
||||
TSI_RS_NEW_SCANNER_1,
|
||||
TSI_RS_SQL_SCAN_PARAM_1,
|
||||
TSI_RS_NEW_SCANNER_2,
|
||||
TSI_RS_SCHEMA_MGR_1,
|
||||
};
|
||||
|
||||
enum TSIProxyserverType
|
||||
{
|
||||
TSI_YUNTI_PROXY_READER_1 = 10001,
|
||||
TSI_YUNTI_PROXY_READER_2,
|
||||
};
|
||||
|
||||
enum TSILiboblogType
|
||||
{
|
||||
TSI_LIBOBLOG_PARTITION = 11001,
|
||||
TSI_LIBOBLOG_DML_STMT,
|
||||
TSI_LIBOBLOG_MYSQL_ADAPTOR,
|
||||
TSI_LIBOBLOG_META_MANAGER,
|
||||
TSI_LIBOBLOG_ROW_VALUE,
|
||||
TSI_LIBOBLOG_SERVER_SELECTOR,
|
||||
TSI_LIBOBLOG_MYSQL_QUERY_RESULT,
|
||||
};
|
||||
|
||||
enum TSICLogType
|
||||
{
|
||||
TSI_CLOG_READER_TSIINFO = 12001,
|
||||
TSI_CLOG_WRITE_AIOPARAM,
|
||||
};
|
||||
|
||||
enum TSIMemtableType
|
||||
{
|
||||
};
|
||||
|
||||
enum TSITransType
|
||||
{
|
||||
};
|
||||
|
||||
#define GET_TSI0(type) ::oceanbase::common::TSIFactory::get_instance<type,0>()
|
||||
#define GET_TSI_MULT0(type, tag) ::oceanbase::common::TSIFactory::get_instance<type,tag>()
|
||||
#define GET_TSI(type) ::oceanbase::common::TSIFactory::get_instance<type,0>()
|
||||
#define GET_TSI_MULT(type, tag) ::oceanbase::common::TSIFactory::get_instance<type,tag>()
|
||||
|
||||
template <class T>
|
||||
class Wrapper
|
||||
{
|
||||
public:
|
||||
Wrapper() : instance_(NULL) {}
|
||||
~Wrapper()
|
||||
{
|
||||
if (NULL != instance_) {
|
||||
delete instance_;
|
||||
instance_ = NULL;
|
||||
}
|
||||
}
|
||||
public:
|
||||
T *&get_instance() { return instance_; }
|
||||
private:
|
||||
T *instance_;
|
||||
};
|
||||
|
||||
#define GET_TSI_ARGS(type, num, args...) \
|
||||
({ \
|
||||
type *__type_ret__ = NULL; \
|
||||
Wrapper<type> *__type_wrapper__ = GET_TSI_MULT(Wrapper<type>, num); \
|
||||
if (NULL != __type_wrapper__) \
|
||||
{ \
|
||||
__type_ret__ = __type_wrapper__->get_instance(); \
|
||||
if (NULL == __type_ret__) \
|
||||
{ \
|
||||
__type_wrapper__->get_instance() = new(std::nothrow) type(args); \
|
||||
__type_ret__ = __type_wrapper__->get_instance(); \
|
||||
} \
|
||||
} \
|
||||
__type_ret__; \
|
||||
})
|
||||
// GET_TSI(Wrapper<type>) ? (GET_TSI(Wrapper<type>)->get_instance() ? (GET_TSI(Wrapper<type>)->get_instance()) :
|
||||
// (GET_TSI(Wrapper<type>)->get_instance() = new(std::nothrow) type(args))) : NULL
|
||||
|
||||
class TSINodeBase
|
||||
{
|
||||
public:
|
||||
TSINodeBase() : next(NULL) {}
|
||||
virtual ~TSINodeBase() { next = NULL; }
|
||||
TSINodeBase *next;
|
||||
};
|
||||
|
||||
template <class T>
|
||||
class TSINode : public TSINodeBase
|
||||
{
|
||||
public:
|
||||
explicit TSINode(T *instance) : instance_(instance) {}
|
||||
virtual ~TSINode()
|
||||
{
|
||||
// FIXME: support print log at this point.
|
||||
//
|
||||
// Currently syslog need COVAR information e.g. TRACE ID,
|
||||
// log_limiter. But when thread exits rouine's context has been
|
||||
// destroyed.
|
||||
if (NULL != instance_) {
|
||||
// _LIB_LOG(INFO, "delete instance [%s] %p", typeid(T).name(), instance_);
|
||||
instance_->~T();
|
||||
instance_ = NULL;
|
||||
}
|
||||
}
|
||||
private:
|
||||
T *instance_;
|
||||
};
|
||||
|
||||
template <class T, int N>
|
||||
class TSINode<T[N]> : public TSINodeBase
|
||||
{
|
||||
public:
|
||||
explicit TSINode(T (*instance)[N]) : instance_(*instance) {}
|
||||
virtual ~TSINode()
|
||||
{
|
||||
if (nullptr != instance_) {
|
||||
for (int i = 0; i < N; i++) {
|
||||
instance_[i].~T();
|
||||
}
|
||||
instance_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
T *instance_;
|
||||
};
|
||||
|
||||
|
||||
|
||||
class ThreadSpecInfo
|
||||
{
|
||||
public:
|
||||
ThreadSpecInfo() : list_(NULL),
|
||||
alloc_(ObModIds::OB_TSI_FACTORY)
|
||||
{}
|
||||
~ThreadSpecInfo()
|
||||
{
|
||||
TSINodeBase *iter = list_;
|
||||
while (NULL != iter) {
|
||||
TSINodeBase *tmp = iter;
|
||||
iter = iter->next;
|
||||
tmp->~TSINodeBase();
|
||||
}
|
||||
list_ = NULL;
|
||||
}
|
||||
public:
|
||||
template <class T>
|
||||
T *get_instance()
|
||||
{
|
||||
T *instance = NULL;
|
||||
TSINode<T> *node = NULL;
|
||||
void *instance_buffer = nullptr;
|
||||
void *node_buffer = nullptr;
|
||||
while (nullptr == instance) {
|
||||
if (nullptr == instance_buffer) {
|
||||
instance_buffer = alloc_.alloc(sizeof(T));
|
||||
}
|
||||
if (nullptr != instance_buffer && nullptr == node_buffer) {
|
||||
node_buffer = alloc_.alloc(sizeof(TSINode<T>));
|
||||
}
|
||||
if (nullptr == instance_buffer || nullptr == node_buffer) {
|
||||
if (REACH_TIME_INTERVAL(10 * 1000L * 1000L)) {
|
||||
LIB_LOG_RET(WARN, OB_ALLOCATE_MEMORY_FAILED, "new instance fail", "type", typeid(T).name());
|
||||
}
|
||||
::usleep(100L * 1000L);
|
||||
} else {
|
||||
instance = reinterpret_cast<T*>(new (instance_buffer) T());
|
||||
node = new(node_buffer) TSINode<T>(instance);
|
||||
_LIB_LOG(INFO, "new instance succ [%s] %p size=%ld, tsi=%p",
|
||||
typeid(T).name(), instance, sizeof(T), this);
|
||||
if (NULL == list_) {
|
||||
list_ = node;
|
||||
list_->next = NULL;
|
||||
} else {
|
||||
node->next = list_;
|
||||
list_ = node;
|
||||
}
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
};
|
||||
private:
|
||||
TSINodeBase *list_;
|
||||
common::ObArenaAllocator alloc_;
|
||||
};
|
||||
|
||||
class TSIFactory;
|
||||
extern TSIFactory &get_tsi_fatcory();
|
||||
|
||||
class TSIFactory
|
||||
{
|
||||
static const pthread_key_t INVALID_THREAD_KEY = INT32_MAX;
|
||||
public:
|
||||
TSIFactory() : key_(INVALID_THREAD_KEY)
|
||||
{
|
||||
if (0 != pthread_key_create(&key_, destroy_thread_data_)) {
|
||||
_LIB_LOG_RET(WARN, OB_ERROR, "pthread_key_create fail errno=%u", errno);
|
||||
}
|
||||
}
|
||||
~TSIFactory()
|
||||
{
|
||||
if (INVALID_THREAD_KEY != key_) {
|
||||
void *ptr = pthread_getspecific(key_);
|
||||
destroy_thread_data_(ptr);
|
||||
if (0 != pthread_key_delete(key_)) {
|
||||
_LIB_LOG_RET(WARN, OB_ERROR, "pthread_key_delete fail errno=%u", errno);
|
||||
} else {
|
||||
key_ = INVALID_THREAD_KEY;
|
||||
}
|
||||
}
|
||||
}
|
||||
public:
|
||||
template <class T>
|
||||
T *new_instance()
|
||||
{
|
||||
T *instance = NULL;
|
||||
if (INVALID_THREAD_KEY != key_) {
|
||||
ThreadSpecInfo *tsi = (ThreadSpecInfo *)pthread_getspecific(key_);
|
||||
if (NULL == tsi) {
|
||||
tsi = new (std::nothrow) ThreadSpecInfo();
|
||||
// As some log call to_cstring and to_cstring->GET_TSI_MULT->get_instance->new_instance,
|
||||
// so this place do not print log.
|
||||
//if (NULL != tsi) {
|
||||
//_LIB_LOG(INFO, "new tsi succ %p key=%d", tsi, key_);
|
||||
//}
|
||||
if (NULL != tsi
|
||||
&& 0 != pthread_setspecific(key_, tsi)) {
|
||||
_LIB_LOG_RET(WARN, OB_ERROR, "pthread_setspecific fail errno=%u key=%d", errno, key_);
|
||||
delete tsi;
|
||||
tsi = NULL;
|
||||
}
|
||||
}
|
||||
if (NULL != tsi) {
|
||||
instance = tsi->get_instance<T>();
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
template <class T, size_t num>
|
||||
OB_INLINE static T *get_instance()
|
||||
{
|
||||
// Note: Accelerating TSI object access.
|
||||
#ifdef TSI_STATIC_SUM
|
||||
static char TSI_PLACEHOLDER[sizeof(T)] __attribute__((section("tsi_placeholder"), used));
|
||||
#endif
|
||||
RLOCAL_INLINE(T *, instance);
|
||||
if (OB_UNLIKELY(NULL == instance)) {
|
||||
instance = get_tsi_fatcory().new_instance<T>();
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
private:
|
||||
static void destroy_thread_data_(void *ptr)
|
||||
{
|
||||
if (NULL != ptr) {
|
||||
ThreadSpecInfo *tsi = (ThreadSpecInfo *)ptr;
|
||||
delete tsi;
|
||||
}
|
||||
}
|
||||
private:
|
||||
pthread_key_t key_;
|
||||
};
|
||||
|
||||
#define GET_TSI0(type) (::oceanbase::common::ObDITls<type,0>::get_instance())
|
||||
#define GET_TSI_MULT0(type, tag) (::oceanbase::common::ObDITls<type,tag>::get_instance())
|
||||
#define GET_TSI(type) (::oceanbase::common::ObDITls<type,0>::get_instance())
|
||||
#define GET_TSI_MULT(type, tag) (::oceanbase::common::ObDITls<type,tag>::get_instance())
|
||||
} // namespace common
|
||||
} // namespace oceanbase
|
||||
|
||||
|
279
deps/oblib/src/lib/utility/ob_macro_utils.h
vendored
279
deps/oblib/src/lib/utility/ob_macro_utils.h
vendored
@ -25,21 +25,8 @@
|
||||
_90, _91, _92, _93, _94, _95, _96, _97, _98, _99, \
|
||||
_100, ...) _100
|
||||
|
||||
#define _SELECT100_(_0, _1, _2, _3, _4, _5, _6, _7, _8, _9, \
|
||||
_10, _11, _12, _13, _14, _15, _16, _17, _18, _19, \
|
||||
_20, _21, _22, _23, _24, _25, _26, _27, _28, _29, \
|
||||
_30, _31, _32, _33, _34, _35, _36, _37, _38, _39, \
|
||||
_40, _41, _42, _43, _44, _45, _46, _47, _48, _49, \
|
||||
_50, _51, _52, _53, _54, _55, _56, _57, _58, _59, \
|
||||
_60, _61, _62, _63, _64, _65, _66, _67, _68, _69, \
|
||||
_70, _71, _72, _73, _74, _75, _76, _77, _78, _79, \
|
||||
_80, _81, _82, _83, _84, _85, _86, _87, _88, _89, \
|
||||
_90, _91, _92, _93, _94, _95, _96, _97, _98, _99, \
|
||||
_100, ...) _100
|
||||
|
||||
// These two macros do same work that select the 100th argument from
|
||||
// argument list.
|
||||
#define _SELECT100(...) _SELECT100_(__VA_ARGS__)
|
||||
#define SELECT100(...) SELECT100_(__VA_ARGS__)
|
||||
|
||||
// Expand to the number of arguments
|
||||
@ -76,129 +63,117 @@
|
||||
#define _CONCAT_(x, y) x ## y
|
||||
#define _CONCAT(x, y) _CONCAT_(x, y)
|
||||
|
||||
// make that many duplicates of X, X should be surrounded by parentheses.
|
||||
#define APPLY_VARGS(f, x) f(x)
|
||||
#define APPLY0(f, ...) __VA_ARGS__
|
||||
#define APPLY1(f, ...) APPLY_VARGS(f, APPLY0(f, ##__VA_ARGS__))
|
||||
#define APPLY2(f, ...) APPLY_VARGS(f, APPLY1(f, ##__VA_ARGS__))
|
||||
#define APPLY3(f, ...) APPLY_VARGS(f, APPLY2(f, ##__VA_ARGS__))
|
||||
#define APPLY4(f, ...) APPLY_VARGS(f, APPLY3(f, ##__VA_ARGS__))
|
||||
#define APPLY5(f, ...) APPLY_VARGS(f, APPLY4(f, ##__VA_ARGS__))
|
||||
#define APPLY6(f, ...) APPLY_VARGS(f, APPLY5(f, ##__VA_ARGS__))
|
||||
#define APPLY7(f, ...) APPLY_VARGS(f, APPLY6(f, ##__VA_ARGS__))
|
||||
#define APPLY8(f, ...) APPLY_VARGS(f, APPLY7(f, ##__VA_ARGS__))
|
||||
#define APPLY9(f, ...) APPLY_VARGS(f, APPLY8(f, ##__VA_ARGS__))
|
||||
#define APPLY10(f, ...) APPLY_VARGS(f, APPLY9(f, ##__VA_ARGS__))
|
||||
#define APPLY11(f, ...) APPLY_VARGS(f, APPLY10(f, ##__VA_ARGS__))
|
||||
#define APPLY12(f, ...) APPLY_VARGS(f, APPLY11(f, ##__VA_ARGS__))
|
||||
#define APPLY13(f, ...) APPLY_VARGS(f, APPLY12(f, ##__VA_ARGS__))
|
||||
#define APPLY14(f, ...) APPLY_VARGS(f, APPLY13(f, ##__VA_ARGS__))
|
||||
#define APPLY15(f, ...) APPLY_VARGS(f, APPLY14(f, ##__VA_ARGS__))
|
||||
#define APPLY16(f, ...) APPLY_VARGS(f, APPLY15(f, ##__VA_ARGS__))
|
||||
#define APPLY17(f, ...) APPLY_VARGS(f, APPLY16(f, ##__VA_ARGS__))
|
||||
#define APPLY18(f, ...) APPLY_VARGS(f, APPLY17(f, ##__VA_ARGS__))
|
||||
#define APPLY19(f, ...) APPLY_VARGS(f, APPLY18(f, ##__VA_ARGS__))
|
||||
#define APPLY20(f, ...) APPLY_VARGS(f, APPLY19(f, ##__VA_ARGS__))
|
||||
#define APPLY21(f, ...) APPLY_VARGS(f, APPLY20(f, ##__VA_ARGS__))
|
||||
#define APPLY22(f, ...) APPLY_VARGS(f, APPLY21(f, ##__VA_ARGS__))
|
||||
#define APPLY23(f, ...) APPLY_VARGS(f, APPLY22(f, ##__VA_ARGS__))
|
||||
#define APPLY24(f, ...) APPLY_VARGS(f, APPLY23(f, ##__VA_ARGS__))
|
||||
#define APPLY25(f, ...) APPLY_VARGS(f, APPLY24(f, ##__VA_ARGS__))
|
||||
#define APPLY26(f, ...) APPLY_VARGS(f, APPLY25(f, ##__VA_ARGS__))
|
||||
#define APPLY27(f, ...) APPLY_VARGS(f, APPLY26(f, ##__VA_ARGS__))
|
||||
#define APPLY28(f, ...) APPLY_VARGS(f, APPLY27(f, ##__VA_ARGS__))
|
||||
#define APPLY29(f, ...) APPLY_VARGS(f, APPLY28(f, ##__VA_ARGS__))
|
||||
#define APPLY30(f, ...) APPLY_VARGS(f, APPLY29(f, ##__VA_ARGS__))
|
||||
#define APPLY31(f, ...) APPLY_VARGS(f, APPLY30(f, ##__VA_ARGS__))
|
||||
#define APPLY32(f, ...) APPLY_VARGS(f, APPLY31(f, ##__VA_ARGS__))
|
||||
#define APPLY33(f, ...) APPLY_VARGS(f, APPLY32(f, ##__VA_ARGS__))
|
||||
#define APPLY34(f, ...) APPLY_VARGS(f, APPLY33(f, ##__VA_ARGS__))
|
||||
#define APPLY35(f, ...) APPLY_VARGS(f, APPLY34(f, ##__VA_ARGS__))
|
||||
#define APPLY36(f, ...) APPLY_VARGS(f, APPLY35(f, ##__VA_ARGS__))
|
||||
#define APPLY37(f, ...) APPLY_VARGS(f, APPLY36(f, ##__VA_ARGS__))
|
||||
#define APPLY38(f, ...) APPLY_VARGS(f, APPLY37(f, ##__VA_ARGS__))
|
||||
#define APPLY39(f, ...) APPLY_VARGS(f, APPLY38(f, ##__VA_ARGS__))
|
||||
#define APPLY40(f, ...) APPLY_VARGS(f, APPLY39(f, ##__VA_ARGS__))
|
||||
#define APPLY41(f, ...) APPLY_VARGS(f, APPLY40(f, ##__VA_ARGS__))
|
||||
#define APPLY42(f, ...) APPLY_VARGS(f, APPLY41(f, ##__VA_ARGS__))
|
||||
#define APPLY43(f, ...) APPLY_VARGS(f, APPLY42(f, ##__VA_ARGS__))
|
||||
#define APPLY44(f, ...) APPLY_VARGS(f, APPLY43(f, ##__VA_ARGS__))
|
||||
#define APPLY45(f, ...) APPLY_VARGS(f, APPLY44(f, ##__VA_ARGS__))
|
||||
#define APPLY46(f, ...) APPLY_VARGS(f, APPLY45(f, ##__VA_ARGS__))
|
||||
#define APPLY47(f, ...) APPLY_VARGS(f, APPLY46(f, ##__VA_ARGS__))
|
||||
#define APPLY48(f, ...) APPLY_VARGS(f, APPLY47(f, ##__VA_ARGS__))
|
||||
#define APPLY49(f, ...) APPLY_VARGS(f, APPLY48(f, ##__VA_ARGS__))
|
||||
#define APPLY50(f, ...) APPLY_VARGS(f, APPLY49(f, ##__VA_ARGS__))
|
||||
#define APPLY51(f, ...) APPLY_VARGS(f, APPLY50(f, ##__VA_ARGS__))
|
||||
#define APPLY52(f, ...) APPLY_VARGS(f, APPLY51(f, ##__VA_ARGS__))
|
||||
#define APPLY53(f, ...) APPLY_VARGS(f, APPLY52(f, ##__VA_ARGS__))
|
||||
#define APPLY54(f, ...) APPLY_VARGS(f, APPLY53(f, ##__VA_ARGS__))
|
||||
#define APPLY55(f, ...) APPLY_VARGS(f, APPLY54(f, ##__VA_ARGS__))
|
||||
#define APPLY56(f, ...) APPLY_VARGS(f, APPLY55(f, ##__VA_ARGS__))
|
||||
#define APPLY57(f, ...) APPLY_VARGS(f, APPLY56(f, ##__VA_ARGS__))
|
||||
#define APPLY58(f, ...) APPLY_VARGS(f, APPLY57(f, ##__VA_ARGS__))
|
||||
#define APPLY59(f, ...) APPLY_VARGS(f, APPLY58(f, ##__VA_ARGS__))
|
||||
#define APPLY60(f, ...) APPLY_VARGS(f, APPLY59(f, ##__VA_ARGS__))
|
||||
#define APPLY61(f, ...) APPLY_VARGS(f, APPLY60(f, ##__VA_ARGS__))
|
||||
#define APPLY62(f, ...) APPLY_VARGS(f, APPLY61(f, ##__VA_ARGS__))
|
||||
#define APPLY63(f, ...) APPLY_VARGS(f, APPLY62(f, ##__VA_ARGS__))
|
||||
#define APPLY64(f, ...) APPLY_VARGS(f, APPLY63(f, ##__VA_ARGS__))
|
||||
#define APPLY65(f, ...) APPLY_VARGS(f, APPLY64(f, ##__VA_ARGS__))
|
||||
#define APPLY66(f, ...) APPLY_VARGS(f, APPLY65(f, ##__VA_ARGS__))
|
||||
#define APPLY67(f, ...) APPLY_VARGS(f, APPLY66(f, ##__VA_ARGS__))
|
||||
#define APPLY68(f, ...) APPLY_VARGS(f, APPLY67(f, ##__VA_ARGS__))
|
||||
#define APPLY69(f, ...) APPLY_VARGS(f, APPLY68(f, ##__VA_ARGS__))
|
||||
#define APPLY70(f, ...) APPLY_VARGS(f, APPLY69(f, ##__VA_ARGS__))
|
||||
#define APPLY71(f, ...) APPLY_VARGS(f, APPLY70(f, ##__VA_ARGS__))
|
||||
#define APPLY72(f, ...) APPLY_VARGS(f, APPLY71(f, ##__VA_ARGS__))
|
||||
#define APPLY73(f, ...) APPLY_VARGS(f, APPLY72(f, ##__VA_ARGS__))
|
||||
#define APPLY74(f, ...) APPLY_VARGS(f, APPLY73(f, ##__VA_ARGS__))
|
||||
#define APPLY75(f, ...) APPLY_VARGS(f, APPLY74(f, ##__VA_ARGS__))
|
||||
#define APPLY76(f, ...) APPLY_VARGS(f, APPLY75(f, ##__VA_ARGS__))
|
||||
#define APPLY77(f, ...) APPLY_VARGS(f, APPLY76(f, ##__VA_ARGS__))
|
||||
#define APPLY78(f, ...) APPLY_VARGS(f, APPLY77(f, ##__VA_ARGS__))
|
||||
#define APPLY79(f, ...) APPLY_VARGS(f, APPLY78(f, ##__VA_ARGS__))
|
||||
#define APPLY80(f, ...) APPLY_VARGS(f, APPLY79(f, ##__VA_ARGS__))
|
||||
#define APPLY81(f, ...) APPLY_VARGS(f, APPLY80(f, ##__VA_ARGS__))
|
||||
#define APPLY82(f, ...) APPLY_VARGS(f, APPLY81(f, ##__VA_ARGS__))
|
||||
#define APPLY83(f, ...) APPLY_VARGS(f, APPLY82(f, ##__VA_ARGS__))
|
||||
#define APPLY84(f, ...) APPLY_VARGS(f, APPLY83(f, ##__VA_ARGS__))
|
||||
#define APPLY85(f, ...) APPLY_VARGS(f, APPLY84(f, ##__VA_ARGS__))
|
||||
#define APPLY86(f, ...) APPLY_VARGS(f, APPLY85(f, ##__VA_ARGS__))
|
||||
#define APPLY87(f, ...) APPLY_VARGS(f, APPLY86(f, ##__VA_ARGS__))
|
||||
#define APPLY88(f, ...) APPLY_VARGS(f, APPLY87(f, ##__VA_ARGS__))
|
||||
#define APPLY89(f, ...) APPLY_VARGS(f, APPLY88(f, ##__VA_ARGS__))
|
||||
#define APPLY90(f, ...) APPLY_VARGS(f, APPLY89(f, ##__VA_ARGS__))
|
||||
#define APPLY91(f, ...) APPLY_VARGS(f, APPLY90(f, ##__VA_ARGS__))
|
||||
#define APPLY92(f, ...) APPLY_VARGS(f, APPLY91(f, ##__VA_ARGS__))
|
||||
#define APPLY93(f, ...) APPLY_VARGS(f, APPLY92(f, ##__VA_ARGS__))
|
||||
#define APPLY94(f, ...) APPLY_VARGS(f, APPLY93(f, ##__VA_ARGS__))
|
||||
#define APPLY95(f, ...) APPLY_VARGS(f, APPLY94(f, ##__VA_ARGS__))
|
||||
#define APPLY96(f, ...) APPLY_VARGS(f, APPLY95(f, ##__VA_ARGS__))
|
||||
#define APPLY97(f, ...) APPLY_VARGS(f, APPLY96(f, ##__VA_ARGS__))
|
||||
#define APPLY98(f, ...) APPLY_VARGS(f, APPLY97(f, ##__VA_ARGS__))
|
||||
#define APPLY99(f, ...) APPLY_VARGS(f, APPLY98(f, ##__VA_ARGS__))
|
||||
#define APPLY100(f, ...) APPLY_VARGS(f, APPLY99(f, ##__VA_ARGS__))
|
||||
#define APPLY(n, f, ...) APPLY ## n (f, ##__VA_ARGS__)
|
||||
|
||||
// select nth argument
|
||||
//
|
||||
// DUP1((=)) => =
|
||||
// DUP2((,)) => ,,
|
||||
// DUP5((a++;)) => a++;a++;a++;a++;a++;
|
||||
//
|
||||
#define DUP1(X) SELF X
|
||||
#define DUP2(X) SELF X DUP1(X)
|
||||
#define DUP3(X) SELF X DUP2(X)
|
||||
#define DUP4(X) SELF X DUP3(X)
|
||||
#define DUP5(X) SELF X DUP4(X)
|
||||
#define DUP6(X) SELF X DUP5(X)
|
||||
#define DUP7(X) SELF X DUP6(X)
|
||||
#define DUP8(X) SELF X DUP7(X)
|
||||
#define DUP9(X) SELF X DUP8(X)
|
||||
#define DUP10(X) SELF X DUP9(X)
|
||||
#define DUP11(X) SELF X DUP10(X)
|
||||
#define DUP12(X) SELF X DUP11(X)
|
||||
#define DUP13(X) SELF X DUP12(X)
|
||||
#define DUP14(X) SELF X DUP13(X)
|
||||
#define DUP15(X) SELF X DUP14(X)
|
||||
#define DUP16(X) SELF X DUP15(X)
|
||||
#define DUP17(X) SELF X DUP16(X)
|
||||
#define DUP18(X) SELF X DUP17(X)
|
||||
#define DUP19(X) SELF X DUP18(X)
|
||||
#define DUP20(X) SELF X DUP19(X)
|
||||
#define DUP21(X) SELF X DUP20(X)
|
||||
#define DUP22(X) SELF X DUP21(X)
|
||||
#define DUP23(X) SELF X DUP22(X)
|
||||
#define DUP24(X) SELF X DUP23(X)
|
||||
#define DUP25(X) SELF X DUP24(X)
|
||||
#define DUP26(X) SELF X DUP25(X)
|
||||
#define DUP27(X) SELF X DUP26(X)
|
||||
#define DUP28(X) SELF X DUP27(X)
|
||||
#define DUP29(X) SELF X DUP28(X)
|
||||
#define DUP30(X) SELF X DUP29(X)
|
||||
#define DUP31(X) SELF X DUP30(X)
|
||||
#define DUP32(X) SELF X DUP31(X)
|
||||
#define DUP33(X) SELF X DUP32(X)
|
||||
#define DUP34(X) SELF X DUP33(X)
|
||||
#define DUP35(X) SELF X DUP34(X)
|
||||
#define DUP36(X) SELF X DUP35(X)
|
||||
#define DUP37(X) SELF X DUP36(X)
|
||||
#define DUP38(X) SELF X DUP37(X)
|
||||
#define DUP39(X) SELF X DUP38(X)
|
||||
#define DUP40(X) SELF X DUP39(X)
|
||||
#define DUP41(X) SELF X DUP40(X)
|
||||
#define DUP42(X) SELF X DUP41(X)
|
||||
#define DUP43(X) SELF X DUP42(X)
|
||||
#define DUP44(X) SELF X DUP43(X)
|
||||
#define DUP45(X) SELF X DUP44(X)
|
||||
#define DUP46(X) SELF X DUP45(X)
|
||||
#define DUP47(X) SELF X DUP46(X)
|
||||
#define DUP48(X) SELF X DUP47(X)
|
||||
#define DUP49(X) SELF X DUP48(X)
|
||||
#define DUP50(X) SELF X DUP49(X)
|
||||
#define DUP51(X) SELF X DUP50(X)
|
||||
#define DUP52(X) SELF X DUP51(X)
|
||||
#define DUP53(X) SELF X DUP52(X)
|
||||
#define DUP54(X) SELF X DUP53(X)
|
||||
#define DUP55(X) SELF X DUP54(X)
|
||||
#define DUP56(X) SELF X DUP55(X)
|
||||
#define DUP57(X) SELF X DUP56(X)
|
||||
#define DUP58(X) SELF X DUP57(X)
|
||||
#define DUP59(X) SELF X DUP58(X)
|
||||
#define DUP60(X) SELF X DUP59(X)
|
||||
#define DUP61(X) SELF X DUP60(X)
|
||||
#define DUP62(X) SELF X DUP61(X)
|
||||
#define DUP63(X) SELF X DUP62(X)
|
||||
#define DUP64(X) SELF X DUP63(X)
|
||||
#define DUP65(X) SELF X DUP64(X)
|
||||
#define DUP66(X) SELF X DUP65(X)
|
||||
#define DUP67(X) SELF X DUP66(X)
|
||||
#define DUP68(X) SELF X DUP67(X)
|
||||
#define DUP69(X) SELF X DUP68(X)
|
||||
#define DUP70(X) SELF X DUP69(X)
|
||||
#define DUP71(X) SELF X DUP70(X)
|
||||
#define DUP72(X) SELF X DUP71(X)
|
||||
#define DUP73(X) SELF X DUP72(X)
|
||||
#define DUP74(X) SELF X DUP73(X)
|
||||
#define DUP75(X) SELF X DUP74(X)
|
||||
#define DUP76(X) SELF X DUP75(X)
|
||||
#define DUP77(X) SELF X DUP76(X)
|
||||
#define DUP78(X) SELF X DUP77(X)
|
||||
#define DUP79(X) SELF X DUP78(X)
|
||||
#define DUP80(X) SELF X DUP79(X)
|
||||
#define DUP81(X) SELF X DUP80(X)
|
||||
#define DUP82(X) SELF X DUP81(X)
|
||||
#define DUP83(X) SELF X DUP82(X)
|
||||
#define DUP84(X) SELF X DUP83(X)
|
||||
#define DUP85(X) SELF X DUP84(X)
|
||||
#define DUP86(X) SELF X DUP85(X)
|
||||
#define DUP87(X) SELF X DUP86(X)
|
||||
#define DUP88(X) SELF X DUP87(X)
|
||||
#define DUP89(X) SELF X DUP88(X)
|
||||
#define DUP90(X) SELF X DUP89(X)
|
||||
#define DUP91(X) SELF X DUP90(X)
|
||||
#define DUP92(X) SELF X DUP91(X)
|
||||
#define DUP93(X) SELF X DUP92(X)
|
||||
#define DUP94(X) SELF X DUP93(X)
|
||||
#define DUP95(X) SELF X DUP94(X)
|
||||
#define DUP96(X) SELF X DUP95(X)
|
||||
#define DUP97(X) SELF X DUP96(X)
|
||||
#define DUP98(X) SELF X DUP97(X)
|
||||
#define DUP99(X) SELF X DUP98(X)
|
||||
|
||||
#define DUP_(n, X) DUP ## n (X)
|
||||
#define DUP(n, X) DUP_(n, X)
|
||||
|
||||
|
||||
// expand to the number of 100 minus n
|
||||
#define COMP100(n) \
|
||||
_SELECT100(DUP(n, (,)), \
|
||||
1, 2, 3, 4, 5, 6, 7, 8, 9, \
|
||||
10, 11, 12, 13, 14, 15, 16, 17, 18, 19, \
|
||||
20, 21, 22, 23, 24, 25, 26, 27, 28, 29, \
|
||||
30, 31, 32, 33, 34, 35, 36, 37, 38, 39, \
|
||||
40, 41, 42, 43, 44, 45, 46, 47, 48, 49, \
|
||||
50, 51, 52, 53, 54, 55, 56, 57, 58, 59, \
|
||||
60, 61, 62, 63, 64, 65, 66, 67, 68, 69, \
|
||||
70, 71, 72, 73, 74, 75, 76, 77, 78, 79, \
|
||||
80, 81, 82, 83, 84, 85, 86, 87, 88, 89, \
|
||||
90, 91, 92, 93, 94, 95, 96, 97, 98, 99)
|
||||
// SELECT(2, arg1, arg2, arg3) => arg2
|
||||
#define EXPAND(...) __VA_ARGS__
|
||||
#define FIRST(x,...) x
|
||||
#define REMOVE_FIRST(x,...) __VA_ARGS__
|
||||
#define SELECT(n, ...) APPLY_VARGS(FIRST, APPLY(n, REMOVE_FIRST, ,##__VA_ARGS__))
|
||||
|
||||
#define LST_DO_0(...)
|
||||
#define LST_DO_1(M, s, P, ...) P(M, 1, ##__VA_ARGS__)
|
||||
@ -307,11 +282,6 @@
|
||||
#define LST_DO(M, s, ...) LST_DO_(ARGS_NUM(__VA_ARGS__), M, s, PROC_ONE, ##__VA_ARGS__)
|
||||
#define LST_DO2(M, s, ...) LST_DO_(ARGS_NUM(__VA_ARGS__), M, s, PROC_ONE2, ##__VA_ARGS__)
|
||||
|
||||
// select nth argument
|
||||
//
|
||||
// SELECT(2, arg1, arg2, arg3) => arg2
|
||||
#define SELECT(n, ...) SELECT100(DUP(COMP100(n), (,)), __VA_ARGS__)
|
||||
|
||||
#define PROC_ONE(M, ...) M(SELECT(__VA_ARGS__))
|
||||
#define PROC_ONE2(M, IDX, ...) M(SELECT(IDX, __VA_ARGS__), IDX)
|
||||
|
||||
@ -351,20 +321,10 @@
|
||||
// IF_PARENT(XXX, MACRO_PAREN, MACRO_NO_PAREN) ==> MACRO_NO_PAREN(XXX)
|
||||
// IF_PARENT((XXX), MACRO_PAREN, MACRO_NO_PAREN) ==> MACRO_PAREN(XXX)
|
||||
//
|
||||
#define OBMPAR IGNORE(
|
||||
#define PAR(x) _SELF(
|
||||
#define OBM_SELF(x) x
|
||||
|
||||
#define PAREN(x, M) CONCAT(OBM, PAR x) M x)
|
||||
|
||||
#define OBMNPNPAR SELF(IGNORE(
|
||||
#define OBMNP_SELF(x) SELF
|
||||
#define NPAR(x) _IGNORE(
|
||||
#define OBMNP_IGNORE(...) IGNORE(
|
||||
|
||||
#define NO_PAREN(x, M) CONCAT(OBMNP, NPAR x)) M (x))
|
||||
|
||||
#define IF_PAREN(x, MP, MNP) PAREN(x, MP) NO_PAREN(x, MNP)
|
||||
#define COMMA_(...) ,
|
||||
#define APPLY_(f, ...) f(__VA_ARGS__)
|
||||
#define IF_IS_PAREN(x, yes_branch, no_branch) APPLY_(SELECT3, COMMA_ x, yes_branch, no_branch)
|
||||
#define IF_PAREN(x, yes_branch, no_branch) EXPAND(IF_IS_PAREN(x, yes_branch, no_branch) IF_IS_PAREN(x, x, (x)))
|
||||
|
||||
#ifndef MIN
|
||||
#define MIN(x, y) ((x) < (y) ? (x) : (y))
|
||||
@ -381,17 +341,6 @@
|
||||
#define CAT(a, ...) PRIMITIVE_CAT(a, __VA_ARGS__)
|
||||
#define PRIMITIVE_CAT(a, ...) a ## __VA_ARGS__
|
||||
|
||||
#define EMPTY()
|
||||
#define INNER_DEFER(id) id EMPTY()
|
||||
#define OBSTRUCT(...) __VA_ARGS__ INNER_DEFER(EMPTY)()
|
||||
#define EXPAND(...) __VA_ARGS__
|
||||
|
||||
#define EVAL(...) EVAL1(EVAL1(EVAL1(__VA_ARGS__)))
|
||||
#define EVAL1(...) EVAL2(EVAL2(EVAL2(__VA_ARGS__)))
|
||||
#define EVAL2(...) EVAL3(EVAL3(EVAL3(__VA_ARGS__)))
|
||||
#define EVAL3(...) EVAL4(EVAL4(EVAL4(__VA_ARGS__)))
|
||||
#define EVAL4(...) EVAL5(EVAL5(EVAL5(__VA_ARGS__)))
|
||||
#define EVAL5(...) __VA_ARGS__
|
||||
|
||||
////////////////////////////////////////////////////////////////
|
||||
/**
|
||||
|
518
deps/oblib/src/lib/utility/ob_unify_serialize.h
vendored
518
deps/oblib/src/lib/utility/ob_unify_serialize.h
vendored
@ -15,59 +15,11 @@
|
||||
|
||||
#include "lib/utility/serialization.h"
|
||||
|
||||
#define UNIS_DEF_HAS_MEMBER(m) \
|
||||
template <typename _T, typename = void> \
|
||||
struct __unis_has_member : std::false_type{}; \
|
||||
\
|
||||
template <typename _T> \
|
||||
struct __unis_has_member< \
|
||||
_T, \
|
||||
decltype((void)_T::m, void()) \
|
||||
> : std::true_type {};
|
||||
|
||||
#define UNIS_HAS_COMPAT(CLS) __unis_has_member<CLS>{}
|
||||
|
||||
namespace oceanbase {
|
||||
namespace lib {
|
||||
inline uint64_t &get_unis_global_compat_version()
|
||||
{
|
||||
static uint64_t global_version;
|
||||
return global_version;
|
||||
}
|
||||
|
||||
inline uint64_t &get_unis_compat_version()
|
||||
{
|
||||
RLOCAL_INLINE(uint64_t, compat_version);
|
||||
return compat_version;
|
||||
}
|
||||
|
||||
class UnisCompatVersionGuard {
|
||||
public:
|
||||
UnisCompatVersionGuard(uint64_t version)
|
||||
: version_(get_unis_compat_version())
|
||||
{
|
||||
get_unis_compat_version() = version;
|
||||
}
|
||||
~UnisCompatVersionGuard()
|
||||
{
|
||||
get_unis_compat_version() = version_;
|
||||
}
|
||||
private:
|
||||
uint64_t version_;
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
#define UNIS_VERSION_GUARD(v) ::oceanbase::lib::UnisCompatVersionGuard __unis_guard(v)
|
||||
|
||||
#define SERIAL_PARAMS char *buf, const int64_t buf_len, int64_t &pos
|
||||
#define DESERIAL_PARAMS const char *buf, const int64_t data_len, int64_t &pos
|
||||
#define SERIALIZE_SIGNATURE(func) int func(SERIAL_PARAMS) const
|
||||
#define DESERIALIZE_SIGNATURE(func) int func(DESERIAL_PARAMS)
|
||||
#define GET_SERIALIZE_SIZE_SIGNATURE(func) int64_t func(void) const
|
||||
#define SERIALIZE_DISPATCH_NAME serialize_dispatch_
|
||||
#define DESERIALIZE_DISPATCH_NAME deserialize_dispatch_
|
||||
#define GET_SERIALIZE_SIZE_DISPATCH_NAME get_serialize_size_dispatch_
|
||||
|
||||
#define UNF_UNUSED_SER ({(void)buf; (void)buf_len; (void)pos;})
|
||||
#define UNF_UNUSED_DES ({(void)buf; (void)data_len; (void)pos;})
|
||||
@ -76,12 +28,20 @@ private:
|
||||
#define RPC_WARN(...) OB_LOG(WARN, __VA_ARGS__)
|
||||
#endif
|
||||
|
||||
#define OB_DEF_SERIALIZE_SIMPLE(CLS) \
|
||||
int CLS::serialize(SERIAL_PARAMS) const
|
||||
|
||||
#define OB_DEF_DESERIALIZE_SIMPLE(CLS) \
|
||||
int CLS::deserialize(DESERIAL_PARAMS)
|
||||
|
||||
#define OB_DEF_SERIALIZE_SIZE_SIMPLE(CLS) \
|
||||
int64_t CLS::get_serialize_size(void) const
|
||||
|
||||
///
|
||||
// define essential macros used for encode/decode single object
|
||||
//----------------------------------------------------------------------
|
||||
#define NS_ ::oceanbase::common::serialization
|
||||
#define OK_ ::oceanbase::common::OB_SUCCESS
|
||||
#define LIB_NS_ ::oceanbase::lib
|
||||
|
||||
#define OB_UNIS_ENCODE(obj) \
|
||||
if (OB_SUCC(ret)) { \
|
||||
@ -111,57 +71,55 @@ private:
|
||||
len += NS_::encoded_length(obj)
|
||||
//-----------------------------------------------------------------------
|
||||
|
||||
/// utility macros to deal with C native array
|
||||
#define OB_UNIS_ENCODE_ARRAY(objs, objs_count) \
|
||||
OB_UNIS_ENCODE((objs_count)); \
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < (objs_count); ++i) { \
|
||||
OB_UNIS_ENCODE(objs[i]); \
|
||||
// serialize_ no header
|
||||
#define OB_SERIALIZE_NOHEADER(CLS, PARENT, SUFFIX, PRED, ...) \
|
||||
int CLS::serialize##SUFFIX(SERIAL_PARAMS) const { \
|
||||
int ret = PARENT::serialize(buf, buf_len, pos); \
|
||||
if (OB_SUCC(ret) && (PRED)) { \
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, ##__VA_ARGS__); \
|
||||
} \
|
||||
return ret; \
|
||||
}
|
||||
|
||||
#define OB_UNIS_ADD_LEN_ARRAY(objs, objs_count) \
|
||||
OB_UNIS_ADD_LEN((objs_count)); \
|
||||
for (int64_t i = 0; i < (objs_count); ++i) { \
|
||||
OB_UNIS_ADD_LEN(objs[i]); \
|
||||
#define OB_DESERIALIZE_NOHEADER(CLS, PARENT, SUFFIX, ...) \
|
||||
int CLS::deserialize##SUFFIX(DESERIAL_PARAMS) { \
|
||||
int ret = PARENT::deserialize(buf, data_len, pos); \
|
||||
if (OB_SUCC(ret)) { \
|
||||
LST_DO_CODE(OB_UNIS_DECODE, ##__VA_ARGS__); \
|
||||
} \
|
||||
return ret; \
|
||||
}
|
||||
|
||||
#define OB_SERIALIZE_SIZE_NOHEADER(CLS, PARENT, SUFFIX, PRED, ...) \
|
||||
int64_t CLS::get_serialize_size##SUFFIX(void) const { \
|
||||
int64_t len = PARENT::get_serialize_size(); \
|
||||
if (PRED) { \
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, ##__VA_ARGS__); \
|
||||
} \
|
||||
return len; \
|
||||
}
|
||||
|
||||
#define OB_UNIS_DECODE_ARRAY(objs, objs_count) \
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < (objs_count); ++i) { \
|
||||
OB_UNIS_DECODE(objs[i]); \
|
||||
#define OB_SERIALIZE_MEMBER_INHERIT(CLS, PARENT, ...) \
|
||||
OB_SERIALIZE_NOHEADER(CLS, PARENT, , true, ##__VA_ARGS__); \
|
||||
OB_DESERIALIZE_NOHEADER(CLS, PARENT, ,##__VA_ARGS__); \
|
||||
OB_SERIALIZE_SIZE_NOHEADER(CLS, PARENT, , true, ##__VA_ARGS__);
|
||||
|
||||
struct EmptyUnisStruct
|
||||
{
|
||||
static int serialize(SERIAL_PARAMS) {
|
||||
UNF_UNUSED_SER;
|
||||
return 0;
|
||||
}
|
||||
|
||||
///
|
||||
// define macros deal with parent class
|
||||
//-----------------------------------------------------------------------
|
||||
#define UNF_CONCAT_(a, b) a##b
|
||||
#define UNF_CONCAT(a, b) UNF_CONCAT_(a, b)
|
||||
#define UNF_IGNORE(...)
|
||||
#define UNF_uSELF(...) __VA_ARGS__
|
||||
#define UNF_SAFE_DO(M) \
|
||||
do { \
|
||||
if (OB_SUCC(ret)) { \
|
||||
if (OB_FAIL((M))) { \
|
||||
RPC_WARN("fail to execute: " #M, K(ret)); \
|
||||
} \
|
||||
} \
|
||||
} while (0)
|
||||
|
||||
#define UNF_MYCLS_ UNF_uSELF(
|
||||
#define MYCLS_(D, B) uSELF( D
|
||||
#define UNF_MYCLS(x) UNF_CONCAT(UNF_, MYCLS_ x) )
|
||||
|
||||
#define UNF_SBASE_ UNF_IGNORE(
|
||||
#define SBASE_(D, B) \
|
||||
uSELF( UNF_SAFE_DO(B::serialize(buf, buf_len, pos))
|
||||
#define BASE_SER(x) UNF_CONCAT(UNF_, SBASE_ x) )
|
||||
|
||||
#define UNF_DBASE_ UNF_IGNORE(
|
||||
#define DBASE_(D, B) \
|
||||
uSELF( UNF_SAFE_DO(B::deserialize(buf, data_len, pos))
|
||||
#define BASE_DESER(x) UNF_CONCAT(UNF_, DBASE_ x) )
|
||||
|
||||
#define UNF_LBASE_ UNF_IGNORE(
|
||||
#define LBASE_(D, B) uSELF( len += B::get_serialize_size()
|
||||
#define BASE_ADD_LEN(x) UNF_CONCAT(UNF_, LBASE_ x) )
|
||||
static int deserialize(DESERIAL_PARAMS) {
|
||||
UNF_UNUSED_DES;
|
||||
return 0;
|
||||
}
|
||||
static int64_t get_serialize_size() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
#define EmptyParent ::oceanbase::lib::EmptyUnisStruct
|
||||
#define OB_SERIALIZE_MEMBER_SIMPLE(CLS, ...) OB_SERIALIZE_MEMBER_INHERIT(CLS, EmptyParent, ##__VA_ARGS__)
|
||||
|
||||
///
|
||||
// define serialize/desrialize wrapper which helps hide "version" and
|
||||
@ -181,104 +139,27 @@ private:
|
||||
} \
|
||||
}
|
||||
|
||||
#define CALL_SERIALIZE_(SUFFIX, ...) \
|
||||
if (OB_SUCC(ret)) { \
|
||||
if (OB_FAIL(serialize_##SUFFIX(buf, buf_len, pos, ##__VA_ARGS__))) { \
|
||||
RPC_WARN("serialize fail", K(ret)); \
|
||||
} \
|
||||
}
|
||||
|
||||
#define CALL_DESERIALIZE_(SLEN, SUFFIX, ...) \
|
||||
if (OB_SUCC(ret)) { \
|
||||
int64_t pos_orig = pos; \
|
||||
pos = 0; \
|
||||
if (OB_FAIL(deserialize_##SUFFIX( \
|
||||
buf + pos_orig, SLEN, pos, ##__VA_ARGS__))) { \
|
||||
RPC_WARN("deserialize_ fail", \
|
||||
"slen", SLEN, K(pos), K(ret)); \
|
||||
} \
|
||||
pos = pos_orig + SLEN; \
|
||||
} \
|
||||
|
||||
#define CALL_GET_SERIALIZE_SIZE_(SUFFIX, ...) \
|
||||
get_serialize_size_##SUFFIX(__VA_ARGS__)
|
||||
|
||||
#define SERIALIZE_HEADER(version) \
|
||||
if (OB_SUCC(ret)) { \
|
||||
OB_UNIS_ENCODE(version); \
|
||||
}
|
||||
|
||||
// dispatch compatible function depend on current
|
||||
// get_unis_compat_version().
|
||||
#define OB_UNIS_DEFINE_DISPATCH() \
|
||||
int SERIALIZE_DISPATCH_NAME(SERIAL_PARAMS, std::false_type) const \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
CALL_SERIALIZE_(); \
|
||||
return ret; \
|
||||
} \
|
||||
int DESERIALIZE_DISPATCH_NAME(DESERIAL_PARAMS, std::false_type) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
CALL_DESERIALIZE_(data_len,); \
|
||||
return ret; \
|
||||
} \
|
||||
int64_t GET_SERIALIZE_SIZE_DISPATCH_NAME(std::false_type) const \
|
||||
{ \
|
||||
return CALL_GET_SERIALIZE_SIZE_(); \
|
||||
}
|
||||
|
||||
#define OB_UNIS_DEFINE_COMPAT_DISPATCH() \
|
||||
int SERIALIZE_DISPATCH_NAME(SERIAL_PARAMS, std::true_type) const \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
if (OB_LIKELY(LIB_NS_::get_unis_compat_version() == 0 || \
|
||||
LIB_NS_::get_unis_compat_version() > compat_ver_)) { \
|
||||
CALL_SERIALIZE_(); \
|
||||
} else { \
|
||||
CALL_SERIALIZE_(compat_); \
|
||||
} \
|
||||
return ret; \
|
||||
} \
|
||||
int DESERIALIZE_DISPATCH_NAME(DESERIAL_PARAMS, std::true_type) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
if (OB_LIKELY(LIB_NS_::get_unis_compat_version() == 0 || \
|
||||
LIB_NS_::get_unis_compat_version() > compat_ver_)) { \
|
||||
CALL_DESERIALIZE_(data_len,); \
|
||||
} else { \
|
||||
CALL_DESERIALIZE_(data_len, compat_); \
|
||||
} \
|
||||
return ret; \
|
||||
} \
|
||||
int64_t GET_SERIALIZE_SIZE_DISPATCH_NAME(std::true_type) const \
|
||||
{ \
|
||||
if (OB_LIKELY(LIB_NS_::get_unis_compat_version() == 0 || \
|
||||
LIB_NS_::get_unis_compat_version() > compat_ver_)) { \
|
||||
return CALL_GET_SERIALIZE_SIZE_(); \
|
||||
} else { \
|
||||
return CALL_GET_SERIALIZE_SIZE_(compat_); \
|
||||
} \
|
||||
}
|
||||
|
||||
#ifdef NDEBUG
|
||||
#define CHECK_SERIALIZE_SIZE(CLS, real_size)
|
||||
#else
|
||||
#define CHECK_SERIALIZE_SIZE(CLS, real_size) \
|
||||
int64_t expect_size = \
|
||||
CALL_GET_SERIALIZE_SIZE_(dispatch_, UNIS_HAS_COMPAT(CLS)); \
|
||||
int64_t expect_size = get_serialize_size(); \
|
||||
assert(expect_size >= real_size);
|
||||
#endif
|
||||
|
||||
#define OB_UNIS_SERIALIZE(CLS) \
|
||||
#define OB_UNIS_SERIALIZE(CLS) \
|
||||
int CLS::serialize(SERIAL_PARAMS) const \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
SERIALIZE_HEADER(UNIS_VERSION); \
|
||||
OB_UNIS_ENCODE(UNIS_VERSION); \
|
||||
if (OB_SUCC(ret)) { \
|
||||
int64_t size_nbytes = NS_::OB_SERIALIZE_SIZE_NEED_BYTES; \
|
||||
int64_t pos_bak = (pos += size_nbytes); \
|
||||
CALL_SERIALIZE_(dispatch_, UNIS_HAS_COMPAT(CLS)); \
|
||||
if (OB_SUCC(ret)) { \
|
||||
if (OB_FAIL(serialize_(buf, buf_len, pos))) { \
|
||||
RPC_WARN("serialize fail", K(ret)); \
|
||||
} \
|
||||
} \
|
||||
int64_t serial_size = pos - pos_bak; \
|
||||
int64_t tmp_pos = 0; \
|
||||
CHECK_SERIALIZE_SIZE(CLS, serial_size); \
|
||||
@ -290,34 +171,32 @@ private:
|
||||
return ret; \
|
||||
}
|
||||
|
||||
#define DESERIALIZE_HEADER(CLS, version, len) \
|
||||
if (OB_SUCC(ret)) { \
|
||||
OB_UNIS_DECODEx(version); \
|
||||
OB_UNIS_DECODEx(len); \
|
||||
CHECK_VERSION_LENGTH(CLS, version, len); \
|
||||
}
|
||||
|
||||
#define OB_UNIS_DESERIALIZE(CLS) \
|
||||
int CLS::deserialize(DESERIAL_PARAMS) \
|
||||
{ \
|
||||
int CLS::deserialize(DESERIAL_PARAMS) { \
|
||||
int ret = OK_; \
|
||||
int64_t version = 0; \
|
||||
int64_t len = 0; \
|
||||
DESERIALIZE_HEADER(CLS, version, len); \
|
||||
CALL_DESERIALIZE_(len, dispatch_, UNIS_HAS_COMPAT(CLS)); \
|
||||
if (OB_SUCC(ret)) { \
|
||||
OB_UNIS_DECODEx(version); \
|
||||
OB_UNIS_DECODEx(len); \
|
||||
CHECK_VERSION_LENGTH(CLS, version, len); \
|
||||
} \
|
||||
if (OB_SUCC(ret)) { \
|
||||
int64_t pos_orig = pos; \
|
||||
pos = 0; \
|
||||
if (OB_FAIL(deserialize_(buf + pos_orig, len, pos))) { \
|
||||
RPC_WARN("deserialize_ fail", "slen", len, K(pos), K(ret)); \
|
||||
} \
|
||||
pos = pos_orig + len; \
|
||||
} \
|
||||
return ret; \
|
||||
}
|
||||
|
||||
#define SERIALIZE_SIZE_HEADER(version) \
|
||||
OB_UNIS_ADD_LEN(version); \
|
||||
len += NS_::OB_SERIALIZE_SIZE_NEED_BYTES;
|
||||
|
||||
#define OB_UNIS_SERIALIZE_SIZE(CLS) \
|
||||
int64_t CLS::get_serialize_size(void) const \
|
||||
{ \
|
||||
int64_t len = CALL_GET_SERIALIZE_SIZE_( \
|
||||
dispatch_, UNIS_HAS_COMPAT(CLS)); \
|
||||
SERIALIZE_SIZE_HEADER(UNIS_VERSION); \
|
||||
int64_t CLS::get_serialize_size(void) const { \
|
||||
int64_t len = get_serialize_size_(); \
|
||||
OB_UNIS_ADD_LEN(UNIS_VERSION); \
|
||||
len += NS_::OB_SERIALIZE_SIZE_NEED_BYTES; \
|
||||
return len; \
|
||||
}
|
||||
|
||||
@ -335,16 +214,6 @@ private:
|
||||
int deserialize_(DESERIAL_PARAMS); \
|
||||
VIR int64_t get_serialize_size() const PURE; \
|
||||
int64_t get_serialize_size_() const; \
|
||||
UNIS_DEF_HAS_MEMBER(compat_ver_); \
|
||||
OB_UNIS_DEFINE_DISPATCH()
|
||||
|
||||
#define OB_DECLARE_UNIS_COMPAT() \
|
||||
int serialize_compat_(SERIAL_PARAMS) const; \
|
||||
int deserialize_compat_(DESERIAL_PARAMS); \
|
||||
int64_t get_serialize_size_compat_() const; \
|
||||
OB_UNIS_DEFINE_COMPAT_DISPATCH()
|
||||
|
||||
//-----------------------------------------------------------------------
|
||||
|
||||
///
|
||||
// public entries, define interfaces of manual serialization
|
||||
@ -362,192 +231,91 @@ const static int64_t UNIS_VERSION = VER
|
||||
#define OB_UNIS_VERSION_PV() \
|
||||
public: OB_DECLARE_UNIS(virtual,=0); private:
|
||||
|
||||
#define OB_DEF_SERIALIZE(CLS, TEMP...) \
|
||||
TEMP OB_UNIS_SERIALIZE(CLS); \
|
||||
#define OB_DEF_SERIALIZE(CLS, TEMP...) \
|
||||
TEMP OB_UNIS_SERIALIZE(CLS); \
|
||||
TEMP int CLS::serialize_(SERIAL_PARAMS) const
|
||||
|
||||
#define OB_DEF_DESERIALIZE(CLS, TEMP...) \
|
||||
TEMP OB_UNIS_DESERIALIZE(CLS); \
|
||||
#define OB_DEF_DESERIALIZE(CLS, TEMP...) \
|
||||
TEMP OB_UNIS_DESERIALIZE(CLS); \
|
||||
TEMP int CLS::deserialize_(DESERIAL_PARAMS)
|
||||
|
||||
#define OB_DEF_SERIALIZE_SIZE(CLS, TEMP...) \
|
||||
TEMP OB_UNIS_SERIALIZE_SIZE(CLS); \
|
||||
#define OB_DEF_SERIALIZE_SIZE(CLS, TEMP...) \
|
||||
TEMP OB_UNIS_SERIALIZE_SIZE(CLS); \
|
||||
TEMP int64_t CLS::get_serialize_size_(void) const
|
||||
|
||||
/// {{{ helper functions
|
||||
#define OB_DEF_SERIALIZE_COMPAT_(CLS, TEMP...) \
|
||||
TEMP int CLS::serialize_compat_(SERIAL_PARAMS) const
|
||||
#define OB_DEF_DESERIALIZE_COMPAT_(CLS, TEMP...) \
|
||||
TEMP int CLS::deserialize_compat_(DESERIAL_PARAMS)
|
||||
#define OB_DEF_SERIALIZE_SIZE_COMPAT_(CLS, TEMP...) \
|
||||
TEMP int64_t CLS::get_serialize_size_compat_(void) const
|
||||
/// }}}
|
||||
#define OB_SERIALIZE_MEMBER_TEMP_INHERIT(TEMP, CLS, PARENT, ...) \
|
||||
TEMP OB_UNIS_SERIALIZE(CLS); \
|
||||
TEMP OB_UNIS_DESERIALIZE(CLS); \
|
||||
TEMP OB_UNIS_SERIALIZE_SIZE(CLS); \
|
||||
TEMP OB_SERIALIZE_NOHEADER(CLS, PARENT, _, true, ##__VA_ARGS__); \
|
||||
TEMP OB_DESERIALIZE_NOHEADER(CLS, PARENT, _,##__VA_ARGS__); \
|
||||
TEMP OB_SERIALIZE_SIZE_NOHEADER(CLS, PARENT, _, true, ##__VA_ARGS__);
|
||||
|
||||
#define OB_DEF_SERIALIZE_COMPAT(VER, CLS, TEMP...) \
|
||||
OB_DEF_SERIALIZE_COMPAT_(CLS, ##TEMP)
|
||||
#define OB_DEF_DESERIALIZE_COMPAT(VER, CLS, TEMP...) \
|
||||
OB_DEF_DESERIALIZE_COMPAT_(CLS, ##TEMP)
|
||||
#define OB_DEF_SERIALIZE_SIZE_COMPAT(VER, CLS, TEMP...) \
|
||||
OB_DEF_SERIALIZE_SIZE_COMPAT_(CLS, ##TEMP)
|
||||
#define CAR(a, b) a
|
||||
#define CDR(a, b) b
|
||||
#define MY_CLS(CLS) IF_IS_PAREN(CLS, CAR CLS, CLS)
|
||||
#define BASE_CLS(CLS) IF_IS_PAREN(CLS, CDR CLS, EmptyParent)
|
||||
#define OB_SERIALIZE_MEMBER_TEMP(TEMP, CLS, ...) OB_SERIALIZE_MEMBER_TEMP_INHERIT(TEMP, MY_CLS(CLS), BASE_CLS(CLS), ##__VA_ARGS__)
|
||||
#define OB_SERIALIZE_MEMBER(CLS, ...) OB_SERIALIZE_MEMBER_TEMP(, CLS, ##__VA_ARGS__)
|
||||
|
||||
|
||||
//-----------------------------------------------------------------------
|
||||
|
||||
///
|
||||
// public entries, define interfaces of list encode/decode members
|
||||
//-----------------------------------------------------------------------
|
||||
#define OB_SERIALIZE_MEMBER_COMPAT_TEMP_IF(COMPAT, TEMP, CLS, PRED, ...) \
|
||||
OB_DEF_SERIALIZE##COMPAT(UNF_MYCLS(CLS), TEMP) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
UNF_UNUSED_SER; \
|
||||
BASE_SER(CLS); \
|
||||
if (PRED) { \
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, ##__VA_ARGS__); \
|
||||
} \
|
||||
return ret; \
|
||||
} \
|
||||
OB_DEF_DESERIALIZE##COMPAT(UNF_MYCLS(CLS), TEMP) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
UNF_UNUSED_DES; \
|
||||
BASE_DESER(CLS); \
|
||||
LST_DO_CODE(OB_UNIS_DECODE, ##__VA_ARGS__); \
|
||||
return ret; \
|
||||
} \
|
||||
OB_DEF_SERIALIZE_SIZE##COMPAT(UNF_MYCLS(CLS), TEMP) \
|
||||
{ \
|
||||
int64_t len = 0; \
|
||||
BASE_ADD_LEN(CLS); \
|
||||
if (PRED) { \
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, ##__VA_ARGS__); \
|
||||
} \
|
||||
return len; \
|
||||
/// utility macros to deal with C native array
|
||||
#define OB_UNIS_ENCODE_ARRAY(objs, objs_count) \
|
||||
OB_UNIS_ENCODE((objs_count)); \
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < (objs_count); ++i) { \
|
||||
OB_UNIS_ENCODE(objs[i]); \
|
||||
}
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_TEMP_IF(TEMP, CLS, PRED, ...) \
|
||||
OB_SERIALIZE_MEMBER_COMPAT_TEMP_IF(, TEMP, CLS, PRED, ##__VA_ARGS__)
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_TEMP_IF_COMPAT(VER, TEMP, CLS, PRED, ...) \
|
||||
OB_SERIALIZE_MEMBER_COMPAT_TEMP_IF(_COMPAT_, TEMP, CLS, PRED, ##__VA_ARGS__)
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_TEMP(TEMP, CLS, ...) \
|
||||
OB_SERIALIZE_MEMBER_TEMP_IF(TEMP, CLS, true, ##__VA_ARGS__)
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_TEMP_COMPAT(VER, TEMP, CLS, ...) \
|
||||
OB_SERIALIZE_MEMBER_TEMP_IF_COMPAT(VER, TEMP, CLS, true, ##__VA_ARGS__)
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_IF(CLS, PRED, ...) \
|
||||
OB_SERIALIZE_MEMBER_TEMP_IF(, CLS, PRED, ##__VA_ARGS__)
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_IF_COMPAT(VER, CLS, PRED, ...) \
|
||||
OB_SERIALIZE_MEMBER_TEMP_IF_COMPAT(VER, , CLS, PRED, ##__VA_ARGS__)
|
||||
|
||||
#define OB_SERIALIZE_MEMBER(CLS, ...) \
|
||||
OB_SERIALIZE_MEMBER_TEMP(, CLS, ##__VA_ARGS__)
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_COMPAT(VER, CLS, ...) \
|
||||
OB_SERIALIZE_MEMBER_TEMP_COMPAT(VER, , CLS, ##__VA_ARGS__)
|
||||
|
||||
#define OB_UNIS_DEF_SERIALIZE(CLS, ...) \
|
||||
OB_DEF_SERIALIZE(UNF_MYCLS(CLS),) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
UNF_UNUSED_SER; \
|
||||
BASE_SER(CLS); \
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, ##__VA_ARGS__); \
|
||||
return ret; \
|
||||
#define OB_UNIS_ADD_LEN_ARRAY(objs, objs_count) \
|
||||
OB_UNIS_ADD_LEN((objs_count)); \
|
||||
for (int64_t i = 0; i < (objs_count); ++i) { \
|
||||
OB_UNIS_ADD_LEN(objs[i]); \
|
||||
}
|
||||
|
||||
#define OB_UNIS_DEF_DESERIALIZE(CLS, ...) \
|
||||
OB_DEF_DESERIALIZE(UNF_MYCLS(CLS),) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
UNF_UNUSED_DES; \
|
||||
BASE_DESER(CLS); \
|
||||
LST_DO_CODE(OB_UNIS_DECODE, ##__VA_ARGS__); \
|
||||
return ret; \
|
||||
#define OB_UNIS_DECODE_ARRAY(objs, objs_count) \
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < (objs_count); ++i) { \
|
||||
OB_UNIS_DECODE(objs[i]); \
|
||||
}
|
||||
|
||||
#define OB_UNIS_DEF_SERIALIZE_SIZE(CLS, ...) \
|
||||
OB_DEF_SERIALIZE_SIZE(UNF_MYCLS(CLS),) \
|
||||
{ \
|
||||
int64_t len = 0; \
|
||||
BASE_ADD_LEN(CLS); \
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, ##__VA_ARGS__); \
|
||||
return len; \
|
||||
#define OB_UNIS_DEF_SERIALIZE(CLS, ...) \
|
||||
OB_UNIS_SERIALIZE(MY_CLS(CLS)); \
|
||||
OB_SERIALIZE_NOHEADER(MY_CLS(CLS), BASE_CLS(CLS), _, true, ##__VA_ARGS__);
|
||||
|
||||
#define OB_UNIS_DEF_SERIALIZE_SIZE(CLS, ...) \
|
||||
OB_UNIS_SERIALIZE_SIZE(MY_CLS(CLS)); \
|
||||
OB_SERIALIZE_SIZE_NOHEADER(MY_CLS(CLS), BASE_CLS(CLS), _, true, ##__VA_ARGS__);
|
||||
|
||||
#define BASE_ADD_LEN(Base) len += BASE_CLS(Base)::get_serialize_size()
|
||||
#define BASE_SER(Base) if (OB_SUCC(ret) && OB_FAIL(BASE_CLS(Base)::serialize(buf, buf_len, pos))) { \
|
||||
RPC_WARN("serialize base failed", K(ret)); \
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------
|
||||
// Compatibility not guaranteed, **DONT** use this any more.
|
||||
|
||||
#define OB_DEF_SERIALIZE_SIMPLE(CLS) \
|
||||
int CLS::serialize(SERIAL_PARAMS) const
|
||||
|
||||
#define OB_DEF_DESERIALIZE_SIMPLE(CLS) \
|
||||
int CLS::deserialize(DESERIAL_PARAMS)
|
||||
|
||||
#define OB_DEF_SERIALIZE_SIZE_SIMPLE(CLS) \
|
||||
int64_t CLS::get_serialize_size(void) const
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_SIMPLE(CLS, ...) \
|
||||
OB_DEF_SERIALIZE_SIMPLE(CLS) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, ##__VA_ARGS__); \
|
||||
return ret; \
|
||||
} \
|
||||
OB_DEF_DESERIALIZE_SIMPLE(CLS) \
|
||||
{ \
|
||||
int ret = OK_; \
|
||||
LST_DO_CODE(OB_UNIS_DECODE, ##__VA_ARGS__); \
|
||||
return ret; \
|
||||
} \
|
||||
OB_DEF_SERIALIZE_SIZE_SIMPLE(CLS) \
|
||||
{ \
|
||||
int64_t len = 0; \
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, ##__VA_ARGS__); \
|
||||
return len; \
|
||||
#define BASE_DESER(Base) if (OB_SUCC(ret) && OB_FAIL(BASE_CLS(Base)::deserialize(buf, data_len, pos))) { \
|
||||
RPC_WARN("deserialize base failed", K(ret)); \
|
||||
}
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_INHERIT(CLS, PARENT, ...) \
|
||||
OB_DEF_SERIALIZE_SIMPLE(CLS) \
|
||||
{ \
|
||||
int ret = PARENT::serialize(buf, buf_len, pos); \
|
||||
if (OB_SUCC(ret)) { \
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, ##__VA_ARGS__); \
|
||||
} \
|
||||
return ret; \
|
||||
} \
|
||||
OB_DEF_DESERIALIZE_SIMPLE(CLS) \
|
||||
{ \
|
||||
int ret = PARENT::deserialize(buf, data_len, pos); \
|
||||
if (OB_SUCC(ret)) { \
|
||||
LST_DO_CODE(OB_UNIS_DECODE, ##__VA_ARGS__); \
|
||||
} \
|
||||
return ret; \
|
||||
} \
|
||||
OB_DEF_SERIALIZE_SIZE_SIMPLE(CLS) \
|
||||
{ \
|
||||
int64_t len = PARENT::get_serialize_size(); \
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, ##__VA_ARGS__); \
|
||||
return len; \
|
||||
}
|
||||
|
||||
#define UNIS_VER(major, minor, patch) \
|
||||
(((uint64_t)major << 32L) + ((uint64_t)minor << 16L) + (uint64_t)patch)
|
||||
#define OB_UNIS_COMPAT(V) \
|
||||
private: \
|
||||
const static uint64_t compat_ver_ = UNIS_##V; \
|
||||
OB_DECLARE_UNIS_COMPAT()
|
||||
|
||||
#define SERIALIZE_SIZE_HEADER(version) \
|
||||
OB_UNIS_ADD_LEN(version); \
|
||||
len += NS_::OB_SERIALIZE_SIZE_NEED_BYTES;
|
||||
// Define dummy node to replace obsolete member.
|
||||
namespace oceanbase { namespace lib {
|
||||
template <int N>
|
||||
struct UNFDummy {
|
||||
struct UNFDummy {
|
||||
OB_UNIS_VERSION(N);
|
||||
};
|
||||
OB_SERIALIZE_MEMBER_TEMP(template<int N>, UNFDummy<N>);
|
||||
|
||||
#define OB_SERIALIZE_MEMBER_IF(CLS, PRED, ...) \
|
||||
OB_UNIS_SERIALIZE(CLS); \
|
||||
OB_UNIS_DESERIALIZE(CLS); \
|
||||
OB_UNIS_SERIALIZE_SIZE(CLS); \
|
||||
OB_SERIALIZE_NOHEADER(CLS, EmptyParent, _, PRED, ##__VA_ARGS__); \
|
||||
OB_DESERIALIZE_NOHEADER(CLS, EmptyParent, _,##__VA_ARGS__); \
|
||||
OB_SERIALIZE_SIZE_NOHEADER(CLS, EmptyParent, _, PRED, ##__VA_ARGS__);
|
||||
|
||||
inline uint64_t &get_unis_compat_version()
|
||||
{
|
||||
static uint64_t x;
|
||||
return x;
|
||||
}
|
||||
|
||||
#define UNIS_VERSION_GUARD(x)
|
||||
}} // namespace oceanbase::lib
|
||||
|
||||
#endif /* _OCEABASE_LIB_UTILITY_OB_UNIFY_SERIALIZE_H_ */
|
||||
|
1
deps/oblib/src/lib/utility/utility.h
vendored
1
deps/oblib/src/lib/utility/utility.h
vendored
@ -18,6 +18,7 @@
|
||||
#include "easy_define.h"
|
||||
#include "io/easy_io_struct.h"
|
||||
#include "lib/allocator/ob_allocator.h"
|
||||
#include "lib/lock/ob_spin_lock.h"
|
||||
#include "lib/stat/ob_diagnose_info.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/utility/ob_utility.h"
|
||||
|
9
deps/oblib/src/lib/worker.cpp
vendored
9
deps/oblib/src/lib/worker.cpp
vendored
@ -44,23 +44,20 @@ __thread Worker *Worker::self_;
|
||||
|
||||
Worker::Worker()
|
||||
: allocator_(nullptr),
|
||||
st_current_priority_(0),
|
||||
session_(nullptr),
|
||||
cur_request_(nullptr),
|
||||
is_blocking_(false),
|
||||
worker_level_(INT32_MAX),
|
||||
curr_request_level_(0),
|
||||
group_id_(0),
|
||||
rpc_stat_srv_(nullptr),
|
||||
st_current_priority_(0),
|
||||
session_(nullptr),
|
||||
timeout_ts_(INT64_MAX),
|
||||
ntp_offset_(0),
|
||||
rpc_tenant_id_(0),
|
||||
tidx_(-1),
|
||||
large_token_expired_(0),
|
||||
disable_wait_(false)
|
||||
{
|
||||
worker_node_.get_data() = this;
|
||||
lq_worker_node_.get_data() = this;
|
||||
lq_waiting_worker_node_.get_data() = this;
|
||||
if (OB_ISNULL(self_)) {
|
||||
self_ = this;
|
||||
} else {
|
||||
|
214
deps/oblib/src/lib/worker.h
vendored
214
deps/oblib/src/lib/worker.h
vendored
@ -15,9 +15,7 @@
|
||||
|
||||
#include <cstdint>
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "lib/rc/context.h"
|
||||
#include "lib/allocator/ob_fifo_allocator.h"
|
||||
#include "lib/runtime.h"
|
||||
|
||||
namespace oceanbase
|
||||
@ -26,9 +24,7 @@ namespace rpc { class ObRequest; }
|
||||
namespace sql { class ObSQLSessionInfo; }
|
||||
namespace lib
|
||||
{
|
||||
using common::ObArenaAllocator;
|
||||
using common::ObIAllocator;
|
||||
using common::ObFIFOAllocator;
|
||||
|
||||
class Worker
|
||||
{
|
||||
@ -41,6 +37,16 @@ public:
|
||||
|
||||
virtual Status check_wait();
|
||||
virtual int check_status() { check_wait(); return common::OB_SUCCESS; }
|
||||
virtual int check_large_query_quota() { return common::OB_SUCCESS; }
|
||||
// check if retry disabled for the query
|
||||
virtual bool can_retry() const { return false; }
|
||||
// Set retry flag so that scheduler will reprocess this request then
|
||||
virtual void set_need_retry() {}
|
||||
// It's used to check whether query need retry. Whenever worker has
|
||||
// observed this query need retry, it should stop processing this
|
||||
// query immediately.
|
||||
virtual bool need_retry() const { return false; }
|
||||
virtual void resume() {}
|
||||
|
||||
// This function is called before worker waiting for some resources
|
||||
// and starting to give cpu out so that Multi-Tenancy would be aware
|
||||
@ -61,123 +67,78 @@ public:
|
||||
// 2. false the worker hasn't right to go ahead
|
||||
bool sched_run(int64_t waittime=0);
|
||||
|
||||
ObIAllocator &get_sql_arena_allocator() ;
|
||||
OB_INLINE ObIAllocator& get_sql_arena_allocator() { return CURRENT_CONTEXT->get_arena_allocator(); }
|
||||
ObIAllocator &get_allocator() ;
|
||||
|
||||
void set_req_flag(const rpc::ObRequest *cur_request);
|
||||
bool has_req_flag();
|
||||
const rpc::ObRequest *get_cur_request();
|
||||
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_; }
|
||||
|
||||
void set_worker_level(const int32_t level) { worker_level_ = level; }
|
||||
int32_t get_worker_level() const { return worker_level_; }
|
||||
OB_INLINE void set_is_blocking(bool v) { is_blocking_ = v; }
|
||||
OB_INLINE bool is_blocking() { return is_blocking_; }
|
||||
|
||||
void set_curr_request_level(const int32_t level) { curr_request_level_ = level; }
|
||||
int32_t get_curr_request_level() const { return curr_request_level_; }
|
||||
OB_INLINE void set_worker_level(const int32_t level) { worker_level_ = level; }
|
||||
OB_INLINE int32_t get_worker_level() const { return worker_level_; }
|
||||
|
||||
void set_group_id(int32_t group_id) { group_id_ = group_id; }
|
||||
int32_t get_group_id() const { return group_id_; }
|
||||
OB_INLINE void set_curr_request_level(const int32_t level) { curr_request_level_ = level; }
|
||||
OB_INLINE int32_t get_curr_request_level() const { return curr_request_level_; }
|
||||
|
||||
void set_rpc_stat_srv(void *rpc_stat_srv) { rpc_stat_srv_ = rpc_stat_srv; }
|
||||
void *get_rpc_stat_srv() const { return rpc_stat_srv_; }
|
||||
OB_INLINE void set_group_id(int32_t group_id) { group_id_ = group_id; }
|
||||
OB_INLINE int32_t get_group_id() const { return group_id_; }
|
||||
|
||||
virtual int check_large_query_quota()
|
||||
{
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
OB_INLINE void set_rpc_stat_srv(void *rpc_stat_srv) { rpc_stat_srv_ = rpc_stat_srv; }
|
||||
OB_INLINE void *get_rpc_stat_srv() const { return rpc_stat_srv_; }
|
||||
|
||||
static void set_compatibility_mode(CompatMode mode);
|
||||
static CompatMode get_compatibility_mode();
|
||||
|
||||
bool is_timeout_ts_valid()
|
||||
{ return INT64_MAX != timeout_ts_;}
|
||||
void set_timeout_ts(int64_t timeout_ts) { timeout_ts_ = timeout_ts; }
|
||||
int64_t get_timeout_ts() const { return timeout_ts_; }
|
||||
void set_ntp_offset(int64_t offset) { ntp_offset_ = offset; }
|
||||
int64_t get_ntp_offset() const { return ntp_offset_; }
|
||||
OB_INLINE bool is_timeout_ts_valid() { return INT64_MAX != timeout_ts_;}
|
||||
OB_INLINE void set_timeout_ts(int64_t timeout_ts) { timeout_ts_ = timeout_ts; }
|
||||
OB_INLINE int64_t get_timeout_ts() const { return timeout_ts_; }
|
||||
OB_INLINE void set_ntp_offset(int64_t offset) { ntp_offset_ = offset; }
|
||||
OB_INLINE int64_t get_ntp_offset() const { return ntp_offset_; }
|
||||
int64_t get_timeout_remain() const;
|
||||
bool is_timeout() const;
|
||||
|
||||
void set_rpc_tenant(uint64_t tenant_id) { rpc_tenant_id_ = tenant_id; }
|
||||
void reset_rpc_tenant() { rpc_tenant_id_ = 0; }
|
||||
uint64_t get_rpc_tenant() const { return rpc_tenant_id_; }
|
||||
|
||||
ObIAllocator &ssstore_allocator();
|
||||
ObFIFOAllocator &ssstore_fifo_allocator();
|
||||
|
||||
void set_tidx(int64_t tidx);
|
||||
void unset_tidx();
|
||||
int64_t get_tidx() const;
|
||||
|
||||
// It's called when current query can't been retry, maybe some
|
||||
// shared states have been published such as part of result has been
|
||||
// sent to client, operation is doing or even has done and can't
|
||||
// restart easily. It doesn't mean this query won't retry, but
|
||||
// indicates following retry flag setting is forbade,
|
||||
virtual void disable_retry();
|
||||
// check if retry disabled for the query
|
||||
virtual bool can_retry() const;
|
||||
// Set retry flag so that scheduler will reprocess this request then
|
||||
virtual void set_need_retry();
|
||||
// It's used to check whether query need retry. Whenever worker has
|
||||
// observed this query need retry, it should stop processing this
|
||||
// query immediately.
|
||||
virtual bool need_retry() const;
|
||||
|
||||
// Set large token expired timestamp.
|
||||
//
|
||||
// Worker prefer process large queries than normal queries until
|
||||
// token expires, that is the timestamp is larger than now.
|
||||
void set_large_token_expired(int64_t timestamp);
|
||||
|
||||
// Get large token expired timestamp.
|
||||
int64_t get_large_token_expired() const;
|
||||
OB_INLINE void set_rpc_tenant(uint64_t tenant_id) { rpc_tenant_id_ = tenant_id; }
|
||||
OB_INLINE void reset_rpc_tenant() { rpc_tenant_id_ = 0; }
|
||||
OB_INLINE uint64_t get_rpc_tenant() const { return rpc_tenant_id_; }
|
||||
|
||||
// check wait is disabled if f is true
|
||||
void set_disable_wait_flag(bool f);
|
||||
bool get_disable_wait_flag() const;
|
||||
|
||||
common::ObDLinkNode<Worker*> worker_node_;
|
||||
common::ObDLinkNode<Worker*> lq_worker_node_;
|
||||
common::ObDLinkNode<Worker*> lq_waiting_worker_node_;
|
||||
common::ObLink wpool_link_;
|
||||
|
||||
virtual void resume() {}
|
||||
void set_disable_wait_flag(bool f) { disable_wait_ = f; }
|
||||
bool get_disable_wait_flag() const { return disable_wait_; }
|
||||
|
||||
void set_sql_throttle_current_priority(int64_t st_current_priority)
|
||||
{ st_current_priority_ = st_current_priority; }
|
||||
void reset_sql_throttle_current_priority()
|
||||
{ set_sql_throttle_current_priority(100); }
|
||||
|
||||
void set_session(sql::ObSQLSessionInfo* session)
|
||||
{
|
||||
session_ = session;
|
||||
}
|
||||
OB_INLINE void set_session(sql::ObSQLSessionInfo* session) { session_ = session; }
|
||||
|
||||
public:
|
||||
static void set_compatibility_mode(CompatMode mode);
|
||||
static CompatMode get_compatibility_mode();
|
||||
static Worker& self();
|
||||
|
||||
public:
|
||||
static __thread Worker *self_;
|
||||
|
||||
public:
|
||||
// static variables
|
||||
static Worker& self();
|
||||
|
||||
common::ObDLinkNode<Worker*> worker_node_;
|
||||
protected:
|
||||
// 线程运行时内存从此分配器分配
|
||||
// 初始tenant_id=500, 在处理request时,tenant_id被更新成request的租户id
|
||||
// 可单独指定ctx_id, 此ctx_id保持不变
|
||||
ObIAllocator *allocator_;
|
||||
int64_t st_current_priority_;
|
||||
sql::ObSQLSessionInfo *session_;
|
||||
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_;
|
||||
void *rpc_stat_srv_;
|
||||
|
||||
protected:
|
||||
int64_t st_current_priority_;
|
||||
sql::ObSQLSessionInfo *session_;
|
||||
|
||||
private:
|
||||
int64_t timeout_ts_;
|
||||
|
||||
//ingnore net time, equal to (receive_ts - send_ts).
|
||||
@ -185,12 +146,6 @@ private:
|
||||
|
||||
uint64_t rpc_tenant_id_;
|
||||
|
||||
// worker index in its tenant
|
||||
int64_t tidx_;
|
||||
|
||||
// timestamp when large token expires.
|
||||
int64_t large_token_expired_;
|
||||
|
||||
// Used to prevent the thread holding the lock from being suspended by check_wait
|
||||
bool disable_wait_;
|
||||
|
||||
@ -208,79 +163,6 @@ inline Worker &Worker::self()
|
||||
return *self_;
|
||||
}
|
||||
|
||||
inline void Worker::set_tidx(int64_t tidx)
|
||||
{
|
||||
tidx_ = tidx;
|
||||
}
|
||||
|
||||
inline void Worker::unset_tidx()
|
||||
{
|
||||
tidx_ = -1;
|
||||
}
|
||||
|
||||
inline int64_t Worker::get_tidx() const
|
||||
{
|
||||
return tidx_;
|
||||
}
|
||||
|
||||
inline void Worker::disable_retry()
|
||||
{
|
||||
}
|
||||
|
||||
inline bool Worker::can_retry() const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
inline void Worker::set_need_retry()
|
||||
{
|
||||
}
|
||||
|
||||
inline bool Worker::need_retry() const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
inline void Worker::set_large_token_expired(int64_t timestamp)
|
||||
{
|
||||
large_token_expired_ = timestamp;
|
||||
}
|
||||
|
||||
inline int64_t Worker::get_large_token_expired() const
|
||||
{
|
||||
return large_token_expired_;
|
||||
}
|
||||
|
||||
inline void Worker::set_disable_wait_flag(bool f)
|
||||
{
|
||||
disable_wait_ = f;
|
||||
}
|
||||
|
||||
inline bool Worker::get_disable_wait_flag() const
|
||||
{
|
||||
return disable_wait_;
|
||||
}
|
||||
|
||||
inline void Worker::set_req_flag(const rpc::ObRequest *cur_request)
|
||||
{
|
||||
cur_request_ = cur_request;
|
||||
}
|
||||
|
||||
inline bool Worker::has_req_flag()
|
||||
{
|
||||
return (NULL != cur_request_);
|
||||
}
|
||||
|
||||
inline const rpc::ObRequest *Worker::get_cur_request()
|
||||
{
|
||||
return cur_request_;
|
||||
}
|
||||
|
||||
inline ObIAllocator &Worker::get_sql_arena_allocator()
|
||||
{
|
||||
return CURRENT_CONTEXT->get_arena_allocator();
|
||||
}
|
||||
|
||||
inline ObIAllocator &Worker::get_allocator()
|
||||
{
|
||||
// 预期只在处理请求过程中调用
|
||||
@ -365,16 +247,16 @@ inline bool is_mysql_mode()
|
||||
return get_compat_mode() == Worker::CompatMode::MYSQL;
|
||||
}
|
||||
|
||||
inline void Worker::set_compatibility_mode(Worker::CompatMode mode)
|
||||
OB_INLINE void Worker::set_compatibility_mode(Worker::CompatMode mode)
|
||||
{
|
||||
set_compat_mode(mode);
|
||||
}
|
||||
inline Worker::CompatMode Worker::get_compatibility_mode()
|
||||
|
||||
OB_INLINE Worker::CompatMode Worker::get_compatibility_mode()
|
||||
{
|
||||
return get_compat_mode();
|
||||
}
|
||||
|
||||
|
||||
} // end of namespace lib
|
||||
} // end of namespace oceanbase
|
||||
|
||||
|
1
deps/oblib/src/rpc/CMakeLists.txt
vendored
1
deps/oblib/src/rpc/CMakeLists.txt
vendored
@ -62,7 +62,6 @@ ob_set_subtarget(oblib_rpc obrpc
|
||||
obrpc/ob_net_keepalive.cpp
|
||||
obrpc/ob_listener.cpp
|
||||
obrpc/ob_net_client.cpp
|
||||
obrpc/ob_poc_nio.cpp
|
||||
obrpc/ob_poc_rpc_proxy.cpp
|
||||
obrpc/ob_poc_rpc_request_operator.cpp
|
||||
obrpc/ob_poc_rpc_server.cpp
|
||||
|
20
deps/oblib/src/rpc/frame/ob_net_easy.cpp
vendored
20
deps/oblib/src/rpc/frame/ob_net_easy.cpp
vendored
@ -14,6 +14,7 @@
|
||||
|
||||
#include "io/easy_io.h"
|
||||
#include "rpc/frame/ob_net_easy.h"
|
||||
#include "rpc/obrpc/ob_poc_rpc_server.h"
|
||||
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/utility/utility.h"
|
||||
@ -796,15 +797,20 @@ int ObNetEasy::start()
|
||||
LOG_ERROR("eio is NULL, not inited", K(is_inited_), KP_(rpc_eio), KP_(mysql_eio), KP_(batch_rpc_eio), K(ret));
|
||||
}
|
||||
|
||||
// TODO: rpc_port_ might not in use
|
||||
global_ob_listener = &rpc_listener_;
|
||||
if (OB_SUCC(ret) && rpc_port_ > 0) {
|
||||
if (OB_FAIL(rpc_listener_.listen_create(rpc_port_))) {
|
||||
LOG_ERROR("create listen failed", K(ret));
|
||||
} else if (OB_FAIL(rpc_listener_.start())) {
|
||||
LOG_ERROR("oblistener start failed!", K(rpc_port_), K(ret));
|
||||
} else if (OB_FAIL(ObNetKeepAlive::get_instance().start())) {
|
||||
LOG_ERROR("oblistener start failed!", K(rpc_port_), K(ret));
|
||||
if (!global_poc_server.has_start()) {
|
||||
if (OB_FAIL(rpc_listener_.listen_create(rpc_port_))) {
|
||||
LOG_ERROR("create listen failed", K(ret));
|
||||
} else if (OB_FAIL(rpc_listener_.start())) {
|
||||
LOG_ERROR("oblistener start failed!", K(rpc_port_), K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(ObNetKeepAlive::get_instance().start())) {
|
||||
LOG_ERROR("ObNetKeepAlive start failed!", K(rpc_port_), K(ret));
|
||||
} else {
|
||||
LOG_INFO("oblistener start!", K(rpc_port_));
|
||||
LOG_INFO("ObNetKeepAlive start!", K(rpc_port_));
|
||||
}
|
||||
}
|
||||
|
||||
|
1
deps/oblib/src/rpc/frame/ob_req_deliver.h
vendored
1
deps/oblib/src/rpc/frame/ob_req_deliver.h
vendored
@ -45,6 +45,7 @@ class ObReqQDeliver
|
||||
{
|
||||
public:
|
||||
explicit ObReqQDeliver(ObiReqQHandler &qhandler);
|
||||
ObiReqQHandler &get_qhandler() { return qhandler_; }
|
||||
|
||||
protected:
|
||||
ObiReqQHandler &qhandler_;
|
||||
|
@ -148,7 +148,7 @@ void ObReqQueue::loop()
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_ERROR("invalid argument", K(qhandler_));
|
||||
} else if (OB_FAIL(qhandler_->onThreadCreated(nullptr))) {
|
||||
LOG_ERROR("do thread craeted fail, thread will exit", K(ret));
|
||||
LOG_ERROR("do thread created fail, thread will exit", K(ret));
|
||||
} else {
|
||||
// The main loop threads process tasks.
|
||||
while (!Thread::current().has_set_stop()) {
|
||||
|
3
deps/oblib/src/rpc/frame/ob_req_transport.h
vendored
3
deps/oblib/src/rpc/frame/ob_req_transport.h
vendored
@ -93,7 +93,7 @@ public:
|
||||
{
|
||||
public:
|
||||
AsyncCB(int pcode)
|
||||
: dst_(), timeout_(0), tenant_id_(0),
|
||||
: low_level_cb_(NULL), dst_(), timeout_(0), tenant_id_(0),
|
||||
err_(0), pcode_(pcode), send_ts_(0), payload_(0)
|
||||
{}
|
||||
virtual ~AsyncCB() {}
|
||||
@ -124,6 +124,7 @@ public:
|
||||
void set_payload(const int64_t payload) { payload_ = payload; }
|
||||
int64_t get_payload() { return payload_; }
|
||||
|
||||
void* low_level_cb_;
|
||||
private:
|
||||
static const int64_t REQUEST_ITEM_COST_RT = 100 * 1000; // 100ms
|
||||
protected:
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/allocator/ob_mod_define.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "rpc/obmysql/ob_mysql_packet.h"
|
||||
|
||||
namespace oceanbase
|
||||
|
154
deps/oblib/src/rpc/obmysql/ob_sql_nio.cpp
vendored
154
deps/oblib/src/rpc/obmysql/ob_sql_nio.cpp
vendored
@ -92,10 +92,9 @@ struct ReadyFlag
|
||||
int32_t pending_ CACHE_ALIGNED;
|
||||
};
|
||||
|
||||
#define futex(...) syscall(SYS_futex,__VA_ARGS__)
|
||||
static int futex_wake(volatile int *p, int val)
|
||||
{
|
||||
return static_cast<int>(futex((int *)p, FUTEX_WAKE_PRIVATE, val, NULL, NULL, 0));
|
||||
return futex((uint *)p, FUTEX_WAKE_PRIVATE, val, NULL);
|
||||
}
|
||||
|
||||
struct SingleWaitCond
|
||||
@ -330,7 +329,7 @@ private:
|
||||
class ObSqlSock: public ObLink
|
||||
{
|
||||
public:
|
||||
ObSqlSock(ObSqlNioImpl& nio, int fd): nio_impl_(nio), fd_(fd), err_(0), read_buffer_(fd),
|
||||
ObSqlSock(ObSqlNioImpl *nio, int fd): nio_impl_(nio), fd_(fd), err_(0), read_buffer_(fd),
|
||||
need_epoll_trigger_write_(false), may_handling_(true), handler_close_flag_(false),
|
||||
need_shutdown_(false), last_decode_time_(0), last_write_time_(0), sql_session_info_(NULL) {
|
||||
memset(sess_, 0, sizeof(sess_));
|
||||
@ -339,7 +338,8 @@ public:
|
||||
int64_t get_remain_sz() const { return read_buffer_.get_remain_sz(); }
|
||||
TO_STRING_KV(KP(this), K_(fd), K_(err), K(last_decode_time_), K(last_write_time_),
|
||||
K(read_buffer_.get_consume_sz()), K(get_pending_flag()), KPC(get_trace_id()));
|
||||
ObSqlNioImpl& get_nio_impl() { return nio_impl_; }
|
||||
ObSqlNioImpl *get_nio_impl() { return nio_impl_; }
|
||||
void set_nio_impl(ObSqlNioImpl *impl) { nio_impl_ = impl; }
|
||||
bool set_error(int err) { return 0 == ATOMIC_TAS(&err_, err); }
|
||||
bool has_error() const { return ATOMIC_LOAD(&err_) != 0; }
|
||||
|
||||
@ -439,7 +439,7 @@ public:
|
||||
ObDLink all_list_link_;
|
||||
ObLink write_task_link_;
|
||||
private:
|
||||
ObSqlNioImpl& nio_impl_;
|
||||
ObSqlNioImpl *nio_impl_;
|
||||
int fd_;
|
||||
int err_;
|
||||
ReadBuffer read_buffer_;
|
||||
@ -457,6 +457,11 @@ public:
|
||||
char sess_[3000] __attribute__((aligned(16)));
|
||||
};
|
||||
|
||||
static ObSqlSock *sess2sock(void *sess)
|
||||
{
|
||||
return CONTAINER_OF(sess, ObSqlSock, sess_);
|
||||
}
|
||||
|
||||
int ObSqlSock::set_ssl_enabled()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -584,8 +589,41 @@ public:
|
||||
ObSqlNioImpl(ObISqlSockHandler& handler):
|
||||
handler_(handler), epfd_(-1), lfd_(-1), tcp_keepalive_enabled_(0),
|
||||
tcp_keepidle_(0), tcp_keepintvl_(0), tcp_keepcnt_(0) {}
|
||||
~ObSqlNioImpl() {}
|
||||
~ObSqlNioImpl() {
|
||||
destroy();
|
||||
}
|
||||
void destroy() {
|
||||
ObDLink *head = all_list_.head();
|
||||
ObLink *cur = head->next_;
|
||||
while (cur != head) {
|
||||
ObSqlSock *s = CONTAINER_OF(cur, ObSqlSock, all_list_link_);
|
||||
cur = cur->next_;
|
||||
free_sql_sock(s);
|
||||
}
|
||||
}
|
||||
int init(int port) {
|
||||
int ret = OB_SUCCESS;
|
||||
if (port == -1) {
|
||||
ret = init_io();
|
||||
} else {
|
||||
ret = init_listen(port);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int init_io() {
|
||||
int ret = OB_SUCCESS;
|
||||
uint32_t epflag = EPOLLIN;
|
||||
if ((epfd_ = epoll_create1(EPOLL_CLOEXEC)) < 0) {
|
||||
ret = OB_IO_ERROR;
|
||||
LOG_WARN("epoll_create fail", K(ret), K(errno));
|
||||
} else if (OB_FAIL(evfd_.create(epfd_))) {
|
||||
LOG_WARN("evfd create fail", K(ret));
|
||||
} else {
|
||||
LOG_INFO("sql_nio init io succ");
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int init_listen(int port) {
|
||||
int ret = OB_SUCCESS;
|
||||
uint32_t epflag = EPOLLIN;
|
||||
if ((epfd_ = epoll_create1(EPOLL_CLOEXEC)) < 0) {
|
||||
@ -600,7 +638,7 @@ public:
|
||||
} else if (OB_FAIL(evfd_.create(epfd_))) {
|
||||
LOG_WARN("evfd create fail", K(ret));
|
||||
} else {
|
||||
LOG_INFO("sql_nio listen succ", K(port));
|
||||
LOG_INFO("sql_nio init listen succ", K(port));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -618,6 +656,25 @@ public:
|
||||
update_tcp_keepalive_parameters();
|
||||
print_session_info();
|
||||
}
|
||||
int regist_sess(void *sess) {
|
||||
int err = 0;
|
||||
ObSqlSock *sock = sess2sock(sess);
|
||||
int fd = sock->get_fd();
|
||||
uint32_t epflag = EPOLLIN | EPOLLOUT | EPOLLERR | EPOLLET | EPOLLRDHUP;
|
||||
ObSqlNioImpl *nio_impl = sock->get_nio_impl();
|
||||
sock->remove_fd_from_epoll(nio_impl->get_epfd());
|
||||
nio_impl->remove_session_info(sock);
|
||||
record_session_info(sock);
|
||||
sock->set_nio_impl(this);
|
||||
if (0 != (err = epoll_regist(epfd_, fd, epflag, sock))) {
|
||||
LOG_WARN_RET(OB_ERR_SYS, "epoll_regist fail", K(fd), K(err));
|
||||
}
|
||||
if (0 != err && NULL != sock) {
|
||||
ObSqlSockSession *sess = (ObSqlSockSession *)sock->sess_;
|
||||
sess->destroy_sock();
|
||||
}
|
||||
return err;
|
||||
}
|
||||
void push_close_req(ObSqlSock* s) {
|
||||
if (s->set_error(EIO)) {
|
||||
LOG_WARN_RET(OB_ERR_SYS, "close sql sock by user req", K(*s));
|
||||
@ -724,7 +781,6 @@ private:
|
||||
} else {
|
||||
if (true == s->sql_session_info_is_null()) {
|
||||
pending_destroy_list_.del(&s->dlink_);
|
||||
remove_session_info(s);
|
||||
s->do_close();
|
||||
free_sql_sock(s);
|
||||
}
|
||||
@ -832,13 +888,14 @@ private:
|
||||
ObSqlSock* alloc_sql_sock(int fd) {
|
||||
ObSqlSock* s = NULL;
|
||||
if (NULL != (s = (ObSqlSock*)direct_alloc(sizeof(*s)))) {
|
||||
new(s)ObSqlSock(*this, fd);
|
||||
new (s) ObSqlSock(this, fd);
|
||||
record_session_info(s);
|
||||
}
|
||||
return s;
|
||||
}
|
||||
void free_sql_sock(ObSqlSock* s) {
|
||||
if (NULL != s) {
|
||||
remove_session_info(s);
|
||||
s->~ObSqlSock();
|
||||
direct_free(s);
|
||||
}
|
||||
@ -893,6 +950,7 @@ private:
|
||||
static void* direct_alloc(int64_t sz) { return common::ob_malloc(sz, common::ObModIds::OB_COMMON_NETWORK); }
|
||||
static void direct_free(void* p) { common::ob_free(p); }
|
||||
|
||||
int get_epfd(){return epfd_;}
|
||||
private:
|
||||
ObISqlSockHandler& handler_;
|
||||
int epfd_;
|
||||
@ -908,21 +966,28 @@ private:
|
||||
uint32_t tcp_keepcnt_;
|
||||
};
|
||||
|
||||
int ObSqlNio::start(int port, ObISqlSockHandler* handler, int n_thread)
|
||||
int ObSqlNio::start(int port, ObISqlSockHandler *handler, int n_thread,
|
||||
const uint64_t tenant_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (NULL == (impl_ = (typeof(impl_))ob_malloc(sizeof(ObSqlNioImpl) * n_thread, "SqlNio"))) {
|
||||
port_ = port;
|
||||
handler_ = handler;
|
||||
tenant_id_ = tenant_id;
|
||||
if (n_thread > MAX_THREAD_CNT) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
} else if (NULL == (impl_ = (typeof(impl_))ob_malloc(
|
||||
sizeof(ObSqlNioImpl) * MAX_THREAD_CNT, "SqlNio"))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("alloc sql nio fail", K(ret));
|
||||
} else {
|
||||
for(int i = 0; OB_SUCCESS == ret && i < n_thread; i++) {
|
||||
new(impl_ + i)ObSqlNioImpl(*handler);
|
||||
for (int i = 0; OB_SUCCESS == ret && i < n_thread; i++) {
|
||||
new (impl_ + i) ObSqlNioImpl(*handler);
|
||||
if (OB_FAIL(impl_[i].init(port))) {
|
||||
LOG_WARN("impl init fail", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
set_thread_count(n_thread);
|
||||
lib::Threads::set_thread_count(n_thread);
|
||||
lib::Threads::start();
|
||||
}
|
||||
}
|
||||
@ -941,14 +1006,23 @@ void ObSqlNio::wait()
|
||||
|
||||
void ObSqlNio::destroy()
|
||||
{
|
||||
for (int i = 0; i < get_thread_count(); i++) {
|
||||
impl_[i].destroy();
|
||||
}
|
||||
}
|
||||
|
||||
int __attribute__((weak)) sql_nio_add_cgroup(const uint64_t tenant_id)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
void ObSqlNio::run(int64_t idx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (NULL != impl_) {
|
||||
lib::set_thread_name("sql_nio", idx);
|
||||
while(!has_set_stop()) {
|
||||
// if (tenant_id_ != common::OB_INVALID_ID) {
|
||||
// obmysql::sql_nio_add_cgroup(tenant_id_);
|
||||
// }
|
||||
while(!has_set_stop() && !(OB_NOT_NULL(&lib::Thread::current()) ? lib::Thread::current().has_set_stop() : false)) {
|
||||
impl_[idx].do_work();
|
||||
}
|
||||
if (has_set_stop()) {
|
||||
@ -957,21 +1031,16 @@ void ObSqlNio::run(int64_t idx)
|
||||
}
|
||||
}
|
||||
|
||||
static ObSqlSock* sess2sock(void* sess)
|
||||
{
|
||||
return CONTAINER_OF(sess, ObSqlSock, sess_);
|
||||
}
|
||||
|
||||
void ObSqlNio::destroy_sock(void* sess)
|
||||
{
|
||||
ObSqlSock* sock = sess2sock(sess);
|
||||
sock->get_nio_impl().push_close_req(sock);
|
||||
sock->get_nio_impl()->push_close_req(sock);
|
||||
}
|
||||
|
||||
void ObSqlNio::revert_sock(void* sess)
|
||||
{
|
||||
ObSqlSock* sock = sess2sock(sess);
|
||||
sock->get_nio_impl().revert_sock(sock);
|
||||
sock->get_nio_impl()->revert_sock(sock);
|
||||
}
|
||||
|
||||
void ObSqlNio::set_shutdown(void* sess)
|
||||
@ -1027,7 +1096,7 @@ void ObSqlNio::async_write_data(void* sess, const char* buf, int64_t sz)
|
||||
{
|
||||
ObSqlSock* sock = sess2sock(sess);
|
||||
sock->init_write_task(buf, sz);
|
||||
sock->get_nio_impl().push_write_req(sock);
|
||||
sock->get_nio_impl()->push_write_req(sock);
|
||||
}
|
||||
|
||||
int ObSqlNio::set_ssl_enabled(void* sess)
|
||||
@ -1042,6 +1111,43 @@ SSL* ObSqlNio::get_ssl_st(void* sess)
|
||||
return sock->get_ssl_st();
|
||||
}
|
||||
|
||||
int ObSqlNio::set_thread_count(const int n_thread)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int cur_thread = get_thread_count();
|
||||
if (n_thread > MAX_THREAD_CNT) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
} else if (n_thread == cur_thread) {
|
||||
// do nothing
|
||||
} else {
|
||||
if (n_thread > cur_thread) {
|
||||
for (int i = cur_thread; OB_SUCCESS == ret && i < n_thread; i++) {
|
||||
new (impl_ + i) ObSqlNioImpl(*handler_);
|
||||
if (OB_FAIL(impl_[i].init(port_))) {
|
||||
LOG_WARN("impl init fail");
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
lib::Threads::set_thread_count(n_thread);
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("decrease thread count not allowed", K(cur_thread),
|
||||
K(n_thread));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSqlNio::regist_sess(void *sess)
|
||||
{
|
||||
int err = 0;
|
||||
((ObSqlSockSession *)sess)->nio_ = this;
|
||||
if (0 != (err = impl_[get_dispatch_idx()].regist_sess(sess))) {
|
||||
LOG_ERROR_RET(OB_ERR_SYS, "regist sess fd fail", K(err));
|
||||
}
|
||||
return err;
|
||||
}
|
||||
|
||||
void ObSqlNio::update_tcp_keepalive_params(int keepalive_enabled, uint32_t tcp_keepidle, uint32_t tcp_keepintvl, uint32_t tcp_keepcnt)
|
||||
{
|
||||
int thread_count = get_thread_count();
|
||||
|
21
deps/oblib/src/rpc/obmysql/ob_sql_nio.h
vendored
21
deps/oblib/src/rpc/obmysql/ob_sql_nio.h
vendored
@ -17,6 +17,8 @@
|
||||
#include "lib/thread/threads.h"
|
||||
#include "lib/ssl/ob_ssl_config.h"
|
||||
|
||||
#define MAX_THREAD_CNT 64
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace obmysql
|
||||
@ -26,9 +28,12 @@ class ObISqlSockHandler;
|
||||
class ObSqlNio: public lib::Threads
|
||||
{
|
||||
public:
|
||||
ObSqlNio(): impl_(NULL) {}
|
||||
ObSqlNio()
|
||||
: impl_(NULL), port_(0), handler_(NULL), dispatch_idx_(0),
|
||||
tenant_id_(common::OB_INVALID_ID) {}
|
||||
virtual ~ObSqlNio() {}
|
||||
int start(int port, ObISqlSockHandler* handler, int n_thread);
|
||||
int start(int port, ObISqlSockHandler *handler, int n_thread,
|
||||
const uint64_t tenant_id);
|
||||
bool has_error(void* sess);
|
||||
void destroy_sock(void* sess);
|
||||
void revert_sock(void* sess);
|
||||
@ -46,13 +51,23 @@ public:
|
||||
void shutdown(void* sess);
|
||||
int set_ssl_enabled(void* sess);
|
||||
SSL* get_ssl_st(void* sess);
|
||||
int get_thread_count() { return lib::Threads::get_thread_count(); }
|
||||
int set_thread_count(const int n_thread);
|
||||
int regist_sess(void *sess);
|
||||
uint64_t get_dispatch_idx() {
|
||||
return ATOMIC_FAA(&dispatch_idx_, 1) % get_thread_count();
|
||||
}
|
||||
void update_tcp_keepalive_params(int keepalive_enabled, uint32_t tcp_keepidle, uint32_t tcp_keepintvl, uint32_t tcp_keepcnt);
|
||||
private:
|
||||
void run(int64_t idx);
|
||||
private:
|
||||
ObSqlNioImpl* impl_;
|
||||
int port_;
|
||||
ObISqlSockHandler* handler_;
|
||||
uint64_t dispatch_idx_;
|
||||
uint64_t tenant_id_;
|
||||
};
|
||||
|
||||
extern int sql_nio_add_cgroup(const uint64_t tenant_id);
|
||||
}; // end namespace obmysql
|
||||
}; // end namespace oceanbase
|
||||
|
||||
|
11
deps/oblib/src/rpc/obmysql/ob_sql_nio_server.cpp
vendored
11
deps/oblib/src/rpc/obmysql/ob_sql_nio_server.cpp
vendored
@ -24,12 +24,21 @@ int ObSqlNioServer::start(int port, rpc::frame::ObReqDeliver* deliver, int n_thr
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(io_handler_.init(deliver))) {
|
||||
LOG_WARN("handler init fail", K(ret));
|
||||
} else if (OB_FAIL(nio_.start(port, &io_handler_, n_thread))) {
|
||||
} else if (OB_FAIL(nio_.start(port, &io_handler_, n_thread, tenant_id_))) {
|
||||
LOG_WARN("sql nio start fail", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSqlNioServer::set_thread_count(const int thread_num)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if(thread_num != get_nio()->get_thread_count()){
|
||||
ret = nio_.set_thread_count(thread_num);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObSqlNioServer::stop()
|
||||
{
|
||||
nio_.stop();
|
||||
|
@ -23,13 +23,19 @@ namespace obmysql
|
||||
class ObSqlNioServer
|
||||
{
|
||||
public:
|
||||
ObSqlNioServer(ObISMConnectionCallback& conn_cb, ObMySQLHandler& mysql_handler): thread_processor_(mysql_handler), io_handler_(conn_cb, thread_processor_, nio_) {}
|
||||
ObSqlNioServer(ObISMConnectionCallback &conn_cb,
|
||||
ObMySQLHandler &mysql_handler,
|
||||
const uint64_t tenant_id = common::OB_INVALID_ID)
|
||||
: thread_processor_(mysql_handler),
|
||||
io_handler_(conn_cb, thread_processor_, nio_), tenant_id_(tenant_id) {}
|
||||
virtual ~ObSqlNioServer() {}
|
||||
ObSqlNio *get_nio() { return &nio_; }
|
||||
int start(int port, rpc::frame::ObReqDeliver* deliver, int n_thread);
|
||||
void revert_sock(void* sess);
|
||||
int peek_data(void* sess, int64_t limit, const char*& buf, int64_t& sz);
|
||||
int consume_data(void* sess, int64_t sz);
|
||||
int write_data(void* sess, const char* buf, int64_t sz);
|
||||
int set_thread_count(const int thread_num);
|
||||
void stop();
|
||||
void wait();
|
||||
void destroy();
|
||||
@ -38,6 +44,7 @@ private:
|
||||
ObSqlSockProcessor thread_processor_; // for tenant worker
|
||||
ObSqlSockHandler io_handler_; // for io thread
|
||||
ObSqlNio nio_;
|
||||
uint64_t tenant_id_;
|
||||
};
|
||||
extern ObSqlNioServer* global_sql_nio_server;
|
||||
}; // end namespace obmysql
|
||||
|
@ -29,7 +29,7 @@ class ObSqlSockHandler: public ObISqlSockHandler
|
||||
{
|
||||
public:
|
||||
ObSqlSockHandler(ObISMConnectionCallback& conn_cb, ObSqlSockProcessor& sock_processor, ObSqlNio& nio):
|
||||
conn_cb_(conn_cb), sock_processor_(sock_processor), deliver_(nullptr), nio_(nio) {}
|
||||
conn_cb_(conn_cb), sock_processor_(sock_processor), deliver_(nullptr), nio_(&nio) {}
|
||||
virtual ~ObSqlSockHandler() {}
|
||||
int init(rpc::frame::ObReqDeliver* deliver);
|
||||
virtual int on_readable(void* sess) override;
|
||||
@ -40,7 +40,7 @@ private:
|
||||
ObISMConnectionCallback& conn_cb_;
|
||||
ObSqlSockProcessor& sock_processor_;
|
||||
rpc::frame::ObReqDeliver* deliver_;
|
||||
ObSqlNio& nio_;
|
||||
ObSqlNio* nio_;
|
||||
};
|
||||
|
||||
}; // end namespace obmysql
|
||||
|
@ -22,7 +22,7 @@ using namespace observer;
|
||||
namespace obmysql
|
||||
{
|
||||
|
||||
ObSqlSockSession::ObSqlSockSession(ObISMConnectionCallback& conn_cb, ObSqlNio& nio):
|
||||
ObSqlSockSession::ObSqlSockSession(ObISMConnectionCallback& conn_cb, ObSqlNio* nio):
|
||||
nio_(nio),
|
||||
sm_conn_cb_(conn_cb),
|
||||
sql_req_(ObRequest::OB_MYSQL, 1),
|
||||
@ -48,7 +48,7 @@ void ObSqlSockSession::destroy()
|
||||
|
||||
void ObSqlSockSession::destroy_sock()
|
||||
{
|
||||
return nio_.destroy_sock((void*)this);
|
||||
return nio_->destroy_sock((void*)this);
|
||||
}
|
||||
|
||||
int ObSqlSockSession::on_disconnect()
|
||||
@ -58,18 +58,18 @@ int ObSqlSockSession::on_disconnect()
|
||||
|
||||
void ObSqlSockSession::set_shutdown()
|
||||
{
|
||||
return nio_.set_shutdown((void *)this);
|
||||
return nio_->set_shutdown((void *)this);
|
||||
}
|
||||
|
||||
void ObSqlSockSession::shutdown()
|
||||
{
|
||||
return nio_.shutdown((void *)this);
|
||||
return nio_->shutdown((void *)this);
|
||||
}
|
||||
|
||||
void ObSqlSockSession::revert_sock()
|
||||
{
|
||||
if (last_pkt_sz_ > 0) {
|
||||
nio_.consume_data((void*)this, last_pkt_sz_);
|
||||
nio_->consume_data((void*)this, last_pkt_sz_);
|
||||
last_pkt_sz_ = 0;
|
||||
}
|
||||
sql_req_.reset_trace_id();
|
||||
@ -78,10 +78,10 @@ void ObSqlSockSession::revert_sock()
|
||||
int64_t sz = pending_write_sz_;
|
||||
pending_write_buf_ = NULL;
|
||||
pending_write_sz_ = 0;
|
||||
nio_.async_write_data((void*)this, data, sz);
|
||||
nio_->async_write_data((void*)this, data, sz);
|
||||
} else {
|
||||
pool_.reuse();
|
||||
nio_.revert_sock((void*)this);
|
||||
nio_->revert_sock((void*)this);
|
||||
}
|
||||
}
|
||||
|
||||
@ -90,12 +90,12 @@ void ObSqlSockSession::on_flushed()
|
||||
/* TODO should not go here*/
|
||||
//abort();
|
||||
pool_.reuse();
|
||||
nio_.revert_sock((void*)this);
|
||||
nio_->revert_sock((void*)this);
|
||||
}
|
||||
|
||||
bool ObSqlSockSession::has_error()
|
||||
{
|
||||
return nio_.has_error((void*)this);
|
||||
return nio_->has_error((void*)this);
|
||||
}
|
||||
|
||||
int ObSqlSockSession::peek_data(int64_t limit, const char*& buf, int64_t& sz)
|
||||
@ -104,7 +104,7 @@ int ObSqlSockSession::peek_data(int64_t limit, const char*& buf, int64_t& sz)
|
||||
if (has_error()) {
|
||||
ret = OB_IO_ERROR;
|
||||
LOG_WARN("sock has error", K(ret));
|
||||
} else if (OB_FAIL(nio_.peek_data((void*)this, limit, buf, sz))) {
|
||||
} else if (OB_FAIL(nio_->peek_data((void*)this, limit, buf, sz))) {
|
||||
destroy_sock();
|
||||
}
|
||||
return ret;
|
||||
@ -112,7 +112,7 @@ int ObSqlSockSession::peek_data(int64_t limit, const char*& buf, int64_t& sz)
|
||||
|
||||
void ObSqlSockSession::clear_sql_session_info()
|
||||
{
|
||||
nio_.reset_sql_session_info(this);
|
||||
nio_->reset_sql_session_info(this);
|
||||
}
|
||||
|
||||
int ObSqlSockSession::consume_data(int64_t sz)
|
||||
@ -121,7 +121,7 @@ int ObSqlSockSession::consume_data(int64_t sz)
|
||||
if (has_error()) {
|
||||
ret = OB_IO_ERROR;
|
||||
LOG_WARN("sock has error", K(ret));
|
||||
} else if (OB_FAIL(nio_.consume_data((void*)this, sz))) {
|
||||
} else if (OB_FAIL(nio_->consume_data((void*)this, sz))) {
|
||||
destroy_sock();
|
||||
}
|
||||
return ret;
|
||||
@ -129,7 +129,7 @@ int ObSqlSockSession::consume_data(int64_t sz)
|
||||
|
||||
void ObSqlSockSession::set_last_decode_succ_and_deliver_time(int64_t time)
|
||||
{
|
||||
nio_.set_last_decode_succ_time((void*)this, time);
|
||||
nio_->set_last_decode_succ_time((void*)this, time);
|
||||
}
|
||||
int ObSqlSockSession::write_data(const char* buf, int64_t sz)
|
||||
{
|
||||
@ -137,7 +137,7 @@ int ObSqlSockSession::write_data(const char* buf, int64_t sz)
|
||||
if (has_error()) {
|
||||
ret = OB_IO_ERROR;
|
||||
LOG_WARN("sock has error", K(ret));
|
||||
} else if (OB_FAIL(nio_.write_data((void*)this, buf, sz))) {
|
||||
} else if (OB_FAIL(nio_->write_data((void*)this, buf, sz))) {
|
||||
destroy_sock();
|
||||
}
|
||||
return ret;
|
||||
@ -158,17 +158,17 @@ int ObSqlSockSession::async_write_data(const char* buf, int64_t sz)
|
||||
|
||||
void ObSqlSockSession::set_sql_session_info(void* sess)
|
||||
{
|
||||
nio_.set_sql_session_info((void *)this, sess);
|
||||
nio_->set_sql_session_info((void *)this, sess);
|
||||
}
|
||||
|
||||
int ObSqlSockSession::set_ssl_enabled()
|
||||
{
|
||||
return nio_.set_ssl_enabled((void *)this);
|
||||
return nio_->set_ssl_enabled((void *)this);
|
||||
}
|
||||
|
||||
SSL* ObSqlSockSession::get_ssl_st()
|
||||
{
|
||||
return nio_.get_ssl_st((void *)this);
|
||||
return nio_->get_ssl_st((void *)this);
|
||||
}
|
||||
|
||||
}; // end namespace obmysql
|
||||
|
@ -39,7 +39,7 @@ private:
|
||||
class ObSqlSockSession
|
||||
{
|
||||
public:
|
||||
ObSqlSockSession(ObISMConnectionCallback& conn_cb, ObSqlNio& nio);
|
||||
ObSqlSockSession(ObISMConnectionCallback& conn_cb, ObSqlNio* nio);
|
||||
~ObSqlSockSession();
|
||||
void* alloc(int64_t sz) { return pool_.alloc(sz); }
|
||||
int init();
|
||||
@ -61,7 +61,7 @@ public:
|
||||
void set_sql_session_info(void* sess);
|
||||
int set_ssl_enabled();
|
||||
SSL* get_ssl_st();
|
||||
ObSqlNio& nio_;
|
||||
ObSqlNio* nio_;
|
||||
ObISMConnectionCallback& sm_conn_cb_;
|
||||
rpc::ObRequest sql_req_;
|
||||
ObSqlSessionMemPool pool_;
|
||||
|
36
deps/oblib/src/rpc/obrpc/ob_listener.cpp
vendored
36
deps/oblib/src/rpc/obrpc/ob_listener.cpp
vendored
@ -365,6 +365,42 @@ void ObListener::do_work()
|
||||
return;
|
||||
}
|
||||
|
||||
int ObListener::do_one_event(int accept_fd) {
|
||||
int err = 0;
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
uint64_t client_magic = 0;
|
||||
uint8_t index = 0;
|
||||
io_threads_pipefd_pool_t *pipefd_pool = NULL;
|
||||
|
||||
if (OB_TMP_FAIL(read_client_magic(accept_fd, client_magic, index))) {
|
||||
index = compatible_balance_assign(pipefd_pool);
|
||||
trace_connection_info(accept_fd);
|
||||
if (OB_TMP_FAIL(connection_redispatch(accept_fd, pipefd_pool, index))) {
|
||||
close(accept_fd);
|
||||
}
|
||||
pipefd_pool = NULL;
|
||||
} else {
|
||||
for (int i = 0; i < MAX_PROTOCOL_TYPE_SIZE; i++) {
|
||||
if (io_wrpipefd_map_[i].used && io_wrpipefd_map_[i].magic == client_magic) {
|
||||
pipefd_pool = &(io_wrpipefd_map_[i].ioth_wrpipefd_pool);
|
||||
}
|
||||
}
|
||||
|
||||
trace_connection_info(accept_fd);
|
||||
|
||||
if (OB_ISNULL(pipefd_pool)) { /* high_prio_rpc_eio exist or not is decided by configuration */
|
||||
index = compatible_balance_assign(pipefd_pool);
|
||||
} else {
|
||||
RPC_LOG(INFO, "dispatch to", K(client_magic), K(index));
|
||||
}
|
||||
if(OB_TMP_FAIL(connection_redispatch(accept_fd, pipefd_pool, index))) {
|
||||
close(accept_fd);
|
||||
}
|
||||
pipefd_pool = NULL;
|
||||
}
|
||||
return err;
|
||||
}
|
||||
|
||||
uint8_t ObListener::compatible_balance_assign(io_threads_pipefd_pool_t * &pipefd_pool)
|
||||
{
|
||||
static uint8_t ioth_index_inc = 0;
|
||||
|
1
deps/oblib/src/rpc/obrpc/ob_listener.h
vendored
1
deps/oblib/src/rpc/obrpc/ob_listener.h
vendored
@ -66,6 +66,7 @@ public:
|
||||
int regist(uint64_t magic, int count, int *pipefd_array);
|
||||
uint8_t compatible_balance_assign(io_threads_pipefd_pool_t *& pipefd_pool);
|
||||
void set_port(int port) {port_ = port;}
|
||||
int do_one_event(int accept_fd);
|
||||
void destroy();
|
||||
|
||||
private:
|
||||
|
4
deps/oblib/src/rpc/obrpc/ob_net_client.cpp
vendored
4
deps/oblib/src/rpc/obrpc/ob_net_client.cpp
vendored
@ -16,6 +16,9 @@
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/net/ob_addr.h"
|
||||
#include "rpc/obrpc/ob_rpc_proxy.h"
|
||||
extern "C" {
|
||||
#include "rpc/pnio/interface/group.h"
|
||||
};
|
||||
|
||||
using namespace oceanbase::common;
|
||||
|
||||
@ -50,6 +53,7 @@ int ObNetClient::init_(const ObNetOptions opts)
|
||||
} else if (OB_FAIL(net_.start())) {
|
||||
LOG_ERROR("Start client network fail", K(ret));
|
||||
} else {
|
||||
pn_provision(-1, 1, 1);
|
||||
inited_ = true;
|
||||
}
|
||||
|
||||
|
72
deps/oblib/src/rpc/obrpc/ob_nio_interface.h
vendored
72
deps/oblib/src/rpc/obrpc/ob_nio_interface.h
vendored
@ -1,72 +0,0 @@
|
||||
/**
|
||||
* 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_OBRPC_OB_NIO_INTERFACE_H_
|
||||
#define OCEANBASE_OBRPC_OB_NIO_INTERFACE_H_
|
||||
#include <stdint.h>
|
||||
#include <pthread.h>
|
||||
#include "lib/net/ob_addr.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace obrpc
|
||||
{
|
||||
|
||||
class IReqHandler
|
||||
{
|
||||
public:
|
||||
IReqHandler() {}
|
||||
virtual ~IReqHandler() {}
|
||||
virtual int handle_req(int64_t resp_id, char* buf, int64_t sz) = 0;
|
||||
};
|
||||
|
||||
class IRespHandler
|
||||
{
|
||||
public:
|
||||
IRespHandler() {}
|
||||
virtual ~IRespHandler() {}
|
||||
virtual void* alloc(int64_t sz) = 0;
|
||||
virtual int handle_resp(int io_err, char* buf, int64_t sz) = 0;
|
||||
};
|
||||
|
||||
class ObINio
|
||||
{
|
||||
public:
|
||||
ObINio(): accept_queue_fd_(-1), req_handler_(NULL) {}
|
||||
virtual ~ObINio() {}
|
||||
void init(IReqHandler* req_handler, int accept_queue_fd) {
|
||||
req_handler_ = req_handler;
|
||||
accept_queue_fd_ = accept_queue_fd;
|
||||
}
|
||||
int start() {
|
||||
return pthread_create(&thread_, NULL, thread_func, this);
|
||||
}
|
||||
virtual int post(const common::ObAddr& addr, const char* req, int64_t req_size, IRespHandler* resp_handler) = 0;
|
||||
virtual int resp(int64_t resp_id, char* buf, int64_t sz) = 0;
|
||||
private:
|
||||
static void* thread_func(void* arg) {
|
||||
((ObINio*)arg)->do_work(0);
|
||||
return NULL;
|
||||
}
|
||||
virtual int do_work(int tid) = 0;
|
||||
private:
|
||||
pthread_t thread_;
|
||||
protected:
|
||||
int accept_queue_fd_;
|
||||
IReqHandler* req_handler_;
|
||||
};
|
||||
|
||||
}; // end namespace obrpc
|
||||
}; // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_OBRPC_OB_NIO_INTERFACE_H_ */
|
||||
|
114
deps/oblib/src/rpc/obrpc/ob_poc_nio.cpp
vendored
114
deps/oblib/src/rpc/obrpc/ob_poc_nio.cpp
vendored
@ -1,114 +0,0 @@
|
||||
/**
|
||||
* 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 "rpc/obrpc/ob_poc_nio.h"
|
||||
#include "lib/oblog/ob_log.h"
|
||||
#include <sys/types.h>
|
||||
#include <sys/socket.h>
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::obrpc;
|
||||
|
||||
static struct sockaddr_in* obaddr2sockaddr(struct sockaddr_in *sin, const ObAddr& addr)
|
||||
{
|
||||
if (NULL != sin) {
|
||||
sin->sin_port = (uint16_t)htons((uint16_t)(addr.get_port() + 123));
|
||||
sin->sin_addr.s_addr = htonl(addr.get_ipv4());
|
||||
sin->sin_family = AF_INET;
|
||||
}
|
||||
return sin;
|
||||
}
|
||||
|
||||
int ObPocNio::post(const common::ObAddr& addr, const char* req, int64_t req_size, IRespHandler* resp_handler)
|
||||
{
|
||||
int ret = 0;
|
||||
int64_t io_limit = 1<<21;
|
||||
RLOCAL(char*, resp_buf);
|
||||
struct sockaddr_in sin;
|
||||
int fd = -1;
|
||||
if ((fd = socket(AF_INET, SOCK_STREAM, 0)) < 0) {
|
||||
ret = errno;
|
||||
RPC_LOG(WARN, "create socket fail", K(errno));
|
||||
} else if (connect(fd, (sockaddr*)obaddr2sockaddr(&sin, addr), sizeof(sin)) < 0) {
|
||||
ret = errno;
|
||||
RPC_LOG(WARN, "connect fail", K(addr), K(errno));
|
||||
} else if (write(fd, req, req_size) != req_size) {
|
||||
ret = errno;
|
||||
RPC_LOG(WARN, "write fail", K(req_size), K(errno));
|
||||
} else if (NULL == resp_buf && NULL == (resp_buf = (char*)malloc(io_limit))) {
|
||||
ret = ENOMEM;
|
||||
RPC_LOG(WARN, "alloc resp buffer fail", K(io_limit), K(errno));
|
||||
} else {
|
||||
shutdown(fd, SHUT_WR);
|
||||
RPC_LOG(INFO, "post succ, shutdown write side, blocking read response");
|
||||
int64_t sz = read(fd, resp_buf, io_limit);
|
||||
if (sz < 0) {
|
||||
RPC_LOG(WARN, "read resp fail", K(errno));
|
||||
sz = 0;
|
||||
}
|
||||
RPC_LOG(INFO, "receive resp finish", K(sz));
|
||||
resp_handler->handle_resp(ret, resp_buf, sz);
|
||||
RPC_LOG(INFO, "resp callback finish");
|
||||
}
|
||||
if (fd >= 0) {
|
||||
close(fd);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPocNio::resp(int64_t resp_id, char* buf, int64_t sz)
|
||||
{
|
||||
int fd = (int)resp_id;
|
||||
if (sz > 0 && fd >= 0) {
|
||||
if (write(fd, buf, sz) < 0) {
|
||||
RPC_LOG_RET(WARN, common::OB_ERR_SYS, "write resp fail", K(errno));
|
||||
} else {
|
||||
RPC_LOG_RET(WARN, common::OB_SUCCESS, "write resp OK");
|
||||
}
|
||||
} else {
|
||||
RPC_LOG_RET(WARN, common::OB_INVALID_ARGUMENT, "resp invalid argument", KP(buf), K(sz), K(fd));
|
||||
}
|
||||
if (fd >= 0) {
|
||||
close(fd);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
int ObPocNio::do_work(int tid)
|
||||
{
|
||||
UNUSED(tid);
|
||||
int64_t io_limit = 1<<22;
|
||||
char* io_buf = (char*)malloc(io_limit);
|
||||
while(1){
|
||||
int fd = -1;
|
||||
if (read(accept_queue_fd_, &fd, sizeof(fd)) != sizeof(fd)) {
|
||||
RPC_LOG_RET(ERROR, common::OB_ERR_SYS, "read accept queue fail");
|
||||
continue;
|
||||
} else {
|
||||
RPC_LOG(INFO, "read accept queue succ", K(fd));
|
||||
}
|
||||
int64_t sz = read(fd, io_buf, io_limit);
|
||||
if (sz < 0 || sz >= io_limit) {
|
||||
RPC_LOG_RET(WARN, common::OB_ERR_SYS, "read socket error", K(errno), K(fd));
|
||||
close(fd);
|
||||
} else {
|
||||
int ret = OB_SUCCESS;
|
||||
RPC_LOG(INFO, "read a complete request", K(fd));
|
||||
if (OB_FAIL(req_handler_->handle_req(fd, io_buf, sz))) {
|
||||
RPC_LOG(WARN, "read socket error", K(ret));
|
||||
close(fd);
|
||||
} else {
|
||||
RPC_LOG(INFO, "handle_req success", K(fd));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
36
deps/oblib/src/rpc/obrpc/ob_poc_nio.h
vendored
36
deps/oblib/src/rpc/obrpc/ob_poc_nio.h
vendored
@ -1,36 +0,0 @@
|
||||
/**
|
||||
* 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_OBRPC_OB_POC_NIO_H_
|
||||
#define OCEANBASE_OBRPC_OB_POC_NIO_H_
|
||||
#include "rpc/obrpc/ob_nio_interface.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace obrpc
|
||||
{
|
||||
class ObPocNio: public ObINio
|
||||
{
|
||||
public:
|
||||
ObPocNio() {}
|
||||
virtual ~ObPocNio() {}
|
||||
int post(const common::ObAddr& addr, const char* req, int64_t req_size, IRespHandler* resp_handler) override;
|
||||
int resp(int64_t resp_id, char* buf, int64_t sz) override;
|
||||
private:
|
||||
int do_work(int tid);
|
||||
};
|
||||
|
||||
extern ObPocNio global_poc_nio_;
|
||||
}; // end namespace obrpc
|
||||
}; // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_OBRPC_OB_POC_NIO_H_ */
|
179
deps/oblib/src/rpc/obrpc/ob_poc_rpc_proxy.cpp
vendored
179
deps/oblib/src/rpc/obrpc/ob_poc_rpc_proxy.cpp
vendored
@ -13,22 +13,53 @@
|
||||
#include "rpc/obrpc/ob_poc_rpc_proxy.h"
|
||||
#include "rpc/obrpc/ob_poc_rpc_server.h"
|
||||
#include "rpc/obrpc/ob_rpc_proxy.h"
|
||||
#include "rpc/obrpc/ob_net_keepalive.h"
|
||||
extern "C" {
|
||||
#include "rpc/pnio/r0/futex.h"
|
||||
}
|
||||
|
||||
using namespace oceanbase::common;
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace obrpc
|
||||
{
|
||||
|
||||
int ObSyncRespCallback::handle_resp(int io_err, char* buf, int64_t sz)
|
||||
const int easy_head_size = 16;
|
||||
int ObSyncRespCallback::handle_resp(int io_err, const char* buf, int64_t sz)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (0 == io_err) {
|
||||
resp_ = buf;
|
||||
if (PNIO_OK != io_err) {
|
||||
if (PNIO_TIMEOUT == io_err || PNIO_DISCONNECT == io_err) {
|
||||
send_ret_ = OB_TIMEOUT;
|
||||
} else {
|
||||
send_ret_ = OB_RPC_SEND_ERROR;
|
||||
RPC_LOG_RET(WARN, send_ret_, "pnio error", KP(buf), K(sz), K(io_err));
|
||||
}
|
||||
} else if (NULL == buf || sz <= easy_head_size) {
|
||||
send_ret_ = OB_ERR_UNEXPECTED;
|
||||
RPC_LOG_RET(WARN, send_ret_, "response is null", KP(buf), K(sz), K(io_err));
|
||||
} else {
|
||||
buf = buf + easy_head_size;
|
||||
sz = sz - easy_head_size; // skip easy header
|
||||
sz_ = sz;
|
||||
resp_ = reinterpret_cast<char *>(alloc(sz_));
|
||||
if (resp_ == NULL) {
|
||||
send_ret_ = OB_ALLOCATE_MEMORY_FAILED;
|
||||
RPC_LOG_RET(WARN, send_ret_, "alloc response buffer fail");
|
||||
} else {
|
||||
memcpy(resp_, buf, sz_);
|
||||
}
|
||||
}
|
||||
int ret = send_ret_;
|
||||
ATOMIC_STORE(&cond_, 1);
|
||||
rk_futex_wake(&cond_, 1);
|
||||
return ret;
|
||||
}
|
||||
int ObSyncRespCallback::wait()
|
||||
{
|
||||
while(ATOMIC_LOAD(&cond_) == 0) {
|
||||
rk_futex_wait(&cond_, 0, NULL);
|
||||
}
|
||||
return send_ret_;
|
||||
}
|
||||
|
||||
class ObPocSPAlloc: public rpc::frame::SPAlloc
|
||||
{
|
||||
@ -42,7 +73,7 @@ private:
|
||||
ObRpcMemPool& pool_;
|
||||
};
|
||||
|
||||
ObAsyncRespCallback* ObAsyncRespCallback::create(ObRpcMemPool& pool, UAsyncCB& ucb)
|
||||
ObAsyncRespCallback* ObAsyncRespCallback::create(ObRpcMemPool& pool, UAsyncCB* ucb)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPocSPAlloc sp_alloc(pool);
|
||||
@ -50,46 +81,142 @@ ObAsyncRespCallback* ObAsyncRespCallback::create(ObRpcMemPool& pool, UAsyncCB& u
|
||||
ObAsyncRespCallback* pcb = NULL;
|
||||
if (NULL == (pcb = (ObAsyncRespCallback*)pool.alloc(sizeof(ObAsyncRespCallback)))) {
|
||||
RPC_LOG(WARN, "alloc resp callback fail", K(ret));
|
||||
} else if (NULL == (cb = ucb.clone(sp_alloc))) {
|
||||
RPC_LOG(WARN, "ucb.clone fail", K(ret));
|
||||
} else {
|
||||
new(pcb)ObAsyncRespCallback(pool, *cb);
|
||||
if (NULL != ucb) {
|
||||
if (NULL == (cb = ucb->clone(sp_alloc))) {
|
||||
RPC_LOG(WARN, "ucb.clone fail", K(ret));
|
||||
} else {
|
||||
cb->low_level_cb_ = pcb;
|
||||
}
|
||||
}
|
||||
new(pcb)ObAsyncRespCallback(pool, cb);
|
||||
}
|
||||
return pcb;
|
||||
}
|
||||
|
||||
int ObAsyncRespCallback::handle_resp(int io_err, char* buf, int64_t sz)
|
||||
int ObAsyncRespCallback::handle_resp(int io_err, const char* buf, int64_t sz)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObRpcPacket* ret_pkt = NULL;
|
||||
if (0 != io_err) {
|
||||
ucb_.set_error(io_err);
|
||||
ucb_.on_error(io_err);
|
||||
if (buf != NULL && sz > easy_head_size) {
|
||||
sz = sz - easy_head_size;
|
||||
buf = buf + easy_head_size;
|
||||
} else {
|
||||
sz = 0;
|
||||
buf = NULL;
|
||||
}
|
||||
if (ucb_ == NULL) {
|
||||
// do nothing
|
||||
} else if (0 != io_err) {
|
||||
ucb_->set_error(io_err);
|
||||
if (OB_SUCCESS != ucb_->on_error(io_err)) {
|
||||
ucb_->on_timeout();
|
||||
}
|
||||
} else if (NULL == buf) {
|
||||
ucb_.on_timeout();
|
||||
ucb_->on_timeout();
|
||||
} else if (OB_FAIL(rpc_decode_ob_packet(pool_, buf, sz, ret_pkt))) {
|
||||
ucb_.on_invalid();
|
||||
ucb_->on_invalid();
|
||||
RPC_LOG(WARN, "rpc_decode_ob_packet fail", K(ret));
|
||||
} else if (OB_FAIL(ucb_.decode(ret_pkt))) {
|
||||
ucb_.on_invalid();
|
||||
} else if (OB_FAIL(ucb_->decode(ret_pkt))) {
|
||||
ucb_->on_invalid();
|
||||
RPC_LOG(WARN, "ucb.decode fail", K(ret));
|
||||
} else {
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
if (OB_SUCCESS != (tmp_ret = ucb_.process())) {
|
||||
if (OB_SUCCESS != (tmp_ret = ucb_->process())) {
|
||||
RPC_LOG(WARN, "ucb.process fail", K(tmp_ret));
|
||||
}
|
||||
}
|
||||
pool_.destroy();
|
||||
return ret;
|
||||
}
|
||||
|
||||
void init_ucb(ObRpcProxy& proxy, UAsyncCB* ucb, const common::ObAddr& dest, int64_t send_ts, int64_t payload_sz)
|
||||
{
|
||||
ucb->set_dst(dest);
|
||||
ucb->set_tenant_id(proxy.get_tenant());
|
||||
ucb->set_timeout(proxy.timeout());
|
||||
ucb->set_send_ts(send_ts);
|
||||
ucb->set_payload(payload_sz);
|
||||
}
|
||||
static easy_addr_t to_ez_addr(const ObAddr &addr)
|
||||
{
|
||||
easy_addr_t ez;
|
||||
memset(&ez, 0, sizeof (ez));
|
||||
if (addr.is_valid()) {
|
||||
ez.port = (htons)(static_cast<uint16_t>(addr.get_port()));
|
||||
ez.cidx = 0;
|
||||
if (addr.using_ipv4()) {
|
||||
ez.family = AF_INET;
|
||||
ez.u.addr = htonl(addr.get_ipv4());
|
||||
} else if (addr.using_unix()) {
|
||||
ez.family = AF_UNIX;
|
||||
snprintf(ez.u.unix_path, UNIX_PATH_MAX, "%s", addr.get_unix_path());
|
||||
} else {
|
||||
ez.family = AF_INET6;
|
||||
(void) addr.get_ipv6(&ez.u.addr6, sizeof(ez.u.addr6));
|
||||
}
|
||||
}
|
||||
return ez;
|
||||
}
|
||||
|
||||
int64_t ObPocClientStub::get_proxy_timeout(ObRpcProxy& proxy) {
|
||||
return proxy.timeout();
|
||||
}
|
||||
void ObPocClientStub::set_rcode(ObRpcProxy& proxy, const ObRpcResultCode& rcode) {
|
||||
proxy.set_result_code(rcode);
|
||||
}
|
||||
void ObPocClientStub::set_handle(ObRpcProxy& proxy, Handle* handle, const ObRpcPacketCode& pcode, const ObRpcOpts& opts, bool is_stream_next, int64_t session_id) {
|
||||
proxy.set_handle_attr(handle, pcode, opts, is_stream_next, session_id);
|
||||
}
|
||||
int ObPocClientStub::translate_io_error(int io_err) {
|
||||
int ret = OB_SUCCESS;
|
||||
if (PNIO_OK == io_err) {
|
||||
} else if (ENOMEM == io_err || -ENOMEM == io_err) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else if (EINVAL == io_err || -EINVAL == io_err) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPocClientStub::log_user_error_and_warn(const ObRpcResultCode &rcode) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(OB_SUCCESS != rcode.rcode_)) {
|
||||
FORWARD_USER_ERROR(rcode.rcode_, rcode.msg_);
|
||||
}
|
||||
for (int i = 0; OB_SUCC(ret) && i < rcode.warnings_.count(); ++i) {
|
||||
const common::ObWarningBuffer::WarningItem warning_item = rcode.warnings_.at(i);
|
||||
if (ObLogger::USER_WARN == warning_item.log_level_) {
|
||||
FORWARD_USER_WARN(warning_item.code_, warning_item.msg_);
|
||||
} else if (ObLogger::USER_NOTE == warning_item.log_level_) {
|
||||
FORWARD_USER_NOTE(warning_item.code_, warning_item.msg_);
|
||||
} else {
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
RPC_LOG(WARN, "unknown log type", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void init_ucb(ObRpcProxy& proxy, UAsyncCB& ucb, const common::ObAddr& dest, int64_t send_ts, int64_t payload_sz)
|
||||
{
|
||||
ucb.set_dst(dest);
|
||||
ucb.set_tenant_id(proxy.get_tenant());
|
||||
ucb.set_timeout(proxy.timeout());
|
||||
ucb.set_send_ts(send_ts);
|
||||
ucb.set_payload(payload_sz);
|
||||
int ObPocClientStub::check_blacklist(const common::ObAddr& addr) {
|
||||
int ret = OB_SUCCESS;
|
||||
if(!addr.is_valid()) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
RPC_LOG(WARN, "invalid addr", K(ret), K(addr));
|
||||
} else {
|
||||
easy_addr_t ez_addr = to_ez_addr(addr);
|
||||
if (ObNetKeepAlive::get_instance().in_black(ez_addr)) {
|
||||
ret = OB_RPC_POST_ERROR;
|
||||
if (REACH_TIME_INTERVAL(1000000)) {
|
||||
RPC_LOG(WARN, "address in blacklist", K(ret), K(addr));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
ObPocClientStub global_poc_client(global_poc_server.get_nio());
|
||||
ObPocClientStub global_poc_client;
|
||||
}; // end namespace obrpc
|
||||
}; // end namespace oceanbase
|
||||
|
178
deps/oblib/src/rpc/obrpc/ob_poc_rpc_proxy.h
vendored
178
deps/oblib/src/rpc/obrpc/ob_poc_rpc_proxy.h
vendored
@ -12,57 +12,79 @@
|
||||
|
||||
#ifndef OCEANBASE_OBRPC_OB_POC_RPC_PROXY_H_
|
||||
#define OCEANBASE_OBRPC_OB_POC_RPC_PROXY_H_
|
||||
#include "rpc/obrpc/ob_nio_interface.h"
|
||||
#include "rpc/obrpc/ob_rpc_endec.h"
|
||||
#include "rpc/frame/ob_req_transport.h"
|
||||
#include "rpc/ob_request.h"
|
||||
#include "rpc/obrpc/ob_poc_rpc_server.h"
|
||||
|
||||
extern "C" {
|
||||
#include "rpc/pnio/interface/group.h"
|
||||
}
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace obrpc
|
||||
{
|
||||
class ObSyncRespCallback: public IRespHandler
|
||||
class ObSyncRespCallback
|
||||
{
|
||||
public:
|
||||
ObSyncRespCallback(ObRpcMemPool& pool): pool_(pool), resp_(NULL), sz_(0) {}
|
||||
virtual ~ObSyncRespCallback() {}
|
||||
ObSyncRespCallback(ObRpcMemPool& pool): pkt_nio_cb_(NULL), pool_(pool), resp_(NULL), sz_(0), cond_(0), send_ret_(common::OB_SUCCESS){}
|
||||
~ObSyncRespCallback() {}
|
||||
void* alloc(int64_t sz) { return pool_.alloc(sz); }
|
||||
int handle_resp(int io_err, char* buf, int64_t sz);
|
||||
char* get_resp(int64_t& sz) {
|
||||
int handle_resp(int io_err, const char* buf, int64_t sz);
|
||||
int wait();
|
||||
const char* get_resp(int64_t& sz) {
|
||||
sz = sz_;
|
||||
return resp_;
|
||||
}
|
||||
static int client_cb(void* arg, int io_err, const char* b, int64_t sz) {
|
||||
int ret = ((ObSyncRespCallback*)arg)->handle_resp(io_err, b, sz);
|
||||
return ret;
|
||||
}
|
||||
private:
|
||||
void* pkt_nio_cb_;
|
||||
ObRpcMemPool& pool_;
|
||||
char* resp_;
|
||||
int64_t sz_;
|
||||
int cond_;
|
||||
int send_ret_;
|
||||
};
|
||||
|
||||
typedef rpc::frame::ObReqTransport::AsyncCB UAsyncCB;
|
||||
class ObAsyncRespCallback: public IRespHandler
|
||||
class Handle;
|
||||
class ObAsyncRespCallback
|
||||
{
|
||||
public:
|
||||
ObAsyncRespCallback(ObRpcMemPool& pool, UAsyncCB& ucb): pool_(pool), ucb_(ucb) {}
|
||||
ObAsyncRespCallback(ObRpcMemPool& pool, UAsyncCB* ucb): pkt_nio_cb_(NULL), pool_(pool), ucb_(ucb) {}
|
||||
~ObAsyncRespCallback() {}
|
||||
static ObAsyncRespCallback* create(ObRpcMemPool& pool, UAsyncCB& ucb);
|
||||
UAsyncCB& get_ucb() { return ucb_; }
|
||||
void* alloc(int64_t sz) { return pool_.alloc(sz); }
|
||||
int handle_resp(int io_err, char* buf, int64_t sz);
|
||||
static ObAsyncRespCallback* create(ObRpcMemPool& pool, UAsyncCB* ucb);
|
||||
UAsyncCB* get_ucb() { return ucb_; }
|
||||
int handle_resp(int io_err, const char* buf, int64_t sz);
|
||||
static int client_cb(void* arg, int io_error, const char* b, int64_t sz) {
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (arg != NULL) {
|
||||
ret = ((ObAsyncRespCallback*)arg)->handle_resp(io_error, b, sz);
|
||||
} else {
|
||||
RPC_LOG(WARN, "async rpc callback is null, it is unexpected", KP(b), K(sz));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
private:
|
||||
void* pkt_nio_cb_;
|
||||
ObRpcMemPool& pool_;
|
||||
UAsyncCB& ucb_;
|
||||
UAsyncCB* ucb_;
|
||||
};
|
||||
|
||||
void init_ucb(ObRpcProxy& proxy, UAsyncCB& ucb, const common::ObAddr& addr, int64_t send_ts, int64_t payload_sz);
|
||||
void init_ucb(ObRpcProxy& proxy, UAsyncCB* ucb, const common::ObAddr& addr, int64_t send_ts, int64_t payload_sz);
|
||||
|
||||
template<typename UCB, typename Input>
|
||||
void set_ucb_args(UCB& ucb, const Input& args)
|
||||
void set_ucb_args(UCB* ucb, const Input& args)
|
||||
{
|
||||
ucb.set_args(args);
|
||||
ucb->set_args(args);
|
||||
}
|
||||
|
||||
template<typename NoneType>
|
||||
void set_ucb_args(UAsyncCB& ucb, const NoneType& none)
|
||||
void set_ucb_args(UAsyncCB* ucb, const NoneType& none)
|
||||
{
|
||||
UNUSED(ucb);
|
||||
UNUSED(none);
|
||||
@ -71,55 +93,119 @@ template<typename NoneType>
|
||||
class ObPocClientStub
|
||||
{
|
||||
public:
|
||||
ObPocClientStub(ObINio& nio): nio_(nio) {}
|
||||
ObPocClientStub() {}
|
||||
~ObPocClientStub() {}
|
||||
static int64_t get_proxy_timeout(ObRpcProxy& proxy);
|
||||
static void set_rcode(ObRpcProxy& proxy, const ObRpcResultCode& rcode);
|
||||
static int check_blacklist(const common::ObAddr& addr);
|
||||
static void set_handle(ObRpcProxy& proxy, Handle* handle, const ObRpcPacketCode& pcode, const ObRpcOpts& opts, bool is_stream_next, int64_t session_id);
|
||||
static uint8_t balance_assign_tidx()
|
||||
{
|
||||
static uint8_t s_rpc_tidx CACHE_ALIGNED;
|
||||
return ATOMIC_FAA(&s_rpc_tidx, 1);
|
||||
}
|
||||
static int translate_io_error(int io_err);
|
||||
template<typename Input, typename Output>
|
||||
int send(ObRpcProxy& proxy, const common::ObAddr& addr, ObRpcPacketCode pcode, const Input& args, Output& out, const ObRpcOpts& opts) {
|
||||
int send(ObRpcProxy& proxy, const common::ObAddr& addr, ObRpcPacketCode pcode, const Input& args, Output& out, Handle* handle, const ObRpcOpts& opts) {
|
||||
int sys_err = 0;
|
||||
int ret = common::OB_SUCCESS;
|
||||
ObRpcMemPool pool;
|
||||
const int64_t start_ts = common::ObTimeUtility::current_time();
|
||||
int64_t src_tenant_id = ob_get_tenant_id();
|
||||
auto &set = obrpc::ObRpcPacketSet::instance();
|
||||
const char* pcode_label = set.name_of_idx(set.idx_of_pcode(pcode));
|
||||
ObRpcMemPool pool(src_tenant_id, pcode_label);
|
||||
ObSyncRespCallback cb(pool);
|
||||
char* req = NULL;
|
||||
int64_t req_sz = 0;
|
||||
char* resp = NULL;
|
||||
const char* resp = NULL;
|
||||
int64_t resp_sz = 0;
|
||||
int resp_ret = common::OB_SUCCESS;
|
||||
if (OB_FAIL(rpc_encode_req(proxy, pool, pcode, args, opts, req, req_sz))) {
|
||||
ObRpcPacket resp_pkt;
|
||||
ObRpcResultCode rcode;
|
||||
sockaddr_in sock_addr;
|
||||
uint8_t thread_id = balance_assign_tidx();
|
||||
if (OB_FAIL(rpc_encode_req(proxy, pool, pcode, args, opts, req, req_sz, false))) {
|
||||
RPC_LOG(WARN, "rpc encode req fail", K(ret));
|
||||
} else if (OB_FAIL(nio_.post(addr, req, req_sz, &cb))) {
|
||||
RPC_LOG(WARN, "nio post fail", K(ret));
|
||||
} else if(OB_FAIL(check_blacklist(addr))) {
|
||||
RPC_LOG(WARN, "check_blacklist failed", K(ret));
|
||||
} else if (0 != (sys_err = pn_send(
|
||||
(1ULL<<32) + thread_id,
|
||||
obaddr2sockaddr(&sock_addr, addr),
|
||||
req,
|
||||
req_sz,
|
||||
static_cast<int16_t>(set.idx_of_pcode(pcode)),
|
||||
start_ts + get_proxy_timeout(proxy),
|
||||
ObSyncRespCallback::client_cb,
|
||||
&cb))) {
|
||||
ret = translate_io_error(sys_err);
|
||||
RPC_LOG(WARN, "pn_send fail", K(sys_err), K(ret));
|
||||
} else if (OB_FAIL(cb.wait())) {
|
||||
RPC_LOG(WARN, "sync rpc execute fail", K(ret), K(addr));
|
||||
} else if (NULL == (resp = cb.get_resp(resp_sz))) {
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
RPC_LOG(WARN, "get NULL response buffer", K(ret));
|
||||
} else if (OB_FAIL(rpc_decode_resp(resp, resp_sz, out))) {
|
||||
RPC_LOG(WARN, "rpc decode response fail", K(ret));
|
||||
} else {
|
||||
if (common::OB_SUCCESS != resp_ret) {
|
||||
ret = resp_ret;
|
||||
RPC_LOG(WARN, "rpc execute fail", K(ret));
|
||||
RPC_LOG(WARN, "sync rpc execute success but resp is null", K(ret), K(addr));
|
||||
} else if (OB_FAIL(rpc_decode_resp(resp, resp_sz, out, resp_pkt, rcode))) {
|
||||
RPC_LOG(WARN, "rpc decode response fail", KP(resp), K(resp_sz), K(ret));
|
||||
}
|
||||
if (rcode.rcode_ != OB_DESERIALIZE_ERROR) {
|
||||
int wb_ret = OB_SUCCESS;
|
||||
if (common::OB_SUCCESS != (wb_ret = log_user_error_and_warn(rcode))) {
|
||||
RPC_OBRPC_LOG(WARN, "fail to log user error and warn", K(ret), K(wb_ret), K((rcode)));
|
||||
}
|
||||
set_rcode(proxy, rcode);
|
||||
if (OB_SUCC(ret) && handle) {
|
||||
set_handle(proxy, handle, pcode, opts, resp_pkt.is_stream_next(), resp_pkt.get_session_id());
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template<typename Input, typename UCB>
|
||||
int post(ObRpcProxy& proxy, const common::ObAddr& addr, ObRpcPacketCode pcode, const Input& args, UCB& ucb, const ObRpcOpts& opts) {
|
||||
int post(ObRpcProxy& proxy, const common::ObAddr& addr, ObRpcPacketCode pcode, const Input& args, UCB* ucb, const ObRpcOpts& opts) {
|
||||
int sys_err = 0;
|
||||
int ret = common::OB_SUCCESS;
|
||||
const int64_t start_ts = common::ObTimeUtility::current_time();
|
||||
ObRpcMemPool* pool = NULL;
|
||||
if (NULL == (pool = ObRpcMemPool::create(sizeof(ObAsyncRespCallback) + sizeof(UCB)))) {
|
||||
uint8_t thread_id = balance_assign_tidx();
|
||||
int64_t src_tenant_id = ob_get_tenant_id();
|
||||
#ifndef PERF_MODE
|
||||
const int init_alloc_sz = 0;
|
||||
#else
|
||||
const int init_alloc_sz = 400<<10;
|
||||
#endif
|
||||
auto &set = obrpc::ObRpcPacketSet::instance();
|
||||
const char* pcode_label = set.name_of_idx(set.idx_of_pcode(pcode));
|
||||
if (NULL == (pool = ObRpcMemPool::create(src_tenant_id, pcode_label, init_alloc_sz))) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
ObAsyncRespCallback* cb = NULL;
|
||||
char* req = NULL;
|
||||
int64_t req_sz = 0;
|
||||
if (OB_FAIL(rpc_encode_req(proxy, *pool, pcode, args, opts, req, req_sz))) {
|
||||
if (OB_FAIL(rpc_encode_req(proxy, *pool, pcode, args, opts, req, req_sz, NULL == ucb))) {
|
||||
RPC_LOG(WARN, "rpc encode req fail", K(ret));
|
||||
} else if(OB_FAIL(check_blacklist(addr))) {
|
||||
RPC_LOG(WARN, "check_blacklist failed", K(addr));
|
||||
} else if (NULL == (cb = ObAsyncRespCallback::create(*pool, ucb))) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
init_ucb(proxy, cb->get_ucb(), addr, start_ts, req_sz);
|
||||
set_ucb_args(ucb, args);
|
||||
if (OB_FAIL(nio_.post(addr, req, req_sz, cb))) {
|
||||
RPC_LOG(WARN, "nio post fail", K(ret));
|
||||
auto newcb = reinterpret_cast<UCB*>(cb->get_ucb());
|
||||
if (newcb) {
|
||||
set_ucb_args(newcb, args);
|
||||
init_ucb(proxy, cb->get_ucb(), addr, start_ts, req_sz);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
sockaddr_in sock_addr;
|
||||
if (0 != (sys_err = pn_send(
|
||||
(1ULL<<32) + thread_id,
|
||||
obaddr2sockaddr(&sock_addr, addr),
|
||||
req,
|
||||
req_sz,
|
||||
static_cast<int16_t>(set.idx_of_pcode(pcode)),
|
||||
start_ts + get_proxy_timeout(proxy),
|
||||
ObAsyncRespCallback::client_cb,
|
||||
cb)
|
||||
)) {
|
||||
ret = translate_io_error(sys_err);
|
||||
RPC_LOG(WARN, "pn_send fail", K(sys_err), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -129,12 +215,20 @@ public:
|
||||
return ret;
|
||||
}
|
||||
|
||||
private:
|
||||
ObINio& nio_;
|
||||
static struct sockaddr_in* obaddr2sockaddr(struct sockaddr_in *sin, const ObAddr& addr)
|
||||
{
|
||||
if (NULL != sin) {
|
||||
sin->sin_port = (uint16_t)htons((uint16_t)(addr.get_port()));
|
||||
sin->sin_addr.s_addr = htonl(addr.get_ipv4());
|
||||
sin->sin_family = AF_INET;
|
||||
}
|
||||
return sin;
|
||||
}
|
||||
int log_user_error_and_warn(const ObRpcResultCode &rcode) const;
|
||||
};
|
||||
|
||||
extern ObPocClientStub global_poc_client;
|
||||
#define POC_RPC_INTERCEPT(func, args...) if (transport_impl_ == rpc::ObRequest::TRANSPORT_PROTO_POC) return global_poc_client.func(*this, args);
|
||||
#define POC_RPC_INTERCEPT(func, args...) if (transport_impl_ == rpc::ObRequest::TRANSPORT_PROTO_POC && global_poc_server.client_use_pkt_nio()) return global_poc_client.func(*this, args);
|
||||
}; // end namespace obrpc
|
||||
}; // end namespace oceanbase
|
||||
|
||||
|
@ -31,6 +31,8 @@ void* ObPocRpcRequestOperator::alloc_response_buffer(ObRequest* req, int64_t siz
|
||||
void ObPocRpcRequestOperator::response_result(ObRequest* req, obrpc::ObRpcPacket* pkt)
|
||||
{
|
||||
get_poc_handle_context(req)->resp(pkt);
|
||||
get_poc_handle_context(req)->destroy();
|
||||
|
||||
}
|
||||
|
||||
ObAddr ObPocRpcRequestOperator::get_peer(const ObRequest* req)
|
||||
|
177
deps/oblib/src/rpc/obrpc/ob_poc_rpc_server.cpp
vendored
177
deps/oblib/src/rpc/obrpc/ob_poc_rpc_server.cpp
vendored
@ -11,13 +11,27 @@
|
||||
*/
|
||||
|
||||
#include "rpc/obrpc/ob_poc_rpc_server.h"
|
||||
#include "lib/oblog/ob_log_module.h"
|
||||
#include "rpc/obrpc/ob_net_keepalive.h"
|
||||
|
||||
#define rk_log_macro(level, ret, format, ...) _OB_LOG_RET(level, ret, "PNIO " format, ##__VA_ARGS__)
|
||||
#include "lib/lock/ob_futex.h"
|
||||
extern "C" {
|
||||
#include "rpc/pnio/interface/group.h"
|
||||
};
|
||||
#include "rpc/obrpc/ob_rpc_endec.h"
|
||||
#define cfgi(k, v) atoi(getenv(k)?:v)
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace obrpc
|
||||
{
|
||||
extern const int easy_head_size;
|
||||
ObPocRpcServer global_poc_server;
|
||||
ObListener* global_ob_listener;
|
||||
bool __attribute__((weak)) enable_pkt_nio() {
|
||||
return false;
|
||||
}
|
||||
}; // end namespace obrpc
|
||||
}; // end namespace oceanbase
|
||||
|
||||
@ -25,52 +39,167 @@ using namespace oceanbase::common;
|
||||
using namespace oceanbase::obrpc;
|
||||
using namespace oceanbase::rpc;
|
||||
|
||||
ObRequest* ObPocServerHandleContext::create(ObINio& nio, int64_t resp_id, char* buf, int64_t sz)
|
||||
frame::ObReqDeliver* global_deliver;
|
||||
int ObPocServerHandleContext::create(int64_t resp_id, const char* buf, int64_t sz, ObRequest*& req)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObRequest* req = NULL;
|
||||
ObPocServerHandleContext* ctx = NULL;
|
||||
ObRpcMemPool* pool = ObRpcMemPool::create(sizeof(ObPocServerHandleContext) + sizeof(ObRequest));
|
||||
if (NULL != pool) {
|
||||
ctx = new(pool + 1)ObPocServerHandleContext(nio, *pool, resp_id);
|
||||
req = new(ctx + 1)ObRequest(ObRequest::OB_RPC, 1);
|
||||
ObRpcPacket* pkt = NULL;
|
||||
if (OB_FAIL(rpc_decode_ob_packet(*pool, buf, sz, pkt))) {
|
||||
RPC_LOG(ERROR, "decode packet fail", K(ret));
|
||||
ObRpcPacket tmp_pkt;
|
||||
#ifndef PERF_MODE
|
||||
const int64_t alloc_payload_sz = sz;
|
||||
#else
|
||||
const int64_t alloc_payload_sz = 0;
|
||||
#endif
|
||||
if (OB_FAIL(tmp_pkt.decode(buf, sz))) {
|
||||
RPC_LOG(ERROR, "decode packet fail", K(ret));
|
||||
} else {
|
||||
obrpc::ObRpcPacketCode pcode = tmp_pkt.get_pcode();
|
||||
auto &set = obrpc::ObRpcPacketSet::instance();
|
||||
const char* pcode_label = set.name_of_idx(set.idx_of_pcode(pcode));
|
||||
const int64_t pool_size = sizeof(ObPocServerHandleContext) + sizeof(ObRequest) + sizeof(ObRpcPacket) + alloc_payload_sz;
|
||||
ObRpcMemPool* pool = ObRpcMemPool::create(tmp_pkt.get_tenant_id(), pcode_label, pool_size);
|
||||
void *temp = NULL;
|
||||
if (OB_ISNULL(pool)) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
RPC_LOG(WARN, "create memory pool failed", K(ret));
|
||||
} else if (OB_ISNULL(temp = pool->alloc(sizeof(ObPocServerHandleContext) + sizeof(ObRequest)))){
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
RPC_LOG(WARN, "pool allocate memory failed", K(ret));
|
||||
} else {
|
||||
req->set_server_handle_context(ctx);
|
||||
req->set_packet(pkt);
|
||||
ctx = new(temp)ObPocServerHandleContext(*pool, resp_id);
|
||||
req = new(ctx + 1)ObRequest(ObRequest::OB_RPC, ObRequest::TRANSPORT_PROTO_POC);
|
||||
ObRpcPacket* pkt = (ObRpcPacket*)pool->alloc(sizeof(ObRpcPacket) + alloc_payload_sz);
|
||||
if (NULL == pkt) {
|
||||
RPC_LOG(WARN, "pool allocate rpc packet memory failed", K(ret));
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
MEMCPY(reinterpret_cast<void *>(pkt), reinterpret_cast<void *>(&tmp_pkt), sizeof(ObRpcPacket));
|
||||
const char* packet_data = NULL;
|
||||
if (alloc_payload_sz > 0) {
|
||||
packet_data = reinterpret_cast<char *>(pkt + 1);
|
||||
MEMCPY(const_cast<char*>(packet_data), tmp_pkt.get_cdata(), tmp_pkt.get_clen());
|
||||
} else {
|
||||
packet_data = tmp_pkt.get_cdata();
|
||||
}
|
||||
int64_t receive_ts = ObTimeUtility::current_time();
|
||||
pkt->set_receive_ts(receive_ts);
|
||||
pkt->set_content(packet_data, tmp_pkt.get_clen());
|
||||
req->set_server_handle_context(ctx);
|
||||
req->set_packet(pkt);
|
||||
req->set_receive_timestamp(pkt->get_receive_ts());
|
||||
req->set_request_arrival_time(pkt->get_receive_ts());
|
||||
req->set_arrival_push_diff(common::ObTimeUtility::current_time());
|
||||
|
||||
const int64_t fly_ts = receive_ts - pkt->get_timestamp();
|
||||
if (fly_ts > oceanbase::common::OB_MAX_PACKET_FLY_TS && TC_REACH_TIME_INTERVAL(100 * 1000)) {
|
||||
RPC_LOG(WARN, "PNIO packet wait too much time between proxy and server_cb", "pcode", pkt->get_pcode(),
|
||||
"fly_ts", fly_ts, "send_timestamp", pkt->get_timestamp());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return req;
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObPocServerHandleContext::resp(ObRpcPacket* pkt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int sys_err = 0;
|
||||
char* buf = NULL;
|
||||
int64_t sz = 0;
|
||||
if (OB_FAIL(rpc_encode_ob_packet(pool_, pkt, buf, sz))) {
|
||||
if (NULL == pkt) {
|
||||
RPC_LOG(WARN, "resp pkt is null", K(pkt));
|
||||
} else if (OB_FAIL(rpc_encode_ob_packet(pool_, pkt, buf, sz))) {
|
||||
RPC_LOG(WARN, "rpc_encode_ob_packet fail", K(pkt));
|
||||
buf = NULL;
|
||||
sz = 0;
|
||||
}
|
||||
nio_.resp(resp_id_, buf, sz);
|
||||
if ((sys_err = pn_resp(resp_id_, buf, sz)) != 0) {
|
||||
RPC_LOG(WARN, "pn_resp fail", K(resp_id_), K(sys_err));
|
||||
}
|
||||
}
|
||||
|
||||
int ObPocRpcServer::start(int port, frame::ObReqDeliver* deliver)
|
||||
int serve_cb(int grp, const char* b, int64_t sz, uint64_t resp_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t POC_RPC_MAGIC = 0;
|
||||
server_req_handler_.init(deliver, &nio_);
|
||||
int accept_queue_fd = listener_.regist(POC_RPC_MAGIC, 0, NULL);
|
||||
nio_.init(&server_req_handler_, accept_queue_fd);
|
||||
if (OB_FAIL( nio_.start())) {
|
||||
RPC_LOG(ERROR, "poc nio start fail", K(ret));
|
||||
} else if (OB_FAIL(listener_.start())) {
|
||||
RPC_LOG(ERROR, "listen fail", K(ret), K(port));
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
if (NULL == b || sz <= easy_head_size) {
|
||||
tmp_ret = OB_INVALID_DATA;
|
||||
RPC_LOG(WARN, "rpc request is invalid", K(tmp_ret), K(b), K(sz));
|
||||
} else {
|
||||
RPC_LOG(INFO, "poc rpc server listen succ");
|
||||
b = b + easy_head_size;
|
||||
sz = sz - easy_head_size;
|
||||
ObRequest* req = NULL;
|
||||
if (OB_TMP_FAIL(ObPocServerHandleContext::create(resp_id, b, sz, req))) {
|
||||
RPC_LOG(WARN, "created req is null", K(tmp_ret), K(sz), K(resp_id));
|
||||
} else {
|
||||
global_deliver->deliver(*req);
|
||||
}
|
||||
}
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
int sys_err = 0;
|
||||
if ((sys_err = pn_resp(resp_id, NULL, 0)) != 0) {
|
||||
RPC_LOG(WARN, "pn_resp fail", K(resp_id), K(sys_err));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPocRpcServer::start(int port, int net_thread_count, frame::ObReqDeliver* deliver)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
// init pkt-nio framework
|
||||
int lfd = -1;
|
||||
int grp = 1;
|
||||
if ((lfd = pn_listen(port, serve_cb)) == -1) {
|
||||
ret = OB_IO_ERROR;
|
||||
RPC_LOG(ERROR, "pn_listen failed", K(ret));
|
||||
} else {
|
||||
global_deliver = deliver;
|
||||
int count = pn_provision(lfd, grp, net_thread_count);
|
||||
if (count != net_thread_count) {
|
||||
RPC_LOG(WARN, "pn_provision error", K(count), K(net_thread_count));
|
||||
}
|
||||
has_start_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int ObPocRpcServer::update_tcp_keepalive_params(int64_t user_timeout) {
|
||||
int ret = OB_SUCCESS;
|
||||
if (pn_set_keepalive_timeout(user_timeout) != user_timeout) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
RPC_LOG(WARN, "invalid user_timeout", K(ret), K(user_timeout));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObPocRpcServer::client_use_pkt_nio() {
|
||||
return has_start() && enable_pkt_nio();
|
||||
}
|
||||
|
||||
extern "C" {
|
||||
void* pkt_nio_malloc(int64_t sz, const char* label) {
|
||||
ObMemAttr attr(OB_SERVER_TENANT_ID, label, ObCtxIds::PKT_NIO);
|
||||
return oceanbase::common::ob_malloc(sz, attr);
|
||||
}
|
||||
void pkt_nio_free(void *ptr) {
|
||||
oceanbase::common::ob_free(ptr);
|
||||
}
|
||||
bool server_in_black(struct sockaddr* sa) {
|
||||
easy_addr_t ez_addr;
|
||||
easy_inet_atoe(sa, &ez_addr);
|
||||
return ObNetKeepAlive::get_instance().in_black(ez_addr);
|
||||
}
|
||||
int dispatch_to_ob_listener(int accept_fd) {
|
||||
int ret = -1;
|
||||
if (oceanbase::obrpc::global_ob_listener) {
|
||||
ret = oceanbase::obrpc::global_ob_listener->do_one_event(accept_fd);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
#define PKT_NIO_MALLOC(sz, label) pkt_nio_malloc(sz, label)
|
||||
#define PKT_NIO_FREE(ptr) pkt_nio_free(ptr)
|
||||
#define SERVER_IN_BLACK(sa) server_in_black(sa)
|
||||
#define DISPATCH_EXTERNAL(accept_fd) dispatch_to_ob_listener(accept_fd)
|
||||
#include "rpc/pnio/pkt-nio.c"
|
||||
};
|
||||
|
40
deps/oblib/src/rpc/obrpc/ob_poc_rpc_server.h
vendored
40
deps/oblib/src/rpc/obrpc/ob_poc_rpc_server.h
vendored
@ -12,7 +12,6 @@
|
||||
|
||||
#ifndef OCEANBASE_OBRPC_OB_POC_RPC_SERVER_H_
|
||||
#define OCEANBASE_OBRPC_OB_POC_RPC_SERVER_H_
|
||||
#include "rpc/obrpc/ob_poc_nio.h"
|
||||
#include "rpc/obrpc/ob_rpc_mem_pool.h"
|
||||
#include "rpc/ob_request.h"
|
||||
#include "rpc/frame/ob_req_deliver.h"
|
||||
@ -26,56 +25,39 @@ namespace obrpc
|
||||
class ObPocServerHandleContext
|
||||
{
|
||||
public:
|
||||
ObPocServerHandleContext(ObINio& nio, ObRpcMemPool& pool, uint64_t resp_id):
|
||||
nio_(nio), pool_(pool), resp_id_(resp_id)
|
||||
ObPocServerHandleContext( ObRpcMemPool& pool, uint64_t resp_id):
|
||||
pool_(pool), resp_id_(resp_id)
|
||||
{}
|
||||
~ObPocServerHandleContext() {
|
||||
destroy();
|
||||
}
|
||||
static rpc::ObRequest* create(ObINio& nio, int64_t resp_id, char* buf, int64_t sz);
|
||||
static int create(int64_t resp_id, const char* buf, int64_t sz, rpc::ObRequest*& req);
|
||||
void destroy() { pool_.destroy(); }
|
||||
void resp(ObRpcPacket* pkt);
|
||||
void* alloc(int64_t sz) { return pool_.alloc(sz); }
|
||||
private:
|
||||
ObINio& nio_;
|
||||
ObRpcMemPool& pool_;
|
||||
int64_t resp_id_;
|
||||
uint64_t resp_id_;
|
||||
};
|
||||
|
||||
class ObPocServerReqHandler: public IReqHandler
|
||||
{
|
||||
public:
|
||||
ObPocServerReqHandler(): deliver_(NULL), nio_(NULL) {}
|
||||
~ObPocServerReqHandler() {}
|
||||
void init(rpc::frame::ObReqDeliver* deliver, ObINio* nio) {
|
||||
deliver_ = deliver;
|
||||
nio_ = nio;
|
||||
}
|
||||
int handle_req(int64_t resp_id, char* buf, int64_t sz) {
|
||||
rpc::ObRequest* req = ObPocServerHandleContext::create(*nio_, resp_id, buf, sz);
|
||||
return deliver_->deliver(*req);
|
||||
}
|
||||
private:
|
||||
rpc::frame::ObReqDeliver* deliver_;
|
||||
ObINio* nio_;
|
||||
};
|
||||
|
||||
class ObPocRpcServer
|
||||
{
|
||||
|
||||
public:
|
||||
ObPocRpcServer() {}
|
||||
ObPocRpcServer() : has_start_(false){}
|
||||
~ObPocRpcServer() {}
|
||||
int start(int port, rpc::frame::ObReqDeliver* deliver);
|
||||
int start(int port, int net_thread_count, rpc::frame::ObReqDeliver* deliver);
|
||||
void stop() {}
|
||||
ObPocNio& get_nio() { return nio_; }
|
||||
bool has_start() {return has_start_;}
|
||||
int update_tcp_keepalive_params(int64_t user_timeout);
|
||||
bool client_use_pkt_nio();
|
||||
private:
|
||||
ObPocNio nio_;
|
||||
ObListener listener_;
|
||||
ObPocServerReqHandler server_req_handler_;
|
||||
bool has_start_;
|
||||
};
|
||||
|
||||
extern ObPocRpcServer global_poc_server;
|
||||
extern ObListener* global_ob_listener;
|
||||
|
||||
}; // end namespace obrpc
|
||||
}; // end namespace oceanbase
|
||||
|
2
deps/oblib/src/rpc/obrpc/ob_rpc_endec.cpp
vendored
2
deps/oblib/src/rpc/obrpc/ob_rpc_endec.cpp
vendored
@ -71,7 +71,7 @@ int init_packet(ObRpcProxy& proxy, ObRpcPacket& pkt, ObRpcPacketCode pcode, cons
|
||||
return proxy.init_pkt(&pkt, pcode, opts, unneed_response);
|
||||
}
|
||||
|
||||
int rpc_decode_ob_packet(ObRpcMemPool& pool, char* buf, int64_t sz, ObRpcPacket*& ret_pkt)
|
||||
int rpc_decode_ob_packet(ObRpcMemPool& pool, const char* buf, int64_t sz, ObRpcPacket*& ret_pkt)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
ObRpcPacket* pkt = (ObRpcPacket*)pool.alloc(sizeof(ObRpcPacket));
|
||||
|
50
deps/oblib/src/rpc/obrpc/ob_rpc_endec.h
vendored
50
deps/oblib/src/rpc/obrpc/ob_rpc_endec.h
vendored
@ -28,13 +28,30 @@ int init_packet(ObRpcProxy& proxy, ObRpcPacket& pkt, ObRpcPacketCode pcode, cons
|
||||
const bool unneed_response);
|
||||
|
||||
template <typename T>
|
||||
int rpc_encode_req(ObRpcProxy& proxy, ObRpcMemPool& pool, ObRpcPacketCode pcode, const T& args, const ObRpcOpts& opts, char*& req, int64_t& req_sz)
|
||||
int rpc_encode_req(
|
||||
ObRpcProxy& proxy,
|
||||
ObRpcMemPool& pool,
|
||||
ObRpcPacketCode pcode,
|
||||
const T& args,
|
||||
const ObRpcOpts& opts,
|
||||
char*& req,
|
||||
int64_t& req_sz,
|
||||
bool unneed_resp,
|
||||
bool is_next = false,
|
||||
bool is_last = false,
|
||||
int64_t session_id = 0
|
||||
)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
ObRpcPacket pkt;
|
||||
const int64_t header_sz = pkt.get_header_size();
|
||||
const int64_t payload_sz = calc_extra_payload_size() + common::serialization::encoded_length(args);
|
||||
char* header_buf = (char*)pool.alloc(header_sz + payload_sz);
|
||||
#ifdef PERF_MODE
|
||||
const int64_t reserve_bytes_for_pnio = 200;
|
||||
#else
|
||||
const int64_t reserve_bytes_for_pnio = 0;
|
||||
#endif
|
||||
char* header_buf = (char*)pool.alloc(reserve_bytes_for_pnio + header_sz + payload_sz) + reserve_bytes_for_pnio;
|
||||
char* payload_buf = header_buf + header_sz;
|
||||
int64_t pos = 0;
|
||||
UNIS_VERSION_GUARD(opts.unis_version_);
|
||||
@ -53,9 +70,20 @@ template <typename T>
|
||||
} else {
|
||||
int64_t header_pos = 0;
|
||||
pkt.set_content(payload_buf, payload_sz);
|
||||
if (OB_FAIL(init_packet(proxy, pkt, pcode, opts, false))) {
|
||||
if (OB_FAIL(init_packet(proxy, pkt, pcode, opts, unneed_resp))) {
|
||||
RPC_OBRPC_LOG(WARN, "init packet fail", K(ret));
|
||||
} else if (OB_FAIL(pkt.encode_header(header_buf, header_sz, header_pos))) {
|
||||
} else {
|
||||
if (is_next) {
|
||||
pkt.set_stream_next();
|
||||
}
|
||||
if (is_last) {
|
||||
pkt.set_stream_last();
|
||||
}
|
||||
if (session_id) {
|
||||
pkt.set_session_id(session_id);
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(pkt.encode_header(header_buf, header_sz, header_pos))) {
|
||||
RPC_OBRPC_LOG(WARN, "encode header fail", K(ret));
|
||||
} else {
|
||||
req = header_buf;
|
||||
@ -66,23 +94,23 @@ template <typename T>
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
int rpc_decode_resp(char* resp_buf, int64_t resp_sz, T& result)
|
||||
int rpc_decode_resp(const char* resp_buf, int64_t resp_sz, T& result, ObRpcPacket &pkt, ObRpcResultCode &rcode)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
ObRpcPacket pkt;
|
||||
int64_t pos = 0;
|
||||
if (OB_FAIL(pkt.decode(resp_buf, resp_sz))) {
|
||||
RPC_OBRPC_LOG(WARN, "decode packet fail", K(ret));
|
||||
} else {
|
||||
ObRpcResultCode rcode;
|
||||
UNIS_VERSION_GUARD(pkt.get_unis_version());
|
||||
if (OB_FAIL(rcode.deserialize(resp_buf, resp_sz, pos))) {
|
||||
const char* payload = pkt.get_cdata();
|
||||
int64_t limit = pkt.get_clen();
|
||||
if (OB_FAIL(rcode.deserialize(payload, limit, pos))) {
|
||||
rcode.rcode_ = common::OB_DESERIALIZE_ERROR;
|
||||
RPC_OBRPC_LOG(WARN, "deserialize result code fail", K(ret));
|
||||
} else {
|
||||
if (rcode.rcode_ != common::OB_SUCCESS) {
|
||||
ret = rcode.rcode_;
|
||||
RPC_OBRPC_LOG(WARN, "execute rpc fail", K(ret));
|
||||
} else if (OB_FAIL(common::serialization::decode(resp_buf, resp_sz, pos, result))) {
|
||||
} else if (OB_FAIL(common::serialization::decode(payload, limit, pos, result))) {
|
||||
RPC_OBRPC_LOG(WARN, "deserialize result fail", K(ret));
|
||||
} else {
|
||||
ret = rcode.rcode_;
|
||||
@ -92,7 +120,7 @@ template <typename T>
|
||||
return ret;
|
||||
}
|
||||
|
||||
int rpc_decode_ob_packet(ObRpcMemPool& pool, char* buf, int64_t sz, ObRpcPacket*& ret_pkt);
|
||||
int rpc_decode_ob_packet(ObRpcMemPool& pool, const char* buf, int64_t sz, ObRpcPacket*& ret_pkt);
|
||||
int rpc_encode_ob_packet(ObRpcMemPool& pool, ObRpcPacket* pkt, char*& buf, int64_t& sz);
|
||||
|
||||
}; // end namespace obrpc
|
||||
|
44
deps/oblib/src/rpc/obrpc/ob_rpc_mem_pool.cpp
vendored
44
deps/oblib/src/rpc/obrpc/ob_rpc_mem_pool.cpp
vendored
@ -37,25 +37,49 @@ struct ObRpcMemPool::Page
|
||||
int64_t cur_;
|
||||
char base_[];
|
||||
};
|
||||
static void* rpc_mem_pool_direct_alloc(int64_t sz) { return common::ob_malloc(sz, common::ObModIds::OB_COMMON_NETWORK); }
|
||||
static void* rpc_mem_pool_direct_alloc(int64_t tenant_id, const char* label, int64_t sz) {
|
||||
if (OB_INVALID_TENANT_ID == tenant_id) {
|
||||
tenant_id = OB_SERVER_TENANT_ID;
|
||||
}
|
||||
ObMemAttr attr(tenant_id, label, common::ObCtxIds::RPC_CTX_ID);
|
||||
auto* ret = common::ob_malloc(sz, attr);
|
||||
if (OB_ISNULL(ret)
|
||||
&& OB_ISNULL(lib::ObMallocAllocator::get_instance()->get_tenant_ctx_allocator(tenant_id, common::ObCtxIds::RPC_CTX_ID))) {
|
||||
attr.tenant_id_ = OB_SERVER_TENANT_ID;
|
||||
ret = common::ob_malloc(sz, attr);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
static void rpc_mem_pool_direct_free(void* p) { common::ob_free(p); }
|
||||
static ObRpcMemPool::Page* rpc_mem_pool_create_page(int64_t sz) {
|
||||
static ObRpcMemPool::Page* rpc_mem_pool_create_page(int64_t tenant_id, const char* label, int64_t sz) {
|
||||
int64_t alloc_sz = std::max(sizeof(ObRpcMemPool::Page) + sz, (uint64_t)ObRpcMemPool::RPC_POOL_PAGE_SIZE);
|
||||
ObRpcMemPool::Page* page = (typeof(page))rpc_mem_pool_direct_alloc(alloc_sz);
|
||||
ObRpcMemPool::Page* page = (typeof(page))rpc_mem_pool_direct_alloc(tenant_id, label, alloc_sz);
|
||||
if (OB_ISNULL(page)) {
|
||||
LOG_ERROR_RET(common::OB_ALLOCATE_MEMORY_FAILED, "rpc memory pool alloc memory failed", K(sz), K(alloc_sz));
|
||||
LOG_WARN_RET(common::OB_ALLOCATE_MEMORY_FAILED, "rpc memory pool alloc memory failed", K(sz), K(alloc_sz));
|
||||
} else {
|
||||
new(page)ObRpcMemPool::Page(alloc_sz);
|
||||
}
|
||||
return page;
|
||||
}
|
||||
static void rpc_mem_pool_destroy_page(ObRpcMemPool::Page* page) {
|
||||
if (OB_NOT_NULL(page)) {
|
||||
page->ObRpcMemPool::Page::~Page();
|
||||
common::ob_free(page);
|
||||
}
|
||||
}
|
||||
|
||||
ObRpcMemPool* ObRpcMemPool::create(int64_t req_sz)
|
||||
ObRpcMemPool* ObRpcMemPool::create(int64_t tenant_id, const char* label, int64_t req_sz)
|
||||
{
|
||||
Page* page = rpc_mem_pool_create_page(req_sz + sizeof(ObRpcMemPool));
|
||||
ObRpcMemPool* pool = (typeof(pool))page->alloc(sizeof(ObRpcMemPool));
|
||||
new(pool)ObRpcMemPool(); // can not be null
|
||||
pool->add_page(page);
|
||||
Page* page = nullptr;
|
||||
ObRpcMemPool* pool = nullptr;
|
||||
if (OB_NOT_NULL(page = rpc_mem_pool_create_page(tenant_id, label, req_sz + sizeof(ObRpcMemPool)))) {
|
||||
if (OB_NOT_NULL(pool = (typeof(pool))page->alloc(sizeof(ObRpcMemPool)))) {
|
||||
new(pool)ObRpcMemPool(tenant_id, label); // can not be null
|
||||
pool->add_page(page);
|
||||
} else {
|
||||
rpc_mem_pool_destroy_page(page);
|
||||
}
|
||||
}
|
||||
return pool;
|
||||
}
|
||||
|
||||
@ -64,7 +88,7 @@ void* ObRpcMemPool::alloc(int64_t sz)
|
||||
void* ret = NULL;
|
||||
Page* page = NULL;
|
||||
if (NULL != last_ && NULL != (ret = last_->alloc(sz))) {
|
||||
} else if (NULL == (page = rpc_mem_pool_create_page(sz))) {
|
||||
} else if (NULL == (page = rpc_mem_pool_create_page(tenant_id_, mem_label_, sz))) {
|
||||
} else {
|
||||
ret = page->alloc(sz);
|
||||
add_page(page);
|
||||
|
9
deps/oblib/src/rpc/obrpc/ob_rpc_mem_pool.h
vendored
9
deps/oblib/src/rpc/obrpc/ob_rpc_mem_pool.h
vendored
@ -21,11 +21,12 @@ namespace obrpc
|
||||
class ObRpcMemPool
|
||||
{
|
||||
public:
|
||||
enum { RPC_POOL_PAGE_SIZE = 1<<16 };
|
||||
enum { RPC_POOL_PAGE_SIZE = (1<<14) - 128};
|
||||
struct Page;
|
||||
ObRpcMemPool(): last_(NULL) {}
|
||||
explicit ObRpcMemPool(): last_(NULL), tenant_id_(OB_INVALID_TENANT_ID), mem_label_("RpcDefault") {}
|
||||
explicit ObRpcMemPool(int64_t tenant_id, const char* label): last_(NULL), tenant_id_(tenant_id), mem_label_(label) {}
|
||||
~ObRpcMemPool() { destroy(); }
|
||||
static ObRpcMemPool* create(int64_t sz);
|
||||
static ObRpcMemPool* create(int64_t tenant_id, const char* label, int64_t req_sz);
|
||||
void* alloc(int64_t sz);
|
||||
void reuse();
|
||||
void destroy();
|
||||
@ -33,6 +34,8 @@ private:
|
||||
void add_page(Page* page);
|
||||
private:
|
||||
Page* last_;
|
||||
int64_t tenant_id_;
|
||||
const char* mem_label_;
|
||||
};
|
||||
|
||||
}; // end namespace obrpc
|
||||
|
2
deps/oblib/src/rpc/obrpc/ob_rpc_opts.h
vendored
2
deps/oblib/src/rpc/obrpc/ob_rpc_opts.h
vendored
@ -26,7 +26,6 @@ struct ObRpcOpts
|
||||
ObRpcPriority pr_; // priority of this RPC packet
|
||||
mutable bool is_stream_; // is this RPC packet a stream packet?
|
||||
mutable bool is_stream_last_; // is this RPC packet the last packet in stream?
|
||||
uint64_t unis_version_;
|
||||
common::ObAddr local_addr_;
|
||||
common::ObString ssl_invited_nodes_;
|
||||
|
||||
@ -38,7 +37,6 @@ struct ObRpcOpts
|
||||
local_addr_(),
|
||||
ssl_invited_nodes_()
|
||||
{
|
||||
unis_version_ = lib::get_unis_global_compat_version();
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -318,7 +318,7 @@ int ObRpcProcessorBase::do_response(const Response &rsp)
|
||||
if (rsp.bad_routing_) {
|
||||
packet->set_bad_routing();
|
||||
}
|
||||
packet->set_unis_version(unis_version_);
|
||||
packet->set_unis_version(0);
|
||||
packet->calc_checksum();
|
||||
opacket_size = packet->get_clen() + packet->get_header_size() + common::OB_NET_HEADER_LENGTH;
|
||||
EVENT_INC(RPC_PACKET_OUT);
|
||||
|
@ -36,8 +36,7 @@ public:
|
||||
: rpc_pkt_(NULL), sh_(NULL), sc_(NULL), is_stream_(false), is_stream_end_(false),
|
||||
bad_routing_(false), preserve_recv_data_(false), preserved_buf_(NULL),
|
||||
uncompressed_buf_(NULL), using_buffer_(NULL), send_timestamp_(0), pkt_size_(0), tenant_id_(0),
|
||||
result_compress_type_(common::INVALID_COMPRESSOR),
|
||||
unis_version_(lib::get_unis_global_compat_version())
|
||||
result_compress_type_(common::INVALID_COMPRESSOR)
|
||||
{}
|
||||
|
||||
virtual ~ObRpcProcessorBase();
|
||||
@ -172,7 +171,6 @@ protected:
|
||||
int64_t tenant_id_;
|
||||
// compress the result if not INVALID_COMPRESSOR
|
||||
common::ObCompressorType result_compress_type_;
|
||||
const uint64_t unis_version_;
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObRpcProcessorBase);
|
||||
}; // end of class ObRpcProcessorBase
|
||||
|
113
deps/oblib/src/rpc/obrpc/ob_rpc_proxy.cpp
vendored
113
deps/oblib/src/rpc/obrpc/ob_rpc_proxy.cpp
vendored
@ -68,101 +68,14 @@ int ObRpcProxy::init(const ObReqTransport *transport,
|
||||
src_cluster_id_ = src_cluster_id;
|
||||
dst_ = dst;
|
||||
init_ = true;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObRpcProxy::rpc_call(ObRpcPacketCode pcode, Handle *handle, const ObRpcOpts &opts)
|
||||
{
|
||||
POC_RPC_INTERCEPT(send, dst_, pcode, None, None, opts);
|
||||
int ret = OB_E(EventTable::EN_6) OB_SUCCESS;
|
||||
const int64_t start_ts = ObTimeUtility::current_time();
|
||||
rpc::RpcStatPiece piece;
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (!active_) {
|
||||
ret = OB_INACTIVE_RPC_PROXY;
|
||||
LOG_WARN("Rpc proxy is inactive", K(ret));
|
||||
}
|
||||
int64_t pos = 0;
|
||||
const int64_t payload = calc_payload_size(0);
|
||||
ObReqTransport::Request req;
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(transport_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("Rpc proxy transport is not inited", K(ret));
|
||||
} else if (OB_UNLIKELY(payload > OB_MAX_RPC_PACKET_LENGTH)) {
|
||||
ret = OB_RPC_PACKET_TOO_LONG;
|
||||
LOG_WARN("obrpc packet payload execced its limit",
|
||||
K(ret), K(payload), "limit", OB_MAX_RPC_PACKET_LENGTH);
|
||||
} else if (OB_FAIL(create_request(pcode, *transport_,
|
||||
req, dst_, payload, timeout_, opts.local_addr_,
|
||||
do_ratelimit_, is_bg_flow_, opts.ssl_invited_nodes_, NULL))) {
|
||||
LOG_WARN("create request fail", K(ret));
|
||||
} else if (OB_ISNULL(req.pkt()) || OB_ISNULL(req.buf())) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("request packet or req buf is NULL", K(ret),
|
||||
"packet", req.pkt(), "buf", req.buf());
|
||||
} else if (OB_FAIL(fill_extra_payload(req, payload, pos))) {
|
||||
LOG_WARN("fill extra payload fail", K(ret), K(pos), K(payload));
|
||||
} else if (OB_FAIL(init_pkt(req.pkt(), pcode, opts, false))) {
|
||||
LOG_WARN("Init packet error", K(ret));
|
||||
} else {
|
||||
rpc::RpcStatPiece piece;
|
||||
piece.size_ = payload;
|
||||
piece.time_ = ObTimeUtility::current_time() - req.pkt()->get_timestamp();
|
||||
RPC_STAT(pcode, tenant_id_, piece);
|
||||
|
||||
ObReqTransport::Result r;
|
||||
if (OB_FAIL(send_request(req, r))) {
|
||||
LOG_WARN("send rpc request fail", K(pcode), K(ret));
|
||||
} else if (OB_ISNULL(r.pkt()) || OB_ISNULL(r.pkt()->get_cdata())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("packet or packet cdata is NULL", K(ret), "pkt", r.pkt());
|
||||
int enable_poc_rpc = atoi(getenv("enable_poc_rpc")?:"1");
|
||||
if (enable_poc_rpc > 0) {
|
||||
transport_impl_ = 1;
|
||||
} else {
|
||||
const char *buf = r.pkt()->get_cdata();
|
||||
int64_t len = r.pkt()->get_clen();
|
||||
pos = 0;
|
||||
UNIS_VERSION_GUARD(r.pkt()->get_unis_version());
|
||||
|
||||
if (OB_FAIL(rcode_.deserialize(buf, len, pos))) {
|
||||
LOG_WARN("deserialize result code fail", K(ret));
|
||||
} else {
|
||||
int wb_ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(OB_SUCCESS != rcode_.rcode_)) {
|
||||
ret = rcode_.rcode_;
|
||||
LOG_WARN("execute rpc fail", K(ret));
|
||||
} else if (OB_SUCC(ret) && NULL != handle) {
|
||||
handle->has_more_ = r.pkt()->is_stream_next();
|
||||
handle->dst_ = dst_;
|
||||
handle->sessid_ = r.pkt()->get_session_id();
|
||||
handle->opts_ = opts;
|
||||
handle->transport_ = transport_;
|
||||
handle->proxy_ = *this;
|
||||
handle->pcode_ = pcode;
|
||||
handle->do_ratelimit_ = do_ratelimit_;
|
||||
handle->is_bg_flow_ = is_bg_flow_;
|
||||
} else {
|
||||
//do nothing
|
||||
}
|
||||
if (OB_SUCCESS != (wb_ret = log_user_error_and_warn(rcode_))) {
|
||||
LOG_WARN("fail to log user error and warn", K(ret), K(wb_ret), K((rcode_)));
|
||||
}
|
||||
}
|
||||
transport_impl_ = 0;
|
||||
}
|
||||
}
|
||||
|
||||
piece.size_ = payload;
|
||||
piece.time_ = ObTimeUtility::current_time() - start_ts;
|
||||
if (OB_FAIL(ret)) {
|
||||
piece.failed_ = true;
|
||||
if (OB_TIMEOUT == ret) {
|
||||
piece.is_timeout_ = true;
|
||||
}
|
||||
}
|
||||
RPC_STAT(pcode, tenant_id_, piece);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -171,7 +84,7 @@ int ObRpcProxy::rpc_post(
|
||||
ObReqTransport::AsyncCB *cb,
|
||||
const ObRpcOpts &opts)
|
||||
{
|
||||
POC_RPC_INTERCEPT(post, dst_, pcode, None, *cb, opts);
|
||||
POC_RPC_INTERCEPT(post, dst_, pcode, None, cb, opts);
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
if (!active_) {
|
||||
@ -259,7 +172,7 @@ int ObRpcProxy::init_pkt(
|
||||
pkt->set_dst_cluster_id(dst_cluster_id_);
|
||||
// For request, src_cluster_id must be the cluster_id of this cluster, directly hard-coded
|
||||
pkt->set_src_cluster_id(src_cluster_id_);
|
||||
pkt->set_unis_version(opts.unis_version_);
|
||||
pkt->set_unis_version(0);
|
||||
pkt->set_group_id((0 != get_group_id()) ? get_group_id() : this_worker().get_group_id());
|
||||
if (need_increment_request_level(pcode)) {
|
||||
if (this_worker().get_worker_level() == INT32_MAX) { // The inner sql request is not sent from the tenant thread, so the worker level is still the initial value, given
|
||||
@ -408,3 +321,17 @@ int ObRpcProxy::create_request(
|
||||
PCodeGuard pcode_guard(pcode);
|
||||
return transport.create_request(req, addr, size, timeout, local_addr, do_ratelimit, is_bg_flow, ob_ssl_invited_nodes, cb);
|
||||
}
|
||||
|
||||
void ObRpcProxy::set_handle_attr(Handle* handle, const ObRpcPacketCode& pcode, const ObRpcOpts& opts, bool is_stream_next, int64_t session_id) {
|
||||
if (handle) {
|
||||
handle->pcode_ = pcode;
|
||||
handle->opts_ = opts;
|
||||
handle->has_more_ = is_stream_next;
|
||||
handle->sessid_ = session_id;
|
||||
handle->dst_ = dst_;
|
||||
handle->proxy_ = *this;
|
||||
handle->do_ratelimit_ = do_ratelimit_;
|
||||
handle->is_bg_flow_ = is_bg_flow_;
|
||||
handle->transport_ = NULL;
|
||||
}
|
||||
}
|
||||
|
20
deps/oblib/src/rpc/obrpc/ob_rpc_proxy.h
vendored
20
deps/oblib/src/rpc/obrpc/ob_rpc_proxy.h
vendored
@ -113,7 +113,7 @@ public:
|
||||
|
||||
public:
|
||||
ObRpcProxy()
|
||||
: transport_(NULL), dst_(), transport_impl_(0), timeout_(MAX_RPC_TIMEOUT),
|
||||
: transport_(NULL), dst_(), transport_impl_(1), timeout_(MAX_RPC_TIMEOUT),
|
||||
tenant_id_(common::OB_SYS_TENANT_ID), group_id_(0),
|
||||
priv_tenant_id_(common::OB_INVALID_TENANT_ID),
|
||||
max_process_handler_time_(0), compressor_type_(common::INVALID_COMPRESSOR),
|
||||
@ -145,6 +145,13 @@ public:
|
||||
void set_compressor_type(const common::ObCompressorType &compressor_type) { compressor_type_ = compressor_type; }
|
||||
void set_dst_cluster(int64_t dst_cluster_id) { dst_cluster_id_ = dst_cluster_id; }
|
||||
void set_transport_impl(int transport_impl) { transport_impl_ = transport_impl; }
|
||||
void set_result_code(const ObRpcResultCode retcode) {
|
||||
rcode_.rcode_ = retcode.rcode_;
|
||||
snprintf(rcode_.msg_, common::OB_MAX_ERROR_MSG_LEN, "%s", retcode.msg_);
|
||||
rcode_.warnings_.reset();
|
||||
rcode_.warnings_ = retcode.warnings_;
|
||||
}
|
||||
void set_handle_attr(Handle* handle, const ObRpcPacketCode& pcode, const ObRpcOpts& opts, bool is_stream_next, int64_t session_id);
|
||||
|
||||
bool need_increment_request_level(int pcode) const {
|
||||
return ((pcode > OB_SQL_PCODE_START && pcode < OB_SQL_PCODE_END)
|
||||
@ -221,17 +228,6 @@ protected:
|
||||
Out &result,
|
||||
Handle *handle,
|
||||
const ObRpcOpts &opts);
|
||||
template <typename Input>
|
||||
int rpc_call(ObRpcPacketCode pcode,
|
||||
const Input &args,
|
||||
Handle *handle,
|
||||
const ObRpcOpts &opts);
|
||||
template <typename Output>
|
||||
int rpc_call(ObRpcPacketCode pcode,
|
||||
Output &result,
|
||||
Handle *handle,
|
||||
const ObRpcOpts &opts);
|
||||
int rpc_call(ObRpcPacketCode pcode, Handle *handle, const ObRpcOpts &opts);
|
||||
|
||||
template <class pcodeStruct>
|
||||
int rpc_post(const typename pcodeStruct::Request &args,
|
||||
|
281
deps/oblib/src/rpc/obrpc/ob_rpc_proxy.ipp
vendored
281
deps/oblib/src/rpc/obrpc/ob_rpc_proxy.ipp
vendored
@ -49,8 +49,49 @@ int SSHandle<pcodeStruct>::get_more(typename pcodeStruct::Response &result)
|
||||
ObReqTransport::Result r;
|
||||
|
||||
if (OB_ISNULL(transport_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
RPC_OBRPC_LOG(WARN, "transport_ is NULL", K(ret));
|
||||
RPC_OBRPC_LOG(INFO, "transport_ is NULL, use poc_rpc", K(sess_id), K(has_more_));
|
||||
const int64_t start_ts = common::ObTimeUtility::current_time();
|
||||
int64_t src_tenant_id = ob_get_tenant_id();
|
||||
auto &set = obrpc::ObRpcPacketSet::instance();
|
||||
const char* pcode_label = set.name_of_idx(set.idx_of_pcode(pcode_));
|
||||
ObRpcMemPool pool(src_tenant_id, pcode_label);
|
||||
ObSyncRespCallback cb(pool);
|
||||
char* pnio_req = NULL;
|
||||
int64_t pnio_req_sz = 0, resp_sz = 0;
|
||||
const char* resp = NULL;
|
||||
ObRpcPacket resp_pkt;
|
||||
sockaddr_in sock_addr;
|
||||
static unsigned int thread_id = 0;
|
||||
thread_id ++;
|
||||
if (OB_FAIL(rpc_encode_req(proxy_, pool, pcode_, NULL, opts_, pnio_req, pnio_req_sz, false, true, false, sessid_))) {
|
||||
RPC_LOG(WARN, "rpc encode req fail", K(ret));
|
||||
} else if(!dst_.is_valid()) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
RPC_LOG(WARN, "invalid addr", K(ret));
|
||||
} else if (OB_FAIL(pn_send(
|
||||
(1ULL<<32) + thread_id,
|
||||
ObPocClientStub::obaddr2sockaddr(&sock_addr, dst_),
|
||||
pnio_req,
|
||||
pnio_req_sz,
|
||||
static_cast<int16_t>(set.idx_of_pcode(pcode_)),
|
||||
start_ts + proxy_.timeout(),
|
||||
ObSyncRespCallback::client_cb,
|
||||
&cb))) {
|
||||
RPC_LOG(WARN, "pnio post fail", K(ret));
|
||||
} else if (OB_FAIL(cb.wait())) {
|
||||
RPC_LOG(WARN, "stream rpc execute fail", K(ret), K(dst_));
|
||||
} else if (NULL == (resp = cb.get_resp(resp_sz))) {
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
RPC_LOG(WARN, "stream rpc execute success but resp is null", K(ret), K(dst_));
|
||||
} else if (OB_FAIL(rpc_decode_resp(resp, resp_sz, result, resp_pkt, rcode_))) {
|
||||
RPC_LOG(WARN, "rpc decode response fail", KP(resp), K(resp_sz), K(ret));
|
||||
} else if (rcode_.rcode_ != OB_SUCCESS) {
|
||||
ret = rcode_.rcode_;
|
||||
RPC_OBRPC_LOG(WARN, "execute rpc fail", K(ret));
|
||||
} else {
|
||||
has_more_ = resp_pkt.is_stream_next();
|
||||
}
|
||||
|
||||
} else if (OB_FAIL(ObRpcProxy::create_request(pcode_, *transport_,
|
||||
req, dst_, PAYLOAD_SIZE, proxy_.timeout(), opts_.local_addr_, do_ratelimit_,
|
||||
is_bg_flow_, opts_.ssl_invited_nodes_, NULL))) {
|
||||
@ -139,8 +180,53 @@ int SSHandle<pcodeStruct>::abort()
|
||||
ObReqTransport::Result r;
|
||||
|
||||
if (OB_ISNULL(transport_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
RPC_OBRPC_LOG(ERROR, "transport_ should not be NULL", K(ret));
|
||||
RPC_OBRPC_LOG(INFO, "transport_ is NULL, use poc_rpc", K(sess_id), K(has_more_));
|
||||
const int64_t start_ts = common::ObTimeUtility::current_time();
|
||||
int64_t src_tenant_id = ob_get_tenant_id();
|
||||
auto &set = obrpc::ObRpcPacketSet::instance();
|
||||
const char* pcode_label = set.name_of_idx(set.idx_of_pcode(pcode_));
|
||||
ObRpcMemPool pool(src_tenant_id, pcode_label);
|
||||
ObSyncRespCallback cb(pool);
|
||||
char* pnio_req = NULL;
|
||||
int64_t pnio_req_sz = 0, resp_sz = 0;
|
||||
const char* resp = NULL;
|
||||
ObRpcPacket resp_pkt;
|
||||
sockaddr_in sock_addr;
|
||||
static unsigned int thread_id = 0;
|
||||
thread_id ++;
|
||||
if (OB_FAIL(rpc_encode_req(proxy_, pool, pcode_, NULL, opts_, pnio_req, pnio_req_sz, false, false, true, sessid_))) {
|
||||
RPC_LOG(WARN, "rpc encode req fail", K(ret));
|
||||
} else if(!dst_.is_valid()) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
RPC_LOG(WARN, "invalid addr", K(ret));
|
||||
} else if (OB_FAIL(pn_send(
|
||||
(1ULL<<32) + thread_id,
|
||||
ObPocClientStub::obaddr2sockaddr(&sock_addr, dst_),
|
||||
pnio_req,
|
||||
pnio_req_sz,
|
||||
static_cast<int16_t>(set.idx_of_pcode(pcode_)),
|
||||
start_ts + proxy_.timeout(),
|
||||
ObSyncRespCallback::client_cb,
|
||||
&cb))) {
|
||||
RPC_LOG(WARN, "pnio post fail", K(ret));
|
||||
} else if (OB_FAIL(cb.wait())) {
|
||||
RPC_LOG(WARN, "stream rpc execute fail", K(ret), K(dst_));
|
||||
} else if (NULL == (resp = cb.get_resp(resp_sz))) {
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
RPC_LOG(WARN, "stream rpc execute success but resp is null", K(ret), K(dst_));
|
||||
} else {
|
||||
typename pcodeStruct::Response result;
|
||||
if (OB_FAIL(rpc_decode_resp(resp, resp_sz, result, resp_pkt, rcode_))) {
|
||||
RPC_LOG(WARN, "rpc decode response fail", KP(resp), K(resp_sz), K(ret));
|
||||
} else if (rcode_.rcode_ != OB_SUCCESS) {
|
||||
ret = rcode_.rcode_;
|
||||
RPC_OBRPC_LOG(WARN, "execute rpc fail", K(ret));
|
||||
} else {
|
||||
//do nothing
|
||||
}
|
||||
has_more_ = false;
|
||||
}
|
||||
|
||||
} else if (OB_FAIL(ObRpcProxy::create_request(pcode_, *transport_,
|
||||
req, dst_, PAYLOAD_SIZE, proxy_.timeout(), opts_.local_addr_, do_ratelimit_,
|
||||
is_bg_flow_, opts_.ssl_invited_nodes_, NULL))) {
|
||||
@ -274,7 +360,7 @@ template <typename Input, typename Out>
|
||||
int ObRpcProxy::rpc_call(ObRpcPacketCode pcode, const Input &args,
|
||||
Out &result, Handle *handle, const ObRpcOpts &opts)
|
||||
{
|
||||
POC_RPC_INTERCEPT(send, dst_, pcode, args, result, opts);
|
||||
POC_RPC_INTERCEPT(send, dst_, pcode, args, result, handle, opts);
|
||||
using namespace oceanbase::common;
|
||||
using namespace rpc::frame;
|
||||
int ret = OB_SUCCESS;
|
||||
@ -408,194 +494,11 @@ int ObRpcProxy::rpc_call(ObRpcPacketCode pcode, const Input &args,
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename Input>
|
||||
int ObRpcProxy::rpc_call(ObRpcPacketCode pcode, const Input &args,
|
||||
Handle *handle, const ObRpcOpts &opts)
|
||||
{
|
||||
POC_RPC_INTERCEPT(send, dst_, pcode, args, None, opts);
|
||||
using namespace oceanbase::common;
|
||||
using namespace rpc::frame;
|
||||
int ret = OB_SUCCESS;
|
||||
UNIS_VERSION_GUARD(opts.unis_version_);
|
||||
|
||||
const int64_t start_ts = ObTimeUtility::current_time();
|
||||
rpc::RpcStatPiece piece;
|
||||
|
||||
if (!init_) {
|
||||
ret = OB_NOT_INIT;
|
||||
} else if (!active_) {
|
||||
ret = OB_INACTIVE_RPC_PROXY;
|
||||
}
|
||||
|
||||
int64_t pos = 0;
|
||||
const int64_t payload = calc_payload_size(common::serialization::encoded_length(args));
|
||||
ObReqTransport::Request req;
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (payload > OB_MAX_RPC_PACKET_LENGTH) {
|
||||
ret = OB_RPC_PACKET_TOO_LONG;
|
||||
RPC_OBRPC_LOG(WARN, "obrpc packet payload execced its limit",
|
||||
K(ret), K(payload), "limit", OB_MAX_RPC_PACKET_LENGTH);
|
||||
} else if (OB_FAIL(ObRpcProxy::create_request(pcode, *transport_,
|
||||
req, dst_, payload, timeout_, opts.local_addr_, do_ratelimit_,
|
||||
is_bg_flow_, opts.ssl_invited_nodes_, NULL))) {
|
||||
RPC_OBRPC_LOG(WARN, "create request fail", K(ret));
|
||||
} else if (NULL == req.pkt()) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
RPC_OBRPC_LOG(WARN, "request packet is NULL", K(ret));
|
||||
} else if (OB_FAIL(common::serialization::encode(req.buf(), payload, pos, args))) {
|
||||
RPC_OBRPC_LOG(WARN, "serialize argument fail", K(ret));
|
||||
} else if (OB_FAIL(fill_extra_payload(req, payload, pos))) {
|
||||
RPC_OBRPC_LOG(WARN, "fill extra payload fail", K(ret), K(pos), K(payload));
|
||||
} else if (OB_FAIL(init_pkt(req.pkt(), pcode, opts, false))) {
|
||||
RPC_OBRPC_LOG(WARN, "Init packet error", K(ret));
|
||||
} else {
|
||||
ObReqTransport::Result r;
|
||||
if (OB_FAIL(send_request(req, r))) {
|
||||
RPC_OBRPC_LOG(WARN, "send rpc request fail", K(pcode), K(ret));
|
||||
} else {
|
||||
const char *buf = r.pkt()->get_cdata();
|
||||
int64_t len = r.pkt()->get_clen();
|
||||
int64_t pos = 0;
|
||||
UNIS_VERSION_GUARD(r.pkt()->get_unis_version());
|
||||
|
||||
if (OB_FAIL(rcode_.deserialize(buf, len, pos))) {
|
||||
RPC_OBRPC_LOG(WARN, "deserialize result code fail", K(ret));
|
||||
} else {
|
||||
int wb_ret = OB_SUCCESS;
|
||||
ret = rcode_.rcode_;
|
||||
if (common::OB_SUCCESS == ret && NULL != handle) {
|
||||
handle->has_more_ = r.pkt()->is_stream_next();
|
||||
handle->dst_ = dst_;
|
||||
handle->sessid_ = r.pkt()->get_session_id();
|
||||
handle->opts_ = opts;
|
||||
handle->transport_ = transport_;
|
||||
handle->proxy_ = *this;
|
||||
handle->do_ratelimit_ = do_ratelimit_;
|
||||
handle->is_bg_flow_ = is_bg_flow_;
|
||||
}
|
||||
if (common::OB_SUCCESS != (wb_ret = log_user_error_and_warn(rcode_))) {
|
||||
RPC_OBRPC_LOG(WARN, "fail to log user error and warn", K(ret), K(wb_ret), K((rcode_)));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
piece.size_ = payload;
|
||||
piece.time_ = ObTimeUtility::current_time() - start_ts;
|
||||
if (OB_FAIL(ret)) {
|
||||
piece.failed_ = true;
|
||||
if (OB_TIMEOUT == ret) {
|
||||
piece.is_timeout_ = true;
|
||||
}
|
||||
}
|
||||
RPC_STAT(pcode, tenant_id_, piece);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename Output>
|
||||
int ObRpcProxy::rpc_call(ObRpcPacketCode pcode, Output &result,
|
||||
Handle *handle, const ObRpcOpts &opts)
|
||||
{
|
||||
POC_RPC_INTERCEPT(send, dst_, pcode, None, result, opts);
|
||||
using namespace oceanbase::common;
|
||||
using namespace rpc::frame;
|
||||
static const int64_t PAYLOAD_SIZE = 0;
|
||||
int ret = OB_SUCCESS;
|
||||
UNIS_VERSION_GUARD(opts.unis_version_);
|
||||
|
||||
const int64_t start_ts = ObTimeUtility::current_time();
|
||||
rpc::RpcStatPiece piece;
|
||||
|
||||
if (!init_) {
|
||||
ret = OB_NOT_INIT;
|
||||
} else if (!active_) {
|
||||
ret = OB_INACTIVE_RPC_PROXY;
|
||||
} else {
|
||||
//do nothing
|
||||
}
|
||||
|
||||
int64_t pos = 0;
|
||||
const int64_t payload = calc_payload_size(PAYLOAD_SIZE);
|
||||
|
||||
ObReqTransport::Request req;
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(ObRpcProxy::create_request(pcode, *transport_,
|
||||
req, dst_, payload, timeout_, opts.local_addr_, do_ratelimit_,
|
||||
is_bg_flow_, opts.ssl_invited_nodes_, NULL))) {
|
||||
RPC_OBRPC_LOG(WARN, "create request fail", K(ret));
|
||||
} else if (NULL == req.pkt()) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
RPC_OBRPC_LOG(WARN, "request packet is NULL", K(ret));
|
||||
} else if (OB_FAIL(fill_extra_payload(req, payload, pos))) {
|
||||
RPC_OBRPC_LOG(WARN, "fill extra payload fail", K(ret), K(pos), K(payload));
|
||||
} else if (OB_FAIL(init_pkt(req.pkt(), pcode, opts, false))) {
|
||||
RPC_OBRPC_LOG(WARN, "Init packet error", K(ret));
|
||||
} else {
|
||||
int64_t timestamp = req.pkt()->get_timestamp();
|
||||
ObReqTransport::Result r;
|
||||
if (OB_FAIL(send_request(req, r))) {
|
||||
RPC_OBRPC_LOG(WARN, "send rpc request fail", K(pcode), K(ret));
|
||||
} else {
|
||||
rpc::RpcStatPiece piece;
|
||||
piece.size_ = 0;
|
||||
piece.time_ = ObTimeUtility::current_time() - timestamp;
|
||||
RPC_STAT(pcode, tenant_id_, piece);
|
||||
const char *buf = r.pkt()->get_cdata();
|
||||
int64_t len = r.pkt()->get_clen();
|
||||
int64_t pos = 0;
|
||||
UNIS_VERSION_GUARD(r.pkt()->get_unis_version());
|
||||
|
||||
if (OB_FAIL(rcode_.deserialize(buf, len, pos))) {
|
||||
RPC_OBRPC_LOG(WARN, "deserialize result code fail", K(ret));
|
||||
} else {
|
||||
int wb_ret = OB_SUCCESS;
|
||||
if (rcode_.rcode_ != OB_SUCCESS) {
|
||||
ret = rcode_.rcode_;
|
||||
RPC_OBRPC_LOG(WARN, "execute rpc fail", K(ret), K_(dst));
|
||||
} else if (OB_FAIL(common::serialization::decode(buf, len, pos, result))) {
|
||||
RPC_OBRPC_LOG(WARN, "deserialize result fail", K(ret));
|
||||
} else {
|
||||
ret = rcode_.rcode_;
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && NULL != handle) {
|
||||
handle->has_more_ = r.pkt()->is_stream_next();
|
||||
handle->dst_ = dst_;
|
||||
handle->sessid_ = r.pkt()->get_session_id();
|
||||
handle->opts_ = opts;
|
||||
handle->transport_ = transport_;
|
||||
handle->proxy_ = *this;
|
||||
handle->pcode_ = pcode;
|
||||
handle->do_ratelimit_ = do_ratelimit_;
|
||||
handle->is_bg_flow_ = is_bg_flow_;
|
||||
}
|
||||
if (common::OB_SUCCESS != (wb_ret = log_user_error_and_warn(rcode_))) {
|
||||
RPC_OBRPC_LOG(WARN, "fail to log user error and warn", K(ret), K(wb_ret), K((rcode_)));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
piece.size_ = PAYLOAD_SIZE;
|
||||
piece.time_ = ObTimeUtility::current_time() - start_ts;
|
||||
if (OB_FAIL(ret)) {
|
||||
piece.failed_ = true;
|
||||
if (OB_TIMEOUT == ret) {
|
||||
piece.is_timeout_ = true;
|
||||
}
|
||||
}
|
||||
RPC_STAT(pcode, tenant_id_, piece);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <class pcodeStruct>
|
||||
int ObRpcProxy::rpc_post(const typename pcodeStruct::Request &args,
|
||||
AsyncCB<pcodeStruct> *cb, const ObRpcOpts &opts)
|
||||
{
|
||||
POC_RPC_INTERCEPT(post, dst_, pcodeStruct::PCODE, args, *cb, opts);
|
||||
POC_RPC_INTERCEPT(post, dst_, pcodeStruct::PCODE, args, cb, opts);
|
||||
using namespace oceanbase::common;
|
||||
using namespace rpc::frame;
|
||||
int ret = OB_SUCCESS;
|
||||
|
337
deps/oblib/src/rpc/obrpc/ob_rpc_proxy_macros.h
vendored
337
deps/oblib/src/rpc/obrpc/ob_rpc_proxy_macros.h
vendored
@ -16,71 +16,6 @@
|
||||
#include "lib/utility/ob_macro_utils.h"
|
||||
#include "lib/oblog/ob_log_module.h"
|
||||
|
||||
#define SELECT4(a, b, c, d, ...) d
|
||||
#define SELECT5(a, b, c, d, e, ...) e
|
||||
|
||||
// AOR_, aka argument or result, accepts one argument which is a
|
||||
// typename. If the argument is surrounded by parenthesis, then the
|
||||
// type is represented as input argument type and result into "const
|
||||
// Type & args", otherwise the argument will be treated as result type
|
||||
// and produces "Type & result". Here's the explanation of expanding
|
||||
// step by step:
|
||||
//
|
||||
// AOR_((Type))
|
||||
// => CONCAT(IS_, NOT_CONST_P_ (Type)) & AOR_P_((Type))
|
||||
// => CONCAT(IS_, CONST_P_ Type) & CONCAT(IS_, RESULT_P_ (Type)) )
|
||||
// => IS_CONST_P_ Type & CONCAT(IS_, ARGS_P_) )
|
||||
// => const Type & IS_ARGS_P_ )
|
||||
// => const Type & args IGNORE_( )
|
||||
// => const Type & args
|
||||
//
|
||||
// AOR_(Type)
|
||||
// => CONCAT(IS_, NOT_CONST_P_ Type) & AOR_P_(Type)
|
||||
// => IS_NOT_CONST_P_ Type & CONCAT(IS_, RESULT_P_ Type) )
|
||||
// => Type & IS_RESULT_P_ Type )
|
||||
// => Type & result IGNORE_( Type )
|
||||
// => Type & result
|
||||
//
|
||||
#define RPM_ARGS(T) const T &args
|
||||
#define RPM_RESULT(T) T &result
|
||||
#define AOR_(T) IF_PAREN(T, RPM_ARGS, RPM_RESULT)
|
||||
|
||||
// AOR_P_ is the core macro used by macro AOR_, return "args" if it's
|
||||
// surrounded by parenthesis, "result" or not.
|
||||
//
|
||||
// AOR_P_((Type)) => args
|
||||
// AOR_P_(Type) => result
|
||||
//
|
||||
#define RPM_ARGS_P(T) args
|
||||
#define RPM_RESULT_P(T) result
|
||||
#define AOR_P_(T) IF_PAREN(T, RPM_ARGS_P, RPM_RESULT_P)
|
||||
|
||||
// SWITCH_IN_OUT_(Type) => (Type)
|
||||
// SWITCH_IN_OUT_((Type)) => Type
|
||||
#define RPM2INPUT(T) (T)
|
||||
#define RPM2OUPUT(T) T
|
||||
#define SWITCH_IN_OUT_(T) IF_PAREN(T, RPM2OUTPUT, RPM2INPUT)
|
||||
|
||||
// INPUT_TYPE_((Type)) => Type
|
||||
// INPUT_TYPE_(Type) => NoneT
|
||||
// OUTPUT_TYPE_((Type)) => NoneT
|
||||
// OUTPUT_TYPE_(Type) => Type
|
||||
#define RPM_SELF_TYPE(T) T
|
||||
#define RPM_NONE_TYPE(T) NoneT
|
||||
#define INPUT_TYPE_(T) IF_PAREN(T, RPM_SELF_TYPE, RPM_NONE_TYPE)
|
||||
#define OUTPUT_TYPE_(T) IF_PAREN(T, RPM_NONE_TYPE, RPM_SELF_TYPE)
|
||||
|
||||
// AP_AOR_(Type) => ,
|
||||
// AP_AOR_((Type)) => const Type &args,
|
||||
#define AP_IGNORE(T)
|
||||
#define AP_INPUT(T) const T &args,
|
||||
#define AP_AOR_(T) IF_PAREN(T, AP_INPUT, AP_IGNORE)
|
||||
|
||||
// AP_AOR_P_(Type) => ,
|
||||
// AP_AOR_P_((Type)) => const Type &args,
|
||||
#define AP_INPUT_P(T) args,
|
||||
#define AP_AOR_P_(T) IF_PAREN(T, AP_INPUT_P, AP_IGNORE)
|
||||
|
||||
#define OROP_ const ObRpcOpts &opts = ObRpcOpts()
|
||||
#define ORSSH_(pcode) SSHandle<pcode> &handle
|
||||
#define ORACB_(pcode) AsyncCB<pcode> *cb
|
||||
@ -110,68 +45,67 @@
|
||||
typedef OUTPUT_TYPE_(Output) Response; \
|
||||
};
|
||||
|
||||
#define OB_DEFINE_RPC_S2(name, pcode, prio, Input, Output) \
|
||||
OB_RPC_STRUCT(pcode, Input, Output) \
|
||||
virtual int name(AOR_(Input), AOR_(Output), OROP_) \
|
||||
#define OB_DEFINE_RPC_STRUCT(pcode, Input, Output) \
|
||||
template <typename IGNORE> \
|
||||
struct ObRpc<pcode, IGNORE> { \
|
||||
static constexpr auto PCODE = pcode; \
|
||||
typedef Input Request; \
|
||||
typedef Output Response; \
|
||||
};
|
||||
|
||||
#define RPC_CALL_DISPATCH(name, ...) \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
return mock_proxy_->name(__VA_ARGS__); \
|
||||
} else { \
|
||||
return name ##_(args, result, opts); \
|
||||
}
|
||||
|
||||
#define OB_DEFINE_RPC_SYNC(name, pcode, prio, Input, Output) \
|
||||
OB_DEFINE_RPC_STRUCT(pcode, Input, Output) \
|
||||
int name ## _(const Input& args, Output& result, OROP_) \
|
||||
{ \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
ret = mock_proxy_->name(args, result, opts); \
|
||||
} else { \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, args, result, NULL, newopts); \
|
||||
} \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, args, result, NULL, newopts); \
|
||||
return ret; \
|
||||
} \
|
||||
}
|
||||
|
||||
#define OB_DEFINE_RPC_S1(name, pcode, prio, InOut) \
|
||||
OB_RPC_STRUCT(pcode, InOut, InOut) \
|
||||
virtual int name(AOR_(InOut), OROP_) \
|
||||
{ \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
ret = mock_proxy_->name(AOR_P_(InOut), opts); \
|
||||
} else { \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, AOR_P_(InOut), NULL, newopts); \
|
||||
} \
|
||||
return ret; \
|
||||
} \
|
||||
#define OB_DEFINE_RPC_S2_(name, pcode, prio, Input, Output) \
|
||||
OB_DEFINE_RPC_SYNC(name, pcode, prio, Input, Output); \
|
||||
virtual int name(const Input& args, Output& result, OROP_) { \
|
||||
RPC_CALL_DISPATCH(name, args, result, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_S2(name, pcode, prio, Input, Output) OB_DEFINE_RPC_S2_(name, pcode, prio, EXPAND Input, Output)
|
||||
|
||||
#define OB_DEFINE_RPC_S0(name, pcode, prio) \
|
||||
OB_RPC_STRUCT(pcode, (NoneT), NoneT) \
|
||||
virtual int name(OROP_) \
|
||||
{ \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
ret = mock_proxy_->name(opts); \
|
||||
} else { \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, NULL, newopts); \
|
||||
} \
|
||||
return ret; \
|
||||
} \
|
||||
#define OB_DEFINE_RPC_S1_INPUT_(name, pcode, prio, Input) \
|
||||
OB_DEFINE_RPC_SYNC(name, pcode, prio, Input, NoneT); \
|
||||
virtual int name(const Input& args, OROP_) { \
|
||||
NoneT result; \
|
||||
RPC_CALL_DISPATCH(name, args, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_S1_INPUT(name, pcode, prio, Input) OB_DEFINE_RPC_S1_INPUT_(name, pcode, prio, EXPAND Input)
|
||||
#define OB_DEFINE_RPC_S1_OUTPUT(name, pcode, prio, Output) \
|
||||
OB_DEFINE_RPC_SYNC(name, pcode, prio, NoneT, Output); \
|
||||
virtual int name(Output& result, OROP_) { \
|
||||
const NoneT args; \
|
||||
RPC_CALL_DISPATCH(name, result, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_S1(name, pcode, prio, InOut) IF_IS_PAREN(InOut, OB_DEFINE_RPC_S1_INPUT, OB_DEFINE_RPC_S1_OUTPUT)(name, pcode, prio, InOut)
|
||||
|
||||
#define OB_DEFINE_RPC_S0(name, pcode, prio) \
|
||||
OB_DEFINE_RPC_SYNC(name, pcode, prio, NoneT, NoneT); \
|
||||
virtual int name(OROP_) { \
|
||||
const NoneT args; \
|
||||
NoneT result; \
|
||||
RPC_CALL_DISPATCH(name, opts); \
|
||||
}
|
||||
|
||||
#define OB_DEFINE_RPC_S(prio, name, pcode, ...) \
|
||||
SELECT4(, \
|
||||
@ -182,58 +116,46 @@
|
||||
|
||||
#define RPC_S(args...) _CONCAT(OB_DEFINE_RPC, _S IGNORE_(args))
|
||||
|
||||
// define synchronized stream interface
|
||||
#define OB_DEFINE_RPC_SS2(name, pcode, prio, Input, Output) \
|
||||
OB_RPC_STRUCT(pcode, Input, Output) \
|
||||
virtual int name(AOR_(Input), AOR_(Output), ORSSH_(pcode), OROP_) \
|
||||
{ \
|
||||
#define OB_DEFINE_RPC_STREAM(name, pcode, prio, Input, Output) \
|
||||
OB_DEFINE_RPC_STRUCT(pcode, Input, Output); \
|
||||
int name ##_(const Input& args, Output& result, ORSSH_(pcode), OROP_) { \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, args, result, &handle, newopts); \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, args, result, &handle, newopts); \
|
||||
return ret; \
|
||||
} \
|
||||
|
||||
#define OB_DEFINE_RPC_SS1(name, pcode, prio, InOut) \
|
||||
OB_RPC_STRUCT(pcode, InOut, InOut) \
|
||||
virtual int name(AOR_(InOut), ORSSH_(pcode), OROP_) \
|
||||
{ \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, AOR_P_(InOut), &handle, newopts); \
|
||||
return ret; \
|
||||
} \
|
||||
}
|
||||
// define synchronized stream interface
|
||||
#define OB_DEFINE_RPC_SS2_(name, pcode, prio, Input, Output) \
|
||||
OB_DEFINE_RPC_STREAM(name, pcode, prio, Input, Output); \
|
||||
virtual int name(const Input& args, Output& result, ORSSH_(pcode), OROP_) { \
|
||||
return name ##_(args, result, handle, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_SS2(name, pcode, prio, Input, Output) OB_DEFINE_RPC_SS2_(name, pcode, prio, EXPAND Input, Output)
|
||||
#define OB_DEFINE_RPC_SS1_INPUT_(name, pcode, prio, Input) \
|
||||
OB_DEFINE_RPC_STREAM(name, pcode, prio, Input, NoneT); \
|
||||
virtual int name(const Input& args, ORSSH_(pcode), OROP_) { \
|
||||
NoneT result; \
|
||||
return name ##_(args, result, handle, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_SS1_INPUT(name, pcode, prio, Input) OB_DEFINE_RPC_SS1_INPUT_(name, pcode, prio, EXPAND Input)
|
||||
#define OB_DEFINE_RPC_SS1_OUTPUT(name, pcode, prio, Output) \
|
||||
OB_DEFINE_RPC_STREAM(name, pcode, prio, NoneT, Output); \
|
||||
virtual int name(Output& result, ORSSH_(pcode), OROP_) { \
|
||||
NoneT args; \
|
||||
return name ##_(args, result, handle, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_SS1(name, pcode, prio, InOut) IF_IS_PAREN(InOut, OB_DEFINE_RPC_SS1_INPUT, OB_DEFINE_RPC_SS1_OUTPUT)(name, pcode, prio, InOut)
|
||||
|
||||
// Theoretically, stream rpc without argument or result is
|
||||
// impossible. We add this SS0 interface just complete our rpc
|
||||
// framework.
|
||||
#define OB_DEFINE_RPC_SS0(name, pcode, prio) \
|
||||
OB_RPC_STRUCT(pcode, (NoneT), NoneT) \
|
||||
virtual int name(ORSSH_(pcode), OROP_) \
|
||||
{ \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_call(pcode, &handle, newopts); \
|
||||
return ret; \
|
||||
} \
|
||||
|
||||
#define OB_DEFINE_RPC_SS0(name, pcode, prio)
|
||||
#define OB_DEFINE_RPC_SS(prio, name, pcode, ...) \
|
||||
SELECT4(, \
|
||||
## __VA_ARGS__, \
|
||||
@ -244,9 +166,17 @@
|
||||
#define RPC_SS(args...) _CONCAT(OB_DEFINE_RPC, _SS IGNORE_(args))
|
||||
|
||||
// define asynchronous interface
|
||||
#define OB_DEFINE_RPC_AP2(name, pcode, prio, Input, Output) \
|
||||
OB_RPC_STRUCT(pcode, Input, Output) \
|
||||
virtual int name(AOR_(Input), ORACB_(pcode), OROP_) \
|
||||
#define OB_RPC_ASYNC_DISPATCH(name, ...) \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
return mock_proxy_->name(__VA_ARGS__); \
|
||||
} else { \
|
||||
return name##_(args, cb, opts); \
|
||||
}
|
||||
|
||||
#define OB_DEFINE_RPC_ASYNC(name, pcode, prio, Input, Output) \
|
||||
OB_DEFINE_RPC_STRUCT(pcode, Input, Output); \
|
||||
int name##_(const Input& args, ORACB_(pcode), OROP_) \
|
||||
{ \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
@ -254,60 +184,39 @@
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
ret = mock_proxy_->name(args, cb, newopts); \
|
||||
} else { \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_post<ObRpc<pcode>>(args, cb, newopts); \
|
||||
} \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_post<ObRpc<pcode>>(args, cb, newopts); \
|
||||
return ret; \
|
||||
}
|
||||
|
||||
#define OB_DEFINE_RPC_AP1(name, pcode, prio, InOut) \
|
||||
OB_RPC_STRUCT(pcode, InOut, InOut) \
|
||||
virtual int name(AP_AOR_(InOut) ORACB_(pcode), OROP_) \
|
||||
{ \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
ret = mock_proxy_->name(AP_AOR_P_(InOut) cb, newopts); \
|
||||
} else { \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_post<ObRpc<pcode>>(AP_AOR_P_(InOut) cb, newopts); \
|
||||
} \
|
||||
return ret; \
|
||||
#define OB_DEFINE_RPC_AP2_(name, pcode, prio, Input, Output) \
|
||||
OB_DEFINE_RPC_ASYNC(name, pcode, prio, Input, Output); \
|
||||
virtual int name(const Input& args, ORACB_(pcode), OROP_) { \
|
||||
OB_RPC_ASYNC_DISPATCH(name, args, cb, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_AP2(name, pcode, prio, Input, Output) OB_DEFINE_RPC_AP2_(name, pcode, prio, EXPAND Input, Output)
|
||||
|
||||
#define OB_DEFINE_RPC_AP1_INPUT_(name, pcode, prio, Input) \
|
||||
OB_DEFINE_RPC_ASYNC(name, pcode, prio, Input, NoneT); \
|
||||
virtual int name(const Input& args, ORACB_(pcode), OROP_) { \
|
||||
OB_RPC_ASYNC_DISPATCH(name, args, cb, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_AP1_OUTPUT(name, pcode, prio, Output) \
|
||||
OB_DEFINE_RPC_ASYNC(name, pcode, prio, NoneT, Output); \
|
||||
virtual int name(ORACB_(pcode), OROP_) { \
|
||||
OB_RPC_ASYNC_DISPATCH(name, cb, opts); \
|
||||
}
|
||||
#define OB_DEFINE_RPC_AP1_INPUT(name, pcode, prio, InOut) OB_DEFINE_RPC_AP1_INPUT_(name, pcode, prio, EXPAND InOut)
|
||||
#define OB_DEFINE_RPC_AP1(name, pcode, prio, InOut) IF_IS_PAREN(InOut, OB_DEFINE_RPC_AP1_INPUT, OB_DEFINE_RPC_AP1_OUTPUT)(name, pcode, prio, InOut)
|
||||
|
||||
#define OB_DEFINE_RPC_AP0(name, pcode, prio) \
|
||||
OB_DEFINE_RPC_ASYNC(name, pcode, prio, NoneT, NoneT); \
|
||||
virtual int name(ORACB_(pcode), OROP_) { \
|
||||
OB_RPC_ASYNC_DISPATCH(name, cb, opts); \
|
||||
}
|
||||
|
||||
#define OB_DEFINE_RPC_AP0(name, pcode, prio) \
|
||||
OB_RPC_STRUCT(pcode, (NoneT), NoneT) \
|
||||
virtual int name(ORACB_(pcode), OROP_) \
|
||||
{ \
|
||||
const static ObRpcPriority PR = prio; \
|
||||
int ret = common::OB_SUCCESS; \
|
||||
ObRpcOpts newopts = opts; \
|
||||
if (newopts.pr_ == ORPR_UNDEF) { \
|
||||
newopts.pr_ = PR; \
|
||||
} \
|
||||
if (mock_proxy_) { \
|
||||
mock_proxy_->set_server(dst_); \
|
||||
ret = mock_proxy_->name(cb, newopts); \
|
||||
} else { \
|
||||
newopts.ssl_invited_nodes_ = GCONF._ob_ssl_invited_nodes.get_value_string(); \
|
||||
newopts.local_addr_ = GCTX.self_addr(); \
|
||||
ret = rpc_post(pcode, cb, newopts); \
|
||||
} \
|
||||
return ret; \
|
||||
}
|
||||
|
||||
|
||||
#define SELECT4(a, b, c, d, ...) d
|
||||
#define OB_DEFINE_RPC_AP(prio, name, pcode, ...) \
|
||||
SELECT4(, \
|
||||
## __VA_ARGS__, \
|
||||
|
18
deps/oblib/src/rpc/pnio/README.md
vendored
Normal file
18
deps/oblib/src/rpc/pnio/README.md
vendored
Normal file
@ -0,0 +1,18 @@
|
||||
# pkt-nio
|
||||
|
||||
## API
|
||||
see interface/group.h
|
||||
|
||||
## test
|
||||
```
|
||||
make test/test-group
|
||||
test/test-group
|
||||
```
|
||||
|
||||
## example
|
||||
```
|
||||
#define rk_log_macro(level, format, ...) _OB_LOG(level, "PNIO " format, ##__VA_ARGS__)
|
||||
#include "interface/group.h"
|
||||
...
|
||||
#include "interface/pkt-nio.c"
|
||||
```
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user