[FEAT MERGE] performance optimzation for OLTP

Co-authored-by: dimstars <liangjinrongcm@gmail.com>
Co-authored-by: pe-99y <315053752@qq.com>
This commit is contained in:
Naynahs 2024-04-10 07:32:27 +00:00 committed by ob-robot
parent 054f5a5a80
commit 3d4ef9741d
177 changed files with 7111 additions and 9708 deletions

View File

@ -57,7 +57,7 @@ endif()
ob_define(AUTO_FDO_OPT "")
if(ENABLE_AUTO_FDO)
# file name pattern [observer.prof.{current_timestamp ms}]
set(AUTO_FDO_OPT "-fprofile-sample-use=${CMAKE_SOURCE_DIR}/observer.prof.1702984872675")
set(AUTO_FDO_OPT "-finline-functions -fprofile-sample-use=${CMAKE_SOURCE_DIR}/observer.prof.1702984872675")
endif()
ob_define(THIN_LTO_OPT "")

View File

@ -144,10 +144,7 @@ public:
if (obj_ptr_ == rhs.obj_ptr_) {
cmp = static_cast<int32_t>(obj_cnt_ - rhs.obj_cnt_);
} else {
ret = compare_prefix(rhs, cmp);
if (0 == cmp) {
cmp = static_cast<int32_t>(obj_cnt_ - rhs.obj_cnt_);
}
cmp = fast_compare(rhs);
}
return ret;
}
@ -158,10 +155,7 @@ public:
if (obj_ptr_ == rhs.obj_ptr_) {
cmp = static_cast<int32_t>(obj_cnt_ - rhs.obj_cnt_);
} else {
cmp = compare_prefix(rhs);
if (0 == cmp) {
cmp = static_cast<int32_t>(obj_cnt_ - rhs.obj_cnt_);
}
cmp = fast_compare(rhs);
}
return cmp;
}
@ -214,6 +208,50 @@ public:
return cmp;
}
OB_INLINE int32_t fast_compare(const ObRowkey &rhs) const
{
int32_t cmp = 0;
int64_t i = 0;
for (; i < obj_cnt_ && 0 == cmp; ++i) {
__builtin_prefetch(&rhs.obj_ptr_[i]);
if (i < rhs.obj_cnt_) {
// optimize for int
if (obj_ptr_[i].is_int32() && rhs.obj_ptr_[i].is_int32()) {
int32_t left = obj_ptr_[i].get_int32();
int32_t right = rhs.obj_ptr_[i].get_int32();
if (left > right) {
cmp = 1;
} else if (left < right) {
cmp = -1;
} else {
cmp = 0;
}
} else if (obj_ptr_[i].is_int() && rhs.obj_ptr_[i].is_int()) {
int64_t left = obj_ptr_[i].get_int();
int64_t right = rhs.obj_ptr_[i].get_int();
if (left > right) {
cmp = 1;
} else if (left < right) {
cmp = -1;
} else {
cmp = 0;
}
} else {
cmp = obj_ptr_[i].compare(rhs.obj_ptr_[i]);
}
} else {
cmp = 1;
}
}
// rhs.cnt > this.cnt
if (0 == cmp && i < rhs.obj_cnt_) {
cmp = -1;
}
return cmp;
}
inline bool operator<(const ObRowkey &rhs) const
{
return compare(rhs) < 0;

View File

@ -73,9 +73,12 @@ void ObTLTaGuard::revert()
}
ObMallocAllocator::ObMallocAllocator()
: locks_(), allocators_(), unrecycled_lock_(), unrecycled_allocators_(),
: locks_(), allocators_(), unrecycled_lock_(false), unrecycled_allocators_(),
reserved_(0), urgent_(0), max_used_tenant_id_(0), create_on_demand_(false)
{
for (int64_t i = 0; i < PRESERVED_TENANT_COUNT; ++i) {
locks_[i].enable_record_stat(false);
}
set_root_allocator();
is_inited_ = true;
}

View File

@ -197,6 +197,12 @@ private:
locks_[i].unlock();
}
}
void enable_record_stat(const bool need_record)
{
for (int64_t i = 0; i < BUCKET_COUNT; ++i) {
locks_[i].enable_record_stat(need_record);
}
}
private:
ObLatch locks_[BUCKET_COUNT];
};

View File

@ -118,19 +118,29 @@ int ObLatchMutex::lock(
//wait
waited = true;
// latch mutex wait is an atomic wait event
ObLatchWaitEventGuard wait_guard(
ObLatchDesc::wait_event_idx(latch_id),
abs_timeout_us / 1000,
reinterpret_cast<uint64_t>(this),
(uint32_t*)&lock_.val(),
0,
true /*is_atomic*/);
if (OB_FAIL(wait(abs_timeout_us, uid))) {
if (OB_TIMEOUT != ret) {
COMMON_LOG(WARN, "Fail to wait the latch, ", K(ret));
if (record_stat_) {
ObLatchWaitEventGuard wait_guard(
ObLatchDesc::wait_event_idx(latch_id),
abs_timeout_us / 1000,
reinterpret_cast<uint64_t>(this),
(uint32_t*)&lock_.val(),
0,
true /*is_atomic*/);
if (OB_FAIL(wait(abs_timeout_us, uid))) {
if (OB_TIMEOUT != ret) {
COMMON_LOG(WARN, "Fail to wait the latch, ", K(ret));
}
} else {
break;
}
} else {
break;
if (OB_FAIL(wait(abs_timeout_us, uid))) {
if (OB_TIMEOUT != ret) {
COMMON_LOG(WARN, "Fail to wait the latch, ", K(ret));
}
} else {
break;
}
}
}
}
@ -146,7 +156,7 @@ int ObLatchMutex::wait(const int64_t abs_timeout_us, const uint32_t uid)
{
// performance critical, do not double check the parameters
int ret = OB_SUCCESS;
ObDiagnoseSessionInfo *dsi = ObDiagnoseSessionInfo::get_local_diagnose_info();
ObDiagnoseSessionInfo *dsi = (!record_stat_ ? NULL : ObDiagnoseSessionInfo::get_local_diagnose_info());
int64_t timeout = 0;
int lock = 0;
@ -570,19 +580,6 @@ void ObLockDiagnose::print()
}
#endif
ObLatch::ObLatch()
: lock_(0)
, record_stat_(true)
{
}
ObLatch::~ObLatch()
{
if (0 != lock_) {
COMMON_LOG(DEBUG, "invalid lock,", K(lock_), KCSTRING(lbt()));
}
}
int ObLatch::try_rdlock(const uint32_t latch_id)
{
int ret = OB_SUCCESS;

View File

@ -39,7 +39,7 @@ extern bool USE_CO_LATCH;
#define TRY_LOCK_RECORD_STAT(latch_id, spin_cnt, ret) \
do { \
if (lib::is_diagnose_info_enabled()) { \
if (record_stat_ && lib::is_diagnose_info_enabled()) { \
ObDiagnoseTenantInfo *di = ObDiagnoseTenantInfo::get_local_diagnose_info(); \
if (NULL != di) { \
ObLatchStat *p_latch_stat = di->get_latch_stats().get_or_create_item(latch_id); \
@ -59,7 +59,7 @@ extern bool USE_CO_LATCH;
#define LOCK_RECORD_STAT(latch_id, waited, spin_cnt, yield_cnt) \
do { \
if (lib::is_diagnose_info_enabled()) { \
if (record_stat_ && lib::is_diagnose_info_enabled()) { \
ObDiagnoseTenantInfo *di = ObDiagnoseTenantInfo::get_local_diagnose_info(); \
if (NULL != di) { \
ObLatchStat *p_latch_stat = di->get_latch_stats().get_or_create_item(latch_id); \
@ -213,8 +213,18 @@ class ObLatch
{
friend class TCRWLock;
public:
ObLatch();
~ObLatch();
ObLatch(const bool need_record_stat = true)
: lock_(0)
, record_stat_(need_record_stat)
{
}
~ObLatch()
{
if (0 != lock_) {
COMMON_LOG(DEBUG, "invalid lock,", K(lock_), KCSTRING(lbt()));
}
}
int try_rdlock(const uint32_t latch_id);
int try_wrlock(const uint32_t latch_id, const uint32_t *puid = NULL);
int rdlock(

View File

@ -63,10 +63,7 @@ inline int ObRecursiveMutex::lock()
inline int ObRecursiveMutex::unlock()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!latch_.is_wrlocked_by())) {
ret = OB_ERR_UNEXPECTED;
COMMON_LOG(WARN, "The ObRecursiveMutex is not locked, ", K_(latch_id), K(ret));
} else if (0 == --lock_cnt_) {
if (0 == --lock_cnt_) {
if (OB_FAIL(latch_.unlock())) {
COMMON_LOG(WARN, "Fail to unlock the ObRecursiveMutex, ", K_(latch_id), K(ret));
}

View File

@ -395,7 +395,6 @@ const int64_t OB_MAX_TABLE_NUM_PER_STMT = 256;
const int32_t OB_TMP_BUF_SIZE_256 = 256;
const int64_t OB_SCHEMA_MGR_MAX_USED_TID_MAP_BUCKET_NUM = 64;
const int64_t OB_MAX_SLAVE_READ_DELAY_TS = 5 * 1000 * 1000;
const int64_t OB_SKIP_RANGE_LIMIT = 256;
const int64_t OB_MAX_DIRECTORY_NAME_LENGTH = 128; // Compatible with Oracle
const int64_t OB_MAX_DIRECTORY_PATH_LENGTH = 4000; // Compatible with Oracle

View File

@ -35,6 +35,7 @@
#include <lib/utility/utility.h>
#include <lib/cpu/ob_cpu_topology.h>
#include "lib/ob_running_mode.h"
#include "lib/lock/ob_small_spin_lock.h"
#define PTR_META2OBJ(x) reinterpret_cast<T*>(reinterpret_cast<char*>(x) + sizeof(Meta));
#define PTR_OBJ2META(y) reinterpret_cast<Meta*>(reinterpret_cast<char*>(y) - sizeof(Meta));
@ -46,7 +47,7 @@ namespace common
struct ObPoolArenaHead
{
ObLatch lock; // Lock, 4 Bytes
common::ObByteLock lock; // Lock, 4 Bytes
int32_t borrow_cnt; // Number of calls
int32_t return_cnt; // Number of calls
int32_t miss_cnt; // Number of direct allocations
@ -61,7 +62,6 @@ struct ObPoolArenaHead
void *next; // Point to the first free object
TO_STRING_KV(
KP(this),
K(lock),
K(borrow_cnt),
K(return_cnt),
K(miss_cnt),
@ -72,7 +72,7 @@ struct ObPoolArenaHead
K(last_miss_return_ts),
KP(next));
void reset() {
new (&lock) ObLatch();
new (&lock) common::ObByteLock();
borrow_cnt = 0;
return_cnt = 0;
miss_cnt = 0;
@ -202,19 +202,11 @@ public:
int64_t itid = get_itid();
int64_t aid = itid % arena_num_;
ObPoolArenaHead &arena = arena_[aid];
int64_t cur_ts = ObClockGenerator::getClock();
{ // Enter the critical area of the arena, the timestamp is obtained outside the lock, and minimize the length of the critical area
ObLatchWGuard lock_guard(arena.lock, ObLatchIds::SERVER_OBJECT_POOL_ARENA_LOCK);
ObSmallSpinLockGuard<common::ObByteLock> lock_guard(arena.lock);
cmeta = static_cast<Meta*>(arena.next);
if (NULL != cmeta) {
arena.next = static_cast<void*>(cmeta->next);
arena.borrow_cnt++;
arena.free_num--;
arena.all_using_cnt++;
arena.last_borrow_ts = cur_ts;
} else {
arena.miss_cnt++;
arena.last_miss_ts = cur_ts;
}
}
if (NULL != cmeta) {
@ -233,10 +225,11 @@ public:
cmeta->magic = 0xFEDCFEDC01230123;
ctx = PTR_META2OBJ(p);
new (ctx) T();
ObLatchWGuard lock_guard(arena.lock, ObLatchIds::SERVER_OBJECT_POOL_ARENA_LOCK);
arena.all_using_cnt++;
}
}
if (NULL != ctx) {
ATOMIC_INC(&arena.all_using_cnt);
}
}
return ctx;
}
@ -252,27 +245,17 @@ public:
if (aid >= 0) {
x->reset();
ObPoolArenaHead &arena = arena_[aid];
int64_t cur_ts = ObClockGenerator::getClock();
{ // Enter the critical area of the arena, the timestamp is obtained outside the lock, and minimize the length of the critical area
ObLatchWGuard lock_guard(arena.lock, ObLatchIds::SERVER_OBJECT_POOL_ARENA_LOCK);
ObSmallSpinLockGuard<common::ObByteLock> lock_guard(arena.lock);
cmeta->next = static_cast<Meta*>(arena.next);
arena.next = static_cast<void*>(cmeta);
arena.return_cnt++;
arena.free_num++;
arena.all_using_cnt--;
arena.last_return_ts = cur_ts;
ATOMIC_DEC(&arena.all_using_cnt);
}
} else {
x->~T();
ob_free(cmeta);
ObPoolArenaHead &arena = arena_[-(aid + 1)];
int64_t cur_ts = ObClockGenerator::getClock();
{ // Enter the critical area of the arena, the timestamp is obtained outside the lock, and minimize the length of the critical area
ObLatchWGuard lock_guard(arena.lock, ObLatchIds::SERVER_OBJECT_POOL_ARENA_LOCK);
arena.miss_return_cnt++;
arena.all_using_cnt--;
arena.last_miss_return_ts = cur_ts;
}
ATOMIC_DEC(&arena.all_using_cnt);
}
}
}
@ -329,18 +312,6 @@ public:
ObPoolArenaHead &arena = arena_[i];
arena.reset();
arena.next = static_cast<void*>(pmeta);
arena.free_num = static_cast<int16_t>(cnt_per_arena_);
}
if (regist_) {
// Register to the global list, display and print the log in the virtual table
if (OB_FAIL(ObServerObjectPoolRegistry::add(typeid(T).name(), arena_, arena_num_))) {
COMMON_LOG(WARN, "add to pool registry failed, can't be monitored", K(ret), K(typeid(T).name()), KP(this), K(this));
} else {
COMMON_LOG(INFO, "register server object pool finish",
"tpye_name", typeid(T).name(),
"type_size", sizeof(T),
KP(this), K(this));
}
}
}
if (OB_SUCC(ret)) {
@ -354,15 +325,14 @@ public:
bool has_unfree = false;
for (int64_t i = 0; !has_unfree && i < arena_num_; ++i) {
ObPoolArenaHead &arena = arena_[i];
ObLatchWGuard lock_guard(arena.lock, ObLatchIds::SERVER_OBJECT_POOL_ARENA_LOCK);
has_unfree = arena.all_using_cnt > 0;
has_unfree = ATOMIC_LOAD(&arena.all_using_cnt) > 0;
}
if (!has_unfree) {
for (int64_t i = 0; i < arena_num_; ++i) {
Meta *meta = NULL;
{
ObPoolArenaHead &arena = arena_[i];
ObLatchWGuard lock_guard(arena.lock, ObLatchIds::SERVER_OBJECT_POOL_ARENA_LOCK);
ObSmallSpinLockGuard<common::ObByteLock> lock_guard(arena.lock);
meta = static_cast<Meta*>(arena.next);
arena.next = NULL;
}

View File

@ -57,7 +57,8 @@ public:
~ObMysqlPktContext() {}
void reset()
{
MEMSET(header_buf_, 0, common::OB_MYSQL_HEADER_LENGTH);
static_assert(common::OB_MYSQL_HEADER_LENGTH == 4, "OB_MYSQL_HEADER_LENGTH != 4");
*reinterpret_cast<uint32_t *>(header_buf_) = 0;
header_buffered_len_ = 0;
payload_buf_alloc_len_ = 0;
payload_buf_ = NULL;

View File

@ -384,14 +384,14 @@ class ObSqlSock: public ObLink
public:
ObSqlSock(ObSqlNioImpl *nio, int fd): dlink_(), all_list_link_(), write_task_link_(), nio_impl_(nio), fd_(fd),
err_(0), reader_(fd), read_buffer_(reader_), need_epoll_trigger_write_(false), may_handling_(true),
handler_close_flag_(false), need_shutdown_(false), last_decode_time_(0), last_write_time_(0),
handler_close_flag_(false), need_shutdown_(false), last_decode_time_(0),
sql_session_info_(NULL), tls_verion_option_(SSL_OP_NO_SSLv2|SSL_OP_NO_SSLv3) {
memset(sess_, 0, sizeof(sess_));
}
~ObSqlSock() {}
int64_t get_remain_sz() const { return read_buffer_.get_remain_sz(); }
TO_STRING_KV(KP(this), "session_id", get_sql_session_id(), "trace_id", get_trace_id(), "sql_handling_stage", get_sql_request_execute_state(), "sql_initiative_shutdown", need_shutdown_,
K_(reader), K_(err), K_(last_decode_time), K_(last_write_time), K_(pending_write_task), K_(need_epoll_trigger_write),
TO_STRING_KV(KP(this), "session_id", get_sql_session_id(), "trace_id", get_trace_id(), "sql_handling_stage", get_sql_request_execute_state(),
"sql_initiative_shutdown", need_shutdown_, K_(reader), K_(err), K_(last_decode_time), K_(pending_write_task), K_(need_epoll_trigger_write),
"consume_size", read_buffer_.get_consume_sz(), "pending_flag", get_pending_flag(), "may_handling_flag", get_may_handling_flag(), K_(handler_close_flag));
ObSqlNioImpl *get_nio_impl() { return nio_impl_; }
void set_nio_impl(ObSqlNioImpl *impl) { nio_impl_ = impl; }
@ -461,7 +461,6 @@ public:
need_epoll_trigger_write_ = false;
LOG_WARN("pending write task write fail", K(ret));
} else if (become_clean) {
last_write_time_ = ObTimeUtility::current_time();
need_epoll_trigger_write_ = false;
LOG_DEBUG("pending write clean", K(this));
} else {
@ -488,7 +487,6 @@ public:
LOG_WARN("write data error", K(errno));
}
}
last_write_time_ = ObTimeUtility::current_time();
return ret;
}
@ -499,9 +497,8 @@ public:
int get_fd() { return fd_; }
void disable_may_handling_flag() { ATOMIC_STORE(&may_handling_, false); }
bool get_may_handling_flag() const { return ATOMIC_LOAD(&may_handling_); }
void set_last_decode_time() { last_decode_time_ = ObTimeUtility::current_time(); }
void set_last_decode_time() { last_decode_time_ = ObClockGenerator::getClock(); }
int64_t get_last_decode_time() const { return last_decode_time_; }
int64_t get_last_write_time() const { return last_write_time_; }
int on_disconnect() {
ObSqlSockSession* sess = (ObSqlSockSession *)sess_;
return sess->on_disconnect();
@ -541,7 +538,6 @@ private:
bool handler_close_flag_;
bool need_shutdown_;
int64_t last_decode_time_;
int64_t last_write_time_;
void* sql_session_info_;
uint64_t tls_verion_option_;
private:
@ -611,7 +607,6 @@ int ObSqlSock::write_handshake_packet(const char* buf, int64_t sz) {
LOG_WARN("write data error", K(fd_), K(errno));
}
}
last_write_time_ = ObTimeUtility::current_time();
return ret;
}
@ -889,14 +884,14 @@ public:
}
void revert_sock(ObSqlSock* s) {
if (OB_UNLIKELY(s->has_error())) {
LOG_TRACE("revert_sock: sock has error", K(*s));
LOG_DEBUG("revert_sock: sock has error", K(*s));
s->disable_may_handling_flag();
} else if (OB_UNLIKELY(s->need_shutdown())) {
LOG_TRACE("sock revert succ and push to close req queue", K(*s));
LOG_DEBUG("sock revert succ and push to close req queue", K(*s));
push_close_req(s);
s->disable_may_handling_flag();
} else if (OB_UNLIKELY(!s->end_handle())) {
LOG_TRACE("revert_sock: sock still readable", K(*s));
LOG_DEBUG("revert_sock: sock still readable", K(*s));
int ret = OB_SUCCESS;
if (OB_FAIL(handler_.on_readable(s->sess_))) {
LOG_WARN("push to omt queue fail, will close socket", K(ret), K(*s));
@ -1160,7 +1155,7 @@ private:
ObSqlSock* s = CONTAINER_OF(cur, ObSqlSock, all_list_link_);
cur = cur->next_;
if (s->get_pending_flag()) {
int64_t time_interval = ObTimeUtility::current_time() - s->get_last_decode_time();
int64_t time_interval = ObClockGenerator::getClock() - s->get_last_decode_time();
if (time_interval > max_process_time) {
LOG_INFO("[sql nio session]", K(*s));
}

View File

@ -127,7 +127,7 @@ int ObSqlSockHandler::on_readable(void* udata)
}
if (OB_SUCCESS != ret || NULL == sql_req) {
} else if (FALSE_IT(sess->set_last_decode_succ_and_deliver_time(ObTimeUtility::current_time()))) {
} else if (FALSE_IT(sess->set_last_decode_succ_and_deliver_time(ObClockGenerator::getClock()))) {
} else if (OB_FAIL(deliver_->deliver(*sql_req))) {
LOG_WARN("deliver sql request fail", K(ret), K(sess->sql_session_id_));
}

View File

@ -136,7 +136,7 @@ enum {
int64_t get_usec()
{
return ObTimeUtility::current_time();
return ObClockGenerator::getClock();
}
int set_nonblocking(int fd)

10929
hotfuncs.txt

File diff suppressed because it is too large Load Diff

View File

@ -62,9 +62,6 @@ TEST_F(TestSharedMacroBlk, test_used_size_mgr)
OK(shared_mgr.add_block(id1, legal_size));
OK(shared_mgr.free_block(id1, legal_size)); // delete id from mgr
ASSERT_EQ(OB_ENTRY_NOT_EXIST, shared_mgr.block_used_size_.get(id1, size));
OK(shared_mgr.free_block(id1, legal_size));
OK(shared_mgr.add_block(id1, legal_size)); // delete id from mgr
ASSERT_EQ(OB_ENTRY_NOT_EXIST, shared_mgr.block_used_size_.get(id1, size));
const int64_t recyclable_size = ObSharedMacroBlockMgr::RECYCLABLE_BLOCK_SIZE - 1;
OK(shared_mgr.add_block(id1, recyclable_size));

View File

@ -47,19 +47,19 @@ int ObMvccWriteGuard::write_auth(storage::ObStoreCtx &) { return OB_SUCCESS; }
ObMvccWriteGuard::~ObMvccWriteGuard() {}
void *ObMemtableCtx::callback_alloc(const int64_t size)
void *ObMemtableCtx::alloc_mvcc_row_callback()
{
void* ret = NULL;
if (OB_ISNULL(ret = std::malloc(size))) {
TRANS_LOG_RET(ERROR, OB_ALLOCATE_MEMORY_FAILED, "callback alloc error, no memory", K(size), K(*this));
if (OB_ISNULL(ret = std::malloc(sizeof(ObMvccRowCallback)))) {
TRANS_LOG_RET(ERROR, OB_ALLOCATE_MEMORY_FAILED, "callback alloc error, no memory", K(*this));
} else {
ATOMIC_FAA(&callback_mem_used_, size);
ATOMIC_FAA(&callback_mem_used_, sizeof(ObMvccRowCallback));
ATOMIC_INC(&callback_alloc_count_);
}
return ret;
}
void ObMemtableCtx::callback_free(ObITransCallback *cb)
void ObMemtableCtx::free_mvcc_row_callback(ObITransCallback *cb)
{
if (OB_ISNULL(cb)) {
TRANS_LOG_RET(ERROR, OB_ERR_UNEXPECTED, "cb is null, unexpected error", KP(cb), K(*this));
@ -412,7 +412,6 @@ public:
store_ctx->mvcc_acc_ctx_.tx_ctx_ = tx_ctx;
store_ctx->mvcc_acc_ctx_.mem_ctx_ = &(tx_ctx->mt_ctx_);
store_ctx->mvcc_acc_ctx_.mem_ctx_->set_trans_ctx(tx_ctx);
store_ctx->mvcc_acc_ctx_.mem_ctx_->get_tx_table_guard()->init(&tx_table_);
tx_ctx->mt_ctx_.log_gen_.set(&(tx_ctx->mt_ctx_.trans_mgr_),
&(tx_ctx->mt_ctx_));
store_ctx->mvcc_acc_ctx_.snapshot_.tx_id_ = tx_id;

View File

@ -440,15 +440,99 @@ TEST_F(TestTrans, transfer_block2)
th.join();
}
TEST_F(TestTrans, tablet_to_ls_cache)
{
// 0. init
int ret = OB_SUCCESS;
uint64_t tenant_id = MTL_ID();
ObLSID ls_id_1(1201);
ObLSID ls_id_2(1202);
ObLS *ls1 = nullptr;
ObLS *ls2 = nullptr;
ObLSTxCtxMgr *ls_tx_ctx_mgr = NULL;
ObTransService *tx_service = MTL(ObTransService*);
ASSERT_EQ(true, tx_service->is_inited_);
ASSERT_EQ(true, tx_service->tablet_to_ls_cache_.is_inited_);
create_ls(tenant_id, ls_id_1, ls1);
create_ls(tenant_id, ls_id_2, ls2);
ASSERT_EQ(OB_SUCCESS, tx_service->tx_ctx_mgr_.get_ls_tx_ctx_mgr(ls_id_1, ls_tx_ctx_mgr));
int64_t base_ref = ls_tx_ctx_mgr->get_ref();
int64_t base_size = tx_service->tablet_to_ls_cache_.size();
// 1. insert tablet
const int TABLET_NUM = 10;
ObSEArray<ObTabletID, TABLET_NUM> tablet_ids;
ObSEArray<ObTabletID, TABLET_NUM> tablet_ids_2;
for (int i = 0; i < TABLET_NUM; i++) {
ObTabletID tablet_id(1300 + i);
ASSERT_EQ(OB_SUCCESS, tablet_ids.push_back(tablet_id));
ObTabletID tablet_id_2(1600 + i);
ASSERT_EQ(OB_SUCCESS, tablet_ids_2.push_back(tablet_id_2));
}
ASSERT_EQ(TABLET_NUM, tablet_ids.count());
ARRAY_FOREACH(tablet_ids, i) {
const ObTabletID &tablet_id = tablet_ids.at(i);
ASSERT_EQ(OB_SUCCESS, tx_service->create_tablet(tablet_id, i < TABLET_NUM/2 ? ls_id_1:ls_id_2));
}
ASSERT_EQ(TABLET_NUM + base_size, tx_service->tablet_to_ls_cache_.size());
ASSERT_EQ(TABLET_NUM/2 + base_ref, ls_tx_ctx_mgr->get_ref());
// repeated inserts will not fail, but cache entries count will not grow.
ARRAY_FOREACH(tablet_ids, i) {
const ObTabletID &tablet_id = tablet_ids.at(i);
ASSERT_EQ(OB_SUCCESS, tx_service->create_tablet(tablet_id, i < TABLET_NUM/2 ? ls_id_1:ls_id_2));
}
ASSERT_EQ(TABLET_NUM + base_size, tx_service->tablet_to_ls_cache_.size());
ASSERT_EQ(TABLET_NUM/2 + base_ref, ls_tx_ctx_mgr->get_ref());
// 2. check and get ls
ObLSID ls_id;
bool is_local = false;
ARRAY_FOREACH(tablet_ids, i) {
// tablet exist
const ObTabletID &tablet_id = tablet_ids.at(i);
ls_id.reset();
is_local = false;
ASSERT_EQ(OB_SUCCESS, tx_service->check_and_get_ls_info(tablet_id, ls_id, is_local));
ASSERT_EQ((i < TABLET_NUM/2 ? ls_id_1.id():ls_id_2.id()), ls_id.id());
ASSERT_EQ(true, is_local);
// tablet not exist
const ObTabletID &tablet_id_2 = tablet_ids_2.at(i);
ASSERT_EQ(OB_ENTRY_NOT_EXIST, tx_service->check_and_get_ls_info(tablet_id_2, ls_id, is_local));
}
ASSERT_EQ(TABLET_NUM + base_size, tx_service->tablet_to_ls_cache_.size());
// 3. remove tablet
ASSERT_EQ(OB_SUCCESS, MTL(ObLSService*)->remove_ls(ls_id_2));
ASSERT_EQ(TABLET_NUM/2, tx_service->tablet_to_ls_cache_.size());
for (int i = 0; i < TABLET_NUM/2; i++) {
const ObTabletID &tablet_id = tablet_ids.at(i);
ls_id.reset();
is_local = false;
ASSERT_EQ(OB_SUCCESS, tx_service->check_and_get_ls_info(tablet_id, ls_id, is_local));
ASSERT_EQ(ls_id_1, ls_id);
ASSERT_EQ(true, is_local);
tx_service->remove_tablet(tablet_id, ls_id);
ASSERT_EQ(OB_ENTRY_NOT_EXIST, tx_service->check_and_get_ls_info(tablet_id, ls_id, is_local));
}
ASSERT_EQ(0, tx_service->tablet_to_ls_cache_.size());
ASSERT_EQ(base_ref, ls_tx_ctx_mgr->get_ref());
// 4. clear
ASSERT_EQ(OB_SUCCESS, tx_service->tx_ctx_mgr_.revert_ls_tx_ctx_mgr(ls_tx_ctx_mgr));
ASSERT_EQ(OB_SUCCESS, MTL(ObLSService*)->remove_ls(ls_id_1));
}
TEST_F(TestTrans, remove_ls)
{
ObLSID ls_id(100);
ObLSID ls_id2(101);
ASSERT_EQ(OB_SUCCESS, MTL(ObLSService*)->remove_ls(ls_id));
ASSERT_EQ(OB_SUCCESS, MTL(ObLSService*)->remove_ls(ls_id2));
ObTransService *tx_service = MTL(ObTransService*);
ASSERT_EQ(0, tx_service->tablet_to_ls_cache_.size());
}
} // end oceanbase
int main(int argc, char **argv)

View File

@ -100,7 +100,7 @@ private:
ObMemtableKey mt_key;
ret = mt_ctx_.lock_mem_ctx_.add_lock_record(lock_op, lock_op_node);
ASSERT_EQ(OB_SUCCESS, ret);
cb = mt_ctx_.alloc_table_lock_callback(mt_ctx_,
cb = mt_ctx_.create_table_lock_callback(mt_ctx_,
&memtable_);
ASSERT_NE(nullptr, cb);
ret = mt_key.encode(&rowkey);

View File

@ -22,6 +22,7 @@
#include "storage/tablelock/ob_lock_memtable.h"
#include "storage/tablelock/ob_mem_ctx_table_lock.h"
#include "table_lock_common_env.h"
#include "storage/memtable/ob_memtable_context.h"
namespace oceanbase
{
@ -48,7 +49,7 @@ public:
TestMemCtxTableLock()
: ls_id_(1),
fake_t3m_(common::OB_SERVER_TENANT_ID),
mem_ctx_(allocator_)
mem_ctx_(fake_memtable_ctx_)
{
LOG_INFO("construct TestMemCtxTableLock");
}
@ -81,6 +82,7 @@ public:
void init_mem_ctx()
{
ASSERT_EQ(OB_SUCCESS, fake_memtable_ctx_.init(common::OB_SERVER_TENANT_ID));
ASSERT_EQ(OB_SUCCESS, mem_ctx_.init(handle_));
}
@ -89,8 +91,8 @@ private:
ObLockMemtable memtable_;
ObTableHandleV2 handle_;
ObTenantMetaMemMgr fake_t3m_;
memtable::ObMemtableCtx fake_memtable_ctx_;
ObFreezer freezer_;
ObLockMemCtx mem_ctx_;
ObArenaAllocator allocator_;
};

View File

@ -112,6 +112,7 @@ ob_unittest_observer(test_transfer_tx_data test_transfer_with_smaller_tx_data.cp
ob_unittest_observer(test_transfer_in_after_abort test_transfer_in_after_abort.cpp)
ob_unittest_observer(test_transfer_commit_action test_transfer_with_commit_action.cpp)
ob_unittest_observer(test_memtable_new_safe_to_destroy test_memtable_new_safe_to_destroy.cpp)
ob_unittest_observer(test_tablet_to_ls_cache test_tablet_to_ls_cache.cpp)
# TODO(muwei.ym): open later
ob_ha_unittest_observer(test_transfer_handler storage_ha/test_transfer_handler.cpp)
ob_ha_unittest_observer(test_transfer_and_restart_basic storage_ha/test_transfer_and_restart_basic.cpp)

View File

@ -333,5 +333,74 @@ int ObSimpleClusterTestBase::check_tenant_exist(bool &bool_ret, const char *tena
return ret;
}
int ObSimpleClusterTestBase::batch_create_table(
const uint64_t tenant_id,
ObMySQLProxy &sql_proxy,
const int64_t TOTAL_NUM,
ObIArray<ObTabletLSPair> &tablet_ls_pairs)
{
int ret = OB_SUCCESS;
tablet_ls_pairs.reset();
ObSqlString sql;
// batch create table
int64_t affected_rows = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < TOTAL_NUM; ++i) {
sql.reset();
if (OB_FAIL(sql.assign_fmt("create table t%ld(c1 int)", i))) {
} else if (OB_FAIL(sql_proxy.write(sql.ptr(), affected_rows))) {
}
}
// batch get table_id
sql.reset();
if (OB_FAIL(sql.assign_fmt("select TABLET_ID, LS_ID from oceanbase.DBA_OB_TABLE_LOCATIONS where table_name in ("))) {
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < TOTAL_NUM; ++i) {
if (OB_FAIL(sql.append_fmt("%s't%ld'", 0 == i ? "" : ",", i))) {}
}
if (FAILEDx(sql.append_fmt(") order by TABLET_ID"))) {};
}
SMART_VAR(ObMySQLProxy::MySQLResult, result) {
if (OB_FAIL(tablet_ls_pairs.reserve(TOTAL_NUM))) {
} else if (OB_UNLIKELY(!is_valid_tenant_id(tenant_id))) {
ret = OB_ERR_UNEXPECTED;
} else if (OB_FAIL(sql_proxy.read(result, sql.ptr()))) {
} else if (OB_ISNULL(result.get_result())) {
ret = OB_ERR_UNEXPECTED;
} else {
sqlclient::ObMySQLResult &res = *result.get_result();
uint64_t tablet_id = ObTabletID::INVALID_TABLET_ID;
int64_t ls_id = ObLSID::INVALID_LS_ID;
while(OB_SUCC(ret) && OB_SUCC(res.next())) {
EXTRACT_INT_FIELD_MYSQL(res, "TABLET_ID", tablet_id, uint64_t);
EXTRACT_INT_FIELD_MYSQL(res, "LS_ID", ls_id, int64_t);
if (OB_FAIL(tablet_ls_pairs.push_back(ObTabletLSPair(tablet_id, ls_id)))) {}
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
} else {
SERVER_LOG(WARN, "fail to generate data", K(sql));
}
}
}
return ret;
}
int ObSimpleClusterTestBase::batch_drop_table(
const uint64_t tenant_id,
ObMySQLProxy &sql_proxy,
const int64_t TOTAL_NUM)
{
int ret = OB_SUCCESS;
ObSqlString sql;
int64_t affected_rows = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < TOTAL_NUM; ++i) {
sql.reset();
if (OB_FAIL(sql.assign_fmt("drop table t%ld", i))) {
} else if (OB_FAIL(sql_proxy.write(sql.ptr(), affected_rows))) {
}
}
return ret;
}
} // end unittest
} // end oceanbase

View File

@ -49,6 +49,13 @@ public:
int get_tenant_id(uint64_t &tenant_id, const char *tenant_name = "tt1");
int exec_write_sql_sys(const char *sql_str, int64_t &affected_rows);
int check_tenant_exist(bool &bool_ret, const char *tenant_name = "tt1");
int batch_create_table(const uint64_t tenant_id,
ObMySQLProxy &sql_proxy,
const int64_t TOTAL_NUM,
ObIArray<ObTabletLSPair> &tablet_ls_pairs);
int batch_drop_table(const uint64_t tenant_id,
ObMySQLProxy &sql_proxy,
const int64_t TOTAL_NUM);
protected:
virtual void SetUp();

View File

@ -0,0 +1,100 @@
/**
* Copyright (c) 2023 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include <gtest/gtest.h>
#include <gmock/gmock.h>
#define USING_LOG_PREFIX SHARE
#define protected public
#define private public
#include "env/ob_simple_cluster_test_base.h"
#include "lib/ob_errno.h"
#include "storage/tx/ob_trans_service.h"
#include "storage/tx_storage/ob_ls_service.h"
namespace oceanbase
{
using namespace unittest;
namespace share
{
using namespace common;
using namespace transaction;
class TestTabletToLSCache : public unittest::ObSimpleClusterTestBase
{
public:
TestTabletToLSCache() : unittest::ObSimpleClusterTestBase("test_tablet_to_ls_cache") {}
};
TEST_F(TestTabletToLSCache, tablet_to_ls_cache)
{
// 0. init
int ret = OB_SUCCESS;
uint64_t tenant_id = OB_INVALID_TENANT_ID;
const int64_t TOTAL_NUM = 10;
ObSEArray<ObTabletLSPair, TOTAL_NUM> tablet_ls_pairs;
ASSERT_EQ(OB_SUCCESS, create_tenant());
ASSERT_EQ(OB_SUCCESS, get_tenant_id(tenant_id));
ASSERT_EQ(true, is_valid_tenant_id(tenant_id));
ASSERT_EQ(OB_SUCCESS, get_curr_simple_server().init_sql_proxy2());
ObMySQLProxy &sql_proxy = get_curr_simple_server().get_sql_proxy2();
MTL_SWITCH(tenant_id) {
ObLSService *ls_service = MTL(ObLSService *);
ASSERT_EQ(true, OB_NOT_NULL(ls_service));
ObTransService *tx_service = MTL(ObTransService *);
ASSERT_EQ(true, OB_NOT_NULL(tx_service));
int64_t base_size = tx_service->tablet_to_ls_cache_.size();
// 1. create tablet
ASSERT_EQ(OB_SUCCESS, batch_create_table(tenant_id, sql_proxy, TOTAL_NUM, tablet_ls_pairs));
ASSERT_EQ(TOTAL_NUM, tablet_ls_pairs.count());
ASSERT_EQ(base_size + TOTAL_NUM, tx_service->tablet_to_ls_cache_.size());
// 2. check and get ls
ObLSID ls_id;
ARRAY_FOREACH(tablet_ls_pairs, i) {
const ObTabletLSPair &pair = tablet_ls_pairs.at(i);
bool is_local = false;
ls_id.reset();
ASSERT_EQ(OB_SUCCESS, tx_service->check_and_get_ls_info(pair.get_tablet_id(), ls_id, is_local));
ASSERT_EQ(pair.get_ls_id().id(), ls_id.id());
ASSERT_EQ(true, is_local);
}
// 3. drop table
ObArray<ObTabletID> remove_tablet_ids;
ObLSHandle ls_handle;
ls_id = tablet_ls_pairs.at(0).get_ls_id();
ASSERT_EQ(OB_SUCCESS, ls_service->get_ls(ls_id, ls_handle, ObLSGetMod::STORAGE_MOD));
ObLSTabletService *ls_tablet_svr = ls_handle.get_ls()->get_tablet_svr();
ARRAY_FOREACH(tablet_ls_pairs, i) {
const ObTabletLSPair &pair = tablet_ls_pairs.at(i);
remove_tablet_ids.push_back(pair.get_tablet_id());
}
ASSERT_EQ(OB_SUCCESS, batch_drop_table(tenant_id, sql_proxy, TOTAL_NUM));
// after drop table, the tablets won't be removed immediately, so need to call remove_tablets
ASSERT_EQ(OB_SUCCESS, ls_tablet_svr->remove_tablets(remove_tablet_ids));
ASSERT_EQ(base_size, tx_service->tablet_to_ls_cache_.size());
}
}
} // namespace share
} // namespace oceanbase
int main(int argc, char **argv)
{
oceanbase::unittest::init_log_and_gtest(argc, argv);
OB_LOGGER.set_log_level("INFO");
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View File

@ -392,7 +392,7 @@ int ObApplyStatus::push_append_cb(AppendCb *cb)
ret = OB_NOT_INIT;
CLOG_LOG(ERROR, "apply service not init", K(ret));
} else {
const int64_t start_ts = ObTimeUtility::fast_current_time();
const int64_t start_ts = ObClockGenerator::getClock();
LSN palf_committed_end_lsn;
const LSN cb_lsn = cb->__get_lsn();
const SCN cb_scn = cb->__get_scn();
@ -416,7 +416,7 @@ int ObApplyStatus::push_append_cb(AppendCb *cb)
}
}
}
const int64_t push_cost = ObTimeUtility::fast_current_time() - start_ts;
const int64_t push_cost = ObClockGenerator::getClock() - start_ts;
if (push_cost > 1 * 1000) { //1ms
CLOG_LOG(INFO, "apply service push_task cost too much time", K(thread_index), K(cb_lsn),
K(cb_sign), KPC(this), K(push_cost));
@ -528,7 +528,7 @@ int ObApplyStatus::try_handle_cb_queue(ObApplyServiceQueueTask *cb_queue,
ret = OB_SUCCESS;
}
}
CLOG_LOG(TRACE, "try_handle_cb_queue finish", KPC(this), KPC(cb_queue), K(ret), K(is_queue_empty), K(is_timeslice_run_out));
CLOG_LOG(DEBUG, "try_handle_cb_queue finish", KPC(this), KPC(cb_queue), K(ret), K(is_queue_empty), K(is_timeslice_run_out));
return ret;
}

View File

@ -249,17 +249,17 @@ private:
PALF_LOG(ERROR, "FixedSlidingWindow revert error", KR(ret), K(r_id), K(begin_sn_), K(end_sn_), K(curr_ref));
} else if (0 == curr_ref) {
int64_t tmp_end = get_end_sn_();
PALF_LOG(TRACE, "revert zero", K(r_id), K(begin_sn_), K(tmp_end));
PALF_LOG(DEBUG, "revert zero", K(r_id), K(begin_sn_), K(tmp_end));
for (int64_t i = idx; i == calc_idx_(tmp_end) && tmp_id < get_begin_sn();) {
// slot mutex lock for revert/slide, revert/revert
ObByteLockGuard guard(array_[i].slot_lock_);
PALF_LOG(TRACE, "try revert reset", K(i), K(tmp_end), K(begin_sn_));
PALF_LOG(DEBUG, "try revert reset", K(i), K(tmp_end), K(begin_sn_));
// double check end_sn_ to avoid that revert operation inc wrong end_sn_
if (!(tmp_end == get_end_sn_() && 0 == ATOMIC_LOAD(&(array_[i].ref_)))) {
break;
} else {
// Order is vital: 1. reset slot 2. inc end_sn_, for get/revert mutex
PALF_LOG(TRACE, "do revert reset", K(i), K(tmp_end), K(begin_sn_));
PALF_LOG(DEBUG, "do revert reset", K(i), K(tmp_end), K(begin_sn_));
array_[i].reset();
if (ATOMIC_BCAS(&(end_sn_), tmp_end, (tmp_end+1))) {
i = calc_idx_(i+1);
@ -274,7 +274,7 @@ private:
}
}
} else {
PALF_LOG(TRACE, "revert succ", K(r_id), K(curr_ref), K(begin_sn_), K(end_sn_));
PALF_LOG(DEBUG, "revert succ", K(r_id), K(curr_ref), K(begin_sn_), K(end_sn_));
}
}
return ret;

View File

@ -352,7 +352,6 @@ int ObMPBase::init_process_var(sql::ObSqlCtx &ctx,
sql::ObSQLSessionInfo &session) const
{
int ret = OB_SUCCESS;
bool enable_udr = false;
if (!packet_sender_.is_conn_valid()) {
ret = OB_CONNECT_ERROR;
LOG_WARN("connection already disconnected", K(ret));
@ -377,13 +376,9 @@ int ObMPBase::init_process_var(sql::ObSqlCtx &ctx,
}
ctx.is_protocol_weak_read_ = pkt.is_weak_read();
ctx.set_enable_strict_defensive_check(GCONF.enable_strict_defensive_check());
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(session.get_effective_tenant_id()));
if (tenant_config.is_valid()) {
enable_udr = tenant_config->enable_user_defined_rewrite_rules;
}
ctx.set_enable_user_defined_rewrite(enable_udr);
LOG_TRACE("protocol flag info", K(ctx.can_reroute_sql_), K(ctx.is_protocol_weak_read_),
K(ctx.get_enable_strict_defensive_check()), K(enable_udr));
ctx.set_enable_user_defined_rewrite(session.enable_udr());
LOG_DEBUG("protocol flag info", K(ctx.can_reroute_sql_), K(ctx.is_protocol_weak_read_),
K(ctx.get_enable_strict_defensive_check()), "enable_udr", session.enable_udr());
}
return ret;
}
@ -616,7 +611,7 @@ int ObMPBase::process_extra_info(sql::ObSQLSessionInfo &session,
{
int ret = OB_SUCCESS;
SessionInfoVerifacation sess_info_verification;
LOG_TRACE("process extra info", K(ret),K(pkt.get_extra_info().exist_sess_info_veri()));
LOG_DEBUG("process extra info", K(ret),K(pkt.get_extra_info().exist_sess_info_veri()));
if (FALSE_IT(session.set_has_query_executed(true))) {
} else if (pkt.get_extra_info().exist_sync_sess_info()
&& OB_FAIL(ObMPUtils::sync_session_info(session,

View File

@ -226,11 +226,8 @@ int ObMPConnect::init_process_single_stmt(const ObMultiStmtItem &multi_stmt_item
} else if (OB_FAIL(set_session_active(sql, session, ObTimeUtil::current_time()))) {
LOG_WARN("fail to set session active", K(ret));
} else {
const bool enable_trace_log = lib::is_trace_log_enabled();
if (enable_trace_log) {
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
ctx.retry_times_ = 0; // 这里是建立连接的时候的初始化sql的执行,不重试
ctx.schema_guard_ = &schema_guard;
HEAP_VAR(ObMySQLResultSet, result, session, allocator) {
@ -253,9 +250,7 @@ int ObMPConnect::init_process_single_stmt(const ObMultiStmtItem &multi_stmt_item
}
ret = (open_ret != OB_SUCCESS) ? open_ret : ret;
}
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
}
//对于tracelog的处理,不影响正常逻辑,错误码无须赋值给ret

View File

@ -1017,7 +1017,7 @@ int ObMPPacketSender::flush_buffer(const bool is_last)
proto20_context_.is_filename_packet_ = false;
}
int64_t buf_sz = ez_buf_->last - ez_buf_->pos;
//int64_t buf_sz = ez_buf_->last - ez_buf_->pos;
if (OB_SUCCESS != ret) {
} else if (ObRequest::TRANSPORT_PROTO_EASY == nio_protocol_) {
ObFlushBufferParam flush_param(*ez_buf_, *req_->get_ez_req(), comp_context_,
@ -1063,6 +1063,7 @@ int ObMPPacketSender::flush_buffer(const bool is_last)
// after flush, set pos to 0
proto20_context_.curr_proto20_packet_start_pos_ = 0;
}
/*
ObSQLSessionInfo *sess = nullptr;
if (OB_FAIL(get_session(sess))) {
LOG_WARN("fail to get session info", K(ret));
@ -1073,7 +1074,7 @@ int ObMPPacketSender::flush_buffer(const bool is_last)
}
if (OB_NOT_NULL(sess)) {
GCTX.session_mgr_->revert_session(sess);
}
}*/
}
}
if (is_last) {

View File

@ -277,6 +277,7 @@ int ObMPQuery::process()
"trans_id", session.get_tx_id(), K(session));
} else if (queries.count() > 1
&& OB_FAIL(try_batched_multi_stmt_optimization(session,
conn,
queries,
parse_stat,
optimization_done,
@ -325,6 +326,7 @@ int ObMPQuery::process()
// 原来的值默认为true,会影响单条sql的二次路由,现在改为用 queries.count() 判断。
bool is_part_of_multi = queries.count() > 1 ? true : false;
ret = process_single_stmt(ObMultiStmtItem(is_part_of_multi, i, queries.at(i)),
conn,
session,
has_more,
force_sync_resp,
@ -349,6 +351,7 @@ int ObMPQuery::process()
EVENT_INC(SQL_SINGLE_QUERY_COUNT);
// 处理普通的Single Statement
ret = process_single_stmt(ObMultiStmtItem(false, 0, sql_),
conn,
session,
has_more,
force_sync_resp,
@ -420,6 +423,7 @@ int ObMPQuery::process()
* for details, please ref to
*/
int ObMPQuery::try_batched_multi_stmt_optimization(sql::ObSQLSessionInfo &session,
ObSMConnection *conn,
common::ObIArray<ObString> &queries,
const ObMPParseStat &parse_stat,
bool &optimization_done,
@ -440,6 +444,7 @@ int ObMPQuery::try_batched_multi_stmt_optimization(sql::ObSQLSessionInfo &sessio
} else if (!use_plan_cache) {
// 不打开plan_cache开关,则优化不支持
} else if (OB_FAIL(process_single_stmt(ObMultiStmtItem(false, 0, sql_, &queries, is_ins_multi_val_opt),
conn,
session,
has_more,
force_sync_resp,
@ -463,6 +468,7 @@ int ObMPQuery::try_batched_multi_stmt_optimization(sql::ObSQLSessionInfo &sessio
}
int ObMPQuery::process_single_stmt(const ObMultiStmtItem &multi_stmt_item,
ObSMConnection *conn,
ObSQLSessionInfo &session,
bool has_more_result,
bool force_sync_resp,
@ -473,7 +479,6 @@ int ObMPQuery::process_single_stmt(const ObMultiStmtItem &multi_stmt_item,
FLTSpanGuard(mpquery_single_stmt);
ctx_.spm_ctx_.reset();
bool need_response_error = true;
const bool enable_trace_log = lib::is_trace_log_enabled();
session.get_raw_audit_record().request_memory_used_ = 0;
observer::ObProcessMallocCallback pmcb(0,
session.get_raw_audit_record().request_memory_used_);
@ -485,14 +490,12 @@ int ObMPQuery::process_single_stmt(const ObMultiStmtItem &multi_stmt_item,
session.set_curr_trans_last_stmt_end_time(0);
//============================ 注意这些变量的生命周期 ================================
ObSessionStatEstGuard stat_est_guard(get_conn()->tenant_->id(), session.get_sessid());
ObSessionStatEstGuard stat_est_guard(conn->tenant_->id(), session.get_sessid());
if (OB_FAIL(init_process_var(ctx_, multi_stmt_item, session))) {
LOG_WARN("init process var failed.", K(ret), K(multi_stmt_item));
} else {
if (enable_trace_log) {
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
// obproxy may use 'SET @@last_schema_version = xxxx' to set newest schema,
// observer will force refresh schema if local_schema_version < last_schema_version;
if (OB_FAIL(check_and_refresh_schema(session.get_login_tenant_id(),

View File

@ -79,6 +79,7 @@ private:
bool &async_resp_used,
bool &need_disconnect);
int process_single_stmt(const sql::ObMultiStmtItem &multi_stmt_item,
ObSMConnection *conn,
sql::ObSQLSessionInfo &session,
bool has_more_result,
bool force_sync_resp,
@ -96,6 +97,7 @@ private:
const share::ObPartitionLocation &partition_loc,
share::ObFBPartitionParam &param);
int try_batched_multi_stmt_optimization(sql::ObSQLSessionInfo &session,
ObSMConnection *conn,
common::ObIArray<ObString> &queries,
const ObMPParseStat &parse_stat,
bool &optimization_done,
@ -106,6 +108,8 @@ private:
int store_params_value_to_str(ObIAllocator &allocator,
sql::ObSQLSessionInfo &session,
common::ParamStore &params);
public:
static const int64_t MAX_SELF_OBJ_SIZE = 2 * 1024L;
private:
DISALLOW_COPY_AND_ASSIGN(ObMPQuery);
private:
@ -122,6 +126,7 @@ private:
int64_t params_value_len_;
char *params_value_;
}; // end of class ObMPQuery
STATIC_ASSERT(sizeof(ObMPQuery) < ObMPQuery::MAX_SELF_OBJ_SIZE, "sizeof ObMPQuery not great to 2KB");
} // end of namespace observer
} // end of namespace oceanbase
#endif /* _OBMP_QUERY_H_ */

View File

@ -1699,17 +1699,14 @@ int ObMPStmtExecute::process_execute_stmt(const ObMultiStmtItem &multi_stmt_item
// 执行setup_wb后,所有WARNING都会写入到当前session的WARNING BUFFER中
setup_wb(session);
const bool enable_trace_log = lib::is_trace_log_enabled();
//============================ 注意这些变量的生命周期 ================================
ObSMConnection *conn = get_conn();
ObSessionStatEstGuard stat_est_guard(conn->tenant_->id(), session.get_sessid());
if (OB_FAIL(init_process_var(ctx_, multi_stmt_item, session))) {
LOG_WARN("init process var failed.", K(ret), K(multi_stmt_item));
} else {
if (enable_trace_log) {
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
// obproxy may use 'SET @@last_schema_version = xxxx' to set newest schema,
// observer will force refresh schema if local_schema_version < last_schema_version;
if (OB_FAIL(check_and_refresh_schema(session.get_login_tenant_id(),
@ -1786,9 +1783,7 @@ int ObMPStmtExecute::process_execute_stmt(const ObMultiStmtItem &multi_stmt_item
ret = OB_SUCC(bak_ret) ? ret : bak_ret;
}
}
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
const int64_t debug_sync_timeout = GCONF.debug_sync_timeout;
if (debug_sync_timeout > 0) {
// ignore thread local debug sync actions to session actions failed

View File

@ -616,11 +616,8 @@ int ObMPStmtFetch::process_fetch_stmt(ObSQLSessionInfo &session,
// 执行setup_wb后,所有WARNING都会写入到当前session的WARNING BUFFER中
setup_wb(session);
ObSessionStatEstGuard stat_est_guard(get_conn()->tenant_->id(), session.get_sessid());
const bool enable_trace_log = lib::is_trace_log_enabled();
if (enable_trace_log) {
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
//set session log_level.Must use ObThreadLogLevelUtils::clear() in pair
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
// obproxy may use 'SET @@last_schema_version = xxxx' to set newest schema,
// observer will force refresh schema if local_schema_version < last_schema_version;
if (OB_FAIL(check_and_refresh_schema(session.get_login_tenant_id(),
@ -635,9 +632,7 @@ int ObMPStmtFetch::process_fetch_stmt(ObSQLSessionInfo &session,
ret = do_process(session, need_response_error);
}
}
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
const int64_t debug_sync_timeout = GCONF.debug_sync_timeout;
if (debug_sync_timeout > 0) {
// ignore thread local debug sync actions to session actions failed

View File

@ -187,14 +187,9 @@ int ObMPStmtGetPieceData::process_get_piece_data_stmt(ObSQLSessionInfo &session)
ObVirtualTableIteratorFactory vt_iter_factory(*gctx_.vt_iter_creator_);
ObSessionStatEstGuard stat_est_guard(get_conn()->tenant_->id(), session.get_sessid());
const bool enable_trace_log = lib::is_trace_log_enabled();
if (enable_trace_log) {
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
ret = do_process(session);
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
//对于tracelog的处理,不影响正常逻辑,错误码无须赋值给ret
int tmp_ret = OB_SUCCESS;

View File

@ -298,10 +298,7 @@ int ObMPStmtPrepare::process_prepare_stmt(const ObMultiStmtItem &multi_stmt_item
if (OB_FAIL(init_process_var(ctx_, multi_stmt_item, session))) {
LOG_WARN("init process var faield.", K(ret), K(multi_stmt_item));
} else {
const bool enable_trace_log = lib::is_trace_log_enabled();
if (enable_trace_log) {
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
if (OB_FAIL(check_and_refresh_schema(session.get_login_tenant_id(),
session.get_effective_tenant_id()))) {
LOG_WARN("failed to check_and_refresh_schema", K(ret));
@ -342,9 +339,7 @@ int ObMPStmtPrepare::process_prepare_stmt(const ObMultiStmtItem &multi_stmt_item
"retry_times", retry_ctrl_.get_retry_times(), K(multi_stmt_item));
}
}
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
}
//对于tracelog的处理,不影响正常逻辑,错误码无须赋值给ret

View File

@ -212,14 +212,9 @@ int ObMPStmtSendLongData::process_send_long_data_stmt(ObSQLSessionInfo &session)
ObVirtualTableIteratorFactory vt_iter_factory(*gctx_.vt_iter_creator_);
ObSessionStatEstGuard stat_est_guard(get_conn()->tenant_->id(), session.get_sessid());
const bool enable_trace_log = lib::is_trace_log_enabled();
if (enable_trace_log) {
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
ret = do_process(session);
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
//对于tracelog的处理,不影响正常逻辑,错误码无须赋值给ret
int tmp_ret = OB_SUCCESS;

View File

@ -193,14 +193,9 @@ int ObMPStmtSendPieceData::process_send_long_data_stmt(ObSQLSessionInfo &session
ObVirtualTableIteratorFactory vt_iter_factory(*gctx_.vt_iter_creator_);
ObSessionStatEstGuard stat_est_guard(get_conn()->tenant_->id(), session.get_sessid());
const bool enable_trace_log = lib::is_trace_log_enabled();
if (enable_trace_log) {
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
}
ObThreadLogLevelUtils::init(session.get_log_id_level_map());
ret = do_process(session);
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
//对于tracelog的处理,不影响正常逻辑,错误码无须赋值给ret
int tmp_ret = OB_SUCCESS;

View File

@ -159,7 +159,7 @@ int ObMPUtils::sync_session_info(sql::ObSQLSessionInfo &sess, const common::ObSt
const char *end = buf + len;
int64_t pos = 0;
LOG_TRACE("sync sess_inf", K(sess.get_is_in_retry()),
LOG_DEBUG("sync sess_inf", K(sess.get_is_in_retry()),
K(sess.get_sessid()), KP(data), K(len), KPHEX(data, len));
// decode sess_info

View File

@ -711,22 +711,6 @@ int ObSrvDeliver::deliver_mysql_request(ObRequest &req)
if (need_update_stat) {
EVENT_INC(MYSQL_PACKET_IN);
EVENT_ADD(MYSQL_PACKET_IN_BYTES, pkt.get_clen() + OB_MYSQL_HEADER_LENGTH);
sql::ObSQLSessionInfo *sess_info = nullptr;
if (OB_ISNULL(conn) || OB_ISNULL(GCTX.session_mgr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("conn or sessoin mgr is NULL", K(ret), KP(conn), K(GCTX.session_mgr_));
} else if (OB_FAIL(GCTX.session_mgr_->get_session(conn->sessid_, sess_info))) {
LOG_WARN("get session fail", K(ret), "sessid", conn->sessid_,
"proxy_sessid", conn->proxy_sessid_);
} else if (OB_ISNULL(sess_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sess_info is null", K(ret));
} else {
sess_info->inc_in_bytes(pkt.get_clen() + OB_MYSQL_HEADER_LENGTH);
}
if (OB_NOT_NULL(sess_info)) {
GCTX.session_mgr_->revert_session(sess_info);
}
}
// The tenant check has been done in the recv_request method. For performance considerations, the check here is removed;
/*

View File

@ -170,6 +170,7 @@ int ObSrvXlator::th_destroy()
typedef union EP_RPCP_BUF {
char rpcp_buffer_[RPCP_BUF_SIZE]; // reserve memory for rpc processor
char ep_buffer_[sizeof (ObErrorP) + sizeof (ObMPError)];
char obmp_query_buffer_[ObMPQuery::MAX_SELF_OBJ_SIZE];
} EP_RPCP_BUF;
// Make sure election rpc processor allocated successfully when OOM occurs
STATIC_ASSERT(RPCP_BUF_SIZE >= sizeof(oceanbase::palf::ElectionPrepareRequestMsgP), "RPCP_BUF_SIZE should be big enough to allocate election processer");
@ -198,89 +199,101 @@ int ObSrvMySQLXlator::translate(rpc::ObRequest &req, ObReqProcessor *&processor)
ret = get_mp_connect_processor(processor);
} else {
const ObMySQLRawPacket &pkt = reinterpret_cast<const ObMySQLRawPacket &>(req.get_packet());
switch (pkt.get_cmd()) {
MYSQL_PROCESSOR(ObMPQuery, gctx_);
MYSQL_PROCESSOR(ObMPQuit, gctx_);
MYSQL_PROCESSOR(ObMPPing, gctx_);
MYSQL_PROCESSOR(ObMPInitDB, gctx_);
MYSQL_PROCESSOR(ObMPChangeUser, gctx_);
MYSQL_PROCESSOR(ObMPStatistic, gctx_);
MYSQL_PROCESSOR(ObMPStmtPrepare, gctx_);
MYSQL_PROCESSOR(ObMPStmtExecute, gctx_);
MYSQL_PROCESSOR(ObMPStmtFetch, gctx_);
MYSQL_PROCESSOR(ObMPStmtReset, gctx_);
MYSQL_PROCESSOR(ObMPStmtPrexecute, gctx_);
MYSQL_PROCESSOR(ObMPStmtSendPieceData, gctx_);
MYSQL_PROCESSOR(ObMPStmtGetPieceData, gctx_);
MYSQL_PROCESSOR(ObMPStmtSendLongData, gctx_);
MYSQL_PROCESSOR(ObMPResetConnection, gctx_);
// ps stmt close request may not response packet.
// Howerver, in get processor phase, it may report
// error due to lack of memory and this response error packet.
// To avoid this situation, we make stmt close processor
// by stack memory
case obmysql::COM_STMT_CLOSE: {
char *closepbuf = (&co_closepbuf)->buffer_;
ObMPStmtClose* p = new (&closepbuf[0]) ObMPStmtClose(gctx_);
if (OB_FAIL(p->init())) {
SERVER_LOG(ERROR, "Init ObMPStmtClose fail", K(ret));
p->~ObMPStmtClose();
} else {
processor = p;
}
break;
if (pkt.get_cmd() == obmysql::COM_QUERY) {
char *buf = (&co_ep_rpcp_buf)->obmp_query_buffer_;
ObMPQuery *p = new (buf) ObMPQuery(gctx_);
if (OB_ISNULL(p)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
} else if (OB_FAIL(p->init())) {
SERVER_LOG(ERROR, "Init ObMPQuery fail", K(ret));
p->~ObMPQuery();
} else {
processor = p;
}
case obmysql::COM_FIELD_LIST: {
/*为了和proxy进行适配,对于COM_FIELD_LIST命令的支持,按照以下原则支持:
* 1. Proxy模式
* 2. Proxy模式
* 2.1. 1.7.6
* 1.7.6
*
* 2.2.
*/
ObSMConnection *conn = reinterpret_cast<ObSMConnection* >(
SQL_REQ_OP.get_sql_session(&req));
if (OB_ISNULL(conn)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(conn), K(ret));
} else if (conn->is_proxy_) {
const char *sup_proxy_min_version = "1.7.6";
uint64_t min_proxy_version = 0;
if (OB_FAIL(ObClusterVersion::get_version(sup_proxy_min_version, min_proxy_version))) {
LOG_WARN("failed to get version", K(ret));
} else if (conn->proxy_version_ < min_proxy_version) {
NEW_MYSQL_PROCESSOR(ObMPDefault, gctx_);
} else {
switch (pkt.get_cmd()) {
MYSQL_PROCESSOR(ObMPQuit, gctx_);
MYSQL_PROCESSOR(ObMPPing, gctx_);
MYSQL_PROCESSOR(ObMPInitDB, gctx_);
MYSQL_PROCESSOR(ObMPChangeUser, gctx_);
MYSQL_PROCESSOR(ObMPStatistic, gctx_);
MYSQL_PROCESSOR(ObMPStmtPrepare, gctx_);
MYSQL_PROCESSOR(ObMPStmtExecute, gctx_);
MYSQL_PROCESSOR(ObMPStmtFetch, gctx_);
MYSQL_PROCESSOR(ObMPStmtReset, gctx_);
MYSQL_PROCESSOR(ObMPStmtPrexecute, gctx_);
MYSQL_PROCESSOR(ObMPStmtSendPieceData, gctx_);
MYSQL_PROCESSOR(ObMPStmtGetPieceData, gctx_);
MYSQL_PROCESSOR(ObMPStmtSendLongData, gctx_);
MYSQL_PROCESSOR(ObMPResetConnection, gctx_);
// ps stmt close request may not response packet.
// Howerver, in get processor phase, it may report
// error due to lack of memory and this response error packet.
// To avoid this situation, we make stmt close processor
// by stack memory
case obmysql::COM_STMT_CLOSE: {
char *closepbuf = (&co_closepbuf)->buffer_;
ObMPStmtClose* p = new (&closepbuf[0]) ObMPStmtClose(gctx_);
if (OB_FAIL(p->init())) {
SERVER_LOG(ERROR, "Init ObMPStmtClose fail", K(ret));
p->~ObMPStmtClose();
} else {
processor = p;
}
break;
}
case obmysql::COM_FIELD_LIST: {
/*为了和proxy进行适配,对于COM_FIELD_LIST命令的支持,按照以下原则支持:
* 1. Proxy模式
* 2. Proxy模式
* 2.1. 1.7.6
* 1.7.6
*
* 2.2.
*/
ObSMConnection *conn = reinterpret_cast<ObSMConnection* >(
SQL_REQ_OP.get_sql_session(&req));
if (OB_ISNULL(conn)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(conn), K(ret));
} else if (conn->is_proxy_) {
const char *sup_proxy_min_version = "1.7.6";
uint64_t min_proxy_version = 0;
if (OB_FAIL(ObClusterVersion::get_version(sup_proxy_min_version, min_proxy_version))) {
LOG_WARN("failed to get version", K(ret));
} else if (conn->proxy_version_ < min_proxy_version) {
NEW_MYSQL_PROCESSOR(ObMPDefault, gctx_);
} else {
NEW_MYSQL_PROCESSOR(ObMPQuery, gctx_);
}
} else {
NEW_MYSQL_PROCESSOR(ObMPQuery, gctx_);
}
} else {
NEW_MYSQL_PROCESSOR(ObMPQuery, gctx_);
break;
}
break;
default:
NEW_MYSQL_PROCESSOR(ObMPDefault, gctx_);
break;
}
default:
NEW_MYSQL_PROCESSOR(ObMPDefault, gctx_);
break;
}
if (OB_SUCC(ret) && pkt.get_cmd() == obmysql::COM_FIELD_LIST) {
if (OB_ISNULL(static_cast<ObMPQuery *>(processor))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(static_cast<ObMPQuery *>(processor)));
} else {
static_cast<ObMPQuery *>(processor)->set_is_com_filed_list();
if (OB_SUCC(ret) && pkt.get_cmd() == obmysql::COM_FIELD_LIST) {
if (OB_ISNULL(static_cast<ObMPQuery *>(processor))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(static_cast<ObMPQuery *>(processor)));
} else {
static_cast<ObMPQuery *>(processor)->set_is_com_filed_list();
}
}
}
if (OB_SUCC(ret) && (pkt.get_cmd() == obmysql::COM_STMT_PREPARE
|| pkt.get_cmd() == obmysql::COM_STMT_PREXECUTE)) {
ObSMConnection *conn = reinterpret_cast<ObSMConnection* >(
SQL_REQ_OP.get_sql_session(&req));
if (OB_ISNULL(conn) || OB_ISNULL(dynamic_cast<ObMPBase *>(processor))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(dynamic_cast<ObMPBase *>(processor)));
} else {
uint64_t proxy_version = conn->is_proxy_ ? conn->proxy_version_ : 0;
static_cast<ObMPBase *>(processor)->set_proxy_version(proxy_version);
if (OB_SUCC(ret) && (pkt.get_cmd() == obmysql::COM_STMT_PREPARE
|| pkt.get_cmd() == obmysql::COM_STMT_PREXECUTE)) {
ObSMConnection *conn = reinterpret_cast<ObSMConnection* >(
SQL_REQ_OP.get_sql_session(&req));
if (OB_ISNULL(conn) || OB_ISNULL(dynamic_cast<ObMPBase *>(processor))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(dynamic_cast<ObMPBase *>(processor)));
} else {
uint64_t proxy_version = conn->is_proxy_ ? conn->proxy_version_ : 0;
static_cast<ObMPBase *>(processor)->set_proxy_version(proxy_version);
}
}
}
}
@ -345,6 +358,7 @@ int ObSrvXlator::release(ObReqProcessor *processor)
const char *epbuf = (&co_ep_rpcp_buf)->ep_buffer_;
const char *cpbuf = (&co_closepbuf)->buffer_;
const char *rpcpbuf = (&co_ep_rpcp_buf)->rpcp_buffer_;
const char *mp_query_buf = (&co_ep_rpcp_buf)->obmp_query_buffer_;
if (NULL == processor) {
ret = OB_INVALID_ARGUMENT;
LOG_ERROR("invalid argument", K(processor), K(ret));
@ -355,6 +369,9 @@ int ObSrvXlator::release(ObReqProcessor *processor)
processor->destroy();
ObRequest::TransportProto nio_protocol = (ObRequest::TransportProto)processor->get_nio_protocol();
processor->~ObReqProcessor();
} else if (reinterpret_cast<char*>(processor) == mp_query_buf) {
processor->destroy();
processor->~ObReqProcessor();
} else {
processor->destroy();

View File

@ -215,7 +215,7 @@ int ObTenantConfig::got_version(int64_t version, const bool remove_repeat)
K_(current_version), K(version));
update_task_.update_local_ = true;
update_task_.version_ = version;
update_task_.scheduled_time_ = ObTimeUtility::current_monotonic_raw_time();
update_task_.scheduled_time_ = ObClockGenerator::getClock();
schedule_task = true;
mutex_.unlock();
}

View File

@ -35,7 +35,7 @@ class ObResourceGroup;
static const int64_t WORKER_CHECK_PERIOD = 500L;
static const int64_t REQUEST_WAIT_TIME = 10 * 1000L;
static const int64_t NESTING_REQUEST_WAIT_TIME = 10 * 1000L;
static const int64_t NESTING_REQUEST_WAIT_TIME = 1 * 1000 * 1000L;
// Quick Queue Priorities
enum { QQ_HIGH = 0, QQ_NORMAL, QQ_LOW, QQ_MAX_PRIO };

View File

@ -99,7 +99,6 @@ int ObWorkerProcessor::process(rpc::ObRequest &req)
}
OB_ATOMIC_EVENT_RESET_RECORDER();
PERF_RESET_RECORDER();
const bool enable_trace_log = lib::is_trace_log_enabled();
const int64_t q_time = THIS_THWORKER.get_query_start_time() - req.get_receive_timestamp();
NG_TRACE_EXT(process_begin,
OB_ID(in_queue_time), q_time,
@ -121,7 +120,7 @@ int ObWorkerProcessor::process(rpc::ObRequest &req)
if (OB_LOGGER.is_info_as_wdiag()) {
ObThreadLogLevelUtils::clear();
} else {
if (enable_trace_log && OB_LOG_LEVEL_NONE != packet.get_log_level()) {
if (OB_LOG_LEVEL_NONE != packet.get_log_level()) {
ObThreadLogLevelUtils::init(packet.get_log_level());
}
}
@ -161,9 +160,7 @@ int ObWorkerProcessor::process(rpc::ObRequest &req)
// cleanup
ObCurTraceId::reset();
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
PERF_GATHER_DATA();
//LOG_INFO("yzf debug", "atomic_op", ATOMIC_EVENT_RECORDER);
OB_ATOMIC_EVENT_GATHER_DATA();

View File

@ -1546,8 +1546,6 @@ int ObPL::trans_sql(PlTransformTreeCtx &trans_ctx, ParseNode *root, ObExecContex
sql_info.need_check_fp_ = false;
ObPCParam *pc_param = NULL;
char *ptr = (char *)trans_ctx.allocator_->alloc(param_num * sizeof(ObPCParam));
pc_ctx.fp_result_.raw_params_.set_allocator(trans_ctx.allocator_);
pc_ctx.fp_result_.raw_params_.set_capacity(param_num);
if (OB_ISNULL(ptr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc memory for pc param", K(ret), K(ptr));

View File

@ -2006,6 +2006,8 @@ int ObRootService::execute_bootstrap(const obrpc::ObBootstrapArg &arg)
LOG_WARN("fail to get baseline schema version", KR(ret));
} else if (OB_FAIL(set_cpu_quota_concurrency_config_())) {
LOG_WARN("failed to update cpu_quota_concurrency", K(ret));
} else if (OB_FAIL(set_enable_trace_log_())) {
LOG_WARN("fail to set one phase commit config", K(ret));
}
if (OB_SUCC(ret)) {
@ -11238,6 +11240,17 @@ void ObRootService::update_cpu_quota_concurrency_in_memory_()
}
}
int ObRootService::set_enable_trace_log_()
{
int64_t affected_rows = 0;
int ret = OB_SUCCESS;
if (OB_FAIL(sql_proxy_.write("ALTER SYSTEM SET enable_record_trace_log = false;", affected_rows))) {
LOG_WARN("update enable_record_trace_log failed", K(ret));
}
return OB_SUCCESS;
}
int ObRootService::set_cpu_quota_concurrency_config_()
{
int64_t affected_rows = 0;

View File

@ -889,6 +889,7 @@ private:
const share::ObServerStatus &server_status);
void update_cpu_quota_concurrency_in_memory_();
int set_cpu_quota_concurrency_config_();
int set_enable_trace_log_();
int try_notify_switch_leader(const obrpc::ObNotifySwitchLeaderArg::SwitchLeaderComment &comment);
int precheck_interval_part(const obrpc::ObAlterTableArg &arg);

View File

@ -450,7 +450,7 @@ int ObConfigManager::got_version(int64_t version, const bool remove_repeat/* = f
if (schedule) {
update_task_.version_ = version;
update_task_.scheduled_time_ = ObTimeUtility::current_monotonic_raw_time();
update_task_.scheduled_time_ = ObClockGenerator::getClock();
if (OB_FAIL(TG_SCHEDULE(lib::TGDefIDs::CONFIG_MGR, update_task_, 0, false))) {
LOG_WARN("Update local config failed, may try later", K(ret));
} else {

View File

@ -97,10 +97,10 @@ public:
Value *next_;
};
template <typename Key, typename Value, typename AllocHandle, typename LockType, int64_t BUCKETS_CNT = 64>
class ObLightHashMap {
typedef common::ObSEArray<Value *, 32> ValueArray;
template<typename Key, typename Value, typename AllocHandle, typename LockType, int64_t BUCKETS_CNT = 64, int64_t LOCKS_CNT = BUCKETS_CNT>
class ObLightHashMap
{
typedef common::ObSEArray<Value *, 32> ValueArray;
public:
ObLightHashMap() : is_inited_(false), total_cnt_(0) { OB_ASSERT(BUCKETS_CNT > 0); }
~ObLightHashMap() { destroy(); }
@ -114,7 +114,7 @@ public:
Value *next = nullptr;
for (int64_t i = 0; i < BUCKETS_CNT; ++i) {
{
BucketWLockGuard guard(buckets_[i].lock_, get_itid());
BucketWLockGuard guard(locks_[i % LOCKS_CNT], get_itid());
curr = buckets_[i].next_;
while (OB_NOT_NULL(curr)) {
@ -128,6 +128,9 @@ public:
// reset bucket
buckets_[i].reset();
}
for (int64_t i = 0; i < LOCKS_CNT; ++i) {
locks_[i].destroy();
}
total_cnt_ = 0;
is_inited_ = false;
}
@ -143,12 +146,12 @@ public:
ret = OB_INIT_TWICE;
SHARE_LOG(WARN, "ObLightHashMap init twice", K(ret));
} else {
// init bucket, init lock in bucket
for (int64_t i = 0; OB_SUCC(ret) && i < BUCKETS_CNT; ++i) {
if (OB_FAIL(buckets_[i].init(mem_attr))) {
SHARE_LOG(WARN, "ObLightHashMap bucket init fail", K(ret));
for (int64_t j = 0; j <= i; ++j) {
buckets_[j].destroy();
// init lock
for (int64_t i = 0 ; OB_SUCC(ret) && i < LOCKS_CNT; ++i) {
if (OB_FAIL(locks_[i].init(mem_attr))) {
TRANS_LOG(WARN, "ObLightHashMap locks init fail", K(ret));
for (int64_t j = 0 ; j <= i; ++j) {
locks_[j].destroy();
}
}
}
@ -173,7 +176,7 @@ public:
SHARE_LOG(WARN, "invalid argument", K(key), KP(value));
} else {
int64_t pos = key.hash() % BUCKETS_CNT;
BucketWLockGuard guard(buckets_[pos].lock_, get_itid());
BucketWLockGuard guard(locks_[pos % LOCKS_CNT], get_itid());
Value *curr = buckets_[pos].next_;
while (OB_NOT_NULL(curr)) {
@ -216,7 +219,7 @@ public:
SHARE_LOG(ERROR, "invalid argument", K(key), KP(value));
} else {
int64_t pos = key.hash() % BUCKETS_CNT;
BucketWLockGuard guard(buckets_[pos].lock_, get_itid());
BucketWLockGuard guard(locks_[pos % LOCKS_CNT], get_itid());
if (buckets_[pos].next_ != value && (NULL == value->prev_ && NULL == value->next_)) {
// do nothing
} else {
@ -261,7 +264,7 @@ public:
Value *tmp_value = NULL;
int64_t pos = key.hash() % BUCKETS_CNT;
BucketRLockGuard guard(buckets_[pos].lock_, get_itid());
BucketRLockGuard guard(locks_[pos % LOCKS_CNT], get_itid());
tmp_value = buckets_[pos].next_;
while (OB_NOT_NULL(tmp_value)) {
@ -341,14 +344,19 @@ public:
const int64_t cnt = array.count();
for (int64_t i = 0; i < cnt; ++i) {
if (fn(array.at(i))) {
BucketWLockGuard guard(buckets_[pos].lock_, get_itid());
BucketWLockGuard guard(locks_[pos % LOCKS_CNT], get_itid());
if (buckets_[pos].next_ != array.at(i) && (NULL == array.at(i)->prev_ && NULL == array.at(i)->next_)) {
// do nothing
} else {
del_from_bucket_(pos, array.at(i));
if (0 == array.at(i)->dec_ref(1)) {
TRANS_LOG(WARN, "ref should not be 0 here", K(ret));
alloc_handle_.free_value(array.at(i));
array.at(i) = NULL;
}
}
}
if (0 == array.at(i)->dec_ref(1)) {
if (OB_NOT_NULL(array.at(i)) && 0 == array.at(i)->dec_ref(1)) {
alloc_handle_.free_value(array.at(i));
}
}
@ -361,7 +369,7 @@ public:
{
int ret = common::OB_SUCCESS;
// read lock
BucketRLockGuard guard(buckets_[bucket_pos].lock_, get_itid());
BucketRLockGuard guard(locks_[bucket_pos % LOCKS_CNT], get_itid());
Value *val = buckets_[bucket_pos].next_;
while (OB_SUCC(ret) && OB_NOT_NULL(val)) {
@ -406,16 +414,13 @@ private:
struct ObLightHashHeader {
Value *next_;
Value *hot_cache_val_;
LockType lock_;
ObLightHashHeader() : next_(NULL), hot_cache_val_(NULL) {}
~ObLightHashHeader() { destroy(); }
int init(const lib::ObMemAttr &mem_attr) { return lock_.init(mem_attr); }
void reset()
{
next_ = NULL;
hot_cache_val_ = NULL;
lock_.destroy();
}
void destroy() { reset(); }
};
@ -508,6 +513,7 @@ private:
// sizeof(QsyncLock) = 4K;
bool is_inited_;
ObLightHashHeader buckets_[BUCKETS_CNT];
LockType locks_[LOCKS_CNT];
int64_t total_cnt_;
#ifndef NDEBUG
public:

View File

@ -1029,7 +1029,7 @@ public:
// New members of ObCreateTableArg need to pay attention to the implementation of is_allow_when_upgrade
bool is_alter_view_;
ObSequenceDDLArg sequence_ddl_arg_;
common::ObSArray<share::schema::ObDependencyInfo> dep_infos_;
common::ObSArray<oceanbase::share::schema::ObDependencyInfo> dep_infos_;
common::ObSArray<ObMVAdditionalInfo> mv_ainfo_;
};
@ -1109,7 +1109,7 @@ ObCreateSynonymArg():
share::schema::ObSynonymInfo synonym_info_;
common::ObString db_name_;
common::ObString obj_db_name_;
share::schema::ObDependencyInfo dependency_info_;
oceanbase::share::schema::ObDependencyInfo dependency_info_;
};
struct ObDropSynonymArg : ObDDLArg
@ -6861,7 +6861,7 @@ public:
share::schema::ObPackageInfo package_info_;
common::ObSArray<share::schema::ObRoutineInfo> public_routine_infos_;
share::schema::ObErrorInfo error_info_;
common::ObSArray<share::schema::ObDependencyInfo> dependency_infos_;
common::ObSArray<oceanbase::share::schema::ObDependencyInfo> dependency_infos_;
};
struct ObAlterPackageArg : public ObDDLArg

View File

@ -62,7 +62,7 @@ int ObDASCtx::init(const ObPhysicalPlan &plan, ObExecContext &ctx)
}
}
}
LOG_TRACE("init das context finish", K(ret), K(normal_locations), K(das_locations), K(table_locs_));
LOG_DEBUG("init das context finish", K(ret), K(normal_locations), K(das_locations), K(table_locs_));
return ret;
}
@ -121,6 +121,7 @@ ObDASTableLoc *ObDASCtx::get_table_loc_by_id(uint64_t table_loc_id, uint64_t ref
if ((*tmp_node)->loc_meta_->table_loc_id_ == table_loc_id &&
(*tmp_node)->loc_meta_->ref_table_id_ == ref_table_id) {
table_loc = *tmp_node;
break;
}
}
return table_loc;
@ -231,21 +232,24 @@ int ObDASCtx::check_same_server(const ObDASTabletLoc *tablet_loc)
ret = OB_INVALID_ARGUMENT;
LOG_WARN("tablet location is null", KR(ret), KP(tablet_loc));
} else if (same_server_) {
ObDASTabletLoc *first_tablet = NULL;
FOREACH_X(table_node, table_locs_, NULL == first_tablet) {
ObDASTableLoc *cur_table_loc = *table_node;
for (DASTabletLocListIter tablet_node = cur_table_loc->tablet_locs_begin();
NULL == first_tablet && tablet_node != cur_table_loc->tablet_locs_end();
++tablet_node) {
first_tablet = *tablet_node;
if (!same_tablet_addr_.is_valid()) {
ObDASTabletLoc *first_tablet = NULL;
FOREACH_X(table_node, table_locs_, NULL == first_tablet)
{
ObDASTableLoc *cur_table_loc = *table_node;
for (DASTabletLocListIter tablet_node = cur_table_loc->tablet_locs_begin();
NULL == first_tablet && tablet_node != cur_table_loc->tablet_locs_end(); ++tablet_node) {
first_tablet = *tablet_node;
}
}
if (OB_ISNULL(first_tablet)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("first tablet location is null", KR(ret), KP(first_tablet));
} else {
same_tablet_addr_ = first_tablet->server_;
}
}
if (OB_ISNULL(first_tablet)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("first tablet location is null", KR(ret), KP(first_tablet));
} else if (tablet_loc->server_ != first_tablet->server_) {
same_server_ = false;
}
same_server_ = (tablet_loc->server_ == same_tablet_addr_);
}
return ret;
}
@ -425,7 +429,51 @@ int ObDASCtx::add_candi_table_loc(const ObDASTableLocMeta &loc_meta,
LOG_WARN("extended tablet loc failed", K(ret));
}
}
LOG_TRACE("das table loc assign finish", K(candi_table_loc), K(loc_meta), K(table_loc->get_tablet_locs()));
LOG_DEBUG("das table loc assign finish", K(candi_table_loc), K(loc_meta), K(table_loc->get_tablet_locs()));
return ret;
}
int ObDASCtx::add_final_table_loc(const ObDASTableLocMeta &loc_meta,
const ObIArray<ObTabletID> &tablet_ids,
const ObIArray<ObObjectID> &partition_ids,
const ObIArray<ObObjectID> &first_level_part_ids)
{
int ret = OB_SUCCESS;
ObDASTableLoc *table_loc = nullptr;
ObDASTableLocMeta *final_meta = nullptr;
LOG_DEBUG("das table loc assign begin", K(loc_meta));
if (OB_FAIL(ObDASUtils::build_table_loc_meta(allocator_, loc_meta, final_meta))) {
LOG_WARN("build table loc meta failed", K(ret));
} else if (OB_FAIL(extended_table_loc(*final_meta, table_loc))) {
LOG_WARN("extended table loc failed", K(ret), K(loc_meta));
}
for (int64_t i = 0; OB_SUCC(ret) && i < tablet_ids.count(); ++i) {
ObDASTabletLoc *tablet_loc = nullptr;
ObObjectID first_level_part_id =
first_level_part_ids.empty() ? OB_INVALID_ID : first_level_part_ids.at(i);
if (OB_FAIL(extended_tablet_loc(*table_loc,
tablet_ids.at(i),
tablet_loc,
partition_ids.at(i),
first_level_part_id))) {
LOG_WARN("extended tablet loc failed", K(ret));
}
}
LOG_DEBUG("das table loc assign finish", K(loc_meta), K(table_loc->get_tablet_locs()));
if (OB_FAIL(ret)) {
clear_all_location_info();
}
return ret;
}
int ObDASCtx::build_table_loc_meta(const ObDASTableLocMeta &src,
ObDASTableLocMeta *&dst)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObDASUtils::build_table_loc_meta(allocator_, src, dst))) {
LOG_WARN("build table loc meta failed", K(ret));
}
return ret;
}

View File

@ -52,6 +52,7 @@ public:
savepoint_(),
write_branch_id_(0),
del_ctx_list_(allocator),
same_tablet_addr_(),
jump_read_group_id_(-1),
flags_(0)
{
@ -93,6 +94,11 @@ public:
ObDASTabletLoc *&talet_loc);
int extended_table_loc(const ObDASTableLocMeta &loc_meta, ObDASTableLoc *&table_loc);
int add_candi_table_loc(const ObDASTableLocMeta &loc_meta, const ObCandiTableLoc &candi_table_loc);
int add_final_table_loc(const ObDASTableLocMeta &loc_meta,
const ObIArray<ObTabletID> &tablet_ids,
const ObIArray<ObObjectID> &partition_ids,
const ObIArray<ObObjectID> &first_level_part_ids);
int build_table_loc_meta(const ObDASTableLocMeta &src, ObDASTableLocMeta *&dst);
int get_das_tablet_mapper(const uint64_t ref_table_id,
ObDASTabletMapper &tablet_mapper,
const DASTableIDArrayWrap *related_table_ids = nullptr);
@ -113,6 +119,7 @@ public:
table_locs_.clear();
related_tablet_map_.clear();
external_table_locs_.clear();
same_tablet_addr_.reset();
same_server_ = 1;
}
ObDASTaskFactory &get_das_factory() { return das_factory_; }
@ -124,6 +131,7 @@ public:
int build_external_table_location(
uint64_t table_loc_id, uint64_t ref_table_id, common::ObIArray<ObAddr> &locations);
int build_related_tablet_map(const ObDASTableLocMeta &loc_meta);
const ObAddr &same_tablet_addr() const { return same_tablet_addr_; }
TO_STRING_KV(K_(table_locs),
K_(external_table_locs),
@ -154,6 +162,7 @@ private:
int16_t write_branch_id_;
//@todo: save snapshot version
DASDelCtxList del_ctx_list_;
ObAddr same_tablet_addr_;
public:
int64_t jump_read_group_id_;
union {

View File

@ -482,7 +482,7 @@ int ObDASWriteBuffer::try_add_row(const ObIArray<ObExpr*> &exprs,
ret = OB_ERR_UNEXPECTED;
LOG_WARN("stored row is null", K(ret));
} else {
LOG_TRACE("add dml_row pay_load here", KPC(stored_row));
LOG_DEBUG("add dml_row pay_load here", KPC(stored_row));
}
}

View File

@ -22,6 +22,7 @@
#include "sql/das/ob_das_utils.h"
#include "sql/ob_sql_context.h"
#include "storage/tx/wrs/ob_black_list.h"
#include "storage/tx/ob_trans_service.h"
#include "lib/rc/context.h"
namespace oceanbase
@ -916,7 +917,7 @@ int ObDASLocationRouter::nonblock_get_candi_tablet_locations(const ObDASTableLoc
LOG_WARN("fail to set partition location with only readable replica",
K(ret),K(i), K(location), K(candi_tablet_locs), K(tablet_ids), K(partition_ids));
}
LOG_TRACE("set partition location with only readable replica",
LOG_DEBUG("set partition location with only readable replica",
K(ret),K(i), K(location), K(candi_tablet_locs), K(tablet_ids), K(partition_ids));
}
} // for end
@ -954,12 +955,24 @@ int ObDASLocationRouter::nonblock_get_leader(const uint64_t tenant_id,
int ret = OB_SUCCESS;
bool is_cache_hit = false;
tablet_loc.tablet_id_ = tablet_id;
ObTransService *trans_service = MTL(ObTransService *);
bool is_local_leader = false;
if (OB_FAIL(all_tablet_list_.push_back(tablet_id))) {
LOG_WARN("store access tablet id failed", K(ret), K(tablet_id));
} else if (OB_FAIL(GCTX.location_service_->nonblock_get(tenant_id,
tablet_id,
tablet_loc.ls_id_))) {
LOG_WARN("nonblock get ls id failed", K(ret), K(tablet_id));
} else if (get_total_retry_cnt() > 0 || OB_FAIL(trans_service->check_and_get_ls_info(tablet_id, tablet_loc.ls_id_, is_local_leader))) {
ret = OB_SUCCESS;
if (OB_FAIL(GCTX.location_service_->nonblock_get(tenant_id,
tablet_id,
tablet_loc.ls_id_))) {
LOG_WARN("nonblock get ls id failed", K(ret), K(tablet_id));
} else if (OB_FAIL(GCTX.location_service_->nonblock_get_leader(GCONF.cluster_id,
tenant_id,
tablet_loc.ls_id_,
tablet_loc.server_))) {
LOG_WARN("nonblock get ls location failed", K(ret), K(tablet_loc));
}
} else if (is_local_leader) {
tablet_loc.server_ = GCTX.self_addr();
} else if (OB_FAIL(GCTX.location_service_->nonblock_get_leader(GCONF.cluster_id,
tenant_id,
tablet_loc.ls_id_,

View File

@ -132,9 +132,12 @@ public:
int insert_related_tablet_map();
void clear()
{
list_.clear();
map_.clear();
if (!empty()) {
list_.clear();
map_.clear();
}
}
bool empty() const { return list_.empty() && map_.empty(); }
const RelatedTabletList &get_list() const { return list_; }
TO_STRING_KV(K_(list), "map_size", map_.size());
private:

View File

@ -278,7 +278,7 @@ int ObDASRef::execute_all_task()
LOG_WARN("failed to process all async remote tasks", KR(ret));
}
ret = COVER_SUCC(tmp_ret);
if (check_rcode_can_retry(ret)) {
if (OB_FAIL(ret) && check_rcode_can_retry(ret)) {
ret = OB_SUCCESS;
}
if (OB_SUCC(ret)) {

View File

@ -134,7 +134,7 @@ int ObDASUpdIterator::get_next_row(ObNewRow *&row)
LOG_WARN("project new storage row failed", K(ret));
} else {
row = old_row_;
LOG_TRACE("DAS update get old row",
LOG_DEBUG("DAS update get old row",
K_(das_ctdef_->old_row_projector),
K_(das_ctdef_->new_row_projector),
"table_id", das_ctdef_->table_id_,

View File

@ -325,7 +325,7 @@ int ObDASIndexDMLAdaptor<N, DMLIterator>::write_tablet(DMLIterator &iter, int64_
{
int ret = common::OB_SUCCESS;
affected_rows = 0;
SQL_DAS_LOG(TRACE, "begin to write the main tablet",
SQL_DAS_LOG(DEBUG, "begin to write the main tablet",
K(ls_id_), K(tablet_id_), K(ctdef_->table_id_), K(ctdef_->index_tid_));
if (ctdef_->is_ignore_) {
if (OB_FAIL(write_tablet_with_ignore(iter, affected_rows))) {
@ -343,7 +343,7 @@ int ObDASIndexDMLAdaptor<N, DMLIterator>::write_tablet(DMLIterator &iter, int64_
RtDefType *related_rtdef = static_cast<RtDefType*>(related_rtdefs_->at(i));
ObTabletID related_tablet_id = related_tablet_ids_->at(i);
int64_t index_affected_rows = 0;
SQL_DAS_LOG(TRACE, "rewind iterator and write local index tablet",
SQL_DAS_LOG(DEBUG, "rewind iterator and write local index tablet",
K(ls_id_), K(related_tablet_id), K(related_ctdef->table_id_), K(related_ctdef->index_tid_));
if (OB_FAIL(iter.rewind(related_ctdef))) {
SQL_DAS_LOG(WARN, "rewind iterator failed", K(ret));

View File

@ -1154,7 +1154,7 @@ int ObTableModifyOp::get_next_row_from_child()
LOG_WARN("fail to get next row", K(ret));
}
} else {
LOG_TRACE("child output row", "row", ROWEXPR2STR(eval_ctx_, child_->get_spec().output_));
LOG_DEBUG("child output row", "row", ROWEXPR2STR(eval_ctx_, child_->get_spec().output_));
}
return ret;
}

View File

@ -679,7 +679,7 @@ int ObTempExpr::eval(ObExecContext &exec_ctx, const ObNewRow &row, ObObj &result
if (!exec_ctx.use_temp_expr_ctx_cache()) {
temp_expr_ctx->~ObTempExprCtx();
}
LOG_TRACE("temp expr result", K(result), K(row), K(rt_exprs_));
LOG_DEBUG("temp expr result", K(result), K(row), K(rt_exprs_));
}
return ret;

View File

@ -258,16 +258,6 @@ int ObExecContext::init_phy_op(const uint64_t phy_op_size)
LOG_WARN("init operator kit store failed", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(gi_task_map_)) {
// Do nothing.
} else if (gi_task_map_->created()) {
// Do nothing. If this map has been created, it means this plan is trying to reopen.
} else if (OB_FAIL(gi_task_map_->create(PARTITION_WISE_JOIN_TSC_HASH_BUCKET_NUM, /* assume no more than 8 table scan in a plan */
ObModIds::OB_SQL_PX))) {
LOG_WARN("create gi task map failed", K(ret));
}
}
return ret;
}

View File

@ -293,7 +293,7 @@ int ObOpSpec::create_operator(ObExecContext &exec_ctx, ObOperator *&op) const
} else if (OB_FAIL(create_exec_feedback_node_recursive(exec_ctx))) {
LOG_WARN("fail to create exec feedback node", K(ret));
}
LOG_TRACE("trace create operator", K(ret), K(lbt()));
LOG_DEBUG("trace create operator", K(ret), K(lbt()));
return ret;
}
@ -312,7 +312,7 @@ int ObOpSpec::create_operator_recursive(ObExecContext &exec_ctx, ObOperator *&op
K(ret), K(id_), KP(kit), KP(children_), K(create_child_cnt), K(type_));
} else {
kit->spec_ = this;
LOG_TRACE("trace create spec", K(ret), K(id_), K(type_));
LOG_DEBUG("trace create spec", K(ret), K(id_), K(type_));
for (int64_t i = 0; OB_SUCC(ret) && i < child_cnt_; i++) {
if (NULL == children_[i]) {
// 这里如果有child但为nullptr,说明是receive算子
@ -341,7 +341,7 @@ int ObOpSpec::create_operator_recursive(ObExecContext &exec_ctx, ObOperator *&op
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL input returned", K(ret));
} else {
LOG_TRACE("trace create input", K(ret), K(id_), K(type_));
LOG_DEBUG("trace create input", K(ret), K(id_), K(type_));
}
}
}

View File

@ -287,7 +287,9 @@ struct AllocOpHelper
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc memory failed", K(ret), K(alloc_size));
} else {
memset(mem, 0, sizeof(OpType *) * child_cnt);
if (child_cnt > 0) {
memset(mem, 0, sizeof(OpType *) * child_cnt);
}
op = new (&mem[child_cnt]) OpType(exec_ctx, spec, input);
if (OB_FAIL(op->set_children_pointer(mem, child_cnt))
|| OB_FAIL(op->init())) {

View File

@ -160,7 +160,6 @@ void PxWorkerFunctor::operator ()(bool need_exec)
ObPxSqcHandler *sqc_handler = task_arg_.get_sqc_handler();
SQCHandlerGuard sqc_handler_guard(sqc_handler);
lib::MemoryContext mem_context = nullptr;
const bool enable_trace_log = lib::is_trace_log_enabled();
//ensure PX worker skip updating timeout_ts_ by ntp offset
THIS_WORKER.set_ntp_offset(0);
if (!need_exec) {
@ -185,7 +184,7 @@ void PxWorkerFunctor::operator ()(bool need_exec)
if (OB_LOGGER.is_info_as_wdiag()) {
ObThreadLogLevelUtils::clear();
} else {
if (OB_LOG_LEVEL_NONE != env_arg_.get_log_level() && enable_trace_log) {
if (OB_LOG_LEVEL_NONE != env_arg_.get_log_level()) {
ObThreadLogLevelUtils::init(env_arg_.get_log_level());
}
}
@ -233,9 +232,7 @@ void PxWorkerFunctor::operator ()(bool need_exec)
}
}
}
if (enable_trace_log) {
ObThreadLogLevelUtils::clear();
}
ObThreadLogLevelUtils::clear();
} else if (OB_ISNULL(sqc_handler)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("Unexpected null sqc handler", K(sqc_handler));

View File

@ -1235,7 +1235,7 @@ int ObTableScanOp::prepare_single_scan_range(int64_t group_idx)
if (OB_SUCC(ret) && MY_SPEC.is_vt_mapping_) {
OZ(vt_result_converter_->convert_key_ranges(MY_INPUT.key_ranges_));
}
LOG_TRACE("prepare single scan range", K(ret), K(key_ranges), K(MY_INPUT.key_ranges_),
LOG_DEBUG("prepare single scan range", K(ret), K(key_ranges), K(MY_INPUT.key_ranges_),
K(MY_INPUT.ss_key_ranges_));
return ret;
}

View File

@ -14,8 +14,6 @@
#include "sql/executor/ob_maintain_dependency_info_task.h"
#include "share/ob_common_rpc_proxy.h"
#include "share/schema/ob_schema_getter_guard.h"
#include "share/schema/ob_multi_version_schema_service.h"
#include "observer/ob_server_struct.h"
namespace oceanbase
{

View File

@ -15,7 +15,6 @@
#include "lib/container/ob_fixed_array.h"
#include "lib/thread/ob_async_task_queue.h"
#include "share/schema/ob_dependency_info.h"
#include "lib/hash/ob_hashset.h"
#include "share/schema/ob_multi_version_schema_service.h"
#include "share/schema/ob_schema_struct.h"

View File

@ -149,7 +149,7 @@ OB_INLINE int ObResultSet::open_plan()
int ObResultSet::open()
{
int ret = OB_SUCCESS;
my_session_.set_process_query_time(ObTimeUtility::current_time());
my_session_.set_process_query_time(ObClockGenerator::getClock());
LinkExecCtxGuard link_guard(my_session_, get_exec_context());
FLTSpanGuard(open);
if (lib::is_oracle_mode() &&
@ -1031,7 +1031,7 @@ OB_INLINE int ObResultSet::auto_end_plan_trans(ObPhysicalPlan& plan,
bool is_rollback = false;
my_session_.get_autocommit(ac);
async = false;
LOG_TRACE("auto_end_plan_trans.start", K(ret),
LOG_DEBUG("auto_end_plan_trans.start", K(ret),
K(in_trans), K(ac), K(explicit_trans),
K(is_async_end_trans_submitted()));
// explicit start trans will disable auto-commit
@ -1121,7 +1121,7 @@ OB_INLINE int ObResultSet::auto_end_plan_trans(ObPhysicalPlan& plan,
}
}
NG_TRACE(auto_end_plan_end);
LOG_TRACE("auto_end_plan_trans.end", K(ret),
LOG_DEBUG("auto_end_plan_trans.end", K(ret),
K(in_trans), K(ac), K(explicit_trans), K(plan.is_need_trans()),
K(is_rollback), K(async),
K(is_async_end_trans_submitted()));

View File

@ -439,7 +439,10 @@ int ObSqlTransControl::do_end_trans_(ObSQLSessionInfo *session,
transaction::ObTxDesc *&tx_ptr = session->get_tx_desc();
bool is_detector_exist = false;
int tmp_ret = OB_SUCCESS;
if (OB_ISNULL(MTL(share::detector::ObDeadLockDetectorMgr*))) {
const int64_t lcl_op_interval = GCONF._lcl_op_interval;
if (lcl_op_interval <= 0) {
// do nothing
} else if (OB_ISNULL(MTL(share::detector::ObDeadLockDetectorMgr*))) {
tmp_ret = OB_BAD_NULL_ERROR;
DETECT_LOG(WARN, "MTL ObDeadLockDetectorMgr is NULL", K(tmp_ret), K(tx_ptr->tid()));
} else if (OB_TMP_FAIL(MTL(share::detector::ObDeadLockDetectorMgr*)->
@ -795,17 +798,23 @@ int ObSqlTransControl::stmt_setup_snapshot_(ObSQLSessionInfo *session,
int64_t stmt_expire_ts = get_stmt_expire_ts(plan_ctx, *session);
share::ObLSID first_ls_id;
bool local_single_ls_plan = false;
bool is_single_tablet = false;
const bool local_single_ls_plan_maybe = plan->is_local_plan() &&
OB_PHY_PLAN_LOCAL == plan->get_location_type();
if (local_single_ls_plan_maybe) {
if (OB_FAIL(get_first_lsid(das_ctx, first_ls_id))) {
if (OB_FAIL(get_first_lsid(das_ctx, first_ls_id, is_single_tablet))) {
} else if (!first_ls_id.is_valid()) {
// do nothing
// get_ls_read_snapshot may degenerate into get_gts, so it can be used even if the ls is not local.
// This is mainly to solve the problem of strong reading performance in some single-tablet scenarios.
} else if (OB_FAIL(txs->get_ls_read_snapshot(tx_desc,
session->get_tx_isolation(),
first_ls_id,
stmt_expire_ts,
snapshot))) {
} else if (is_single_tablet) {
// performance for single tablet scenario
local_single_ls_plan = true;
} else {
local_single_ls_plan = has_same_lsid(das_ctx, snapshot, first_ls_id);
}
@ -935,7 +944,7 @@ uint32_t ObSqlTransControl::get_real_session_id(ObSQLSessionInfo &session)
return session.get_xid().empty() ? 0 : (session.get_proxy_sessid() != 0 ? session.get_proxy_sessid() : session.get_sessid());
}
int ObSqlTransControl::get_first_lsid(const ObDASCtx &das_ctx, share::ObLSID &first_lsid)
int ObSqlTransControl::get_first_lsid(const ObDASCtx &das_ctx, share::ObLSID &first_lsid, bool &is_single_tablet)
{
int ret = OB_SUCCESS;
const DASTableLocList &table_locs = das_ctx.get_table_loc_list();
@ -946,6 +955,7 @@ int ObSqlTransControl::get_first_lsid(const ObDASCtx &das_ctx, share::ObLSID &fi
const ObDASTabletLoc *tablet_loc = tablet_locs.get_first();
first_lsid = tablet_loc->ls_id_;
}
is_single_tablet = (1 == table_locs.size() && 1 == tablet_locs.size());
}
return ret;
}

View File

@ -263,7 +263,7 @@ private:
transaction::ObTransService *txs,
bool &start_hook);
static uint32_t get_real_session_id(ObSQLSessionInfo &session);
static int get_first_lsid(const ObDASCtx &das_ctx, share::ObLSID &first_lsid);
static int get_first_lsid(const ObDASCtx &das_ctx, share::ObLSID &first_lsid, bool &is_single_tablet);
static bool has_same_lsid(const ObDASCtx &das_ctx,
const transaction::ObTxReadSnapshot &snapshot,
share::ObLSID &first_lsid);

View File

@ -1420,6 +1420,79 @@ int ObTableLocation::calculate_candi_tablet_locations(
return ret;
}
int ObTableLocation::calculate_single_tablet_partition(ObExecContext &exec_ctx,
const ParamStore &params,
const ObDataTypeCastParams &dtc_params) const
{
int ret = OB_SUCCESS;
ObDASTabletMapper tablet_mapper;
ObDASCtx &das_ctx = exec_ctx.get_das_ctx();
tablet_mapper.set_non_partitioned_table_ids(tablet_id_, object_id_, &related_list_);
if (OB_FAIL(das_ctx.get_das_tablet_mapper(loc_meta_.ref_table_id_,
tablet_mapper,
&loc_meta_.related_table_ids_))) {
LOG_WARN("failed to get das tablet mapper", K(ret));
} else {
DASRelatedTabletMap *map =
static_cast<DASRelatedTabletMap*>(tablet_mapper.get_related_table_info().related_map_);
if (OB_NOT_NULL(map) && !related_list_.empty() && OB_FAIL(map->assign(related_list_))) {
LOG_WARN("failed to assign related map list", K(ret));
}
}
LOG_DEBUG("calculate single tablet id end", K(loc_meta_), K(object_id_), K(tablet_id_));
NG_TRACE(tl_calc_part_id_end);
ObDASTableLoc *table_loc = nullptr;
ObDASTabletLoc *tablet_loc = nullptr;
ObDASTableLocMeta *final_meta = nullptr;
LOG_DEBUG("das table loc assign begin", K_(loc_meta));
if (OB_FAIL(das_ctx.build_table_loc_meta(loc_meta_, final_meta))) {
LOG_WARN("build table loc meta failed", K(ret));
} else if (OB_FAIL(das_ctx.extended_table_loc(*final_meta, table_loc))) {
LOG_WARN("extended table loc failed", K(ret), K(loc_meta_));
} else if (OB_FAIL(das_ctx.extended_tablet_loc(*table_loc, tablet_id_, tablet_loc, object_id_))) {
LOG_WARN("extended tablet loc failed", K(ret));
}
if (OB_FAIL(ret)) {
das_ctx.clear_all_location_info();
}
return ret;
}
int ObTableLocation::calculate_final_tablet_locations(ObExecContext &exec_ctx,
const ParamStore &params,
const ObDataTypeCastParams &dtc_params) const
{
int ret = OB_SUCCESS;
if (!inited_) {
ret = OB_NOT_INIT;
LOG_WARN("ObTableLocation not inited", K(ret));
} else if (is_non_partition_optimized_) {
// fast path for single tablet
if (OB_FAIL(calculate_single_tablet_partition(exec_ctx, params, dtc_params))) {
LOG_WARN("failed to calculate single final tablet location", K(ret));
}
} else {
ObSEArray<ObObjectID, 8> partition_ids;
ObSEArray<ObObjectID, 8> first_level_part_ids;
ObSEArray<ObTabletID, 8> tablet_ids;
if (OB_FAIL(calculate_tablet_ids(exec_ctx,
params,
tablet_ids,
partition_ids,
first_level_part_ids,
dtc_params))) {
LOG_WARN("failed to calculate final tablet ids", K(ret));
} else if (OB_FAIL(add_final_tablet_locations(exec_ctx.get_das_ctx(),
tablet_ids,
partition_ids,
first_level_part_ids))) {
LOG_WARN("failed to add final tablet locations to das_ctx", K(ret));
}
}
return ret;
}
int ObTableLocation::init_partition_ids_by_rowkey2(ObExecContext &exec_ctx,
ObSQLSessionInfo &session_info,
ObSchemaGetterGuard &schema_guard,
@ -1654,7 +1727,6 @@ int ObTableLocation::calculate_tablet_ids(ObExecContext &exec_ctx,
{
int ret = OB_SUCCESS;
ObDASTabletMapper tablet_mapper;
ObPartitionIdMap partition_id_map;
if (!inited_) {
ret = OB_NOT_INIT;
LOG_WARN("ObTableLocation not inited", K(ret));
@ -1665,6 +1737,7 @@ int ObTableLocation::calculate_tablet_ids(ObExecContext &exec_ctx,
loc_meta_.ref_table_id_, tablet_mapper, &loc_meta_.related_table_ids_))) {
LOG_WARN("fail to get das tablet mapper", K(ret));
} else {
ObPartitionIdMap partition_id_map;
if (is_in_hit_) { //判断是否是in类型
if (OB_FAIL(calc_partition_ids_by_in_expr(exec_ctx, tablet_mapper, tablet_ids, partition_ids, dtc_params))) {
LOG_WARN("fail to calc_partition_ids_by_in_expr", K(ret));
@ -1742,7 +1815,7 @@ int ObTableLocation::calculate_tablet_ids(ObExecContext &exec_ctx,
}
}
LOG_TRACE("calculate tablet ids end", K(loc_meta_), K(partition_ids), K(tablet_ids));
LOG_DEBUG("calculate tablet ids end", K(loc_meta_), K(partition_ids), K(tablet_ids));
NG_TRACE(tl_calc_part_id_end);
return ret;
}
@ -1760,6 +1833,17 @@ int ObTableLocation::get_tablet_locations(ObDASCtx &das_ctx,
candi_tablet_locs);
}
int ObTableLocation::add_final_tablet_locations(ObDASCtx &das_ctx,
const ObIArray<ObTabletID> &tablet_ids,
const ObIArray<ObObjectID> &partition_ids,
const ObIArray<ObObjectID> &first_level_part_ids) const
{
return das_ctx.add_final_table_loc(loc_meta_,
tablet_ids,
partition_ids,
first_level_part_ids);
}
int ObTableLocation::get_part_col_type(const ObRawExpr *expr,
ObObjType &col_type,
ObCollationType &collation_type,

View File

@ -627,6 +627,15 @@ public:
const ParamStore &params,
ObCandiTabletLocIArray &candi_tablet_locs,
const common::ObDataTypeCastParams &dtc_params) const;
int calculate_single_tablet_partition(ObExecContext &exec_ctx,
const ParamStore &params,
const ObDataTypeCastParams &dtc_params) const;
int calculate_final_tablet_locations(ObExecContext &exec_ctx,
const ParamStore &params,
const common::ObDataTypeCastParams &dtc_params) const;
/**
* Calculate tablet ids from input parameters.
*/
@ -669,6 +678,11 @@ public:
const ObIArray<ObObjectID> &first_level_part_ids,
ObCandiTabletLocIArray &candi_tablet_locs) const;
int add_final_tablet_locations(ObDASCtx &das_ctx,
const ObIArray<ObTabletID> &tablet_ids,
const ObIArray<ObObjectID> &partition_ids,
const ObIArray<ObObjectID> &first_level_part_ids) const;
static int send_add_interval_partition_rpc_new_engine(ObIAllocator &allocator,
ObSQLSessionInfo *session,
ObSchemaGetterGuard *schema_guard,

View File

@ -425,7 +425,7 @@ int ObFastParserBase::parser_insert_str(common::ObIAllocator &allocator,
can_batch_opt = is_valid;
}
}
LOG_TRACE("after parser insert print curr_sql", K(old_no_param_sql), K(new_truncated_sql),
LOG_DEBUG("after parser insert print curr_sql", K(old_no_param_sql), K(new_truncated_sql),
K(can_batch_opt), K(params_count), K(row_count));
return ret;
}

View File

@ -69,7 +69,6 @@ int ObDistPlans::get_plan(ObPlanCacheCtx &pc_ctx,
// for single dist plan without px, we already fill the phy locations while calculating plan type
// for multi table px plan, physical location is calculated in match step
ObArray<ObCandiTableLoc> candi_table_locs;
bool need_check_on_same_server = false;
if (OB_ISNULL(plan_set_)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid null plan set", K(ret), K(plan_set_));
@ -77,12 +76,13 @@ int ObDistPlans::get_plan(ObPlanCacheCtx &pc_ctx,
// do nothing
} else if (OB_FAIL(ObPhyLocationGetter::get_phy_locations(plan->get_table_locations(),
pc_ctx,
candi_table_locs,
need_check_on_same_server))) {
candi_table_locs))) {
LOG_WARN("failed to get physical table locations", K(ret));
} else if (OB_FAIL(ObPhyLocationGetter::build_table_locs(pc_ctx.exec_ctx_.get_das_ctx(),
plan->get_table_locations(),
candi_table_locs))) {
} else if (candi_table_locs.empty()) {
// do nothing.
} else if (OB_FAIL(ObPhyLocationGetter::build_candi_table_locs(pc_ctx.exec_ctx_.get_das_ctx(),
plan->get_table_locations(),
candi_table_locs))) {
LOG_WARN("fail to init table locs", K(ret));
}
}

View File

@ -140,7 +140,7 @@ int ObPCVSet::inner_get_cache_obj(ObILibCacheCtx &ctx,
}
}
if (pc_ctx.exec_ctx_.get_min_cluster_version() != GET_MIN_CLUSTER_VERSION()) {
LOG_TRACE("Lob Debug, using remote min cluster version",
LOG_DEBUG("Lob Debug, using remote min cluster version",
K(pc_ctx.exec_ctx_.get_min_cluster_version()),
K(GET_MIN_CLUSTER_VERSION()));
}
@ -181,7 +181,7 @@ int ObPCVSet::inner_get_cache_obj(ObILibCacheCtx &ctx,
bool need_check_schema = true;
DLIST_FOREACH(pcv, pcv_list_) {
bool is_same = false;
LOG_TRACE("get plan, pcv", K(pcv));
LOG_DEBUG("get plan, pcv", K(pcv));
if (OB_FAIL(pcv->get_all_dep_schema(pc_ctx,
pc_ctx.sql_ctx_.session_info_->get_database_id(),
new_tenant_schema_version,

View File

@ -740,8 +740,6 @@ int ObPlanCache::construct_fast_parser_result(common::ObIAllocator &allocator,
K(batch_count), K(first_truncated_sql), K(pc_ctx.raw_sql_), K(fp_result));
} else {
fp_result.raw_params_.reset();
fp_result.raw_params_.set_allocator(&allocator);
fp_result.raw_params_.set_capacity(pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0)->count());
if (OB_FAIL(fp_result.raw_params_.assign(*pc_ctx.insert_batch_opt_info_.multi_raw_params_.at(0)))) {
LOG_WARN("fail to assign raw_param", K(ret));
} else {
@ -1229,18 +1227,18 @@ int ObPlanCache::get_cache_obj(ObILibCacheCtx &ctx,
SQL_PC_LOG(TRACE, "failed to get cache node from lib cache by key", K(ret));
} else if (OB_UNLIKELY(NULL == cache_node)) {
ret = OB_SQL_PC_NOT_EXIST;
SQL_PC_LOG(TRACE, "cache obj does not exist!", K(key));
SQL_PC_LOG(DEBUG, "cache obj does not exist!", K(key));
} else {
LOG_TRACE("inner_get_cache_obj", K(key), K(cache_node));
LOG_DEBUG("inner_get_cache_obj", K(key), K(cache_node));
if (OB_FAIL(cache_node->update_node_stat(ctx))) {
SQL_PC_LOG(WARN, "failed to update node stat", K(ret));
} else if (OB_FAIL(cache_node->get_cache_obj(ctx, key, cache_obj))) {
if (OB_SQL_PC_NOT_EXIST != ret) {
LOG_TRACE("cache_node fail to get cache obj", K(ret));
LOG_DEBUG("cache_node fail to get cache obj", K(ret));
}
} else {
guard.cache_obj_ = cache_obj;
LOG_TRACE("succ to get cache obj", KPC(key));
LOG_DEBUG("succ to get cache obj", KPC(key));
}
// release lock whatever
(void)cache_node->unlock();

View File

@ -222,7 +222,6 @@ private:
public:
ObFastParserResult()
: inner_alloc_("FastParserRes"),
raw_params_(&inner_alloc_),
parameterized_params_(&inner_alloc_),
cache_params_(NULL),
values_token_pos_(0),
@ -231,7 +230,7 @@ public:
reset_question_mark_ctx();
}
ObPlanCacheKey pc_key_; //plan cache key, parameterized by fast parser
common::ObFixedArray<ObPCParam *, common::ObIAllocator> raw_params_;
common::ObSEArray<ObPCParam *, 4> raw_params_;
common::ObFixedArray<const common::ObObjParam *, common::ObIAllocator> parameterized_params_;
ParamStore *cache_params_;
ObQuestionMarkCtx question_mark_ctx_;
@ -261,7 +260,6 @@ public:
{
int ret = OB_SUCCESS;
pc_key_ = other.pc_key_;
raw_params_.set_allocator(&inner_alloc_);
parameterized_params_.set_allocator(&inner_alloc_);
cache_params_ = other.cache_params_;
question_mark_ctx_ = other.question_mark_ctx_;

View File

@ -251,7 +251,6 @@ int ObPhyLocationGetter::reselect_duplicate_table_best_replica(const ObIArray<Ob
return ret;
}
//need_check_on_same_server: out, 是否需要检查分区在同一server, 如果这里检查过了就置为false
int ObPhyLocationGetter::get_phy_locations(const ObIArray<ObTableLocation> &table_locations,
const ObPlanCacheCtx &pc_ctx,
ObIArray<ObCandiTableLoc> &candi_table_locs,
@ -336,6 +335,106 @@ int ObPhyLocationGetter::get_phy_locations(const ObIArray<ObTableLocation> &tabl
return ret;
}
int ObPhyLocationGetter::get_phy_locations(const ObIArray<ObTableLocation> &table_locations,
const ObPlanCacheCtx &pc_ctx,
ObIArray<ObCandiTableLoc> &candi_table_locs)
{
int ret = OB_SUCCESS;
bool has_duplicate_tbl_not_in_dml = false;
ObExecContext &exec_ctx = pc_ctx.exec_ctx_;
const ObDataTypeCastParams dtc_params = ObBasicSessionInfo::create_dtc_params(pc_ctx.sql_ctx_.session_info_);
ObPhysicalPlanCtx *plan_ctx = exec_ctx.get_physical_plan_ctx();
const ParamStore &params = plan_ctx->get_param_store();
int64_t N = table_locations.count();
bool is_retrying = false;
if (OB_ISNULL(plan_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid executor ctx!", K(ret), K(plan_ctx));
} else {
int64_t calculate_candi_table_num = N;
for (int64_t i = 0; OB_SUCC(ret) && i < N ; i++) {
const ObTableLocation &table_location = table_locations.at(i);
if (table_location.get_loc_meta().select_leader_) {
if (OB_FAIL(table_location.calculate_final_tablet_locations(exec_ctx,
params,
dtc_params))) {
LOG_WARN("failed to calculate final tablet locations", K(table_location), K(ret));
} else {
calculate_candi_table_num -= 1;
}
}
} // for end
if (OB_SUCC(ret) && calculate_candi_table_num > 0) {
ObSEArray<const ObTableLocation *, 2> table_location_ptrs;
ObSEArray<ObCandiTableLoc *, 2> phy_location_info_ptrs;
if (OB_FAIL(candi_table_locs.prepare_allocate(calculate_candi_table_num))) {
LOG_WARN("phy_locations_info prepare allocate error", K(ret), K(calculate_candi_table_num));
} else {
for (int64_t i = 0, j = 0; OB_SUCC(ret) && i < N && j < calculate_candi_table_num; i++) {
const ObTableLocation &table_location = table_locations.at(i);
if (!table_location.get_loc_meta().select_leader_) {
ObCandiTableLoc &candi_table_loc = candi_table_locs.at(j);
NG_TRACE(calc_partition_location_begin);
// 这里认为materialized view的复制表是每个server都有副本的,
// 因此这里不判断是否能生成materialized view了,一定都能生成
if (OB_FAIL(table_location.calculate_candi_tablet_locations(exec_ctx,
params,
candi_table_loc.get_phy_part_loc_info_list_for_update(),
dtc_params))) {
LOG_WARN("failed to calculate partition location", K(ret));
} else {
NG_TRACE(calc_partition_location_end);
if (table_location.is_duplicate_table_not_in_dml()) {
has_duplicate_tbl_not_in_dml = true;
}
candi_table_loc.set_duplicate_type(table_location.get_duplicate_type());
candi_table_loc.set_table_location_key(
table_location.get_table_id(), table_location.get_ref_table_id());
LOG_DEBUG("plan cache util", K(candi_table_loc));
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_location_ptrs.push_back(&table_location))) {
LOG_WARN("failed to push back table location ptrs", K(ret), K(i),
K(N), K(table_locations.at(i)));
} else if (OB_FAIL(phy_location_info_ptrs.push_back(&candi_table_loc))) {
LOG_WARN("failed to push back phy location info ptrs", K(ret), K(i),
K(N), K(candi_table_locs.at(j)));
} else if (OB_FAIL(pc_ctx.is_retry_for_dup_tbl(is_retrying))) {
LOG_WARN("failed to test if retrying", K(ret));
} else if (is_retrying) {
LOG_INFO("Physical Location from Location Cache", K(candi_table_loc));
}
j += 1;
}
}
} // for end
}
//Only check the on_same_server when has table location in the phy_plan.
if (OB_SUCC(ret)) {
bool on_same_server = true;
if (OB_FAIL(ObLogPlan::select_replicas(exec_ctx, table_location_ptrs,
exec_ctx.get_addr(),
phy_location_info_ptrs))) {
LOG_WARN("failed to select replicas", K(ret), K(table_locations),
K(exec_ctx.get_addr()), K(phy_location_info_ptrs));
} else if (!has_duplicate_tbl_not_in_dml || is_retrying) {
// do nothing
} else if (OB_FAIL(reselect_duplicate_table_best_replica(candi_table_locs,
on_same_server))) {
LOG_WARN("failed to reselect replicas", K(ret));
}
LOG_TRACE("after select_replicas", K(on_same_server), K(has_duplicate_tbl_not_in_dml),
K(candi_table_locs), K(table_locations), K(ret));
}
}
}
return ret;
}
int ObPhyLocationGetter::build_table_locs(ObDASCtx &das_ctx,
const ObIArray<ObTableLocation> &table_locations,
const ObIArray<ObCandiTableLoc> &candi_table_locs)
@ -354,6 +453,28 @@ int ObPhyLocationGetter::build_table_locs(ObDASCtx &das_ctx,
return ret;
}
int ObPhyLocationGetter::build_candi_table_locs(ObDASCtx &das_ctx,
const ObIArray<ObTableLocation> &table_locations,
const ObIArray<ObCandiTableLoc> &candi_table_locs)
{
int ret = OB_SUCCESS;
CK(table_locations.count() >= candi_table_locs.count());
for (int64_t i = 0, j = 0; OB_SUCC(ret) && i < table_locations.count() && j < candi_table_locs.count(); i++) {
if (!table_locations.at(i).get_loc_meta().select_leader_) {
if (OB_FAIL(das_ctx.add_candi_table_loc(table_locations.at(i).get_loc_meta(), candi_table_locs.at(j)))) {
LOG_WARN("add candi table location failed", K(ret), K(table_locations.at(i).get_loc_meta()));
} else {
j += 1;
}
}
}
if (OB_FAIL(ret)) {
das_ctx.clear_all_location_info();
}
return ret;
}
//this function will rewrite the related tablet map info in DASCtx
int ObPhyLocationGetter::build_related_tablet_info(const ObTableLocation &table_location,
ObExecContext &exec_ctx,

View File

@ -953,9 +953,19 @@ struct ObPhyLocationGetter
{
public:
// used for getting plan
// In this interface, we first process the table locations that were marked select_leader, the tablet
// locations of them will be added to das_ctx directly, without the need to construct candi_table_locs.
// For the remaining table locations that are not marked select_leader, continue to use the previous
// logic where a candi_table_loc is generated for each table location. These candi_table_locs will be
// added to das_ctx by @build_candi_table_locs().
static int get_phy_locations(const ObIArray<ObTableLocation> &table_locations,
const ObPlanCacheCtx &pc_ctx,
ObIArray<ObCandiTableLoc> &phy_location_infos,
ObIArray<ObCandiTableLoc> &phy_location_infos);
// used for matching plan
static int get_phy_locations(const ObIArray<ObTableLocation> &table_locations,
const ObPlanCacheCtx &pc_ctx,
ObIArray<ObCandiTableLoc> &candi_table_locs,
bool &need_check_on_same_server);
// used for adding plan
@ -966,6 +976,9 @@ public:
static int build_table_locs(ObDASCtx &das_ctx,
const common::ObIArray<ObTableLocation> &table_locations,
const common::ObIArray<ObCandiTableLoc> &candi_table_locs);
static int build_candi_table_locs(ObDASCtx &das_ctx,
const common::ObIArray<ObTableLocation> &table_locations,
const common::ObIArray<ObCandiTableLoc> &candi_table_locs);
static int build_related_tablet_info(const ObTableLocation &table_location,
ObExecContext &exec_ctx,
DASRelatedTabletMap *&related_map);

View File

@ -112,11 +112,6 @@ int ObPlanMatchHelper::match_plan(const ObPlanCacheCtx &pc_ctx,
LOG_WARN("failed to set refactored", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(das_ctx.add_candi_table_loc(out_tbl_locations.at(i).get_loc_meta(), src_location))) {
LOG_WARN("add candi table loc failed", K(ret), K(out_tbl_locations.at(i).get_loc_meta()), K(src_location));
}
}
}
}
}

View File

@ -2048,22 +2048,21 @@ void ObSqlPlanSet::reset()
//need_check_on_same_server: out, 是否需要检查分区在同一server, 如果里面检查过且不在同一server则置为false
int ObSqlPlanSet::get_phy_locations(const ObIArray<ObTableLocation> &table_locations,
ObPlanCacheCtx &pc_ctx,
ObIArray<ObCandiTableLoc> &candi_table_locs,
bool &need_check_on_same_server)
ObIArray<ObCandiTableLoc> &candi_table_locs)
{
int ret = OB_SUCCESS;
need_check_on_same_server = true;
DAS_CTX(pc_ctx.exec_ctx_).clear_all_location_info();
if (OB_FAIL(ObPhyLocationGetter::get_phy_locations(table_locations,
pc_ctx,
candi_table_locs,
need_check_on_same_server))) {
candi_table_locs))) {
LOG_WARN("failed to get phy locations", K(ret), K(table_locations));
} else if (OB_FAIL(ObPhyLocationGetter::build_table_locs(pc_ctx.exec_ctx_.get_das_ctx(),
table_locations,
candi_table_locs))) {
LOG_WARN("fail to init table locs", K(ret));
}
} else if (candi_table_locs.empty()) {
// do nothing.
} else if (OB_FAIL(ObPhyLocationGetter::build_candi_table_locs(pc_ctx.exec_ctx_.get_das_ctx(),
table_locations,
candi_table_locs))) {
LOG_WARN("fail to init table locs", K(ret));
}
return ret;
}
@ -2079,57 +2078,47 @@ int ObSqlPlanSet::get_phy_locations(const ObIArray<ObTableLocation> &table_locat
* FALSE: we know partitions on different servers via ObPhyLocationGetter::get_phy_locations
* (when there are duplicate tables not in DML), no need to check again
*/
int ObSqlPlanSet::calc_phy_plan_type_v2(const ObIArray<ObCandiTableLoc> &candi_table_locs,
ObPhyPlanType &plan_type,
bool need_check_on_same_server)
int ObSqlPlanSet::calc_phy_plan_type_v2(const common::ObIArray<ObCandiTableLoc> &candi_table_locs,
const ObPlanCacheCtx &pc_ctx,
ObPhyPlanType &plan_type)
{
int ret = OB_SUCCESS;
int64_t N = candi_table_locs.count();
ObDASCtx &das_ctx = pc_ctx.exec_ctx_.get_das_ctx();
const DASTableLocList &table_locs = das_ctx.get_table_loc_list();
int64_t N = table_locs.size();
if (0 == N) {
plan_type = OB_PHY_PLAN_LOCAL;
SQL_PC_LOG(DEBUG, "no table used, thus local plan");
} else {
bool is_all_empty = true;
bool is_all_single_partition = true;
for (int i = 0; is_all_single_partition && i < N; ++i) {
if (candi_table_locs.at(i).get_partition_cnt() != 0) {
FOREACH_X(table_loc, table_locs, is_all_single_partition)
{
const DASTabletLocList &tablet_locs = (*table_loc)->get_tablet_locs();
if (tablet_locs.size() != 0) {
is_all_empty = false;
}
if (candi_table_locs.at(i).get_partition_cnt() > 1) {
if (tablet_locs.size() > 1) {
is_all_single_partition = false;
}
}
if (is_all_empty) {
plan_type = OB_PHY_PLAN_LOCAL;
} else if (is_all_single_partition) {
bool is_same = true;
ObAddr my_address = GCTX.self_addr();
ObAddr first_addr;
if (!need_check_on_same_server) {
is_same = false;
}
if (is_same && OB_FAIL(is_partition_in_same_server(candi_table_locs,
is_same,
first_addr))) {
SQL_PC_LOG(WARN, "fail to calculate whether all partitions in same server",
K(ret),
K(candi_table_locs));
} else {
if (is_same) {
if (my_address == first_addr) {
plan_type = OB_PHY_PLAN_LOCAL;
} else {
plan_type = OB_PHY_PLAN_REMOTE;
}
if (das_ctx.same_server_) {
if (GCTX.self_addr() == das_ctx.same_tablet_addr()) {
plan_type = OB_PHY_PLAN_LOCAL;
} else {
plan_type = OB_PHY_PLAN_DISTRIBUTED;
plan_type = OB_PHY_PLAN_REMOTE;
}
} else {
plan_type = OB_PHY_PLAN_DISTRIBUTED;
}
} else {
plan_type = OB_PHY_PLAN_DISTRIBUTED;
}
}
return ret;
}
@ -2347,17 +2336,15 @@ int ObSqlPlanSet::get_plan_type(const ObIArray<ObTableLocation> &table_locations
ObPhyPlanType &plan_type)
{
int ret = OB_SUCCESS;
bool need_check_on_same_server = true;
candi_table_locs.reuse();
if (OB_FAIL(get_phy_locations(table_locations,
pc_ctx,
candi_table_locs,
need_check_on_same_server))) {
candi_table_locs))) {
LOG_WARN("failed to get physical locations", K(ret));
} else if (OB_FAIL(calc_phy_plan_type_v2(candi_table_locs,
plan_type,
need_check_on_same_server))) {
pc_ctx,
plan_type))) {
LOG_WARN("failed to calcute physical plan type", K(ret));
} else {
// Lookup算子支持压到远程去执行:

View File

@ -349,9 +349,9 @@ public:
// calculate phy_plan type:
// @param [in] phy_locations
// @param [out] plan_type
static int calc_phy_plan_type_v2(const common::ObIArray<ObCandiTableLoc> &phy_locations,
ObPhyPlanType &plan_type,
bool need_check_on_same_server);
static int calc_phy_plan_type_v2(const common::ObIArray<ObCandiTableLoc> &candi_table_locs,
const ObPlanCacheCtx &pc_ctx,
ObPhyPlanType &plan_type);
static int calc_phy_plan_type(const common::ObIArray<ObCandiTableLoc> &phy_locations,
ObPhyPlanType &plan_type);
inline bool has_duplicate_table() const { return has_duplicate_table_; }
@ -411,8 +411,7 @@ private:
ObPhysicalPlan *&plan);
int get_phy_locations(const ObIArray<ObTableLocation> &table_locations,
ObPlanCacheCtx &pc_ctx,
ObIArray<ObCandiTableLoc> &candi_table_locs,
bool &need_check_on_same_server);
ObIArray<ObCandiTableLoc> &candi_table_locs);
int get_phy_locations(const ObTablePartitionInfoArray &partition_infos,
ObIArray<ObCandiTableLoc> &candi_table_locs);

View File

@ -1502,8 +1502,6 @@ int ObSqlParameterization::fast_parser(ObIAllocator &allocator,
ObPCParam *pc_param = NULL;
char *ptr = (char *)allocator.alloc(param_num * sizeof(ObPCParam));
fp_result.raw_params_.reset();
fp_result.raw_params_.set_allocator(&allocator);
fp_result.raw_params_.set_capacity(param_num);
if (OB_ISNULL(ptr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
SQL_PC_LOG(WARN, "fail to alloc memory for pc param", K(ret), K(ptr));
@ -1533,8 +1531,6 @@ int ObSqlParameterization::fast_parser(ObIAllocator &allocator,
ObPCParam *pc_param = NULL;
ParamList *p_list = parse_result.param_nodes_;
char *ptr = (char *)allocator.alloc(param_num * sizeof(ObPCParam));
fp_result.raw_params_.set_allocator(&allocator);
fp_result.raw_params_.set_capacity(param_num);
if (OB_ISNULL(ptr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
SQL_PC_LOG(WARN, "fail to alloc memory for pc param", K(ret), K(ptr));

View File

@ -12,8 +12,6 @@
#define USING_LOG_PREFIX SQL_PC
#include "sql/plan_cache/ob_values_table_compression.h"
#include "sql/plan_cache/ob_plan_cache_struct.h"
#include "sql/parser/ob_fast_parser.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/resolver/ob_resolver_utils.h"
#include "sql/engine/expr/ob_expr_version.h"
@ -382,8 +380,6 @@ int ObValuesTableCompression::try_batch_exec_params(ObIAllocator &allocator,
} else if (can_fold_params) {
fp_result.pc_key_.name_.assign_ptr(new_no_param_sql.ptr(), new_no_param_sql.length());
fp_result.raw_params_.reset();
fp_result.raw_params_.set_allocator(&allocator);
fp_result.raw_params_.set_capacity(temp_store.count());
for (int64_t i = 0; i < temp_store.count(); i++) {
// checked null before
temp_store.at(i)->node_->pos_ = no_param_pos.at(i);
@ -866,4 +862,4 @@ int ObValuesTableCompression::parser_values_row_str(ObIAllocator &allocator,
}
}
}
}

View File

@ -95,4 +95,4 @@ private:
}
}
#endif /* _OB_VALUES_TABLE_COMPRESSION_H */
#endif /* _OB_VALUES_TABLE_COMPRESSION_H */

View File

@ -8814,7 +8814,7 @@ int ObResolverUtils::resolver_param(ObPlanCacheCtx &pc_ctx,
}
}
is_param = true;
LOG_TRACE("is_param", K(param_idx), K(obj_param), K(raw_param->type_), K(raw_param->value_),
LOG_DEBUG("is_param", K(param_idx), K(obj_param), K(raw_param->type_), K(raw_param->value_),
"str_value", ObString(raw_param->str_len_, raw_param->str_value_));
}
return ret;

View File

@ -12,8 +12,7 @@
#define USING_LOG_PREFIX SQL_REWRITE
#include "sql/rewrite/ob_transform_predicate_move_around.h"
#include "sql/rewrite/ob_stmt_comparer.h"
#include "sql/rewrite/ob_transform_utils.h"
//#include "sql/rewrite/ob_stmt_comparer.h"
#include "sql/optimizer/ob_optimizer_util.h"
#include "sql/resolver/expr/ob_raw_expr_util.h"
#include "sql/rewrite/ob_predicate_deduce.h"
@ -4143,4 +4142,4 @@ bool ObTempTableColumnCheckContext::compare_column(const ObColumnRefRawExpr &lef
bret = true;
}
return bret;
}
}

View File

@ -14,6 +14,7 @@
#define OB_TRANSFORM_PREDICATE_MOVE_AROUND_H
#include "sql/rewrite/ob_transform_rule.h"
#include "sql/rewrite/ob_transform_utils.h"
#include "sql/resolver/dml/ob_select_stmt.h"
#include "sql/rewrite/ob_stmt_comparer.h"

View File

@ -180,7 +180,7 @@ ObBasicSessionInfo::~ObBasicSessionInfo()
bool ObBasicSessionInfo::is_server_status_in_transaction() const
{
bool in_txn = OB_NOT_NULL(tx_desc_) && tx_desc_->in_tx_for_free_route();
LOG_TRACE("decide flag: server in transaction", K(in_txn));
LOG_DEBUG("decide flag: server in transaction", K(in_txn));
return in_txn;
}
@ -5774,7 +5774,6 @@ int ObBasicSessionInfo::is_timeout(bool &is_timeout)
int ObBasicSessionInfo::is_trx_commit_timeout(transaction::ObITxCallback *&callback, int &retcode)
{
int ret = OB_SUCCESS;
int64_t cur_time = ::oceanbase::common::ObTimeUtility::current_time();
if (is_in_transaction() && tx_desc_->is_committing()) {
if (tx_desc_->is_tx_timeout()) {
callback = tx_desc_->cancel_commit_cb();
@ -6236,7 +6235,7 @@ int ObBasicSessionInfo::update_timezone_info()
{
int ret = OB_SUCCESS;
const int64_t UPDATE_PERIOD = 1000 * 1000 * 5; //5s
int64_t cur_time = ObTimeUtility::current_time();
int64_t cur_time = ObClockGenerator::getClock();
if (cur_time - last_update_tz_time_ > UPDATE_PERIOD) {
ObTZMapWrap tz_map_wrap;
ObTimeZoneInfoManager *tz_info_mgr = NULL;

View File

@ -57,7 +57,7 @@ int ObSessInfoVerify::sync_sess_info_veri(sql::ObSQLSessionInfo &sess,
const int64_t len = sess_info_veri.length();
const char *end = buf + len;
int64_t pos = 0;
LOG_TRACE("start sync proxy sess info verification", K(sess.get_is_in_retry()),
LOG_DEBUG("start sync proxy sess info verification", K(sess.get_is_in_retry()),
K(sess.get_sessid()), KP(data), K(len), KPHEX(data, len));
// decode sess_info
@ -66,7 +66,7 @@ int ObSessInfoVerify::sync_sess_info_veri(sql::ObSQLSessionInfo &sess,
int16_t extra_id = 0;
int32_t info_len = 0;
char *sess_buf = NULL;
LOG_TRACE("sync field sess_inf", K(sess.get_sessid()),
LOG_DEBUG("sync field sess_inf", K(sess.get_sessid()),
KP(data), K(pos), K(len), KPHEX(data+pos, len-pos));
if (OB_FAIL(ObProtoTransUtil::resolve_type_and_len(buf, len, pos, extra_id, info_len))) {
LOG_WARN("failed to resolve type and len", K(ret), K(len), K(pos));
@ -83,10 +83,10 @@ int ObSessInfoVerify::sync_sess_info_veri(sql::ObSQLSessionInfo &sess,
sess_info_verification))) {
LOG_WARN("failed to resolve value", K(extra_id), KP(buf), K(len), K(pos), K(info_len));
} else {
LOG_TRACE("success to resolve value", K(extra_id), K(len), K(pos), K(info_len));
LOG_DEBUG("success to resolve value", K(extra_id), K(len), K(pos), K(info_len));
}
}
LOG_TRACE("success to get sess info verification requied by proxy",
LOG_DEBUG("success to get sess info verification requied by proxy",
K(sess_info_verification), K(sess.get_sessid()),
K(sess.get_proxy_sessid()));
}
@ -139,7 +139,7 @@ int ObSessInfoVerify::verify_session_info(sql::ObSQLSessionInfo &sess,
value_buffer.assign_buffer(ptr, result.verify_info_buf_.length());
value_buffer.write(result.verify_info_buf_.ptr(), result.verify_info_buf_.length());
}
LOG_TRACE("need verify", K(&result), K(result.need_verify_), K(result.verify_info_buf_));
LOG_DEBUG("need verify", K(&result), K(result.need_verify_), K(result.verify_info_buf_));
if (OB_FAIL(ret)) {
} else if (result.need_verify_) {
// verification error injection.
@ -153,12 +153,12 @@ int ObSessInfoVerify::verify_session_info(sql::ObSQLSessionInfo &sess,
LOG_ERROR("session info self-verification failed", K(ret), K(sess.get_sessid()),
K(sess.get_proxy_sessid()), K(sess_info_verification));
} else {
LOG_TRACE("session info self-verification success", K(ret));
LOG_DEBUG("session info self-verification success", K(ret));
}
} else {
LOG_TRACE("session info no need self-verification", K(ret));
LOG_DEBUG("session info no need self-verification", K(ret));
}
LOG_TRACE("verify end", K(sess.get_sessid()),
LOG_DEBUG("verify end", K(sess.get_sessid()),
K(sess.get_proxy_sessid()), K(sess_info_verification));
}
} else {

View File

@ -315,6 +315,7 @@ ob_set_subtarget(ob_storage tx
tx/ob_location_adapter.cpp
tx/ob_one_phase_committer.cpp
tx/ob_standby_timestamp_service.cpp
tx/ob_tablet_to_ls_cache.cpp
tx/ob_timestamp_access.cpp
tx/ob_timestamp_service.cpp
tx/ob_time_wheel.cpp

View File

@ -246,7 +246,7 @@ int ObMultipleScanMerge::locate_blockscan_border()
if (OB_ISNULL(iter)) {
ret = common::OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null iter", K(ret), K(iter));
} else if (OB_FAIL(iter->get_next_row_ext(item.row_, item.iter_flag_))) {
} else if (OB_FAIL(iter->get_next_row(item.row_))) {
if (OB_ITER_END != ret) {
LOG_WARN("Failed to get next row from iterator", K(ret), "index", iter_idx, "iterator", *iter);
} else {
@ -277,7 +277,7 @@ int ObMultipleScanMerge::locate_blockscan_border()
ret = OB_ERR_UNEXPECTED;
LOG_WARN("item or row is null", K(ret), KP(top_item));
} else {
LOG_DEBUG("get top item", K(top_item->iter_idx_), KPC(top_item->row_), K(top_item->iter_flag_));
LOG_DEBUG("get top item", K(top_item->iter_idx_), KPC(top_item->row_));
const int64_t rowkey_cnt = access_param_->iter_param_.get_schema_rowkey_count();
if (OB_FAIL(border_key.assign(top_item->row_->storage_datums_, rowkey_cnt))) {
LOG_WARN("Fail to assign border key", K(ret), K(rowkey_cnt));
@ -347,7 +347,7 @@ int ObMultipleScanMerge::supply_consume()
if (NULL == iter) {
ret = common::OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "Unexpected error", K(ret), K(iter));
} else if (OB_FAIL(iter->get_next_row_ext(item.row_, item.iter_flag_))) {
} else if (OB_FAIL(iter->get_next_row(item.row_))) {
if (OB_ITER_END != ret) {
if (OB_PUSHDOWN_STATUS_CHANGED != ret) {
STORAGE_LOG(WARN, "failed to get next row from iterator", K(ret), "index", iter_idx, "iterator", *iter);
@ -416,7 +416,7 @@ int ObMultipleScanMerge::inner_get_next_row(ObDatumRow &row)
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "Unexpected null iter", K(ret), K_(consumer_cnt));
} else if (iter->can_blockscan()) {
if (OB_FAIL(iter->get_next_row_ext(item.row_, item.iter_flag_))) {
if (OB_FAIL(iter->get_next_row(item.row_))) {
if (OB_ITER_END == ret) {
consumer_cnt_ = 0;
ret = OB_SUCCESS;
@ -506,7 +506,7 @@ int ObMultipleScanMerge::inner_merge_row(ObDatumRow &row)
STORAGE_LOG(WARN, "item or row is null", K(ret), KP(top_item));
} else {
STORAGE_LOG(DEBUG, "top_item", K(top_item->iter_idx_), K(*top_item->row_), K(row),
K(has_same_rowkey), K(first_row), K(top_item->iter_flag_));
K(has_same_rowkey), K(first_row));
}
if (OB_SUCC(ret)) {

View File

@ -24,19 +24,17 @@ struct ObScanMergeLoserTreeItem
{
const blocksstable::ObDatumRow *row_;
int64_t iter_idx_;
uint8_t iter_flag_;
bool equal_with_next_;
ObScanMergeLoserTreeItem() : row_(NULL), iter_idx_(0), iter_flag_(0), equal_with_next_(false)
ObScanMergeLoserTreeItem() : row_(NULL), iter_idx_(0), equal_with_next_(false)
{}
~ObScanMergeLoserTreeItem() = default;
void reset()
{
row_ = NULL;
iter_idx_ = 0;
iter_flag_ = 0;
equal_with_next_ = false;
}
TO_STRING_KV(K_(iter_idx), K_(iter_flag), KPC(row_));
TO_STRING_KV(K_(iter_idx), KPC(row_));
};
class ObScanMergeLoserTreeCmp

View File

@ -78,12 +78,6 @@ public:
ObTableAccessContext &context,
ObITable *table,
const void *query_range);
virtual int get_next_row_ext(const blocksstable::ObDatumRow *&row, uint8_t& flag) {
int ret = get_next_row(row);
flag = get_iter_flag();
return ret;
}
virtual uint8_t get_iter_flag() { return 0; }
virtual bool is_sstable_iter() const { return is_sstable_iter_; }
virtual int refresh_blockscan_checker(const blocksstable::ObDatumRowkey &rowkey)
{

View File

@ -1794,7 +1794,7 @@ int ObIndexBlockRowScanner::locate_range(
}
} else {
range_ = &range;
LOG_TRACE("Locate range in index block by range", K(ret), K(range), KPC(iter_),
LOG_DEBUG("Locate range in index block by range", K(ret), K(range), KPC(iter_),
K(is_left_border), K(is_right_border), KP(this));
}
return ret;

View File

@ -1011,7 +1011,7 @@ void ObBlockManager::mark_and_sweep()
}
if (OB_FAIL(ret)) {
// do nothing
ATOMIC_FAA(&alloc_num_, alloc_num); // add alloc_num back to trigger next round mark
} else {
tmp_status.pending_free_count_ += mark_info.count();
tmp_status.mark_cost_time_ = ObTimeUtility::fast_current_time() - tmp_status.start_time_;
@ -1608,7 +1608,8 @@ const int64_t ObBlockManager::InspectBadBlockTask::MAX_SEARCH_COUNT_PER_ROUND =
ObBlockManager::InspectBadBlockTask::InspectBadBlockTask(ObBlockManager &blk_mgr)
: blk_mgr_(blk_mgr),
last_macro_idx_(0)
last_macro_idx_(-1),
last_check_time_(0)
{
}
@ -1619,12 +1620,21 @@ ObBlockManager::InspectBadBlockTask::~InspectBadBlockTask()
void ObBlockManager::InspectBadBlockTask::reset()
{
last_macro_idx_ = 0;
last_macro_idx_ = -1;
last_check_time_ = 0;
}
void ObBlockManager::InspectBadBlockTask::runTimerTask()
{
inspect_bad_block();
const int64_t next_check_time = last_check_time_ + ACCESS_TIME_INTERVAL;
if (next_check_time <= ObTimeUtility::fast_current_time()) { /* exceed 2 days */
inspect_bad_block();
if (last_macro_idx_ == -1) { // finished
last_check_time_ = ObTimeUtility::fast_current_time();
}
} else {
LOG_INFO("skip inspect bad block", K_(last_check_time), K_(last_macro_idx));
}
}
int ObBlockManager::InspectBadBlockTask::check_block(ObMacroBlockHandle &macro_block_handle)
@ -1813,7 +1823,12 @@ void ObBlockManager::InspectBadBlockTask::inspect_bad_block()
&& check_count < max_check_count_per_round
&& (ObTimeUtility::current_time() - begin_time) < inspect_timeout_us;
++i) {
last_macro_idx_ = (last_macro_idx_ + 1) % total_used_macro_block_count;
++last_macro_idx_;
if (last_macro_idx_ >= total_used_macro_block_count) {
last_macro_idx_ = -1; // finished this round
break;
}
const MacroBlockId &macro_id = macro_ids.at(last_macro_idx_);
ObMacroBlockInfo block_info;
ObMacroBlockHandle macro_block_handle;

View File

@ -464,6 +464,7 @@ private:
ObBlockManager &blk_mgr_;
int64_t last_macro_idx_;
int64_t last_check_time_;
private:
DISALLOW_COPY_AND_ASSIGN(InspectBadBlockTask);
};

View File

@ -80,6 +80,7 @@ ObSharedMacroBlockMgr::ObSharedMacroBlockMgr()
block_used_size_(),
defragmentation_task_(*this),
tg_id_(-1),
need_defragment_(false),
is_inited_(false)
{
}
@ -101,6 +102,7 @@ void ObSharedMacroBlockMgr::destroy()
}
common_header_buf_ = nullptr;
block_used_size_.destroy();
need_defragment_ = false;
is_inited_ = false;
}
@ -373,12 +375,11 @@ int ObSharedMacroBlockMgr::add_block(const MacroBlockId &block_id, const int64_t
lib::ObMutexGuard guard(blocks_mutex_);
if (OB_FAIL(block_used_size_.get(block_id, curr_size)) && OB_ENTRY_NOT_EXIST != ret) {
LOG_WARN("fail to get block id from map", K(ret), K(block_id));
} else if ((curr_size += block_size) == 0) {
if (OB_FAIL(block_used_size_.erase(block_id))) {
LOG_WARN("fail to erase id from map", K(ret), K(block_id));
}
} else if (FALSE_IT(curr_size += block_size)) {
} else if (OB_FAIL(block_used_size_.insert_or_update(block_id, curr_size))) {
LOG_WARN("fail to add block to map", K(ret), K(block_id), K(curr_size));
} else if (is_recyclable(block_id, curr_size)) {
ATOMIC_SET(&need_defragment_, true);
}
}
return ret;
@ -402,6 +403,8 @@ int ObSharedMacroBlockMgr::free_block(const MacroBlockId &block_id, const int64_
}
} else if (OB_FAIL(block_used_size_.insert_or_update(block_id, curr_size))) {
LOG_WARN("fail to set block used size", K(ret), K(block_id), K(block_size), K(curr_size));
} else if (is_recyclable(block_id, curr_size)) {
ATOMIC_SET(&need_defragment_, true);
}
}
return ret;
@ -430,6 +433,7 @@ int ObSharedMacroBlockMgr::get_recyclable_blocks(ObIAllocator &allocator, ObIArr
if (OB_FAIL(ret)) {
// do nothing
} else {
ATOMIC_SET(&need_defragment_, false);
int tmp_ret = OB_SUCCESS;
for (int64_t i = 0; i < recycled_block_ids.count(); ++i) { // ignore tmp_ret
const MacroBlockId &block_id = recycled_block_ids.at(i);
@ -457,6 +461,8 @@ int ObSharedMacroBlockMgr::defragment()
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LOG_WARN("ObSharedMacroBlockMgr hasn't been initiated", K(ret));
} else if (!(ATOMIC_LOAD(&need_defragment_))) { // skip
LOG_INFO("skip defragment task", K(ret), K_(need_defragment));
} else if (OB_FAIL(macro_ids.init(MAX_RECYCLABLE_BLOCK_CNT))) {
LOG_WARN("fail to init macro ids", K(ret));
} else if (OB_FAIL(get_recyclable_blocks(task_allocator, macro_ids))) {
@ -499,6 +505,8 @@ int ObSharedMacroBlockMgr::defragment()
if (OB_ITER_END == ret || OB_SUCC(ret)) {
ret = OB_SUCCESS;
FLOG_INFO("successfully defragment data blocks", K(ret), K(rewrite_cnt), K(block_used_size_.count()));
} else {
ATOMIC_SET(&need_defragment_, true); // set need_defragment_ true to trigger next round
}
if (nullptr != sstable_index_builder) {

View File

@ -68,13 +68,13 @@ public:
int write_block(const char* buf, const int64_t size, ObBlockInfo &block_info, ObMacroBlocksWriteCtx &write_ctx);
int add_block(const MacroBlockId &block_id, const int64_t block_size);
int free_block(const MacroBlockId &block_id, const int64_t block_size);
bool is_recyclable(const MacroBlockId &macro_id, const int64_t &used_size) const ;
TO_STRING_KV(K_(macro_handle), K_(offset), K_(header_size));
static int mtl_init(ObSharedMacroBlockMgr* &shared_block_mgr);
private:
bool is_recyclable(const MacroBlockId &macro_id, const int64_t &used_size) const ;
class ObBlockDefragmentationTask : public common::ObTimerTask
{
public:
@ -199,6 +199,7 @@ private:
ObLinearHashMap<MacroBlockId, int32_t> block_used_size_;
ObBlockDefragmentationTask defragmentation_task_;
int tg_id_;
bool need_defragment_;
bool is_inited_;
};

View File

@ -812,8 +812,7 @@ int ObDDLMacroBlockIterator::open(ObSSTable *sstable, const ObDatumRange &query_
ObDatumRowkeyWrapper(&query_range.get_start_key(), &read_info.get_datum_utils()),
query_range.is_left_open(),
ObDatumRowkeyWrapper(&query_range.get_end_key(), &read_info.get_datum_utils()),
query_range.is_right_open(),
INT64_MAX/*version*/))) {
query_range.is_right_open()))) {
LOG_WARN("ddl memtable locate range failed", K(ret));
}
} else if (sstable->is_ddl_merge_sstable()) { // co ddl partial data, need scan macro block

View File

@ -231,7 +231,6 @@ void ObBlockMetaTree::destroy()
void ObBlockMetaTree::destroy_tree_value()
{
int ret = OB_SUCCESS;
const int64_t version = INT64_MAX;
blocksstable::DDLBtreeIterator tmp_iter;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
@ -239,8 +238,7 @@ void ObBlockMetaTree::destroy_tree_value()
ObDatumRowkeyWrapper(&ObDatumRowkey::MIN_ROWKEY, datum_utils_),
false,
ObDatumRowkeyWrapper(&ObDatumRowkey::MAX_ROWKEY, datum_utils_),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else {
while (OB_SUCC(ret)) {
@ -320,7 +318,6 @@ int ObBlockMetaTree::get_sorted_meta_array(ObIArray<ObDDLBlockMeta> &meta_array)
{
int ret = OB_SUCCESS;
meta_array.reset();
const int64_t version = INT64_MAX;
blocksstable::DDLBtreeIterator tmp_iter;
if (OB_UNLIKELY(!is_inited_)) {
@ -330,8 +327,7 @@ int ObBlockMetaTree::get_sorted_meta_array(ObIArray<ObDDLBlockMeta> &meta_array)
ObDatumRowkeyWrapper(&ObDatumRowkey::MIN_ROWKEY, datum_utils_),
false,
ObDatumRowkeyWrapper(&ObDatumRowkey::MAX_ROWKEY, datum_utils_),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else if (OB_FAIL(meta_array.reserve(macro_blocks_.count()))) {
LOG_WARN("reserve meta array failed", K(ret), K(macro_blocks_.count()));
@ -419,7 +415,6 @@ int ObBlockMetaTree::lower_bound(const blocksstable::ObDatumRowkey *target_rowke
int ret = OB_SUCCESS;
rowkey = nullptr;
tree_value = nullptr;
const int64_t version = INT64_MAX;
bool find = false;
blocksstable::DDLBtreeIterator tmp_iter;
if (OB_ISNULL(target_rowkey)) {
@ -429,8 +424,7 @@ int ObBlockMetaTree::lower_bound(const blocksstable::ObDatumRowkey *target_rowke
ObDatumRowkeyWrapper(target_rowkey, &datum_utils),
false,
ObDatumRowkeyWrapper(&ObDatumRowkey::MAX_ROWKEY, &datum_utils),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else {
while (OB_SUCC(ret)) {
@ -468,7 +462,6 @@ int ObBlockMetaTree::upper_bound(const blocksstable::ObDatumRowkey *target_rowke
int ret = OB_SUCCESS;
rowkey = nullptr;
tree_value = nullptr;
const int64_t version = INT64_MAX;
bool find = false;
blocksstable::DDLBtreeIterator tmp_iter;
if (OB_ISNULL(target_rowkey)) {
@ -478,8 +471,7 @@ int ObBlockMetaTree::upper_bound(const blocksstable::ObDatumRowkey *target_rowke
ObDatumRowkeyWrapper(target_rowkey, &datum_utils),
true,
ObDatumRowkeyWrapper(&ObDatumRowkey::MAX_ROWKEY, &datum_utils),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else {
while (OB_SUCC(ret)) {
@ -516,7 +508,6 @@ int ObBlockMetaTree::locate_key(const blocksstable::ObDatumRange &range,
{
int ret = OB_SUCCESS;
cur_tree_value = nullptr;
const int64_t version = INT64_MAX;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
@ -533,8 +524,7 @@ int ObBlockMetaTree::locate_key(const blocksstable::ObDatumRange &range,
ObDatumRowkeyWrapper(rowkey, &datum_utils),
true,
ObDatumRowkeyWrapper(rowkey, &datum_utils),
true,
version))) {
true))) {
LOG_WARN("locate range failed", K(ret));
} else {
cur_tree_value = tree_value;
@ -554,7 +544,6 @@ int ObBlockMetaTree::locate_range(const blocksstable::ObDatumRange &range,
{
int ret = OB_SUCCESS;
cur_tree_value = nullptr;
const int64_t version = INT64_MAX;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
@ -606,8 +595,7 @@ int ObBlockMetaTree::locate_range(const blocksstable::ObDatumRange &range,
ObDatumRowkeyWrapper(start_rowkey /*ObDatumRowkey::MIN_ROWKEY*/, &datum_utils),
false,
ObDatumRowkeyWrapper(end_rowkey, &datum_utils),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else if (OB_FAIL(iter.get_next(rowkey_wrapper_left, tree_value_left))) {
if (OB_ITER_END != ret) {
@ -630,8 +618,7 @@ int ObBlockMetaTree::locate_range(const blocksstable::ObDatumRange &range,
ObDatumRowkeyWrapper(start_rowkey, &datum_utils),
true,
ObDatumRowkeyWrapper(end_rowkey, &datum_utils),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else {
cur_tree_value = start_tree_value;
@ -645,8 +632,7 @@ int ObBlockMetaTree::locate_range(const blocksstable::ObDatumRange &range,
ObDatumRowkeyWrapper(end_rowkey /*ObDatumRowkey::MAX_ROWKEY*/, &datum_utils),
false,
ObDatumRowkeyWrapper(start_rowkey, &datum_utils),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else if (OB_FAIL(iter.get_next(rowkey_wrapper_right, tree_value_right))) {
if (OB_ITER_END != ret) {
@ -669,8 +655,7 @@ int ObBlockMetaTree::locate_range(const blocksstable::ObDatumRange &range,
ObDatumRowkeyWrapper(end_rowkey, &datum_utils),
true,
ObDatumRowkeyWrapper(start_rowkey, &datum_utils),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else {
cur_tree_value = end_tree_value;
@ -759,7 +744,6 @@ int ObBlockMetaTree::get_next_tree_value(blocksstable::DDLBtreeIterator &iter,
int ObBlockMetaTree::get_last_rowkey(const ObDatumRowkey *&last_rowkey)
{
int ret = OB_SUCCESS;
const int64_t version = INT64_MAX;
last_rowkey = nullptr;
blocksstable::DDLBtreeIterator tmp_iter;
if (OB_UNLIKELY(!is_inited_)) {
@ -770,8 +754,7 @@ int ObBlockMetaTree::get_last_rowkey(const ObDatumRowkey *&last_rowkey)
ObDatumRowkeyWrapper(&ObDatumRowkey::MIN_ROWKEY, datum_utils_),
false,
ObDatumRowkeyWrapper(&ObDatumRowkey::MAX_ROWKEY, datum_utils_),
false,
version))) {
false))) {
LOG_WARN("locate range failed", K(ret));
} else {
bool find = false;

View File

@ -519,6 +519,7 @@ int ObLSTabletService::inner_remove_tablet(
const ObTabletMapKey key(ls_id, tablet_id);
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTenantDirectLoadMgr *tenant_direct_load_mgr = MTL(ObTenantDirectLoadMgr *);
ObTransService *tx_svr = MTL(ObTransService*);
if (OB_FAIL(tablet_id_set_.erase(tablet_id))) {
if (OB_HASH_NOT_EXIST == ret) {
@ -527,6 +528,8 @@ int ObLSTabletService::inner_remove_tablet(
} else {
LOG_WARN("fail to erase tablet id from set", K(ret), K(ls_id), K(tablet_id));
}
} else if (OB_FAIL(tx_svr->remove_tablet(tablet_id, ls_id))) {
LOG_ERROR("fail to remove tablet to ls cache", K(ret), K(tablet_id), K(ls_id));
}
if (OB_SUCC(ret)) {
@ -908,6 +911,7 @@ int ObLSTabletService::migrate_create_tablet(
ObTabletHandle tmp_tablet_hdl;
ObTabletHandle tablet_handle;
ObMetaDiskAddr disk_addr;
ObTransService *tx_svr = MTL(ObTransService*);
if (OB_FAIL(ObTabletCreateDeleteHelper::create_tmp_tablet(key, allocator, tmp_tablet_hdl))) {
LOG_WARN("fail to create temporary tablet", K(ret), K(key));
@ -922,6 +926,8 @@ int ObLSTabletService::migrate_create_tablet(
LOG_WARN("fail to create tablet and cas", K(ret), K(ls_id), K(tablet_id), K(disk_addr));
} else if (OB_FAIL(tablet_handle.get_obj()->start_direct_load_task_if_need())) {
LOG_WARN("start ddl if need failed", K(ret));
} else if (OB_FAIL(tx_svr->create_tablet(key.tablet_id_, key.ls_id_))) {
LOG_WARN("fail to create tablet cache", K(ret), K(key), K(tablet_handle));
} else {
LOG_INFO("migrate create tablet succeed", K(ret), K(key), K(disk_addr));
}
@ -1814,6 +1820,7 @@ int ObLSTabletService::replay_create_tablet(
int ret = OB_SUCCESS;
bool b_exist = false;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTransService *tx_svr = MTL(ObTransService*);
ObFreezer *freezer = ls_->get_freezer();
const ObLSID &ls_id = ls_->get_ls_id();
if (IS_NOT_INIT) {
@ -1876,6 +1883,8 @@ int ObLSTabletService::replay_create_tablet(
LOG_WARN("start ddl if need failed", K(ret));
} else if (OB_FAIL(tablet->inc_macro_ref_cnt())) {
LOG_WARN("fail to increase macro blocks' ref cnt for meta and data", K(ret));
} else if (OB_FAIL(tx_svr->create_tablet(key.tablet_id_, key.ls_id_))) {
LOG_WARN("fail to create tablet cache", K(ret), K(key), K(tablet_hdl));
}
if (OB_SUCC(ret)) {
@ -1924,11 +1933,13 @@ int ObLSTabletService::get_tablet_with_timeout(
int ObLSTabletService::direct_get_tablet(const common::ObTabletID &tablet_id, ObTabletHandle &handle)
{
TIMEGUARD_INIT(STORAGE, 10_ms);
#ifdef ENABLE_DEBUG_LOG
ObTimeGuard tg("direct_get_tablet", 10_ms);
#endif
int ret = OB_SUCCESS;
const ObTabletMapKey key(ls_->get_ls_id(), tablet_id);
if (CLICK_FAIL(ObTabletCreateDeleteHelper::get_tablet(key, handle))) {
if (OB_FAIL(ObTabletCreateDeleteHelper::get_tablet(key, handle))) {
if (OB_TABLET_NOT_EXIST != ret) {
LOG_WARN("failed to get tablet from t3m", K(ret), K(key));
}
@ -2022,6 +2033,7 @@ int ObLSTabletService::create_tablet(
common::ObArenaAllocator tmp_allocator(common::ObMemAttr(MTL_ID(), "CreateTab"));
common::ObArenaAllocator *allocator = nullptr;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTransService *tx_svr = MTL(ObTransService*);
const ObTabletMapKey key(ls_id, tablet_id);
ObTablet *tablet = nullptr;
ObFreezer *freezer = ls_->get_freezer();
@ -2048,6 +2060,8 @@ int ObLSTabletService::create_tablet(
LOG_WARN("failed to compare and swap tablet", K(ret), K(key), K(tablet_handle), K(param));
} else if (OB_FAIL(tablet_id_set_.set(tablet_id))) {
LOG_WARN("fail to insert tablet id", K(ret), K(ls_id), K(tablet_id));
} else if (OB_FAIL(tx_svr->create_tablet(key.tablet_id_, key.ls_id_))) {
LOG_WARN("fail to create tablet cache", K(ret), K(key), K(tablet_handle));
} else {
report_tablet_to_rs(tablet_id);
}
@ -2077,6 +2091,7 @@ int ObLSTabletService::create_inner_tablet(
int ret = OB_SUCCESS;
common::ObArenaAllocator allocator(common::ObMemAttr(MTL_ID(), "LSCreateTab"));
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTransService *tx_svr = MTL(ObTransService*);
const ObTabletMapKey key(ls_id, tablet_id);
ObTablet *tmp_tablet = nullptr;
ObFreezer *freezer = ls_->get_freezer();
@ -2102,6 +2117,8 @@ int ObLSTabletService::create_inner_tablet(
} else if (FALSE_IT(disk_addr = tablet_handle.get_obj()->get_tablet_addr())) {
} else if (OB_FAIL(safe_create_cas_tablet(ls_id, tablet_id, disk_addr, tablet_handle, time_guard))) {
LOG_WARN("fail to refresh tablet", K(ret), K(ls_id), K(tablet_id), K(disk_addr), K(tablet_handle));
} else if (OB_FAIL(tx_svr->create_tablet(key.tablet_id_, key.ls_id_))) {
LOG_WARN("fail to create tablet cache", K(ret), K(key), K(tablet_handle));
}
if (OB_SUCC(ret)) {
@ -2122,6 +2139,7 @@ int ObLSTabletService::create_transfer_in_tablet(
{
int ret = OB_SUCCESS;
ObTenantMetaMemMgr *t3m = MTL(ObTenantMetaMemMgr*);
ObTransService *tx_svr = MTL(ObTransService*);
const common::ObTabletID &tablet_id = tablet_meta.tablet_id_;
const ObTabletMapKey key(ls_id, tablet_id);
ObTablet *tablet = nullptr;
@ -2176,6 +2194,8 @@ int ObLSTabletService::create_transfer_in_tablet(
LOG_WARN("failed to compare and swap tablet", K(ret), K(key), K(tablet_handle), K(param));
} else if (OB_FAIL(tablet_id_set_.set(tablet_id))) {
LOG_WARN("fail to insert tablet id", K(ret), K(ls_id), K(tablet_id));
} else if (OB_FAIL(tx_svr->create_tablet(key.tablet_id_, key.ls_id_))) {
LOG_WARN("fail to create tablet cache", K(ret), K(key), K(tablet_meta));
} else {
time_guard.click("Swap");
}

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