[FEAT MERGE]4.2 PX Feature

This commit is contained in:
qianchanger 2023-04-30 04:41:31 +00:00 committed by ob-robot
parent 09ed904b58
commit 3ae36f5774
134 changed files with 10295 additions and 1480 deletions

View File

@ -244,6 +244,29 @@ public:
}
return ret;
};
// erase key value pair if pred is met
// thread safe erase, will add write lock to the bucket
// return value:
// OB_SUCCESS for success
// OB_HASH_NOT_EXIST for node not exists
// others for error
template<class _pred>
int erase_if(const _key_type &key, _pred &pred, bool &is_erased, _value_type *value = NULL)
{
int ret = OB_SUCCESS;
pair_type pair;
if (NULL != value) {
if (OB_FAIL(ht_.erase_if(key, pred, is_erased, &pair))) {
} else if (is_erased && OB_FAIL(copy_assign(*value, pair.second))) {
HASH_WRITE_LOG(HASH_FATAL, "copy assign failed, ret=%d", ret);
}
} else {
ret = ht_.erase_if(key, pred, is_erased);
}
return ret;
}
template <class _archive>
int serialization(_archive &archive)
{

View File

@ -1304,6 +1304,73 @@ public:
return ret;
}
// erase key value pair if pred is met
// thread safe erase, will add write lock to the bucket
// return value:
// OB_SUCCESS for success
// OB_HASH_NOT_EXIST for node not exists
// others for error
template<class _pred>
int erase_if(const _key_type &key, _pred &pred, bool &is_erased, _value_type *value = NULL)
{
return erase_if(key, pred, preproc_, is_erased, value);
}
// erase key value pair if pred is met
// thread safe erase, will add write lock to the bucket
template<class _pred, class _preproc>
int erase_if(const _key_type &key, _pred &pred, _preproc &preproc,
bool &is_erased, _value_type *value = NULL)
{
int ret = OB_SUCCESS;
uint64_t hash_value = 0;
is_erased = false;
if (OB_UNLIKELY(!inited(buckets_)) || OB_UNLIKELY(NULL == allocer_)) {
HASH_WRITE_LOG(HASH_WARNING, "hashtable not init");
ret = OB_NOT_INIT;
} else if (OB_FAIL(hashfunc_(key, hash_value))) {
HASH_WRITE_LOG(HASH_WARNING, "hash key failed, ret = %d", ret);
} else {
int64_t bucket_pos = hash_value % bucket_num_;
hashbucket &bucket = buckets_[bucket_pos];
bucket_lock_cond blc(bucket);
writelocker locker(blc.lock());
hashnode *node = bucket.node;
hashnode *prev = NULL;
ret = OB_HASH_NOT_EXIST;
while (NULL != node) {
if (equal_(getkey_(node->data), key)) {
ret = OB_SUCCESS;
if (pred(preproc(node->data))) {
if (NULL != value) {
if (OB_FAIL(copy_assign(*value, node->data))) {
HASH_WRITE_LOG(HASH_FATAL, "failed to copy data, ret = %d", ret);
}
}
if (OB_SUCC(ret)) {
if (NULL == prev) {
bucket.node = node->next;
} else {
prev->next = node->next;
}
allocer_->free(node);
node = NULL;
ATOMIC_DEC((uint64_t *) &size_);
bucket_lock_cond blc(bucket);
cond_broadcaster()(blc.cond());
is_erased = true;
}
}
break;
} else {
prev = node;
node = node->next;
}
}
}
return ret;
}
// thread safe scan, will add read lock to the bucket,
// the modification to the value is forbidden
// @param callback

View File

@ -611,6 +611,12 @@ class EventTable
EN_PX_SQC_INIT_FAILED = 601,
EN_PX_SQC_INIT_PROCESS_FAILED = 602,
EN_PX_PRINT_TARGET_MONITOR_LOG = 603,
EN_PX_SQC_NOT_REPORT_TO_QC = 604,
EN_PX_QC_EARLY_TERMINATE = 605,
EN_PX_SINGLE_DFO_NOT_ERASE_DTL_INTERM_RESULT = 606,
EN_PX_TEMP_TABLE_NOT_DESTROY_REMOTE_INTERM_RESULT = 607,
EN_PX_NOT_ERASE_P2P_DH_MSG = 608,
EN_PX_SLOW_PROCESS_SQC_FINISH_MSG = 609,
// please add new trace point after 700 or before 600
// Compaction Related 700-750

View File

@ -504,6 +504,8 @@ PCODE_DEF(OB_LOAD_DATA_SHUFFLE, 0x519)
PCODE_DEF(OB_LOAD_DATA_INSERT, 0x51A)
PCODE_DEF(OB_AP_PING_SQL_TASK, 0x51B)
PCODE_DEF(OB_REMOTE_SYNC_EXECUTE, 0x51C) //remote sync execute with sql
PCODE_DEF(OB_PX_CLAER_DH_MSG, 0x51D) //for clear px p2p datahub
PCODE_DEF(OB_PX_P2P_DH_MSG, 0x51E) // for px p2p datahub
PCODE_DEF(OB_PX_FAST_INIT_SQC, 0x51F)
PCODE_DEF(OB_CHECK_BUILD_INDEX_TASK_EXIST, 0x520) // check build index task exist
PCODE_DEF(OB_DAS_SYNC_ACCESS, 0x521) //access execute with sync rpc
@ -972,5 +974,8 @@ PCODE_DEF(OB_LOAD_EXTERNAL_FILE_LIST, 0x1571)
// 1581-1590 for workload repository
// PCODE_DEF(OB_WR_ASYNC_SNAPSHOT_TASK, 0x1581)
// session info verification
PCODE_DEF(OB_SESS_INFO_VERIFICATION, 0x1592)
// for detect manager
PCODE_DEF(OB_DETECT_RPC_CALL, 0x1595)

View File

@ -48,3 +48,4 @@ ob_unittest_observer(test_ob_simple_rto test_ob_simple_rto.cpp)
ob_unittest_observer(test_all_virtual_proxy_partition_info_default_value test_all_virtual_proxy_partition_info_default_value.cpp)
ob_unittest_observer(test_get_stopped_zone_list test_get_stopped_zone_list.cpp)
ob_unittest_observer(test_lock_table_with_tx test_lock_table_with_tx.cpp)
ob_unittest_observer(test_ob_detect_manager_in_simple_server test_ob_detect_manager_in_simple_server.cpp)

View File

@ -0,0 +1,284 @@
/**
* 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.
*/
#define USING_LOG_PREFIX SERVER
#include <gtest/gtest.h>
#define private public
#define protected public
#include "share/detect/ob_detect_manager.h"
#include "share/detect/ob_detect_manager_utils.h"
#include "lib/oblog/ob_log_module.h"
#include "lib/thread/thread_mgr.h"
#include "lib/alloc/memory_dump.h"
// simple server
#include "env/ob_simple_cluster_test_base.h"
#include "env/ob_fast_bootstrap.h"
#include "lib/mysqlclient/ob_mysql_result.h"
using namespace oceanbase;
using namespace oceanbase::common;
using namespace oceanbase::lib;
using namespace std;
namespace oceanbase
{
namespace unittest
{
static const uint64_t ACTIVATE_DELAY_TIME_SEC = 5;
static uint64_t s_tenant_id = OB_SYS_TENANT_ID;
ObDetectManager *dm = nullptr;
ObDetectManagerThread &dm_thr = ObDetectManagerThread::instance();
class ObMockResource
{
public:
static int free_cnt;
public:
ObMockResource() {}
void destroy()
{
int ret = OB_SUCCESS;
ATOMIC_FAA(&ObMockResource::free_cnt, 1);
LOG_WARN("ObMockResource destoryed");
}
};
int ObMockResource::free_cnt = 0;
class ObMockDetectCB : public ObIDetectCallback
{
public:
ObMockDetectCB(const ObArray<ObPeerTaskState> &peer_states, ObMockResource *resource)
: ObIDetectCallback(peer_states), resouce_(resource) {}
int do_callback() override
{
int ret = OB_SUCCESS;
if (OB_NOT_NULL(resouce_)) {
resouce_->destroy();
ob_free(resouce_);
resouce_ = nullptr;
}
return ret;
}
int64_t get_detect_callback_type() const override { return (int64_t)DetectCallBackType::VIRTUAL; }
private:
ObMockResource* resouce_;
};
ObMockResource *allocate_mock_resource()
{
void *buf = nullptr;
buf = ob_malloc(sizeof(ObMockResource), "mock resource");
ObMockResource *rsc = new(buf) ObMockResource;
return rsc;
}
static int init_dm()
{
int ret = OB_SUCCESS;
// hack an local addr to dm
ObAddr self;
self.set_ip_addr("127.0.0.1", 8086);
MTL_SWITCH(s_tenant_id) {
dm = MTL(ObDetectManager*);
dm->self_ = self;
dm_thr.self_ = self;
}
return ret;
}
// check memory具有延迟性,从测试的结果来看,大致阈值是5s
// 调用 check_memory_leak 前需要前置sleep
int check_memory_leak()
{
sleep(15);
int ret = OB_SUCCESS;
uint64_t tenant_id = OB_SYS_TENANT_ID;
for (int ctx_id = 0; ctx_id < ObCtxIds::MAX_CTX_ID; ctx_id++) {
auto ta = ObMallocAllocator::get_instance()->get_tenant_ctx_allocator(tenant_id, ctx_id);
if (nullptr == ta) {
ta = ObMallocAllocator::get_instance()->get_tenant_ctx_allocator_unrecycled(tenant_id, ctx_id);
}
if (nullptr == ta) {
continue;
}
if (OB_SUCC(ret)) {
ret = ta->iter_label([&](lib::ObLabel &label, LabelItem *l_item)
{
if (strcmp(label.str_, "DetectManager") == 0) {
LOG_WARN("dm leak memory:", K(tenant_id), K(ctx_id), K(label.str_), K(l_item->hold_), K(l_item->used_), K(l_item->count_));
}
return OB_SUCCESS;
});
}
}
return ret;
}
class TestRunCtx
{
public:
uint64_t tenant_id_ = 0;
int time_sec_ = 0;
};
TestRunCtx RunCtx;
class ObSimpleClusterExampleTest : public ObSimpleClusterTestBase
{
public:
// 指定case运行目录前缀 test_ob_simple_cluster_
ObSimpleClusterExampleTest() : ObSimpleClusterTestBase("test_ob_simple_cluster_") {}
};
TEST_F(ObSimpleClusterExampleTest, observer_start)
{
SERVER_LOG(INFO, "observer_start succ");
}
TEST_F(ObSimpleClusterExampleTest, init_dm)
{
ASSERT_EQ(OB_SUCCESS, init_dm());
}
TEST_F(ObSimpleClusterExampleTest, test_detect_local)
{
// mock an local addr
ObAddr local_addr;
local_addr.set_ip_addr("127.0.0.1", 8086);
ObPeerTaskState peer_state(local_addr);
ObArray<ObPeerTaskState> peer_states;
peer_states.push_back(peer_state);
// cb1 managerd by dm
ObDetectableId id1;
ObDetectManagerUtils::generate_detectable_id(id1, s_tenant_id);
ObMockDetectCB *cb1 = nullptr;
ObMockResource *rsc1 = allocate_mock_resource();
uint64_t node_sequence_id1 = 0;
// cb2 managerd by dm
ObDetectableId id2;
ObDetectManagerUtils::generate_detectable_id(id2, s_tenant_id);
ObMockDetectCB *cb2 = nullptr;
ObMockResource *rsc2 = allocate_mock_resource();
uint64_t node_sequence_id2 = 0;
// register id
ASSERT_EQ(OB_SUCCESS, dm->register_detectable_id(id1));
// register cb
ASSERT_EQ(OB_SUCCESS, dm->register_check_item(id1, cb1, node_sequence_id1, true/* need_ref */, peer_states, rsc1));
ASSERT_EQ(OB_INVALID_ARGUMENT, dm->register_check_item(id1, cb1, node_sequence_id1, true/* need_ref */, peer_states, rsc1));
ASSERT_EQ(OB_SUCCESS, dm->register_check_item(id2, cb2, node_sequence_id2, false/* need_ref */, peer_states, rsc2));
// unregister id1 after 2s
sleep(ACTIVATE_DELAY_TIME_SEC);
ASSERT_EQ(OB_SUCCESS, dm->unregister_detectable_id(id1));
// wait detect loop done
sleep(15);
// cb1 and cb2 are executed;
ASSERT_EQ(2, ATOMIC_LOAD(&ObMockResource::free_cnt));
ATOMIC_SAF(&ObMockResource::free_cnt, 2);
// unregister cb1
cb1->dec_ref_count();
ASSERT_EQ(OB_SUCCESS, dm->unregister_check_item(id1, node_sequence_id1));
check_memory_leak();
}
TEST_F(ObSimpleClusterExampleTest, test_detect_remote)
{
// get rpc addr
ObAddr remote_addr = get_curr_simple_server().get_addr();
ObPeerTaskState peer_state(remote_addr);
ObArray<ObPeerTaskState> peer_states;
peer_states.push_back(peer_state);
ObDetectableId id1;
ObDetectManagerUtils::generate_detectable_id(id1, s_tenant_id);
// cb1 managerd by user
ObMockDetectCB *cb1 = nullptr;
ObMockResource *rsc1 = allocate_mock_resource();
uint64_t node_sequence_id1 = 0;
// cb2 managerd by dm
ObDetectableId id2;
ObDetectManagerUtils::generate_detectable_id(id2, s_tenant_id);
ObMockDetectCB *cb2 = nullptr;
ObMockResource *rsc2 = allocate_mock_resource();
uint64_t node_sequence_id2 = 0;
// register
ASSERT_EQ(OB_SUCCESS, dm->register_detectable_id(id1));
ASSERT_EQ(OB_SUCCESS, dm->register_check_item(id1, cb1, node_sequence_id1, true/* need_ref */, peer_states, rsc1));
ASSERT_EQ(OB_SUCCESS, dm->register_check_item(id2, cb2, node_sequence_id2, false/* need_ref */, peer_states, rsc2));
// unregister id1 after 2s
sleep(ACTIVATE_DELAY_TIME_SEC);
ASSERT_EQ(OB_SUCCESS, dm->unregister_detectable_id(id1));
// wait detect loop done
sleep(15);
ASSERT_EQ(2, ATOMIC_LOAD(&ObMockResource::free_cnt));
// unregister cb1.
cb1->dec_ref_count();
ASSERT_EQ(OB_SUCCESS, dm->unregister_check_item(id1, node_sequence_id1));
check_memory_leak();
}
TEST_F(ObSimpleClusterExampleTest, end)
{
if (RunCtx.time_sec_ > 0) {
::sleep(RunCtx.time_sec_);
}
}
} // end unittest
} // end oceanbase
int main(int argc, char **argv)
{
int c = 0;
int time_sec = 0;
char *log_level = (char*)"INFO";
while(EOF != (c = getopt(argc,argv,"t:l:"))) {
switch(c) {
case 't':
time_sec = atoi(optarg);
break;
case 'l':
log_level = optarg;
oceanbase::unittest::ObSimpleClusterTestBase::enable_env_warn_log_ = false;
break;
default:
break;
}
}
oceanbase::unittest::init_log_and_gtest(argc, argv);
int ret = OB_SUCCESS;
system("rm -f test_ob_detect_manager_in_simple_server.log*");
OB_LOGGER.set_file_name("test_ob_detect_manager_in_simple_server.log", true, true);
OB_LOGGER.set_log_level("INFO");
LOG_INFO("main>>>");
oceanbase::unittest::RunCtx.time_sec_ = time_sec;
::testing::InitGoogleTest(&argc, argv);
ret = RUN_ALL_TESTS();
return ret;
}

View File

@ -203,7 +203,7 @@ typedef enum ObItemType
T_OP_COLL_PRED = 177, // collection predicate, such as: xx is member of collection
T_OP_BOOL = 178,
T_OP_STACK_OVERFLOW_CHECK = 179, // stack over flow check expr.
T_OP_JOIN_BLOOM_FILTER = 180,
T_OP_RUNTIME_FILTER = 180,
T_OP_TO_OUTFILE_ROW = 181,
// The aggregation version arithmetic operator is used to ignore the double overflow error,
// because mysql do not check overflow for double in aggregation.

View File

@ -251,6 +251,7 @@ ob_set_subtarget(ob_server virtual_table
virtual_table/ob_all_virtual_ps_stat.cpp
virtual_table/ob_all_virtual_px_target_monitor.cpp
virtual_table/ob_all_virtual_px_worker_stat.cpp
virtual_table/ob_all_virtual_px_p2p_datahub.cpp
virtual_table/ob_all_virtual_dtl_interm_result_monitor.cpp
virtual_table/ob_all_virtual_raid_stat.cpp
virtual_table/ob_all_virtual_ls_archive_stat.cpp

View File

@ -68,6 +68,7 @@
#include "sql/dtl/ob_dtl.h"
#include "sql/engine/cmd/ob_load_data_utils.h"
#include "sql/engine/px/ob_px_worker.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
#include "sql/ob_sql_init.h"
#include "sql/ob_sql_task.h"
#include "storage/ob_i_store.h"
@ -101,6 +102,7 @@
#include "storage/ddl/ob_ddl_redo_log_writer.h"
#include "observer/ob_server_utils.h"
#include "observer/table_load/ob_table_load_partition_calc.h"
#include "share/detect/ob_detect_manager.h"
using namespace oceanbase::lib;
using namespace oceanbase::common;
@ -406,6 +408,8 @@ int ObServer::init(const ObServerOptions &opts, const ObPLogWriterCfg &log_cfg)
LOG_ERROR("init ObBackupInfo failed", KR(ret));
} else if (OB_FAIL(ObPxBloomFilterManager::instance().init())) {
LOG_ERROR("init px blomm filter manager failed", KR(ret));
} else if (OB_FAIL(PX_P2P_DH.init())) {
LOG_ERROR("init px p2p datahub failed", KR(ret));
} else if (OB_FAIL(ObBackupFileLockMgr::get_instance().init())) {
LOG_ERROR("init backup file lock mgr failed", KR(ret));
} else if (OB_FAIL(ObDagWarningHistoryManager::get_instance().init())) {
@ -437,7 +441,9 @@ int ObServer::init(const ObServerOptions &opts, const ObPLogWriterCfg &log_cfg)
} else if (OB_FAIL(ObDDLRedoLogWriter::get_instance().init())) {
LOG_WARN("init DDL redo log writer failed", KR(ret));
}
else {
else if (OB_FAIL(ObDetectManagerThread::instance().init(GCTX.self_addr(), net_frame_.get_req_transport()))) {
LOG_WARN("init ObDetectManagerThread failed", KR(ret));
} else {
GDS.set_rpc_proxy(&rs_rpc_proxy_);
}
}
@ -978,6 +984,10 @@ int ObServer::stop()
TG_STOP(lib::TGDefIDs::Blacklist);
FLOG_INFO("server blacklist stopped");
FLOG_INFO("begin to stop detect manager detect thread");
TG_STOP(lib::TGDefIDs::DetectManager);
FLOG_INFO("detect manager detect thread stopped");
FLOG_INFO("begin to stop ObNetKeepAlive");
ObNetKeepAlive::get_instance().stop();
FLOG_INFO("ObNetKeepAlive stopped");

View File

@ -26,6 +26,7 @@
#include "sql/engine/cmd/ob_kill_executor.h"
#include "sql/engine/cmd/ob_load_data_rpc.h"
#include "sql/engine/px/ob_px_rpc_processor.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.h"
#include "sql/das/ob_das_id_rpc.h"
#include "sql/dtl/ob_dtl_rpc_processor.h"
#include "storage/tablelock/ob_table_lock_rpc_processor.h"
@ -161,6 +162,8 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator *xlator) {
RPC_PROCESSOR(ObInitSqcP, gctx_);
RPC_PROCESSOR(ObInitTaskP, gctx_);
RPC_PROCESSOR(ObInitFastSqcP, gctx_);
RPC_PROCESSOR(ObPxP2pDhMsgP, gctx_);
RPC_PROCESSOR(ObPxP2pDhClearMsgP, gctx_);
RPC_PROCESSOR(ObPxTenantTargetMonitorP, gctx_);
RPC_PROCESSOR(ObPxCleanDtlIntermResP, gctx_);
// SQL Estimate
@ -258,7 +261,7 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator *xlator) {
// checkpoint slog rpc
RPC_PROCESSOR(ObCheckpointSlogP, gctx_);
// check connectivity
// check connectivity
RPC_PROCESSOR(ObRpcCheckBackupDestConnectivityP, gctx_);
// global auto increment service rpc

View File

@ -48,6 +48,7 @@
#include "observer/dbms_job/ob_dbms_job_rpc_processor.h"
#include "storage/tx_storage/ob_tenant_freezer_rpc.h"
#include "observer/dbms_scheduler/ob_dbms_sched_job_rpc_processor.h"
#include "share/detect/ob_detect_rpc_processor.h"
#include "share/external_table/ob_external_table_file_rpc_processor.h"
@ -95,6 +96,8 @@ void oceanbase::observer::init_srv_xlator_for_sys(ObSrvRpcXlator *xlator) {
RPC_PROCESSOR(ObBlacklistReqP);
RPC_PROCESSOR(ObBlacklistRespP);
RPC_PROCESSOR(ObDetectRpcP);
// election provided
// RPC_PROCESSOR(ObElectionP);
RPC_PROCESSOR(ObRequestHeartbeatP, gctx_);

View File

@ -103,6 +103,7 @@
#include "sql/plan_cache/ob_plan_cache.h"
#include "sql/plan_cache/ob_ps_cache.h"
#include "rootserver/ob_heartbeat_service.h"
#include "share/detect/ob_detect_manager.h"
using namespace oceanbase;
using namespace oceanbase::lib;
@ -444,6 +445,7 @@ int ObMultiTenant::init(ObAddr myaddr,
MTL_BIND2(mtl_new_default, ObPlanCache::mtl_init, nullptr, ObPlanCache::mtl_stop, nullptr, mtl_destroy_default);
MTL_BIND2(mtl_new_default, ObPsCache::mtl_init, nullptr, ObPsCache::mtl_stop, nullptr, mtl_destroy_default);
MTL_BIND2(server_obj_pool_mtl_new<ObPartTransCtx>, nullptr, nullptr, nullptr, nullptr, server_obj_pool_mtl_destroy<ObPartTransCtx>);
MTL_BIND(ObDetectManager::mtl_init, ObDetectManager::mtl_destroy);
if (GCONF._enable_new_sql_nio && GCONF._enable_tenant_sql_net_thread) {
MTL_BIND2(nullptr, nullptr, start_mysql_queue, mtl_stop_default,
mtl_wait_default, mtl_destroy_default);

View File

@ -0,0 +1,186 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#include "observer/virtual_table/ob_all_virtual_px_p2p_datahub.h"
#include "observer/ob_server_utils.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
namespace oceanbase
{
using namespace common;
using namespace sql;
namespace observer
{
static const char *msg_type_str[] = {
"NOT_INIT",
"BLOOM_FILTER_MSG",
"RANGE_FILTER_MSG",
"IN_FILTER_MSG",
"MAX_TYPE"
};
int ObAllPxP2PDatahubTable::P2PMsgTraverseCall::operator() (
common::hash::HashMapPair<ObP2PDhKey,
ObP2PDatahubMsgBase *> &entry)
{
int ret = OB_SUCCESS;
if (OB_NOT_NULL(entry.second)) {
if (!is_sys_tenant(tenant_id_) &&
entry.second->get_tenant_id() != tenant_id_) {
/*do nothing*/
} else {
P2PDatahubNode node;
node.p2p_datahub_id_ = entry.second->get_p2p_datahub_id();
node.msg_type_ = (int64_t)entry.second->get_msg_type();
node.start_time_ = entry.second->get_start_time();
node.tenant_id_ = entry.second->get_tenant_id();
node.trace_id_ = entry.second->get_trace_id();
node.timeout_ts_ = entry.second->get_timeout_ts();
if (OB_FAIL(node_array_.push_back(node))) {
SERVER_LOG(WARN, "fail to push back node", K(ret));
}
}
}
return ret;
}
ObAllPxP2PDatahubTable::ObAllPxP2PDatahubTable():addr_(NULL), start_to_read_(false),
node_array_(), index_(0)
{
}
ObAllPxP2PDatahubTable::~ObAllPxP2PDatahubTable()
{
}
void ObAllPxP2PDatahubTable::reset()
{
addr_ = NULL;
start_to_read_ = false;
node_array_.reset();
index_ = 0;
}
int ObAllPxP2PDatahubTable::p2p_datahub_map_to_array()
{
int ret = OB_SUCCESS;
P2PMsgTraverseCall call(node_array_, effective_tenant_id_);
if (OB_FAIL(PX_P2P_DH.get_map().foreach_refactored(call))) {
SERVER_LOG(WARN, "fail to convert map to array", K(ret));
}
return ret;
}
int ObAllPxP2PDatahubTable::inner_get_next_row(ObNewRow *&row)
{
int ret = OB_SUCCESS;
ObObj *cells = cur_row_.cells_;
ObString ipstr;
if (OB_ISNULL(allocator_) || OB_ISNULL(addr_)) {
ret = OB_NOT_INIT;
SERVER_LOG(WARN, "allocator_ or addr_ is null", K_(allocator), K_(addr), K(ret));
} else if (OB_ISNULL(cells)) {
ret = OB_ERR_UNEXPECTED;
SERVER_LOG(WARN, "cur row cell is NULL", K(ret));
} else if (OB_FAIL(ObServerUtils::get_server_ip(allocator_, ipstr))) {
SERVER_LOG(ERROR, "get server ip failed", K(ret));
} else {
if (!start_to_read_) {
if (OB_FAIL(p2p_datahub_map_to_array())) {
SERVER_LOG(WARN, "fail to convert map to array", K(ret));
}
}
if (index_ >= node_array_.size()) {
ret = OB_ITER_END;
}
for (int64_t cell_idx = 0;
OB_SUCC(ret) && cell_idx < output_column_ids_.count();
++cell_idx) {
const uint64_t column_id = output_column_ids_.at(cell_idx);
switch(column_id) {
case SVR_IP: {
cells[cell_idx].set_varchar(ipstr);
cells[cell_idx].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case SVR_PORT: {
cells[cell_idx].set_int(addr_->get_port());
break;
}
case TRACE_ID: {
int len = node_array_.at(index_).trace_id_.to_string(trace_id_, sizeof(trace_id_));
cells[cell_idx].set_varchar(trace_id_, len);
cells[cell_idx].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
break;
}
case DATAHUB_ID: {
int64_t p2p_datahub_id = node_array_.at(index_).p2p_datahub_id_;
cells[cell_idx].set_int(p2p_datahub_id);
break;
}
case TENANT_ID: {
int64_t tenant_id = node_array_.at(index_).tenant_id_;
cells[cell_idx].set_int(tenant_id);
break;
}
case MESSAGE_TYPE: {
int64_t msg_idx = node_array_.at(index_).msg_type_;
int64_t str_cnt = sizeof(msg_type_str) / sizeof(msg_type_str[0]);
if (msg_idx >= str_cnt || msg_idx < 0) {
ret = OB_ERR_UNEXPECTED;
SERVER_LOG(WARN, "unexpected msg type", K(ret), K(msg_idx), K(str_cnt), K(node_array_.at(index_).msg_type_));
} else {
const char *msg_str = msg_type_str[msg_idx];
cells[cell_idx].set_varchar(msg_str, strlen(msg_str));
cells[cell_idx].set_collation_type(
ObCharset::get_default_collation(ObCharset::get_default_charset()));
}
break;
}
case HOLD_SIZE: {
int64_t hold_size = node_array_.at(index_).hold_size_;
cells[cell_idx].set_int(hold_size);
break;
}
case TIMEOUT_TS: {
int64_t timeout_ts = node_array_.at(index_).timeout_ts_;
cells[cell_idx].set_timestamp(timeout_ts);
break;
}
case START_TIME: {
int64_t start_time = node_array_.at(index_).start_time_;
cells[cell_idx].set_timestamp(start_time);
break;
}
default: {
ret = OB_ERR_UNEXPECTED;
SERVER_LOG(WARN, "invalid column id", K(cell_idx),
K_(output_column_ids), K(ret));
break;
}
}
}
++index_;
}
if (OB_SUCC(ret)) {
start_to_read_ = true;
row = &cur_row_;
}
return ret;
}
}/* ns observer*/
}/* ns oceanbase */

View File

@ -0,0 +1,84 @@
/**
* Copyright (c) 2021 OceanBase
* OceanBase CE is licensed under Mulan PubL v2.
* You can use this software according to the terms and conditions of the Mulan PubL v2.
* You may obtain a copy of Mulan PubL v2 at:
* http://license.coscl.org.cn/MulanPubL-2.0
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
* See the Mulan PubL v2 for more details.
*/
#ifndef OCEANBASE_OBSERVER_VIRTUAL_TABLE_ALL_PX_P2P_DATAHUB_TABLE_
#define OCEANBASE_OBSERVER_VIRTUAL_TABLE_ALL_PX_P2P_DATAHUB_TABLE_
#include "share/ob_virtual_table_scanner_iterator.h"
#include "lib/net/ob_addr.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
namespace oceanbase
{
namespace observer
{
class ObAllPxP2PDatahubTable : public common::ObVirtualTableScannerIterator
{
public:
struct P2PDatahubNode {
common::ObCurTraceId::TraceId trace_id_;
int64_t p2p_datahub_id_;
int64_t tenant_id_;
int64_t msg_type_;
int64_t hold_size_;
int64_t timeout_ts_;
int64_t start_time_;
TO_STRING_KV(K(trace_id_), K(p2p_datahub_id_), K(tenant_id_),
K(msg_type_), K(hold_size_), K(timeout_ts_), K(start_time_));
};
public:
struct P2PMsgTraverseCall
{
P2PMsgTraverseCall(common::ObArray<P2PDatahubNode> &node_array, int64_t tenant_id) :
node_array_(node_array), tenant_id_(tenant_id) {};
~P2PMsgTraverseCall() = default;
int operator() (common::hash::HashMapPair<sql::ObP2PDhKey, sql::ObP2PDatahubMsgBase *> &entry);
common::ObArray<P2PDatahubNode> &node_array_;
int64_t tenant_id_;
};
public:
ObAllPxP2PDatahubTable();
virtual ~ObAllPxP2PDatahubTable();
virtual void reset();
virtual int inner_get_next_row(common::ObNewRow *&row);
inline void set_addr(common::ObAddr &addr) { addr_ = &addr; }
private:
int p2p_datahub_map_to_array();
private:
common::ObAddr *addr_;
bool start_to_read_;
char trace_id_[128];
common::ObArray<P2PDatahubNode> node_array_;
int64_t index_;
enum INSPECT_COLUMN
{
SVR_IP = common::OB_APP_MIN_COLUMN_ID,
SVR_PORT,
TRACE_ID,
DATAHUB_ID,
MESSAGE_TYPE,
TENANT_ID,
HOLD_SIZE,
TIMEOUT_TS,
START_TIME
};
private:
DISALLOW_COPY_AND_ASSIGN(ObAllPxP2PDatahubTable);
};
} // namespace observer
} // namespace oceanbase
#endif // OCEANBASE_OBSERVER_VIRTUAL_TABLE_ALL_PX_P2P_DATAHUB_TABLE_

View File

@ -198,6 +198,7 @@
#include "observer/virtual_table/ob_all_virtual_sql_plan.h"
#include "observer/virtual_table/ob_all_virtual_opt_stat_gather_monitor.h"
#include "observer/virtual_table/ob_all_virtual_thread.h"
#include "observer/virtual_table/ob_all_virtual_px_p2p_datahub.h"
namespace oceanbase
{
@ -1887,6 +1888,15 @@ int ObVTIterCreator::create_vt_iter(ObVTableScanParam &params,
}
break;
}
case OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TID: {
ObAllPxP2PDatahubTable *px_p2p_datahub = NULL;
if (OB_SUCC(NEW_VIRTUAL_TABLE(ObAllPxP2PDatahubTable, px_p2p_datahub))) {
px_p2p_datahub->set_allocator(&allocator);
px_p2p_datahub->set_addr(addr_);
vt_iter = static_cast<ObVirtualTableIterator *>(px_p2p_datahub);
}
break;
}
case OB_ALL_VIRTUAL_TABLET_STORE_STAT_TID: {
ObAllVirtualTabletStoreStat *part_table_store_stat = NULL;
if (OB_SUCC(NEW_VIRTUAL_TABLE(ObAllVirtualTabletStoreStat, part_table_store_stat))) {

View File

@ -242,6 +242,12 @@ ob_set_subtarget(ob_share common_mixed
table/ob_table_load_rpc_struct.cpp
table/ob_table_load_shared_allocator.cpp
table/ob_table_load_row.cpp
detect/ob_detect_manager.cpp
detect/ob_detectable_id.cpp
detect/ob_detect_callback.cpp
detect/ob_detect_rpc_proxy.cpp
detect/ob_detect_rpc_processor.cpp
detect/ob_detect_manager_utils.cpp
)
ob_set_subtarget(ob_share tablet

View File

@ -25,6 +25,7 @@
#include "observer/ob_server_struct.h"
#include "share/ob_rpc_struct.h"
#include "sql/plan_cache/ob_plan_cache_util.h"
#include "sql/optimizer/ob_log_join_filter.h"
#include "share/ob_encryption_util.h"
#include "share/ob_resource_limit.h"
@ -702,5 +703,68 @@ bool ObAutoIncrementModeChecker::check(const ObConfigItem &t) const
return is_valid;
}
int64_t ObConfigRuntimeFilterChecker::get_runtime_filter_type(const char *str, int64_t len)
{
int64_t rf_type = -1;
int64_t l = 0, r = len;
if (0 == len) {
rf_type = 0;
} else {
int64_t l = 0, r = len;
bool is_valid = true;
int flag[3] = {0, 0, 0};
auto fill_flag = [&] (ObString &p_str) {
bool valid = true;
ObString trim_str = p_str.trim();
if (0 == trim_str.case_compare("bloom_filter")) {
flag[0]++;
} else if (0 == trim_str.case_compare("range")) {
flag[1]++;
} else if (0 == trim_str.case_compare("in")) {
flag[2]++;
} else {
valid = false;
}
if (valid) {
if (flag[0] > 1 || flag[1] > 1 || flag[2] > 1) {
valid = false;
}
}
return valid;
};
for (int i = 0; i < len && is_valid; ++i) {
if (str[i] == ',') {
r = i;
ObString p_str(r - l, str + l);
is_valid = fill_flag(p_str);
l = i + 1;
continue;
}
}
if (is_valid) {
ObString p_str(len - l, str + l);
is_valid = fill_flag(p_str);
}
if (is_valid) {
rf_type = flag[0] << 1 |
flag[1] << 2 |
flag[2] << 3;
} else {
rf_type = -1;
}
}
return rf_type;
}
bool ObConfigRuntimeFilterChecker::check(const ObConfigItem &t) const
{
int64_t len = strlen(t.str());
const char *p = t.str();
int64_t rf_type = get_runtime_filter_type(t.str(), len);
return rf_type >= 0;
}
} // end of namepace common
} // end of namespace oceanbase

View File

@ -495,6 +495,18 @@ private:
DISALLOW_COPY_AND_ASSIGN(ObConfigEnableDefensiveChecker);
};
class ObConfigRuntimeFilterChecker
: public ObConfigChecker
{
public:
ObConfigRuntimeFilterChecker() {}
virtual ~ObConfigRuntimeFilterChecker() {}
bool check(const ObConfigItem &t) const;
static int64_t get_runtime_filter_type(const char *str, int64_t len);
private:
DISALLOW_COPY_AND_ASSIGN(ObConfigRuntimeFilterChecker);
};
// config item container
class ObConfigStringKey
{

View File

@ -0,0 +1,269 @@
/**
* 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 "share/detect/ob_detect_callback.h"
#include "share/detect/ob_detect_manager.h"
#include "sql/dtl/ob_dtl_basic_channel.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
#include "sql/dtl/ob_dtl_rpc_channel.h"
namespace oceanbase {
namespace common {
const int64_t DM_INTERRUPT_MSG_MAX_LENGTH = 128;
ObIDetectCallback::ObIDetectCallback(const ObArray<ObPeerTaskState> &peer_states)
: ref_count_(0)
{
int ret = OB_SUCCESS;
if (OB_FAIL(peer_states_.assign(peer_states))) {
alloc_succ_ = false;
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to assign peer_states");
} else {
alloc_succ_ = true;
}
}
int ObIDetectCallback::atomic_set_finished(const common::ObAddr &addr, ObTaskState *state)
{
int ret = OB_SEARCH_NOT_FOUND;
ARRAY_FOREACH_NORET(peer_states_, idx) {
if (peer_states_.at(idx).peer_addr_ == addr) {
ObTaskState old_val = (ObTaskState)ATOMIC_SET(
(int32_t*)&peer_states_.at(idx).peer_state_,
(int32_t)ObTaskState::FINISHED);
if (OB_NOT_NULL(state)) {
*state = old_val;
}
ret = OB_SUCCESS;
break;
}
}
return ret;
}
int64_t ObIDetectCallback::inc_ref_count(int64_t count)
{
return ATOMIC_AAF(&ref_count_, count);
}
int64_t ObIDetectCallback::dec_ref_count()
{
return ATOMIC_SAF(&ref_count_, 1);
}
class ObDmInterruptQcCall
{
public:
ObDmInterruptQcCall(const common::ObAddr &from_svr_addr, sql::ObDfo &dfo,
int err,
int64_t timeout_ts,
bool need_set_not_alive) : from_svr_addr_(from_svr_addr), dfo_(dfo), err_(err),
need_interrupt_(false), timeout_ts_(timeout_ts)
{
need_interrupt_ = true;
}
~ObDmInterruptQcCall() = default;
void operator() (hash::HashMapPair<ObInterruptibleTaskID,
ObInterruptCheckerNode *> &entry);
int mock_sqc_finish_msg(sql::ObPxSqcMeta &sqc);
public:
const common::ObAddr &from_svr_addr_;
sql::ObDfo &dfo_;
int err_;
bool need_interrupt_;
int64_t timeout_ts_;
};
void ObDmInterruptQcCall::operator()(hash::HashMapPair<ObInterruptibleTaskID,
ObInterruptCheckerNode *> &entry)
{
UNUSED(entry);
common::ObIArray<sql::ObPxSqcMeta> &sqcs= dfo_.get_sqcs();
ARRAY_FOREACH_NORET(sqcs, i) {
sql::ObPxSqcMeta &sqc = sqcs.at(i);
if (sqc.get_exec_addr() == from_svr_addr_ && !sqc.is_thread_finish()) {
if (sqc.is_ignore_vtable_error()) {
// mock sqc finish if only visit virtual table
mock_sqc_finish_msg(sqc);
} else {
// no longer need to report
sqc.set_need_report(false);
// mark for rollback trans
sqc.set_interrupt_by_dm(true);
}
break;
}
}
}
int ObDmInterruptQcCall::mock_sqc_finish_msg(sql::ObPxSqcMeta &sqc)
{
int ret = OB_SUCCESS;
dtl::ObDtlBasicChannel *ch = reinterpret_cast<dtl::ObDtlBasicChannel *>(
sqc.get_qc_channel());
if (OB_ISNULL(ch)) {
ret = OB_ERR_UNEXPECTED;
LIB_LOG(WARN, "[DM] ch is unexpected", K(ret));
} else {
MTL_SWITCH(ch->get_tenant_id()) {
ObPxFinishSqcResultMsg finish_msg;
finish_msg.rc_ = err_;
finish_msg.dfo_id_ = sqc.get_dfo_id();
finish_msg.sqc_id_ = sqc.get_sqc_id();
dtl::ObDtlMsgHeader header;
header.nbody_ = static_cast<int32_t>(finish_msg.get_serialize_size());
header.type_ = static_cast<int16_t>(finish_msg.get_type());
int64_t need_size = header.get_serialize_size() + finish_msg.get_serialize_size();
dtl::ObDtlLinkedBuffer *buffer = nullptr;
if (OB_ISNULL(buffer = ch->alloc_buf(need_size))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] alloc buffer failed", K(ret));
} else {
auto buf = buffer->buf();
auto size = buffer->size();
auto &pos = buffer->pos();
buffer->set_data_msg(false);
buffer->timeout_ts() = timeout_ts_;
buffer->set_msg_type(dtl::ObDtlMsgType::FINISH_SQC_RESULT);
if (OB_FAIL(common::serialization::encode(buf, size, pos, header))) {
LIB_LOG(WARN, "[DM] fail to encode buffer", K(ret));
} else if (OB_FAIL(common::serialization::encode(buf, size, pos, finish_msg))) {
LIB_LOG(WARN, "[DM] serialize RPC channel message fail", K(ret));
} else if (FALSE_IT(buffer->size() = pos)) {
} else if (FALSE_IT(pos = 0)) {
} else if (FALSE_IT(buffer->tenant_id() = ch->get_tenant_id())) {
} else if (OB_FAIL(ch->attach(buffer))) {
LIB_LOG(WARN, "[DM] fail to feedup buffer", K(ret));
} else if (FALSE_IT(ch->free_buffer_count())) {
} else {
need_interrupt_ = false;
}
}
if (NULL != buffer) {
ch->free_buffer_count();
}
}
}
return ret;
}
ObQcDetectCB::ObQcDetectCB(const ObArray<ObPeerTaskState> &peer_states, const ObInterruptibleTaskID &tid, sql::ObDfo &dfo,
const ObArray<sql::dtl::ObDtlChannel *> &dtl_channels)
: ObIDetectCallback(peer_states), tid_(tid), dfo_(dfo)
{
// if ObIDetectCallback constructed succ
if (alloc_succ_) {
int ret = OB_SUCCESS;
if (OB_FAIL(dtl_channels_.assign(dtl_channels))) {
alloc_succ_ = false;
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to assign dtl_channels_");
} else {
alloc_succ_ = true;
}
}
}
void ObQcDetectCB::destroy()
{
ObIDetectCallback::destroy();
dtl_channels_.reset();
}
int ObQcDetectCB::do_callback()
{
int ret = OB_SUCCESS;
ObGlobalInterruptManager *manager = ObGlobalInterruptManager::getInstance();
if (OB_NOT_NULL(manager)) {
ObDmInterruptQcCall call(from_svr_addr_, dfo_, OB_RPC_CONNECT_ERROR, timeout_ts_, false);
if (OB_FAIL(manager->get_map().atomic_refactored(tid_, call))) {
ret = ret == OB_HASH_NOT_EXIST ? OB_SUCCESS : ret;
LIB_LOG(WARN, "[DM] fail to set need report, qc already exits", K(tid_), K_(trace_id));
} else if (call.need_interrupt_) {
ObInterruptCode int_code(OB_RPC_CONNECT_ERROR,
GETTID(),
from_svr_addr_,
"Dm interrupt qc");
if (OB_FAIL(manager->interrupt(tid_, int_code))) {
LIB_LOG(WARN, "[DM] fail to send interrupt message", K(int_code), K(tid_), K_(trace_id));
}
}
}
LIB_LOG(WARN, "[DM] interrupt qc", K(tid_), K_(trace_id));
return ret;
}
int ObQcDetectCB::atomic_set_finished(const common::ObAddr &addr, ObTaskState *state)
{
int ret = OB_SEARCH_NOT_FOUND;
for (int i = 0; i < get_peer_states().count(); ++i) {
if (get_peer_states().at(i).peer_addr_ == addr) {
sql::dtl::ObDtlRpcChannel* dtl_rpc_channel = static_cast<sql::dtl::ObDtlRpcChannel*>(dtl_channels_.at(i));
if (dtl_rpc_channel->recv_sqc_fin_res()) {
ATOMIC_SET((int32_t*)&get_peer_states().at(i).peer_state_, (int32_t)ObTaskState::FINISHED);
if (OB_NOT_NULL(state)) {
*state = ObTaskState::FINISHED;
}
}
ret = OB_SUCCESS;
break;
}
}
return ret;
}
int ObSqcDetectCB::do_callback()
{
int ret = OB_SUCCESS;
ObInterruptCode int_code(OB_RPC_CONNECT_ERROR,
GETTID(),
from_svr_addr_,
"Dm interrupt sqc");
if (OB_FAIL(ObGlobalInterruptManager::getInstance()->interrupt(tid_, int_code))) {
LIB_LOG(WARN, "[DM] fail to send interrupt message", K(int_code), K(tid_), K_(trace_id));
}
LIB_LOG(WARN, "[DM] interrupt sqc", K(tid_), K_(trace_id));
return ret;
}
int ObSingleDfoDetectCB::do_callback()
{
int ret = OB_SUCCESS;
ret= sql::dtl::ObDTLIntermResultManager::getInstance().erase_interm_result_info(key_, false);
ret = ret == OB_HASH_NOT_EXIST ? OB_SUCCESS : ret;
LIB_LOG(WARN, "[DM] single dfo erase_interm_result_info", K(ret), K(key_), K_(trace_id));
return ret;
}
int ObTempTableDetectCB::do_callback()
{
int ret = OB_SUCCESS;
ret = sql::dtl::ObDTLIntermResultManager::getInstance().erase_interm_result_info(key_, false);
ret = ret == OB_HASH_NOT_EXIST ? OB_SUCCESS : ret;
LIB_LOG(WARN, "[DM] temp table erase_interm_result_info", K(ret), K(key_), K_(trace_id));
return ret;
}
int ObP2PDataHubDetectCB::do_callback()
{
int ret = OB_SUCCESS;
ObP2PDatahubMsgBase *msg = nullptr;
bool is_erased = false;
ret = PX_P2P_DH.erase_msg_if(key_, msg, is_erased, false/* need unregister dm */);
ret = ret == OB_HASH_NOT_EXIST ? OB_SUCCESS : ret;
LIB_LOG(WARN, "[DM] p2p dh erase p2p msg", K(ret), K(key_), K_(trace_id), K(is_erased));
return ret;
}
} // end namespace common
} // end namespace oceanbase

View File

@ -0,0 +1,173 @@
/**
* 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.
*/
#pragma once
#include "lib/net/ob_addr.h"
#include "share/interrupt/ob_global_interrupt_call.h"
#include "sql/dtl/ob_dtl_interm_result_manager.h"
#include "share/detect/ob_detectable_id.h"
#include "sql/engine/px/ob_dfo.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h"
namespace oceanbase {
namespace sql {
class ObDfo;
class ObPxSqcMeta;
namespace dtl {
class ObDtlChannel;
}
}
namespace common {
enum class ObTaskState
{
RUNNING = 0,
FINISHED = 1,
};
struct ObPeerTaskState {
ObPeerTaskState() : peer_addr_(), peer_state_(ObTaskState::RUNNING) {}
explicit ObPeerTaskState(const common::ObAddr &addr) : peer_addr_(addr), peer_state_(ObTaskState::RUNNING) {}
ObPeerTaskState(const common::ObAddr &addr, const ObTaskState &state) : peer_addr_(addr), peer_state_(state) {}
void operator=(const ObPeerTaskState &other)
{
peer_addr_ = other.peer_addr_;
peer_state_ = other.peer_state_;
}
inline int64_t to_string(char *buf, const int64_t len) const
{
int64_t pos = 0;
databuff_printf(buf, len, pos, "peer_state: (%d)", peer_state_);
pos += peer_addr_.to_string(buf + pos, len - pos);
return pos;
}
common::ObAddr peer_addr_;
ObTaskState peer_state_;
};
enum class DetectCallBackType
{
VIRTUAL = 0,
QC_DETECT_CB = 1,
SQC_DETECT_CB = 2,
SINGLE_DFO_DETECT_CB = 3,
TEMP_TABLE_DETECT_CB = 4,
P2P_DATAHUB_DETECT_CB = 5,
};
class ObIDetectCallback
{
public:
// constructor for pass peer_states from derived class
explicit ObIDetectCallback(const ObArray<ObPeerTaskState> &peer_states);
virtual void destroy()
{
peer_states_.reset();
}
virtual int do_callback() = 0;
virtual int64_t get_detect_callback_type() const = 0;
virtual bool reentrant() const { return false; }
ObArray<ObPeerTaskState> &get_peer_states() { return peer_states_; }
virtual int atomic_set_finished(const common::ObAddr &addr, ObTaskState *state=nullptr);
int64_t get_ref_count() { return ATOMIC_LOAD(&ref_count_); }
int64_t inc_ref_count(int64_t count = 1);
int64_t dec_ref_count();
void set_from_svr_addr(const common::ObAddr &from) { from_svr_addr_ = from; }
const common::ObAddr &get_from_svr_addr() { return from_svr_addr_; }
void set_trace_id(const common::ObCurTraceId::TraceId &trace_id) { trace_id_ = trace_id; }
const common::ObCurTraceId::TraceId & get_trace_id() { return trace_id_; }
bool alloc_succ() { return alloc_succ_; }
inline int64_t to_string(char *buf, const int64_t len) const { return 0; }
private:
int64_t ref_count_;
ObArray<ObPeerTaskState> peer_states_;
protected:
common::ObAddr from_svr_addr_; // in which server the task is detected as finished
common::ObCurTraceId::TraceId trace_id_;
bool alloc_succ_;
};
class ObQcDetectCB : public ObIDetectCallback
{
public:
ObQcDetectCB(const ObArray<ObPeerTaskState> &peer_states, const ObInterruptibleTaskID &tid, sql::ObDfo &dfo,
const ObArray<sql::dtl::ObDtlChannel *> &dtl_channels);
void destroy() override;
int do_callback() override;
int64_t get_detect_callback_type() const override { return (int64_t)DetectCallBackType::QC_DETECT_CB; }
// this DetectCallback is reentrant, because several sqcs can be detected as finished and should set_need_report(false)
bool reentrant() const override { return true; }
int atomic_set_finished(const common::ObAddr &addr, ObTaskState *state=nullptr) override;
private:
ObInterruptibleTaskID tid_;
sql::ObDfo &dfo_;
int64_t timeout_ts_;
ObArray<sql::dtl::ObDtlChannel *> dtl_channels_;
};
class ObSqcDetectCB : public ObIDetectCallback
{
public:
ObSqcDetectCB(const ObArray<ObPeerTaskState> &peer_states, const ObInterruptibleTaskID &tid)
: ObIDetectCallback(peer_states), tid_(tid) {}
int do_callback() override;
int64_t get_detect_callback_type() const override { return (int64_t)DetectCallBackType::SQC_DETECT_CB; }
private:
ObInterruptibleTaskID tid_;
};
class ObSingleDfoDetectCB : public ObIDetectCallback
{
public:
ObSingleDfoDetectCB(const ObArray<ObPeerTaskState> &peer_states, const sql::dtl::ObDTLIntermResultKey &key)
: ObIDetectCallback(peer_states), key_(key) {}
int do_callback() override;
int64_t get_detect_callback_type() const override { return (int64_t)DetectCallBackType::SINGLE_DFO_DETECT_CB; }
private:
sql::dtl::ObDTLIntermResultKey key_;
};
class ObTempTableDetectCB : public ObIDetectCallback
{
public:
ObTempTableDetectCB(const ObArray<ObPeerTaskState> &peer_states, const sql::dtl::ObDTLIntermResultKey &key)
: ObIDetectCallback(peer_states), key_(key) {}
int do_callback() override;
int64_t get_detect_callback_type() const override { return (int64_t)DetectCallBackType::TEMP_TABLE_DETECT_CB; }
private:
sql::dtl::ObDTLIntermResultKey key_;
};
class ObP2PDataHubDetectCB : public ObIDetectCallback
{
public:
ObP2PDataHubDetectCB(const ObArray<ObPeerTaskState> &peer_states, const sql::ObP2PDhKey &key)
: ObIDetectCallback(peer_states), key_(key) {}
int do_callback() override;
int64_t get_detect_callback_type() const override { return (int64_t)DetectCallBackType::P2P_DATAHUB_DETECT_CB; }
private:
sql::ObP2PDhKey key_;
};
} // end namespace common
} // end namespace oceanbase

View File

@ -0,0 +1,763 @@
/**
* 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 "share/detect/ob_detect_manager.h"
#include "lib/ob_running_mode.h"
#include "share/rc/ob_context.h"
namespace oceanbase {
namespace common {
struct ObDetectableIdWrapper : public common::ObLink
{
ObDetectableId detectable_id_;
int64_t activate_tm_;
};
ObDetectableIdGen::ObDetectableIdGen()
{
detect_sequence_id_ = ObTimeUtil::current_time();
callback_node_sequence_id_ = ObTimeUtil::current_time();
}
ObDetectableIdGen &ObDetectableIdGen::instance()
{
static ObDetectableIdGen d_id_gen;
return d_id_gen;
}
int ObDetectableIdGen::generate_detectable_id(ObDetectableId &detectable_id, uint64_t tenant_id)
{
int ret = OB_SUCCESS;
// use server id to ensure that the detectable_id is unique in cluster
uint64_t server_id = GCTX.server_id_;
if (0 == server_id) {
ret = OB_SERVER_IS_INIT;
LIB_LOG(WARN, "[DM] server id is invalid");
} else {
detectable_id.first_ = get_detect_sequence_id();
// use timestamp to ensure that the detectable_id is unique during the process
uint64_t timestamp = ObTimeUtility::current_time();
// [ server_id (16bits) ][ timestamp (32bits) ]
// only if qps > 2^48 or same server reboots after 2^48 the detectable_id be repeated
detectable_id.second_ = (server_id) << 48 | (timestamp & 0x0000FFFFFFFFFFFF);
detectable_id.tenant_id_ = tenant_id;
}
return ret;
}
int ObDetectManager::mtl_init(ObDetectManager *&dm)
{
int ret = OB_SUCCESS;
uint64_t tenant_id = MTL_ID();
int64_t mem_limit = lib::get_tenant_memory_limit(tenant_id);
double mem_factor = mem_limit >= MAX_TENANT_MEM_LIMIT ? 1.0 : (static_cast<double>(mem_limit) / MAX_TENANT_MEM_LIMIT);
// less memory for meta tenant
if (is_meta_tenant(tenant_id)) {
mem_factor = mem_factor * 0.01;
}
dm = OB_NEW(ObDetectManager, ObMemAttr(tenant_id, "DetectManager"), tenant_id);
if (OB_ISNULL(dm)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to alloc detect manager", K(ret));
} else if (OB_FAIL(dm->init(GCTX.self_addr(), mem_factor))) {
LIB_LOG(WARN, "[DM] failed to init detect manager", K(ret));
}
return ret;
}
void ObDetectManager::mtl_destroy(ObDetectManager *&dm)
{
if (nullptr != dm) {
dm->destroy();
ob_delete(dm);
dm = nullptr;
}
}
void ObDetectManager::destroy()
{
int ret = OB_SUCCESS;
// destroy wrapper in fifo_que_
int64_t que_size = fifo_que_.size();
for (int64_t idx = 0; idx < que_size; ++idx) {
common::ObLink *p = nullptr;
IGNORE_RETURN fifo_que_.pop(p);
ObDetectableIdWrapper *wrapper = static_cast<ObDetectableIdWrapper *>(p);
if (OB_ISNULL(wrapper)) {
LIB_LOG(WARN, "[DM] wrapper is null");
continue;
}
mem_context_->get_malloc_allocator().free(wrapper);
}
// destroy node and callback in all_check_items_
FOREACH(iter, all_check_items_) {
ObDetectCallbackNode *node = iter->second;
while (OB_NOT_NULL(node)) {
ObDetectCallbackNode *next_node = node->next_;
delete_cb_node(node);
node = next_node;
}
}
all_check_items_.destroy();
still_need_check_id_.destroy();
detectable_ids_.destroy();
DESTROY_CONTEXT(mem_context_);
LIB_LOG(INFO, "[DM] destory dm", K_(tenant_id));
}
int ObDetectManager::init(const ObAddr &self, double mem_factor)
{
int ret = OB_SUCCESS;
lib::ContextParam param;
param.set_mem_attr(tenant_id_, "DetectManager")
.set_properties(lib::ALLOC_THREAD_SAFE)
.set_parallel(4);
int64_t check_map_bucket_count = static_cast<int64_t>(DEFAULT_CHECK_MAP_BUCKETS_COUNT * mem_factor);
int64_t detectable_ids_bucket_count = static_cast<int64_t>(DEFAULT_CHECK_MAP_BUCKETS_COUNT * mem_factor);
int64_t still_need_check_id_bucket_count = static_cast<int64_t>(MIDDLE_SET_BUCKETS_COUNT * mem_factor);
if (IS_INIT) {
ret = OB_INIT_TWICE;
} else if (!self.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] invalid arguments", K(ret), K(self));
} else if (OB_FAIL(ROOT_CONTEXT->CREATE_CONTEXT(mem_context_, param))) {
LIB_LOG(WARN, "[DM] create memory context failed", K(ret));
} else if (OB_FAIL(all_check_items_.create(check_map_bucket_count,
"HashBuckDmChe",
"HashNodeDmChe",
tenant_id_))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] create hash table failed", K(ret));
} else if (OB_FAIL(detectable_ids_.create(detectable_ids_bucket_count,
"HashBuckDmId",
"HashNodeDmId",
tenant_id_))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] create hash set failed", K(ret));
} else if (OB_FAIL(still_need_check_id_.create(still_need_check_id_bucket_count,
"HashBuckDmId",
"HashNodeDmId",
tenant_id_))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] create hash set failed", K(ret));
} else {
self_ = self;
is_inited_ = true;
LIB_LOG(INFO, "[DM] ObDetectManager init success", K(self), K_(tenant_id), K(mem_factor));
}
return ret;
}
int ObDetectManager::register_detectable_id(const ObDetectableId &detectable_id)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LIB_LOG(ERROR, "[DM] detect manager not inited", K(ret));
} else if (detectable_id.is_invalid()) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] invaild detectable_id, please generate from ObDetectManager", K(ret), K(common::lbt()));
} else if (OB_FAIL(detectable_ids_.set_refactored(detectable_id, 0/* flag */))) {
if (OB_HASH_EXIST != ret) {
LIB_LOG(WARN, "[DM] detectable_ids_ set_refactored failed", K(ret));
} else {
LIB_LOG(WARN, "[DM] detectable_id already exists in detectable_ids_");
}
}
return ret;
}
int ObDetectManager::unregister_detectable_id(const ObDetectableId &detectable_id)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LIB_LOG(ERROR, "[DM] detect manager not inited", K(ret));
} else if (detectable_id.is_invalid()) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] invaild detectable_id", K(common::lbt()));
} else if (OB_FAIL(detectable_ids_.erase_refactored((detectable_id)))) {
LIB_LOG(WARN, "[DM] detectable_ids_ erase_refactored failed", K(ret), K(detectable_id));
}
return ret;
}
int ObDetectManager::do_register_check_item(const ObDetectableId &detectable_id, ObIDetectCallback *cb,
uint64_t &node_sequence_id, bool need_ref)
{
int ret = OB_SUCCESS;
ObDetectCallbackNode *cb_node = nullptr;
if (OB_ISNULL(cb)) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] invaild cb pointer");
} else if (OB_FAIL(create_cb_node(cb, cb_node))) {
cb->destroy();
mem_context_->free(cb);
LIB_LOG(WARN, "[DM] fail to create cb node", K(ret));
} else {
ObDetectableIdWrapper *wrapper = OB_NEWx(ObDetectableIdWrapper, &mem_context_->get_malloc_allocator());
LIB_LOG(DEBUG, "[DM] dm new wrapper ", K(wrapper));
if (OB_ISNULL(wrapper)) {
delete_cb_node(cb_node);
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to create wrapper");
} else {
// if need_ref is true, which means that work thread may use cb, so add ref_count in case of deleting cb during ObDetectCallbackNodeExecuteCall
// typical scene: qc may change sqc's task state in callback upon receiving the report message from sqc, but dm may free callback already
if (need_ref) {
cb->inc_ref_count();
}
node_sequence_id = cb_node->sequence_id_;
// A slightly more complicated but safe inspection operation
// Since map does not provide the operation of "create or modify", nor does it provide the ability to hold bucket locks
// Therefore, add cyclic verification to prevent the occurrence of
// thread a failed to create --> thread b erased --> thread a failed to modify
// Such a situation
do {
if (OB_HASH_EXIST == (ret = all_check_items_.set_refactored(detectable_id, cb_node))) {
ObDetectCallbackNodeAddCall add_node_call(cb_node);
ret = all_check_items_.atomic_refactored(detectable_id, add_node_call);
// If it is an empty queue, it means that another thread wants to delete the node but unexpectedly got the lock by this thread
// So do not delete, try to put again according to HASH_NOT_EXIST
if (add_node_call.is_empty()) {
ret = OB_HASH_NOT_EXIST;
}
}
} while (ret == OB_HASH_NOT_EXIST);
if (OB_SUCC(ret)) {
// For short queries, we do not need to detect
// Use a fifo que to make detect delay, all queries existed more than ACTIVATE_DELAY_TIME will be activate and be detected by dm.
wrapper->detectable_id_ = detectable_id;
wrapper->activate_tm_ = ObTimeUtility::current_time() + ACTIVATE_DELAY_TIME;
// push is never fail
IGNORE_RETURN fifo_que_.push(wrapper);
}
// hashmap may set_refactored for alloc failed
if OB_FAIL(ret) {
delete_cb_node(cb_node);
}
}
}
if (OB_SUCC(ret)) {
LIB_LOG(DEBUG, "[DM] register_check_item", K(ret), K(detectable_id),
K(cb->get_detect_callback_type()), K(node_sequence_id));
}
return ret;
}
int ObDetectManager::unregister_check_item(const ObDetectableId &detectable_id, const uint64_t &node_sequence_id)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LIB_LOG(ERROR, "[DM] detect manager not inited");
} else if (detectable_id.is_invalid()) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] invaild detectable_id", K(node_sequence_id) ,K(common::lbt()));
} else {
// CHECK_MAP key: ObDetectableId, CHECK_MAP value: linked list with node type ObDetectCallbackNode*
// remove node from CHECK_MAP with detectable_id(CHECK_MAP key) and node_sequence_id(identify specific node in the linked list)
// remove kv pair from CHECK_MAP if this node is the last one in the linked list
ObDetectCallbackNodeRemoveCall remove_node_call(this, node_sequence_id);
bool is_erased = false;
if (OB_FAIL(all_check_items_.erase_if(detectable_id, remove_node_call, is_erased))) {
if (OB_HASH_NOT_EXIST == ret) {
// if not found, the possible reason is that node is removed by ObDetectCallbackNodeExecuteCall
LIB_LOG(WARN, "[DM] unregister cb failed, maybe removed by other thread",
K(ret), K(detectable_id), K(node_sequence_id));
} else {
LIB_LOG(WARN, "[DM] unregister cb failed", K(ret), K(detectable_id), K(node_sequence_id));
}
}
}
LIB_LOG(DEBUG, "[DM] unregister_check_item", K(ret), K(detectable_id), K(node_sequence_id));
return ret;
}
int ObDetectManager::gather_requests(REQUEST_MAP &req_map, lib::MemoryContext &req_map_context)
{
int ret = OB_SUCCESS;
int64_t que_size = fifo_que_.size();
int64_t cur_time = ObTimeUtil::current_time();
// don't break if failed
for (int64_t idx = 0; idx < que_size; ++idx) {
common::ObLink *p = nullptr;
IGNORE_RETURN fifo_que_.pop(p);
ObDetectableIdWrapper *wrapper = static_cast<ObDetectableIdWrapper *>(p);
if (OB_ISNULL(wrapper)) {
LIB_LOG(WARN, "[DM] wrapper is null");
continue;
} else if (wrapper->activate_tm_ > cur_time) {
// the check item is not activated, push to fifo que again
// push_front never failed, because wrapper is not null
IGNORE_RETURN fifo_que_.push_front(wrapper);
break;
}
// push all activated check item into still_need_check_id_
if (OB_FAIL(still_need_check_id_.set_refactored(wrapper->detectable_id_, 0/* flag */))) {
if (OB_HASH_EXIST != ret) {
LIB_LOG(WARN, "[DM] failed to set_refactored", K(ret), K(wrapper->detectable_id_));
} else {
ret = OB_SUCCESS;
}
}
mem_context_->get_malloc_allocator().free(wrapper);
LIB_LOG(DEBUG, "[DM] dm free wrapper ", K(wrapper));
}
ObSEArray<ObDetectableId, 32> remove_list;
FOREACH(iter, still_need_check_id_) {
const ObDetectableId &detectable_id = iter->first;
ObDetectReqGetCall req_get_call(req_map, req_map_context);
ret = all_check_items_.read_atomic(detectable_id, req_get_call);
if (OB_HASH_NOT_EXIST == ret) {
// already unregistered, remove it. if push_back failed, let next detect loop do this
if (OB_FAIL(remove_list.push_back(detectable_id))) {
LIB_LOG(WARN, "[DM] failed to push_back to remove_list", K(ret));
}
ret = OB_SUCCESS;
}
}
ARRAY_FOREACH_NORET(remove_list, idx) {
if (OB_FAIL(still_need_check_id_.erase_refactored(remove_list.at(idx)))) {
LIB_LOG(WARN, "[DM] failed to erase_refactored from still_need_check_id_");
}
}
LIB_LOG(DEBUG, "[DM] gather_requests ", K(req_map.size()), K(ret));
return ret;
}
void ObDetectManager::do_detect_local(const ObDetectableId &detectable_id)
{
int ret = OB_SUCCESS;
bool task_alive = is_task_alive(detectable_id);
if (!task_alive) {
ObCheckStateFinishCall check_state_finish_call(self_);
if (OB_SUCC(all_check_items_.read_atomic(detectable_id, check_state_finish_call))) {
if (check_state_finish_call.is_finished()) {
// nop
} else {
// peer exits unexpectly, do detect callbacks
ObDetectCallbackNodeExecuteCall execute_call(this, detectable_id, self_);
bool is_erased = false;
IGNORE_RETURN all_check_items_.erase_if(detectable_id, execute_call, is_erased);
}
}
}
}
void ObDetectManager::do_handle_one_result(const ObDetectableId &detectable_id, const obrpc::ObDetectRpcStatus &rpc_status)
{
int ret = OB_SUCCESS;
ObCheckStateFinishCall check_state_finish_call(rpc_status.dst_);
if (OB_SUCC(all_check_items_.read_atomic(detectable_id, check_state_finish_call))) {
if (check_state_finish_call.is_finished()) {
// nop
} else {
ObDetectCallbackNodeExecuteCall execute_call(this, detectable_id, rpc_status.dst_);
bool is_erased = false;
IGNORE_RETURN all_check_items_.erase_if(detectable_id, execute_call, is_erased);
}
}
}
int ObDetectManager::create_cb_node(ObIDetectCallback *cb, ObDetectCallbackNode *&cb_node)
{
int ret = OB_SUCCESS;
// each node has its own unique node_sequence_id for identify itself in the linked list
cb_node = OB_NEWx(ObDetectCallbackNode, &mem_context_->get_malloc_allocator(),
cb, ObDetectableIdGen::instance().get_callback_node_sequence_id());
if (OB_ISNULL(cb_node)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to new cb_node");
}
LIB_LOG(DEBUG, "[DM] dm new cb node ", K(cb_node));
return ret;
}
void ObDetectManager::delete_cb_node(ObDetectCallbackNode *&cb_node)
{
int ret = OB_SUCCESS;
LIB_LOG(DEBUG, "[DM] dm free cb ", K(cb_node->cb_));
cb_node->cb_->destroy();
mem_context_->free(cb_node->cb_);
cb_node->cb_ = nullptr;
LIB_LOG(DEBUG, "[DM] dm free cbnode ", K(cb_node));
mem_context_->free(cb_node);
cb_node = nullptr;
}
void ObDetectManager::ObDetectCallbackNodeAddCall::operator()(hash::HashMapPair<ObDetectableId,
ObDetectCallbackNode *> &entry)
{
// The map only stores the head of the linked list, so add it directly from the head
if (entry.second != nullptr) {
cb_node_->next_ = entry.second;
entry.second->prev_ = cb_node_;
entry.second = cb_node_;
is_empty_ = false;
} else {
is_empty_ = true;
}
}
bool ObDetectManager::ObDetectCallbackNodeRemoveCall::operator()(hash::HashMapPair<ObDetectableId,
ObDetectCallbackNode *> &entry)
{
auto node = entry.second;
int node_cnt = 0; // node_cnt in linked list, only if node_cnt == 0 the kv pair can be deleted.
while (OB_NOT_NULL(node)) {
node_cnt++;
if (node_sequence_id_ == node->sequence_id_) {
found_node_ = true;
if (node->next_ != nullptr) {
node->next_->prev_ = node->prev_;
// loop will be break after found node, but next node is not null
// add node_cnt to prevent to delete the kv pair
node_cnt++;
}
if (node->prev_ != nullptr) {
node->prev_->next_ = node->next_;
} else {
// Prev is empty, which means that the current deleted element is the head of the linked list pointed to by map_value, and head is set to next
entry.second = node->next_;
}
dm_->delete_cb_node(node);
node_cnt--;
break;
}
node = node->next_;
}
// 0 == node_cnt means that the linked list is empty, return true so that erase_if can erase the kv pair
return (0 == node_cnt);
}
bool ObDetectManager::ObDetectCallbackNodeExecuteCall::operator()(hash::HashMapPair<ObDetectableId,
ObDetectCallbackNode *> &entry)
{
int ret = OB_SUCCESS;
const ObDetectableId &detectable_id = entry.first;
ObDetectCallbackNode *node = entry.second;
int node_cnt = 0; // node_cnt in linked list, remove kv pair if the last node is removed
while (OB_NOT_NULL(node)) {
node_cnt++;
ObDetectCallbackNode *next_node = node->next_;
// if a callback can be reentrant, don't set is_executed so that it can be do for several times
// typical scene: qc detects sqc, if at least 2 sqc failed, both of them should be set not_need_report,
// the callback should be reentrant and node not set to executed
if (!node->is_executed()) {
if (!node->cb_->reentrant()) {
node->set_executed();
}
LIB_LOG(WARN, "[DM] DM found peer not exist, execute detect callback",
K(node->cb_->get_trace_id()), K(from_svr_addr_),
K(detectable_id), K(node->cb_->get_detect_callback_type()), K(node->sequence_id_));
node->cb_->set_from_svr_addr(from_svr_addr_);
if (OB_FAIL(node->cb_->do_callback())) {
LIB_LOG(WARN, "[DM] failed to do_callback",
K(node->cb_->get_trace_id()), K(from_svr_addr_), K(detectable_id),
K(node->cb_->get_detect_callback_type()), K(node->sequence_id_));
}
// ref_count > 0 means that cb is still referred by work thread, don‘t remove it from the linked list
int64_t ref_count = node->cb_->get_ref_count();
if (0 == ref_count) {
if (node->next_ != nullptr) {
node->next_->prev_ = node->prev_;
}
if (node->prev_ != nullptr) {
node->prev_->next_ = node->next_;
} else {
// Prev is empty, which means that the current deleted element is the head of the linked list pointed to by map_value, and head is set to next
entry.second = node->next_;
}
dm_->delete_cb_node(node);
node_cnt--;
}
}
node = next_node;
}
return 0 == node_cnt;
}
void ObDetectManager::ObCheckStateFinishCall::operator()(hash::HashMapPair<ObDetectableId,
ObDetectCallbackNode *> &entry)
{
ObDetectCallbackNode *node = entry.second;
// check task has already been marked as finished
while (OB_NOT_NULL(node)) {
ObTaskState state;
if (OB_SUCCESS == node->cb_->atomic_set_finished(addr_, &state)) {
if (ObTaskState::FINISHED == state) {
finished_ = true;
}
break;
}
node = node->next_;
}
}
void ObDetectManager::ObDetectReqGetCall::operator()(hash::HashMapPair<ObDetectableId, ObDetectCallbackNode *> &entry)
{
int ret = OB_SUCCESS;
const ObDetectableId &peer_id = entry.first;
ObDetectCallbackNode *cb_node = entry.second;
while (OB_NOT_NULL(cb_node)) {
if (!cb_node->is_executed()) {
ObArray<ObPeerTaskState> &peer_states = cb_node->cb_->get_peer_states();
ARRAY_FOREACH_NORET(peer_states, idx) {
ObPeerTaskState &peer_state = peer_states.at(idx);
// only detect running tasks
if ((int32_t)ObTaskState::FINISHED == ATOMIC_LOAD((int32_t*)&peer_state)) {
continue;
}
obrpc::ObTaskStateDetectReq **req_ptr_ptr = req_map_.get(peer_state.peer_addr_);
if (OB_NOT_NULL(req_ptr_ptr)) {
if (OB_FAIL((*req_ptr_ptr)->peer_ids_.set_refactored(peer_id, 0/* flag */))) {
if (OB_HASH_EXIST != ret) {
LIB_LOG(WARN, "[DM] peer_ids_ set_refactored failed", K(peer_id),
K(peer_state.peer_addr_), K(cb_node->cb_->get_trace_id()));
} else {
// this peer id has already been set, ignore
ret = OB_SUCCESS;
}
}
} else {
obrpc::ObTaskStateDetectReq *req_ptr =
OB_NEWx(obrpc::ObTaskStateDetectReq, &req_map_context_->get_arena_allocator());
if (OB_ISNULL(req_ptr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to new ObTaskStateDetectReq", K(peer_id),
K(peer_state.peer_addr_), K(cb_node->cb_->get_trace_id()));
} else if (OB_FAIL(req_ptr->peer_ids_.set_refactored(peer_id, 0/* flag */))) {
LIB_LOG(WARN, "[DM] peer_ids_ set_refactored failed", K(peer_id),
K(peer_state.peer_addr_), K(cb_node->cb_->get_trace_id()));
} else if (OB_FAIL(req_map_.set_refactored(peer_state.peer_addr_, req_ptr))) {
LIB_LOG(WARN, "[DM] req_map_ set req failed", K(peer_state.peer_addr_),
K(ret), K(peer_state.peer_addr_), K(cb_node->cb_->get_trace_id()));
}
}
}
}
cb_node = cb_node->next_;
}
}
ObDetectManagerThread &ObDetectManagerThread::instance()
{
static ObDetectManagerThread dm_thread;
return dm_thread;
}
int ObDetectManagerThread::init(const ObAddr &self, rpc::frame::ObReqTransport *transport)
{
int ret = OB_SUCCESS;
if (IS_INIT) {
ret = OB_INIT_TWICE;
} else if (!self.is_valid() || nullptr == transport) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] invalid arguments", K(ret), K(self), KP(transport));
} else if (OB_FAIL(rpc_proxy_.init(transport, self))) {
LIB_LOG(WARN, "[DM] rpc_proxy_.init failed", K(ret));
} else if (OB_FAIL(req_map_.create(DEFAULT_REQUEST_MAP_BUCKETS_COUNT,
"HashBuckDmReq",
"HashNodeDmReq"))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] create hash set failed", K(ret));
} else if (OB_FAIL(cond_.init(common::ObWaitEventIds::DEFAULT_COND_WAIT))) {
LIB_LOG(WARN, "[DM] failed to init cond_", K(ret));
} else if (OB_FAIL(TG_SET_RUNNABLE_AND_START(lib::TGDefIDs::DetectManager, *this))) {
LIB_LOG(WARN, "[DM] schedule detect without timer failed", K(ret));
} else {
self_ = self;
is_inited_ = true;
LIB_LOG(INFO, "[DM] ObDetectManagerThread init success", K(self));
}
return ret;
}
void ObDetectManagerThread::run1()
{
lib::set_thread_name("ObDetectManagerThread");
IGNORE_RETURN detect();
}
int ObDetectManagerThread::detect() {
static int loop_time = 0;
int ret = OB_SUCCESS;
while (!has_set_stop()) {
ret = OB_SUCCESS;
const int64_t start_time = ObTimeUtility::current_time();
int64_t send_cnt = 0; // mark rpc cnt
// 1. Gather all request from CHECK_MAP into REQUEST_MAP, clustered by addr.
// 2. Send all requests ip by ip. If is local, directly access detectable_ids_.
// 3. Process rpc result
// All requests are allocated from TEMP_CONTEXT to simplify memory management.
CREATE_WITH_TEMP_CONTEXT(lib::ContextParam().set_label(ObModIds::OB_TEMP_VARIABLES)) {
lib::MemoryContext &temp_mem_context = CURRENT_CONTEXT;
common::ObArray<uint64_t> tenant_ids;
if (OB_ISNULL(GCTX.omt_)) {
ret = OB_ERR_UNEXPECTED;
LIB_LOG(WARN, "[DM] unexpected null of GCTX.omt_", K(ret));
} else if (OB_FAIL(GCTX.omt_->get_mtl_tenant_ids(tenant_ids))) {
LIB_LOG(WARN, "[DM] fail to get_mtl_tenant_ids", K(ret));
} else {
for (int64_t i = 0; i < tenant_ids.size(); i++) {
MTL_SWITCH(tenant_ids.at(i)) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(tenant_ids.at(i)));
} else if (OB_FAIL(dm->gather_requests(req_map_, temp_mem_context))) {
LIB_LOG(WARN, "[DM] failed to gather_requests", K(tenant_ids.at(i)));
}
}
// ignore errors at switching tenant
ret = OB_SUCCESS;
}
if (FALSE_IT(send_requests(req_map_, send_cnt, temp_mem_context))) {
} else if (FALSE_IT(handle_rpc_results(send_cnt, temp_mem_context))) {
}
FOREACH(it, req_map_) {
it->second->destroy();
}
}
}
req_map_.reuse();
const int64_t cost_time = ObTimeUtility::current_time() - start_time;
if (cost_time > DETECT_COST_TIME_THRESHOLD) {
LIB_LOG_RET(WARN, OB_ERR_TOO_MUCH_TIME, "[DM] detect_loop_ cost too much time", K(cost_time));
}
int32_t sleep_time = DETECT_INTERVAL - static_cast<const int32_t>(cost_time);
if (sleep_time < 0) {
sleep_time = 0;
} else {
ob_usleep(sleep_time);
}
++loop_time;
LIB_LOG(DEBUG, "[DM] detect has execute ", K(loop_time));
}
return ret;
}
void ObDetectManagerThread::send_requests(REQUEST_MAP &req_map, int64_t &send_cnt, lib::MemoryContext &mem_context)
{
FOREACH(iter, req_map) {
const common::ObAddr &dst = iter->first;
obrpc::ObTaskStateDetectReq *req = iter->second;
if (dst == self_) {
detect_local(req);
} else {
detect_remote(dst, req, send_cnt, mem_context);
}
}
}
void ObDetectManagerThread::detect_local(const obrpc::ObTaskStateDetectReq *req)
{
int ret = OB_SUCCESS;
FOREACH(iter, req->peer_ids_) {
const ObDetectableId &detectable_id = iter->first;
MTL_SWITCH(detectable_id.tenant_id_) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(detectable_id.tenant_id_));
} else {
dm->do_detect_local(detectable_id);
}
}
}
}
void ObDetectManagerThread::detect_remote(const common::ObAddr &dst,
const obrpc::ObTaskStateDetectReq *req, int64_t &send_cnt, lib::MemoryContext &mem_context)
{
int ret = OB_SUCCESS;
obrpc::ObDetectRpcStatus *rpc_status = OB_NEWx(obrpc::ObDetectRpcStatus, &mem_context->get_malloc_allocator(), dst);
LIB_LOG(DEBUG, "[DM] dm new rpc_status");
if (OB_ISNULL(rpc_status)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to allocate rpc_status");
} else if (OB_FAIL(rpc_statuses_.push_back(rpc_status))) {
rpc_status->destroy();
mem_context->free(rpc_status);
LIB_LOG(WARN, "[DM] failed to push_back rpc_status into rpc_statuses_");
} else {
obrpc::ObTaskStateDetectAsyncCB cb(cond_, rpc_status, dst);
if (OB_FAIL(rpc_proxy_.to(dst).by(OB_SYS_TENANT_ID).timeout(DETECT_MSG_TIMEOUT).detect_task_state(*req, &cb))) {
rpc_statuses_.pop_back();
rpc_status->destroy();
mem_context->free(rpc_status);
LIB_LOG(WARN, "[DM] failed do rpc detect_task_state");
} else {
++send_cnt;
}
}
}
void ObDetectManagerThread::handle_rpc_results(int64_t &send_cnt, lib::MemoryContext &mem_context)
{
int ret = OB_SUCCESS;
int64_t return_cb_cnt = 0;
while (return_cb_cnt < send_cnt) {
ObThreadCondGuard guard(cond_);
// wait for timeout or until notified.
cond_.wait_us(COND_WAIT_TIME_USEC);
ARRAY_FOREACH_NORET(rpc_statuses_, idx) {
obrpc::ObDetectRpcStatus &rpc_status = *rpc_statuses_.at(idx);
if (!rpc_status.is_visited() && rpc_status.is_timeout()) {
return_cb_cnt++;
rpc_status.set_visited(true);
} else if (!rpc_status.is_visited() && rpc_status.is_processed()) {
return_cb_cnt++;
rpc_status.set_visited(true);
IGNORE_RETURN handle_one_result(rpc_status);
}
}
}
ARRAY_FOREACH_NORET(rpc_statuses_, idx) {
obrpc::ObDetectRpcStatus *rpc_status = rpc_statuses_.at(idx);
LIB_LOG(DEBUG, "[DM] dm free rpc_status");
rpc_status->destroy();
mem_context->get_malloc_allocator().free(rpc_status);
}
rpc_statuses_.reset();
}
void ObDetectManagerThread::handle_one_result(const obrpc::ObDetectRpcStatus &rpc_status)
{
int ret = OB_SUCCESS;
const obrpc::ObTaskStateDetectResp &cb_result = rpc_status.response_;
ARRAY_FOREACH_NORET(cb_result.task_infos_, idx) {
const obrpc::TaskInfo &task = cb_result.task_infos_.at(idx);
if (common::ObTaskState::FINISHED == task.task_state_) {
const ObDetectableId &detectable_id = task.task_id_;
MTL_SWITCH(detectable_id.tenant_id_) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(detectable_id.tenant_id_));
} else {
dm->do_handle_one_result(detectable_id, rpc_status);
}
}
}
}
}
} // end namespace common
} // end namespace oceanbase

View File

@ -0,0 +1,276 @@
/**
* 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.
*/
#pragma once
#include "share/detect/ob_detectable_id.h"
#include "share/detect/ob_detect_callback.h"
#include "share/detect/ob_detect_rpc_proxy.h"
#include "lib/container/ob_array.h"
#include "lib/hash/ob_hashset.h"
#include "lib/queue/ob_link_queue.h"
#include "lib/thread/thread_mgr_interface.h"
namespace oceanbase {
namespace common {
struct ObDetectCallbackNode
{
ObDetectCallbackNode(ObIDetectCallback *cb, uint64_t sequence_id) :
next_(nullptr), prev_(nullptr), cb_(cb), sequence_id_(sequence_id), executed_(false) {}
ObDetectCallbackNode() : next_(nullptr), prev_(nullptr), cb_(nullptr) {}
~ObDetectCallbackNode() = default;
void set_executed() { executed_ = true; }
bool is_executed() { return executed_; }
ObDetectCallbackNode *next_;
ObDetectCallbackNode *prev_;
ObIDetectCallback *cb_;
uint64_t sequence_id_;
bool executed_;
};
class ObDetectableIdGen
{
public:
static ObDetectableIdGen &instance();
uint64_t get_detect_sequence_id() { return ATOMIC_AAF(&detect_sequence_id_, 1); }
uint64_t get_callback_node_sequence_id() { return ATOMIC_AAF(&callback_node_sequence_id_, 1); }
int generate_detectable_id(ObDetectableId &detectable_id, uint64_t tenant_id);
private:
DISALLOW_COPY_AND_ASSIGN(ObDetectableIdGen);
ObDetectableIdGen();
volatile uint64_t detect_sequence_id_; // for DetectableId
volatile uint64_t callback_node_sequence_id_; // for mark specific node in linked list(value of CHECK_MAP)
};
static const int64_t DEFAULT_REQUEST_MAP_BUCKETS_COUNT = 100; //100
typedef hash::ObHashMap<common::ObAddr, obrpc::ObTaskStateDetectReq *,
hash::SpinReadWriteDefendMode,
hash::hash_func<common::ObAddr>,
hash::equal_to<common::ObAddr>> REQUEST_MAP;
// design doc:
class ObDetectManager
{
private:
static const int64_t MAX_TENANT_MEM_LIMIT = 17179869184; //16G
// for CHECK_MAP
static const int64_t DEFAULT_CHECK_MAP_BUCKETS_COUNT = 1000000; //100w
static const int64_t MINI_MODE_CHECK_MAP_BUCKETS_COUNT = 10000; //1w
// for detectable_ids_
static const int64_t DEFAULT_SET_BUCKETS_COUNT = 1000000; //100w
static const int64_t MINI_MODE_SET_BUCKETS_COUNT = 10000; //1w
// for still_need_check_id_
static const int64_t MIDDLE_SET_BUCKETS_COUNT = 100000; //10w
static const uint64_t ACTIVATE_DELAY_TIME = 5 * 1000L * 1000L; // dm only detects checkitems that have been present for at least "ACTIVATE_DELAY_TIME" seconds
public:
static int mtl_init(ObDetectManager *&dm);
static void mtl_destroy(ObDetectManager *&dm);
public:
/* tool classes */
typedef hash::ObHashMap<ObDetectableId, ObDetectCallbackNode *,
hash::SpinReadWriteDefendMode, hash::hash_func<ObDetectableId>,
hash::equal_to<ObDetectableId>> CHECK_MAP;
/// Atomic insertion callback
class ObDetectCallbackNodeAddCall
{
public:
void operator()(hash::HashMapPair<ObDetectableId, ObDetectCallbackNode *> &entry);
explicit ObDetectCallbackNodeAddCall(ObDetectCallbackNode *cb_node) :
cb_node_(cb_node), is_empty_(false) {};
inline bool is_empty()
{
return is_empty_;
}
private:
ObDetectCallbackNode *cb_node_;
bool is_empty_;
};
/// Atomic removal callback, lock the bucket to avoid reading and inserting operations during removal
class ObDetectCallbackNodeRemoveCall
{
public:
bool operator()(hash::HashMapPair<ObDetectableId, ObDetectCallbackNode *> &entry);
ObDetectCallbackNodeRemoveCall(ObDetectManager *dm, const uint64_t &node_sequence_id)
: found_node_(false), dm_(dm), node_sequence_id_(node_sequence_id) {};
bool is_found_node() { return found_node_; };
private:
bool found_node_;
ObDetectManager *dm_;
uint64_t node_sequence_id_;
};
// Atomic execute
class ObDetectCallbackNodeExecuteCall
{
public:
bool operator()(hash::HashMapPair<ObDetectableId, ObDetectCallbackNode *> &entry);
ObDetectCallbackNodeExecuteCall(ObDetectManager *dm, const ObDetectableId &detectable_id,
const common::ObAddr &addr) : dm_(dm), detectable_id_(detectable_id), from_svr_addr_(addr) {}
private:
friend class ObDetectManager;
ObDetectManager *dm_;
ObDetectableId detectable_id_;
common::ObAddr from_svr_addr_;
};
// Atomic check task is finish
class ObCheckStateFinishCall
{
public:
void operator()(hash::HashMapPair<ObDetectableId, ObDetectCallbackNode *> &entry);
explicit ObCheckStateFinishCall(const common::ObAddr &addr) : addr_(addr), finished_(false) {}
bool is_finished() { return finished_; }
private:
const common::ObAddr &addr_;
bool finished_;
};
// Atomic get request
class ObDetectReqGetCall
{
public:
ObDetectReqGetCall(REQUEST_MAP &req_map, lib::MemoryContext &req_map_context) : req_map_(req_map), req_map_context_(req_map_context) {};
void operator()(hash::HashMapPair<ObDetectableId, ObDetectCallbackNode *> &entry);
private:
REQUEST_MAP &req_map_;
lib::MemoryContext &req_map_context_;
};
public:
ObDetectManager(uint64_t tenant_id) : tenant_id_(tenant_id), mem_context_(nullptr), is_inited_(false) {}
int init(const ObAddr &self, double mem_factor);
void destroy();
int register_detectable_id(const ObDetectableId &detectable_id);
int unregister_detectable_id(const ObDetectableId &detectable_id);
// @param node_sequence_id, return the unique id of node for unregister
// @param need_ref, if is true, means that work thread want to use callback.
// will add ref_count for callbak so that dm can't free it until ref_count changes to 0
// typical scene: qc detects sqc, qc wants to moidfy the state of sqc in callback while callback be freed by dm, add ref_count is necessary
// @param args, used in constructor of T
template <typename T, typename... U>
int register_check_item(const ObDetectableId &detectable_id, T *&cb,
uint64_t &node_sequence_id, bool need_ref, U& ... args)
{
int ret = OB_SUCCESS;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
LIB_LOG(ERROR, "[DM] detect manager not inited", K(ret));
} else if (OB_UNLIKELY(nullptr != cb)) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] cb must allocated from detect manager", K(cb));
} else if (detectable_id.is_invalid()) {
ret = OB_INVALID_ARGUMENT;
LIB_LOG(WARN, "[DM] invaild detectable_id", K(common::lbt()));
} else {
T* ptr = NULL;
ObIAllocator &allocator = get_mem_context()->get_malloc_allocator();
void *buf = allocator.alloc(sizeof(T));
if (OB_NOT_NULL(buf)) {
ptr = new(buf) T(args...);
if (!ptr->alloc_succ()) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to new cb ", K(ptr));
} else {
ObCurTraceId::TraceId *cur_thread_id = ObCurTraceId::get_trace_id();
ptr->set_trace_id(*cur_thread_id);
LIB_LOG(DEBUG, "[DM] dm new cb ", K(ptr));
if (OB_FAIL(do_register_check_item(detectable_id, ptr, node_sequence_id, need_ref))) {
LIB_LOG(WARN, "[DM] failed to register_check_item", K(ptr));
} else {
cb = ptr;
}
}
} else {
ret = OB_ALLOCATE_MEMORY_FAILED;
LIB_LOG(WARN, "[DM] failed to new cb ", K(ptr));
}
}
return ret;
}
// @param detectable_id, key of CHECK_MAP
// @param node_sequence_id, mark which node to be removed in the linklist(value of CHECK_MAP)
int unregister_check_item(const ObDetectableId &detectable_id, const uint64_t &node_sequence_id);
bool is_task_alive(const ObDetectableId &detectable_id)
{
return OB_HASH_EXIST == detectable_ids_.exist_refactored(detectable_id);
}
lib::MemoryContext &get_mem_context() { return mem_context_; }
private:
int do_register_check_item(const ObDetectableId &detectable_id, ObIDetectCallback *cb,
uint64_t &node_sequence_id, bool need_ref = false);
int create_cb_node(ObIDetectCallback *cb, ObDetectCallbackNode *&cb_node);
void delete_cb_node(ObDetectCallbackNode *&cb_node);
int gather_requests(REQUEST_MAP &req_map, lib::MemoryContext &req_map_context);
void do_detect_local(const ObDetectableId &detectable_id);
void do_handle_one_result(const ObDetectableId &detectable_id, const obrpc::ObDetectRpcStatus &rpc_status);
private:
DISALLOW_COPY_AND_ASSIGN(ObDetectManager);
friend class ObDetectManagerThread;
hash::ObHashSet<ObDetectableId, hash::SpinReadWriteDefendMode> detectable_ids_;
ObLinkQueue fifo_que_;
CHECK_MAP all_check_items_;
// still_need_check_id_ only operated by dm's detect thread, there is no data race.
hash::ObHashSet<ObDetectableId, hash::NoPthreadDefendMode> still_need_check_id_;
ObAddr self_;
uint64_t tenant_id_;
lib::MemoryContext mem_context_;
bool is_inited_;
};
class ObDetectManagerThread : public lib::TGRunnable
{
private:
static const int64_t DETECT_MSG_TIMEOUT = 1 * 1000 * 1000; // 1 second
static const uint64_t COND_WAIT_TIME_USEC = 500; // 500 us
static const uint64_t DETECT_COST_TIME_THRESHOLD = 2 * 1000L * 1000L; // for log only
static const uint64_t DETECT_INTERVAL = 10 * 1000L * 1000L; // detect loop interval, default 10 seconds
public:
public:
static ObDetectManagerThread &instance();
int init(const ObAddr &self, rpc::frame::ObReqTransport *transport);
void run1() override;
private:
ObDetectManagerThread() : self_(), is_inited_(false) {}
int detect();
void send_requests(REQUEST_MAP &req_map, int64_t &send_cnt, lib::MemoryContext &mem_context);
void detect_local(const obrpc::ObTaskStateDetectReq *req);
void detect_remote(const common::ObAddr &dst, const obrpc::ObTaskStateDetectReq *req, int64_t &send_cnt, lib::MemoryContext &mem_context);
void handle_rpc_results(int64_t &send_cnt, lib::MemoryContext &mem_context);
void handle_one_result(const obrpc::ObDetectRpcStatus &rpc_status);
private:
DISALLOW_COPY_AND_ASSIGN(ObDetectManagerThread);
REQUEST_MAP req_map_;
obrpc::ObDetectRpcProxy rpc_proxy_;
ObArray<obrpc::ObDetectRpcStatus *> rpc_statuses_;
ObAddr self_;
bool is_inited_;
ObThreadCond cond_;
};
} // end namespace common
} // end namespace oceanbase

View File

@ -0,0 +1,343 @@
/**
* 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 "share/detect/ob_detect_manager_utils.h"
#include "share/detect/ob_detect_manager.h"
#include "sql/engine/px/ob_dfo.h"
#include "share/interrupt/ob_global_interrupt_call.h"
#include "sql/dtl/ob_dtl_interm_result_manager.h"
#include "sql/engine/px/ob_px_sqc_handler.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h"
using namespace oceanbase::sql;
using namespace oceanbase::sql::dtl;
namespace oceanbase {
namespace common {
int ObDetectManagerUtils::generate_detectable_id(ObDetectableId &detectable_id, uint64_t tenant_id) {
return ObDetectableIdGen::instance().generate_detectable_id(detectable_id, tenant_id);
}
void ObDetectManagerUtils::prepare_register_dm_info(common::ObRegisterDmInfo &register_dm_info,
ObPxSqcHandler *handler)
{
register_dm_info.detectable_id_ = handler->get_sqc_init_arg().sqc_.get_px_detectable_ids().qc_detectable_id_;
register_dm_info.addr_ = handler->get_sqc_init_arg().sqc_.get_qc_addr();
}
int ObDetectManagerUtils::qc_register_detectable_id_into_dm(ObDetectableId &detectable_id,
bool &register_detectable_id,
uint64_t tenant_id)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObDetectManagerUtils::generate_detectable_id(detectable_id, tenant_id))) {
LIB_LOG(WARN, "[DM] failed to generate_detectable_id", K(tenant_id));
} else {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(tenant_id));
} else if (OB_FAIL(dm->register_detectable_id(detectable_id))) {
LIB_LOG(WARN, "[DM] QC fail to register detectable_id", K(ret), K(detectable_id), K(tenant_id));
} else {
register_detectable_id = true;
}
LIB_LOG(TRACE, "[DM] QC register detectable_id_", K(ret), K(detectable_id), K(tenant_id));
}
return ret;
}
void ObDetectManagerUtils::qc_unregister_detectable_id_from_dm(const ObDetectableId &detectable_id,
bool &register_detectable_id)
{
if (register_detectable_id) {
int ret = OB_SUCCESS;
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(detectable_id));
} else {
ret = dm->unregister_detectable_id(detectable_id);
}
if (OB_SUCCESS != ret && OB_HASH_NOT_EXIST != ret) {
LIB_LOG(WARN, "[DM] QC failed to unregister_detectable_id", K(ret), K(detectable_id));
}
LIB_LOG(TRACE, "[DM] QC unregister detectable_id_", K(ret), K(detectable_id));
register_detectable_id = false;
}
}
int ObDetectManagerUtils::qc_register_check_item_into_dm(ObDfo &dfo,
const ObArray<ObPeerTaskState> &peer_states,
const ObArray<ObDtlChannel *> &dtl_channels)
{
int ret = OB_SUCCESS;
ObDetectableId sqc_detectable_id = dfo.get_px_detectable_ids().sqc_detectable_id_;
ObQcDetectCB *cb = nullptr;
uint64_t node_sequence_id = 0;
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(sqc_detectable_id));
} else if (OB_SUCC(dm->register_check_item(
sqc_detectable_id, cb, node_sequence_id, true,
peer_states, dfo.get_interrupt_id().query_interrupt_id_, dfo, dtl_channels))) {
dfo.set_detect_cb(cb);
dfo.set_node_sequence_id(node_sequence_id);
} else {
LIB_LOG(WARN, "[DM] qc failed register check item", K(sqc_detectable_id));
}
return ret;
}
void ObDetectManagerUtils::qc_unregister_check_item_from_dm(sql::ObDfo *dfo, ObDetectManager* dm)
{
int ret = OB_SUCCESS;
ObIDetectCallback *cb = dfo->get_detect_cb();
const common::ObDetectableId &sqc_detectable_id = dfo->get_px_detectable_ids().sqc_detectable_id_;
if (OB_NOT_NULL(cb)) {
// only if ref_count_ decrease to 0, unregister cb
if (0 == cb->dec_ref_count()) {
if (OB_ISNULL(dm)) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(sqc_detectable_id));
} else {
int ret = dm->unregister_check_item(sqc_detectable_id, dfo->get_node_sequence_id());
if (OB_SUCCESS != ret && OB_HASH_NOT_EXIST != ret) {
LIB_LOG(WARN, "[DM] qc failed to unregister_check_item ", K(ret),
K(sqc_detectable_id), K(dfo->get_node_sequence_id()));
}
}
} else {
int ret = dm->unregister_check_item(sqc_detectable_id, dfo->get_node_sequence_id());
if (OB_SUCCESS != ret && OB_HASH_NOT_EXIST != ret) {
LIB_LOG(WARN, "[DM] qc failed to unregister_check_item ", K(ret),
K(sqc_detectable_id), K(dfo->get_node_sequence_id()));
}
}
}
dfo->set_detect_cb(nullptr);
}
}
void ObDetectManagerUtils::qc_unregister_all_check_items_from_dm(const ObIArray<ObDfo *> &dfos)
{
int ret = OB_SUCCESS;
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null");
} else {
for (int64_t idx = 0; idx < dfos.count(); ++idx) {
ObDfo *dfo = dfos.at(idx);
if (OB_NOT_NULL(dfo)) {
ObDetectManagerUtils::qc_unregister_check_item_from_dm(dfo, dm);
}
}
}
}
int ObDetectManagerUtils::sqc_register_into_dm(ObPxSqcHandler *sqc_handler, ObPxSqcMeta &sqc)
{
int ret = OB_SUCCESS;
const ObDetectableId &sqc_detectable_id = sqc.get_px_detectable_ids().sqc_detectable_id_;
const ObDetectableId &qc_detectable_id = sqc.get_px_detectable_ids().qc_detectable_id_;
// 1.register sqc's state as running, MUST be success or qc be interrupted by dm,
// must be success or sql being interrupted unexpectedly
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null");
} else if (OB_FAIL(dm->register_detectable_id(sqc_detectable_id))) {
LIB_LOG(WARN, "[DM] sqc failed to register_detectable_id", K(ret), K(sqc_detectable_id), K(qc_detectable_id));
} else {
// 2.register a check item for detecting qc,
// can be failed and not return error code, only drop the ability to detect qc
ObArray<ObPeerTaskState> peer_states;
if (OB_SUCCESS != peer_states.push_back(ObPeerTaskState(sqc.get_qc_addr()))) {
LIB_LOG(WARN, "[DM] failed to push_back", K(ret), K(sqc_detectable_id), K(qc_detectable_id));
} else {
uint64_t node_sequence_id = 0;
ObSqcDetectCB *cb = nullptr;
const ObInterruptibleTaskID &px_interrupt_id = sqc.get_interrupt_id().px_interrupt_id_;
int reg_ret = dm->register_check_item(
qc_detectable_id, cb, node_sequence_id, false, peer_states, px_interrupt_id);
if (OB_SUCCESS != reg_ret) {
LIB_LOG(WARN, "[DM] sqc failed to register_check_item", K(reg_ret), K(sqc_detectable_id), K(qc_detectable_id));
} else {
// for unregister check item in ObPxSqcHandler::destroy_sqc()
sqc_handler->set_node_sequence_id(node_sequence_id);
}
}
}
LIB_LOG(TRACE, "[DM] sqc register dm", K(ret), K(sqc_detectable_id), K(qc_detectable_id));
return ret;
}
void ObDetectManagerUtils::sqc_unregister_detectable_id_from_dm(const common::ObDetectableId &detectable_id)
{
int ret = OB_SUCCESS;
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(detectable_id));
} else {
ret = dm->unregister_detectable_id(detectable_id);
if (OB_SUCCESS != ret && OB_HASH_NOT_EXIST != ret) {
LIB_LOG(WARN, "[DM] failed to unregister_detectable_id ", K(ret), K(detectable_id));
}
}
LIB_LOG(TRACE, "[DM] sqc unregister detectable_id_", K(ret), K(detectable_id));
}
void ObDetectManagerUtils::sqc_unregister_check_item_from_dm(const ObDetectableId &detectable_id,
const uint64_t &node_sequence_id)
{
int ret = OB_SUCCESS;
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(detectable_id));
} else {
ret = dm->unregister_check_item(detectable_id, node_sequence_id);
if (OB_SUCCESS != ret && OB_HASH_NOT_EXIST != ret) {
LIB_LOG(WARN, "[DM] sqc failed to unregister_check_item ",
K(ret), K(detectable_id), K(node_sequence_id));
}
}
}
int ObDetectManagerUtils::single_dfo_register_check_item_into_dm(const common::ObRegisterDmInfo &register_dm_info,
const ObDTLIntermResultKey &key,
ObDTLIntermResultInfo *result_info)
{
int ret = OB_SUCCESS;
if (register_dm_info.is_valid()) {
uint64_t node_sequence_id = 0;
common::ObSingleDfoDetectCB *cb = nullptr;
ObArray<ObPeerTaskState> peer_states;
if (OB_FAIL(peer_states.push_back(ObPeerTaskState{register_dm_info.addr_}))) {
LIB_LOG(WARN, "[DM] failed to push_back to peer_states", K(ret), K(register_dm_info), K(key));
}
if (OB_SUCC(ret)) {
uint64_t tenant_id = register_dm_info.detectable_id_.tenant_id_;
MTL_SWITCH(tenant_id) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(tenant_id));
} else if (OB_FAIL(dm->register_check_item(
register_dm_info.detectable_id_, cb, node_sequence_id, false,
peer_states, key))) {
LIB_LOG(WARN, "[DM] single dfo register_check_item failed ", K(ret), K(register_dm_info), K(key));
} else {
result_info->unregister_dm_info_.detectable_id_ = register_dm_info.detectable_id_;
result_info->unregister_dm_info_.node_sequence_id_ = node_sequence_id;
}
}
}
}
return ret;
}
int ObDetectManagerUtils::temp_table_register_check_item_into_dm(const common::ObDetectableId &qc_detectable_id,
const common::ObAddr &qc_addr,
const dtl::ObDTLIntermResultKey &dtl_int_key,
ObDTLIntermResultInfo *&row_store)
{
int ret = OB_SUCCESS;
uint64_t node_sequence_id = 0;
common::ObTempTableDetectCB *cb = nullptr;
ObArray<ObPeerTaskState> peer_states;
if (OB_FAIL(peer_states.push_back(ObPeerTaskState{qc_addr}))) {
LIB_LOG(WARN, "[DM] failed to push_back", K(ret), K(qc_detectable_id),
K(qc_addr), K(dtl_int_key));
}
if (OB_SUCC(ret)) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(qc_detectable_id));
} else if (OB_FAIL(dm->register_check_item(
qc_detectable_id, cb, node_sequence_id, false,
peer_states, dtl_int_key))) {
LIB_LOG(WARN, "[DM] temp table register_check_item failed ", K(ret),
K(qc_detectable_id), K(qc_addr), K(dtl_int_key));
} else {
row_store->unregister_dm_info_.detectable_id_ = qc_detectable_id;
row_store->unregister_dm_info_.node_sequence_id_ = node_sequence_id;
}
}
return ret;
}
void ObDetectManagerUtils::intern_result_unregister_check_item_from_dm(ObDTLIntermResultInfo *result_info)
{
int ret = OB_SUCCESS;
if (OB_NOT_NULL(result_info) && result_info->unregister_dm_info_.is_valid()) {
uint64_t tenant_id = result_info->unregister_dm_info_.detectable_id_.tenant_id_;
MTL_SWITCH(tenant_id) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(tenant_id));
} else {
ret = dm->unregister_check_item(result_info->unregister_dm_info_.detectable_id_,
result_info->unregister_dm_info_.node_sequence_id_);
if (OB_SUCCESS != ret && OB_HASH_NOT_EXIST != ret) {
LIB_LOG(WARN, "[DM] failed to unregister_check_item ", K(ret), K(result_info->unregister_dm_info_));
}
}
}
}
}
int ObDetectManagerUtils::p2p_datahub_register_check_item_into_dm(const common::ObRegisterDmInfo &register_dm_info,
const sql::ObP2PDhKey &p2p_key, uint64_t &node_sequence_id)
{
int ret = OB_SUCCESS;
ObP2PDataHubDetectCB *cb = nullptr;
const common::ObDetectableId &qc_detectable_id = register_dm_info.detectable_id_;
const common::ObAddr &qc_addr = register_dm_info.addr_;
ObArray<ObPeerTaskState> peer_states;
if (OB_FAIL(peer_states.push_back(ObPeerTaskState{qc_addr}))) {
LIB_LOG(WARN, "[DM] failed to push_back", K(ret), K(qc_detectable_id),
K(qc_addr), K(p2p_key));
} else {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(qc_detectable_id));
} else if (OB_SUCC(dm->register_check_item(
qc_detectable_id, cb, node_sequence_id, false,
peer_states, p2p_key))) {
} else {
LIB_LOG(WARN, "[DM] p2p datahub failed register check item", K(qc_detectable_id));
}
}
return ret;
}
void ObDetectManagerUtils::p2p_datahub_unregister_check_item_from_dm(const common::ObDetectableId &detectable_id,
uint64_t node_sequence_id)
{
int ret = OB_SUCCESS;
if (!detectable_id.is_invalid() && 0 != node_sequence_id) {
uint64_t tenant_id = detectable_id.tenant_id_;
MTL_SWITCH(tenant_id) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(tenant_id));
} else {
ret = dm->unregister_check_item(detectable_id, node_sequence_id);
if (OB_SUCCESS != ret && OB_HASH_NOT_EXIST != ret) {
LIB_LOG(WARN, "[DM] failed to unregister_check_item ", K(ret), K(detectable_id));
}
}
}
}
}
} // end namespace common
} // end namespace oceanbase

View File

@ -0,0 +1,74 @@
/**
* 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.
*/
#pragma once
#include "share/detect/ob_detectable_id.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h"
namespace oceanbase {
namespace sql {
class ObDfo;
class ObPxSqcHandler;
class ObPxSqcMeta;
namespace dtl {
class ObDTLIntermResultKey;
class ObDTLIntermResultInfo;
class ObDtlChannel;
} // end namespace dtl
} // end namespace sql
} // end namespace oceanbase
namespace oceanbase {
namespace common{
class ObPeerTaskState;
class ObDetectManager;
// Dm: detect manager, declared in ob_detect_manager.h
class ObDetectManagerUtils {
public:
static int generate_detectable_id(ObDetectableId &detectable_id, uint64_t tenant_id);
static void prepare_register_dm_info(common::ObRegisterDmInfo &register_dm_info, sql::ObPxSqcHandler *handler);
static int qc_register_detectable_id_into_dm(common::ObDetectableId &detectable_id,
bool &register_detectable_id, uint64_t tenant_id);
static void qc_unregister_detectable_id_from_dm(const common::ObDetectableId &detectable_id,
bool &register_detectable_id);
static int qc_register_check_item_into_dm(sql::ObDfo &dfo,
const ObArray<common::ObPeerTaskState> &peer_states,
const ObArray<sql::dtl::ObDtlChannel *> &dtl_channels);
static void qc_unregister_check_item_from_dm(sql::ObDfo *dfo, ObDetectManager* dm=nullptr);
static void qc_unregister_all_check_items_from_dm(const ObIArray<sql::ObDfo *> &dfos);
static int sqc_register_into_dm(sql::ObPxSqcHandler *sqc_handler, sql::ObPxSqcMeta &sqc);
static void sqc_unregister_detectable_id_from_dm(const common::ObDetectableId &detectable_id);
static void sqc_unregister_check_item_from_dm(const common::ObDetectableId &detectable_id,
const uint64_t &node_sequence_id);
static int single_dfo_register_check_item_into_dm(const common::ObRegisterDmInfo &register_dm_info,
const sql::dtl::ObDTLIntermResultKey &key,
sql::dtl::ObDTLIntermResultInfo *result_info);
static int temp_table_register_check_item_into_dm(const common::ObDetectableId &qc_detectable_id,
const common::ObAddr &qc_addr,
const sql::dtl::ObDTLIntermResultKey &dtl_int_key,
sql::dtl::ObDTLIntermResultInfo *&row_store);
// both for single_dfo and temp_table
static void intern_result_unregister_check_item_from_dm(sql::dtl::ObDTLIntermResultInfo *result_info);
static int p2p_datahub_register_check_item_into_dm(const common::ObRegisterDmInfo &register_dm_info,
const sql::ObP2PDhKey &p2p_key,
uint64_t &node_sequence_id_);
static void p2p_datahub_unregister_check_item_from_dm(const common::ObDetectableId &detectable_id,
uint64_t node_sequence_id);
};
} // end namespace common
} // end namespace oceanbase

View File

@ -0,0 +1,48 @@
/**
* 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 "share/detect/ob_detect_rpc_processor.h"
#include "share/detect/ob_detect_manager.h"
namespace oceanbase {
namespace obrpc {
int ObDetectRpcP::process()
{
int ret = OB_SUCCESS;
const ObTaskStateDetectReq &req = arg_;
ObTaskStateDetectResp &response = result_;
ARRAY_FOREACH_NORET(req.uniq_ids_, i) {
const ObDetectableId &detectable_id = req.uniq_ids_.at(i);
MTL_SWITCH(detectable_id.tenant_id_) {
ObDetectManager* dm = MTL(ObDetectManager*);
if (OB_ISNULL(dm)) {
LIB_LOG(WARN, "[DM] dm is null", K(detectable_id.tenant_id_));
continue;
}
bool task_alive = dm->is_task_alive(detectable_id);
if (!task_alive) {
LIB_LOG(DEBUG, "[DM] tast has already exit", K(detectable_id));
}
TaskInfo task_info;
task_info.task_id_ = detectable_id;
task_info.task_state_ = task_alive ? common::ObTaskState::RUNNING : common::ObTaskState::FINISHED;
if (OB_FAIL(response.task_infos_.push_back(task_info))) {
LIB_LOG(WARN, "[DM] failed to push_back", K(detectable_id), K(task_alive));
}
}
}
return ret;
}
} // end namespace obrpc
} // end namespace oceanbase

View File

@ -0,0 +1,33 @@
/**
* 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.
*/
#pragma once
#include "rpc/obrpc/ob_rpc_processor.h"
#include "share/detect/ob_detect_rpc_proxy.h"
namespace oceanbase {
namespace obrpc {
class ObDetectRpcP
: public obrpc::ObRpcProcessor<obrpc::ObDetectRpcProxy::ObRpc<obrpc::OB_DETECT_RPC_CALL>>
{
public:
ObDetectRpcP() {}
virtual ~ObDetectRpcP() = default;
protected:
int process();
private:
DISALLOW_COPY_AND_ASSIGN(ObDetectRpcP);
};
} // end namespace obrpc
} // end namespace oceanbase

View File

@ -0,0 +1,107 @@
/**
* 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 "share/detect/ob_detect_rpc_proxy.h"
namespace oceanbase {
namespace obrpc {
OB_SERIALIZE_MEMBER(TaskInfo, task_id_, task_state_);
OB_SERIALIZE_MEMBER(ObTaskStateDetectResp, task_infos_);
int ObTaskStateDetectReq::serialize(char *buf, const int64_t buf_len, int64_t &pos) const
{
int ret = OB_SUCCESS;
int64_t peer_id_size = peer_ids_.size();
if (NULL == buf || buf_len <= 0) {
ret = OB_INVALID_ARGUMENT;
COMMON_LOG(WARN, "[DM] invalid arguments.", KP(buf), K(buf_len), K(ret));
} else if (OB_FAIL(serialization::encode_vi64(
buf, buf_len, pos, peer_id_size))) {
COMMON_LOG(WARN, "[DM] serialize peer_id_size failed.",
KP(buf), K(buf_len), K(pos), K(peer_id_size), K(ret));
} else {
FOREACH_X(iter, peer_ids_, OB_SUCC(ret)) {
const ObDetectableId &peer_id = iter->first;
if (OB_FAIL(peer_id.serialize(buf, buf_len, pos))) {
COMMON_LOG(WARN, "[DM] serialize peer_id failed.",
KP(buf), K(buf_len), K(pos), K(peer_id), K(ret));
}
}
}
return ret;
}
int ObTaskStateDetectReq::deserialize(const char *buf, const int64_t data_len, int64_t &pos)
{
int ret = OB_SUCCESS;
int64_t peer_id_size = 0;
if (NULL == buf || data_len <= 0) {
ret = OB_INVALID_ARGUMENT;
COMMON_LOG(WARN, "[DM] invalid arguments.", KP(buf), K(data_len), K(ret));
} else if (serialization::decode_vi64(
buf, data_len, pos, reinterpret_cast<int64_t *>(&peer_id_size))) {
COMMON_LOG(WARN, "[DM] deserialize peer_id_size failed.",
KP(buf), K(data_len), K(pos), K(peer_id_size), K(ret));
} else {
for (int64_t i = 0; i < peer_id_size && OB_SUCC(ret); ++i) {
ObDetectableId peer_id;
if (OB_FAIL(peer_id.deserialize(buf, data_len, pos))) {
COMMON_LOG(WARN, "[DM] deserialize peer_id failed.",
KP(buf), K(data_len), K(pos), K(peer_id), K(ret));
} else {
if (OB_FAIL(uniq_ids_.push_back(peer_id))) {
COMMON_LOG(WARN, "[DM] failed to push back to uniq_ids_", K(ret));
}
}
}
}
return ret;
}
int64_t ObTaskStateDetectReq::get_serialize_size(void) const
{
int64_t total_size = 0;
int64_t peer_id_size = peer_ids_.size();
total_size += serialization::encoded_length_vi64(peer_id_size);
FOREACH(iter, peer_ids_) {
const ObDetectableId &peer_id = iter->first;
total_size += peer_id.get_serialize_size();
}
int ret = OB_SUCCESS;
return total_size;
}
int ObTaskStateDetectAsyncCB::process()
{
ObThreadCondGuard guard(cond_);
int ret = OB_SUCCESS;
if (OB_FAIL(rpc_status_->response_.assign(result_))) {
COMMON_LOG(WARN, "[DM] failed to assign", K(ret), K_(dst));
}
rpc_status_->set_processed(true);
ret = cond_.broadcast();
return ret;
}
void ObTaskStateDetectAsyncCB::on_timeout()
{
ObThreadCondGuard guard(cond_);
int ret = OB_SUCCESS;
COMMON_LOG(WARN, "[DM] detect async rpc timeout", K_(dst));
rpc_status_->set_is_timeout(true);
ret = cond_.broadcast();
}
} // end namespace obrpc
} // end namespace oceanbase

View File

@ -0,0 +1,157 @@
/**
* 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.
*/
#pragma once
#include "share/detect/ob_detect_callback.h"
#include "rpc/obrpc/ob_rpc_proxy.h"
#include "lib/hash/ob_hashset.h"
#include "share/config/ob_server_config.h"
#include "observer/ob_server_struct.h"
namespace oceanbase {
namespace obrpc {
struct ObTaskStateDetectReq
{
OB_UNIS_VERSION(1);
public:
ObTaskStateDetectReq()
{
peer_ids_.create(1);
}
~ObTaskStateDetectReq()
{
destroy();
}
void destroy()
{
peer_ids_.destroy();
uniq_ids_.reset();
}
hash::ObHashSet<ObDetectableId> peer_ids_; // serialize from hashset
common::ObSEArray<ObDetectableId, 1> uniq_ids_; // deserialize into array
};
struct TaskInfo {
OB_UNIS_VERSION(1);
public:
TO_STRING_KV(K_(task_id), K_(task_state));
ObDetectableId task_id_;
common::ObTaskState task_state_;
};
struct ObTaskStateDetectResp
{
OB_UNIS_VERSION(1);
public:
~ObTaskStateDetectResp()
{
destroy();
}
void destroy()
{
task_infos_.reset();
}
int assign(const ObTaskStateDetectResp &other)
{
int ret = OB_SUCCESS;
ARRAY_FOREACH_X(other.task_infos_, idx, cnt, OB_SUCC(ret)) {
if (OB_FAIL(task_infos_.push_back(other.task_infos_.at(idx)))) {
COMMON_LOG(WARN, "[DM] failed to push back");
}
}
return ret;
}
common::ObSEArray<TaskInfo, 1> task_infos_;
};
struct ObDetectRpcStatus
{
public:
ObDetectRpcStatus(const common::ObAddr &dst)
: dst_(dst), is_processed_(false), is_timeout_(false), is_visited_(false) {}
void destroy()
{
response_.destroy();
}
bool is_processed() const { return ATOMIC_LOAD(&is_processed_); }
void set_processed(bool value) { ATOMIC_SET(&is_processed_, value); }
bool is_timeout() const { return ATOMIC_LOAD(&is_timeout_); }
void set_is_timeout(bool value) { ATOMIC_SET(&is_timeout_, value); }
bool is_visited() const { return is_visited_; }
void set_visited(bool value) { is_visited_ = value; }
inline int64_t to_string(char *buf, const int64_t len) const
{
int64_t pos = 0;
databuff_printf(buf, len, pos, "ObDetectRpcStatus: (%d, %d, %d)",
is_processed_, is_timeout_, is_visited_);
return pos;
}
public:
common::ObAddr dst_;
bool is_processed_;
bool is_timeout_;
bool is_visited_;
ObTaskStateDetectResp response_;
};
class ObTaskStateDetectAsyncCB;
class ObDetectRpcProxy : public ObRpcProxy
{
private:
static const int64_t DETECT_MSG_TIMEOUT = 10 * 1000 * 1000;
public:
DEFINE_TO(ObDetectRpcProxy);
RPC_AP(PR1 detect_task_state, OB_DETECT_RPC_CALL, (ObTaskStateDetectReq), ObTaskStateDetectResp);
};
class ObTaskStateDetectAsyncCB : public obrpc::ObDetectRpcProxy::AsyncCB<obrpc::OB_DETECT_RPC_CALL>
{
public:
ObTaskStateDetectAsyncCB(ObThreadCond &cond, ObDetectRpcStatus *rpc_status, const common::ObAddr &dst)
: cond_(cond), rpc_status_(rpc_status), dst_(dst) {}
virtual ~ObTaskStateDetectAsyncCB() {}
// const ObTaskStateDetectResp &get_result() const { return result_; }
// const obrpc::ObRpcResultCode get_ret_code() const { return rcode_; }
int process() override;
virtual void on_timeout() override;
rpc::frame::ObReqTransport::AsyncCB *clone(
const rpc::frame::SPAlloc &alloc) const
{
void *buf = alloc(sizeof(*this));
rpc::frame::ObReqTransport::AsyncCB *newcb = NULL;
if (NULL != buf) {
newcb = new (buf) ObTaskStateDetectAsyncCB(cond_, rpc_status_, dst_);
}
return newcb;
}
virtual void set_args(const Request &arg) { UNUSED(arg); }
private:
ObThreadCond &cond_;
ObDetectRpcStatus *rpc_status_;
common::ObAddr dst_;
};
} // end namespace obrpc
} // end namespace oceanbase

View File

@ -0,0 +1,22 @@
/**
* 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 "share/detect/ob_detectable_id.h"
namespace oceanbase {
namespace common {
OB_SERIALIZE_MEMBER(ObDetectableId, first_, second_, tenant_id_);
OB_SERIALIZE_MEMBER(ObRegisterDmInfo, detectable_id_, addr_);
} // end namespace common
} // end namespace oceanbase

View File

@ -0,0 +1,102 @@
/**
* 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.
*/
#pragma once
#include "lib/utility/ob_unify_serialize.h"
#include "lib/utility/ob_print_utils.h"
#include "lib/net/ob_addr.h"
namespace oceanbase {
namespace common {
struct ObDetectableId
{
OB_UNIS_VERSION(1);
public:
ObDetectableId() : first_(0), second_(0), tenant_id_(0) {}
void operator=(const ObDetectableId &other)
{
first_ = other.first_;
second_ = other.second_;
tenant_id_ = other.tenant_id_;
}
bool operator==(const ObDetectableId &other) const
{
return first_ == other.first_ && second_ == other.second_ && tenant_id_ == other.tenant_id_;
}
bool operator!=(const ObDetectableId &other) const
{
return !(*this == other);
}
inline int64_t to_string(char *buf, const int64_t len) const
{
int64_t pos = 0;
databuff_printf(buf, len, pos, "ObDetectableId: (%ld,%ld,%ld)", first_, second_, tenant_id_);
return pos;
}
uint64_t hash() const
{
return first_;
}
int hash(uint64_t &hash_val) const { hash_val = hash(); return OB_SUCCESS; }
bool is_invalid() const
{
return (0 == first_ && 0 == second_);
}
public:
uint64_t first_;
uint64_t second_;
uint64_t tenant_id_;
};
struct ObRegisterDmInfo
{
OB_UNIS_VERSION(1);
public:
ObRegisterDmInfo() :detectable_id_(), addr_() {}
ObRegisterDmInfo(const common::ObDetectableId &detectable_id, const common::ObAddr &addr)
: detectable_id_(detectable_id), addr_(addr) {}
void operator=(const ObRegisterDmInfo& other)
{
detectable_id_ = other.detectable_id_;
addr_ = other.addr_;
}
bool is_valid() const { return (!detectable_id_.is_invalid() && addr_.is_valid()); }
TO_STRING_KV(K_(detectable_id), K_(addr));
common::ObDetectableId detectable_id_;
common::ObAddr addr_;
};
struct ObUnregisterDmInfo
{
public:
ObUnregisterDmInfo() : detectable_id_(), node_sequence_id_(0) {}
ObUnregisterDmInfo(const common::ObDetectableId &detectable_id, uint64_t node_sequence_id)
: detectable_id_(detectable_id), node_sequence_id_(node_sequence_id) {}
void operator=(const ObUnregisterDmInfo& other)
{
detectable_id_ = other.detectable_id_;
node_sequence_id_ = other.node_sequence_id_;
}
bool is_valid() const { return (!detectable_id_.is_invalid() && node_sequence_id_ != 0); }
TO_STRING_KV(K_(detectable_id), K_(node_sequence_id));
common::ObDetectableId detectable_id_;
uint64_t node_sequence_id_;
};
} // end namespace common
} // end namespace oceanbase

View File

@ -24,11 +24,15 @@ SQL_MONITOR_STATNAME_DEF(DTL_LOOP_TOTAL_MISS, sql_monitor_statname::INT, "dtl mi
SQL_MONITOR_STATNAME_DEF(DTL_LOOP_TOTAL_MISS_AFTER_DATA, sql_monitor_statname::INT, "dtl miss count after data", "the total count of dtl loop miss after get data")
SQL_MONITOR_STATNAME_DEF(HASH_INIT_BUCKET_COUNT, sql_monitor_statname::INT, "hash bucket init size", "init hash bucket count")
SQL_MONITOR_STATNAME_DEF(DISTINCT_BLOCK_MODE, sql_monitor_statname::INT, "hash distinct block mode", "hash distinct block mode")
// JOIN BLOOM FILTER
// JOIN FILTER
SQL_MONITOR_STATNAME_DEF(JOIN_FILTER_FILTERED_COUNT, sql_monitor_statname::INT, "filtered row count", "filtered row count in join filter")
SQL_MONITOR_STATNAME_DEF(JOIN_FILTER_TOTAL_COUNT, sql_monitor_statname::INT, "total row count", "total row count in join filter")
SQL_MONITOR_STATNAME_DEF(JOIN_FILTER_CHECK_COUNT, sql_monitor_statname::INT, "check row count", "the row count of participate in check bloom filter")
SQL_MONITOR_STATNAME_DEF(JOIN_FILTER_READY_TIMESTAMP, sql_monitor_statname::TIMESTAMP, "filter ready time", "bloom filter ready time")
SQL_MONITOR_STATNAME_DEF(JOIN_FILTER_ID, sql_monitor_statname::INT, "filter id", "join filter id in plan")
SQL_MONITOR_STATNAME_DEF(JOIN_FILTER_LENGTH, sql_monitor_statname::INT, "filter length", "join filter length")
SQL_MONITOR_STATNAME_DEF(JOIN_FILTER_BIT_SET, sql_monitor_statname::INT, "filter bitset", "join filter bitset")
// PDML
SQL_MONITOR_STATNAME_DEF(PDML_PARTITION_FLUSH_TIME, sql_monitor_statname::INT, "clock time cost write storage", "total time cost writing data to storage by pdml op")
SQL_MONITOR_STATNAME_DEF(PDML_PARTITION_FLUSH_COUNT, sql_monitor_statname::INT, "times write to storage", "total times writing data to storage by pdml op")

View File

@ -5152,6 +5152,197 @@ int ObInnerTableSchema::all_virtual_timestamp_service_schema(ObTableSchema &tabl
return ret;
}
int ObInnerTableSchema::all_virtual_px_p2p_datahub_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_INVALID_ID);
table_schema.set_database_id(OB_SYS_DATABASE_ID);
table_schema.set_table_id(OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(0);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(VIRTUAL_TABLE);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("svr_ip", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
1, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
MAX_IP_ADDR_LENGTH, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("svr_port", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
2, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("trace_id", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
OB_MAX_TRACE_ID_BUFFER_SIZE, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("datahub_id", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("message_type", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_INVALID, //column_collation_type
256, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("tenant_id", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("hold_size", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObIntType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(int64_t), //column_length
20, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("timeout_ts", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA_TS("start_time", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampType, //column_type
CS_TYPE_INVALID, //column_collation_type
sizeof(ObPreciseDateTime), //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false, //is_autoincrement
false); //is_on_update_for_timestamp
}
if (OB_SUCC(ret)) {
table_schema.get_part_option().set_part_num(1);
table_schema.set_part_level(PARTITION_LEVEL_ONE);
table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_LIST_COLUMNS);
if (OB_FAIL(table_schema.get_part_option().set_part_expr("svr_ip, svr_port"))) {
LOG_WARN("set_part_expr failed", K(ret));
} else if (OB_FAIL(table_schema.mock_list_partition_array())) {
LOG_WARN("mock list partition array failed", K(ret));
}
}
table_schema.set_index_using_type(USING_HASH);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(0);
table_schema.set_max_used_column_id(column_id);
return ret;
}
} // end namespace share
} // end namespace oceanbase

View File

@ -25,6 +25,195 @@ using namespace common;
namespace share
{
int ObInnerTableSchema::all_virtual_px_p2p_datahub_ora_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1;
//generated fields:
table_schema.set_tenant_id(OB_SYS_TENANT_ID);
table_schema.set_tablegroup_id(OB_INVALID_ID);
table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID);
table_schema.set_table_id(OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TID);
table_schema.set_rowkey_split_pos(0);
table_schema.set_is_use_bloomfilter(false);
table_schema.set_progressive_merge_num(0);
table_schema.set_rowkey_column_num(0);
table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK);
table_schema.set_table_type(VIRTUAL_TABLE);
table_schema.set_index_type(INDEX_TYPE_IS_NOT);
table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL);
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_table_name(OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TNAME))) {
LOG_ERROR("fail to set table_name", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) {
LOG_ERROR("fail to set compress_func_name", K(ret));
}
}
table_schema.set_part_level(PARTITION_LEVEL_ZERO);
table_schema.set_charset_type(ObCharset::get_default_charset());
table_schema.set_collation_type(ObCollationType::CS_TYPE_UTF8MB4_BIN);
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("SVR_IP", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
1, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_UTF8MB4_BIN, //column_collation_type
MAX_IP_ADDR_LENGTH, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("SVR_PORT", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
2, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TRACE_ID", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_UTF8MB4_BIN, //column_collation_type
OB_MAX_TRACE_ID_BUFFER_SIZE, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("DATAHUB_ID", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("MESSAGE_TYPE", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObVarcharType, //column_type
CS_TYPE_UTF8MB4_BIN, //column_collation_type
256, //column_length
2, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TENANT_ID", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("HOLD_SIZE", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObNumberType, //column_type
CS_TYPE_INVALID, //column_collation_type
38, //column_length
38, //column_precision
0, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("TIMEOUT_TS", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampLTZType, //column_type
CS_TYPE_INVALID, //column_collation_type
0, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
ADD_COLUMN_SCHEMA("START_TIME", //column_name
++column_id, //column_id
0, //rowkey_id
0, //index_id
0, //part_key_pos
ObTimestampLTZType, //column_type
CS_TYPE_INVALID, //column_collation_type
0, //column_length
-1, //column_precision
-1, //column_scale
false, //is_nullable
false); //is_autoincrement
}
if (OB_SUCC(ret)) {
table_schema.get_part_option().set_part_num(1);
table_schema.set_part_level(PARTITION_LEVEL_ONE);
table_schema.get_part_option().set_part_func_type(PARTITION_FUNC_TYPE_LIST);
if (OB_FAIL(table_schema.get_part_option().set_part_expr("SVR_IP, SVR_PORT"))) {
LOG_WARN("set_part_expr failed", K(ret));
} else if (OB_FAIL(table_schema.mock_list_partition_array())) {
LOG_WARN("mock list partition array failed", K(ret));
}
}
table_schema.set_index_using_type(USING_HASH);
table_schema.set_row_store_type(ENCODING_ROW_STORE);
table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL);
table_schema.set_progressive_merge_round(1);
table_schema.set_storage_format_version(3);
table_schema.set_tablet_id(0);
table_schema.set_max_used_column_id(column_id);
return ret;
}
int ObInnerTableSchema::all_virtual_timestamp_service_ora_schema(ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;

View File

@ -1667,7 +1667,7 @@ int ObInnerTableSchema::v_ob_sql_plan_schema(ObTableSchema &table_schema)
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT SQL_ID, DB_ID, PLAN_HASH, PLAN_ID, GMT_CREATE, OPERATOR, OBJECT_NODE, OBJECT_ID, OBJECT_OWNER, OBJECT_NAME, OBJECT_ALIAS, OBJECT_TYPE, OPTIMIZER, ID, PARENT_ID, DEPTH, POSITION, COST, REAL_COST, CARDINALITY, REAL_CARDINALITY, IO_COST, CPU_COST, BYTES, ROWSET, OTHER_TAG, PARTITION_START, OTHER, ACCESS_PREDICATES, FILTER_PREDICATES, STARTUP_PREDICATES, PROJECTION, SPECIAL_PREDICATES, QBLOCK_NAME, REMARKS, OTHER_XML FROM OCEANBASE.GV$OB_SQL_PLAN WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) {
if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT SQL_ID, DB_ID, PLAN_HASH, PLAN_ID, GMT_CREATE, OPERATOR, OBJECT_NODE, OBJECT_ID, OBJECT_OWNER, OBJECT_NAME, OBJECT_ALIAS, OBJECT_TYPE, OPTIMIZER, ID, PARENT_ID, DEPTH, POSITION, COST, REAL_COST, CARDINALITY, REAL_CARDINALITY, IO_COST, CPU_COST, BYTES, ROWSET, OTHER_TAG, PARTITION_START, OTHER, ACCESS_PREDICATES, FILTER_PREDICATES, STARTUP_PREDICATES, PROJECTION, SPECIAL_PREDICATES, QBLOCK_NAME, REMARKS, OTHER_XML FROM OCEANBASE.GV$OB_SQL_PLAN WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) {
LOG_ERROR("fail to set view_definition", K(ret));
}
}

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View File

@ -560,7 +560,7 @@ int ObInnerTableSchema::v_ob_sql_plan_ora_schema(ObTableSchema &table_schema)
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT DB_ID, SQL_ID, PLAN_HASH, PLAN_ID, GMT_CREATE, OPERATOR, OBJECT_NODE, OBJECT_ID, OBJECT_OWNER, OBJECT_NAME, OBJECT_ALIAS, OBJECT_TYPE, OPTIMIZER, ID, PARENT_ID, DEPTH, POSITION, COST, REAL_COST, CARDINALITY, REAL_CARDINALITY, IO_COST, CPU_COST, BYTES, ROWSET, OTHER_TAG, PARTITION_START, OTHER, ACCESS_PREDICATES, FILTER_PREDICATES, STARTUP_PREDICATES, PROJECTION, SPECIAL_PREDICATES, QBLOCK_NAME, REMARKS, OTHER_XML FROM SYS.GV$OB_SQL_PLAN WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) {
if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT DB_ID, SQL_ID, PLAN_HASH, PLAN_ID, GMT_CREATE, OPERATOR, OBJECT_NODE, OBJECT_ID, OBJECT_OWNER, OBJECT_NAME, OBJECT_ALIAS, OBJECT_TYPE, OPTIMIZER, ID, PARENT_ID, DEPTH, POSITION, COST, REAL_COST, CARDINALITY, REAL_CARDINALITY, IO_COST, CPU_COST, BYTES, ROWSET, OTHER_TAG, PARTITION_START, OTHER, ACCESS_PREDICATES, FILTER_PREDICATES, STARTUP_PREDICATES, PROJECTION, SPECIAL_PREDICATES, QBLOCK_NAME, REMARKS, OTHER_XML FROM SYS.GV$OB_SQL_PLAN WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) {
LOG_ERROR("fail to set view_definition", K(ret));
}
}
@ -810,7 +810,7 @@ int ObInnerTableSchema::gv_session_longops_ora_schema(ObTableSchema &table_schem
table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset()));
if (OB_SUCC(ret)) {
if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT CAST(sid AS NUMBER) AS SID, CAST(trace_id AS VARCHAR2(64)) AS TRACE_ID, CAST(op_name AS VARCHAR2(64)) AS OPNAME, CAST(TARGET AS VARCHAR2(64)) AS TARGET, CAST(SVR_IP AS VARCHAR2(46)) AS SVR_IP, CAST(SVR_PORT AS NUMBER) AS SVR_PORT, TO_CHAR(TO_DATE('19700101','YYYYMMDD') + START_TIME / 86400 / 1000000 + TO_NUMBER(SUBSTR(TZ_OFFSET(SESSIONTIMEZONE), 1, 3)) / 24) AS START_TIME, CAST(ELAPSED_TIME/1000000 AS NUMBER) AS ELAPSED_SECONDS, CAST(REMAINING_TIME AS NUMBER) AS TIME_REMAINING, TO_CHAR(TO_DATE('19700101','YYYYMMDD') + LAST_UPDATE_TIME / 86400 / 1000000 + TO_NUMBER(SUBSTR(TZ_OFFSET(SESSIONTIMEZONE), 1, 3)) / 24) AS LAST_UPDATE_TIME, CAST(MESSAGE AS VARCHAR2(512)) AS MESSAGE FROM SYS.ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) {
if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT CAST(sid AS NUMBER) AS SID, CAST(trace_id AS VARCHAR2(64)) AS TRACE_ID, CAST(op_name AS VARCHAR2(64)) AS OPNAME, CAST(TARGET AS VARCHAR2(64)) AS TARGET, CAST(SVR_IP AS VARCHAR2(46)) AS SVR_IP, CAST(SVR_PORT AS NUMBER) AS SVR_PORT, TO_CHAR(TO_DATE('19700101','YYYYMMDD') + START_TIME / 86400 / 1000000 + TO_NUMBER(SUBSTR(TZ_OFFSET(SESSIONTIMEZONE), 1, 3)) / 24) AS START_TIME, CAST(ELAPSED_TIME/1000000 AS NUMBER) AS ELAPSED_SECONDS, CAST(REMAINING_TIME AS NUMBER) AS TIME_REMAINING, TO_CHAR(TO_DATE('19700101','YYYYMMDD') + LAST_UPDATE_TIME / 86400 / 1000000 + TO_NUMBER(SUBSTR(TZ_OFFSET(SESSIONTIMEZONE), 1, 3)) / 24) AS LAST_UPDATE_TIME, CAST(MESSAGE AS VARCHAR2(512)) AS MESSAGE FROM SYS.ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT WHERE TENANT_ID = EFFECTIVE_TENANT_ID() )__"))) {
LOG_ERROR("fail to set view_definition", K(ret));
}
}

View File

@ -881,6 +881,7 @@ public:
static int all_virtual_thread_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_virtual_long_ops_status_mysql_sys_agent_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_timestamp_service_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_px_p2p_datahub_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_sql_audit_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_plan_stat_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_plan_cache_plan_explain_ora_schema(share::schema::ObTableSchema &table_schema);
@ -1095,6 +1096,7 @@ public:
static int all_virtual_opt_stat_gather_monitor_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_long_ops_status_sys_agent_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_thread_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_px_p2p_datahub_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_virtual_timestamp_service_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_plan_cache_stat_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_plan_cache_plan_stat_schema(share::schema::ObTableSchema &table_schema);
@ -1394,6 +1396,10 @@ public:
static int v_ob_timestamp_service_schema(share::schema::ObTableSchema &table_schema);
static int dba_ob_external_table_files_schema(share::schema::ObTableSchema &table_schema);
static int all_ob_external_table_files_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_px_p2p_datahub_schema(share::schema::ObTableSchema &table_schema);
static int v_ob_px_p2p_datahub_schema(share::schema::ObTableSchema &table_schema);
static int gv_sql_join_filter_schema(share::schema::ObTableSchema &table_schema);
static int v_sql_join_filter_schema(share::schema::ObTableSchema &table_schema);
static int dba_ob_table_stat_stale_info_schema(share::schema::ObTableSchema &table_schema);
static int cdb_ob_external_table_files_schema(share::schema::ObTableSchema &table_schema);
static int dba_synonyms_schema(share::schema::ObTableSchema &table_schema);
@ -1622,6 +1628,10 @@ public:
static int dba_ob_table_opt_stat_gather_history_ora_schema(share::schema::ObTableSchema &table_schema);
static int dba_ob_external_table_files_ora_schema(share::schema::ObTableSchema &table_schema);
static int all_ob_external_table_files_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_px_p2p_datahub_ora_schema(share::schema::ObTableSchema &table_schema);
static int v_ob_px_p2p_datahub_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_sql_join_filter_ora_schema(share::schema::ObTableSchema &table_schema);
static int v_sql_join_filter_ora_schema(share::schema::ObTableSchema &table_schema);
static int dba_ob_table_stat_stale_info_ora_schema(share::schema::ObTableSchema &table_schema);
static int gv_ob_sql_audit_ora_schema(share::schema::ObTableSchema &table_schema);
static int v_ob_sql_audit_ora_schema(share::schema::ObTableSchema &table_schema);
@ -3046,6 +3056,7 @@ const schema_create_func virtual_table_schema_creators [] = {
ObInnerTableSchema::all_virtual_thread_schema,
ObInnerTableSchema::all_virtual_virtual_long_ops_status_mysql_sys_agent_schema,
ObInnerTableSchema::all_virtual_timestamp_service_schema,
ObInnerTableSchema::all_virtual_px_p2p_datahub_schema,
ObInnerTableSchema::all_virtual_sql_plan_monitor_all_virtual_sql_plan_monitor_i1_schema,
ObInnerTableSchema::all_virtual_sql_audit_all_virtual_sql_audit_i1_schema,
ObInnerTableSchema::all_virtual_sysstat_all_virtual_sysstat_i1_schema,
@ -3269,6 +3280,7 @@ const schema_create_func virtual_table_schema_creators [] = {
ObInnerTableSchema::all_virtual_opt_stat_gather_monitor_ora_schema,
ObInnerTableSchema::all_virtual_long_ops_status_sys_agent_schema,
ObInnerTableSchema::all_virtual_thread_ora_schema,
ObInnerTableSchema::all_virtual_px_p2p_datahub_ora_schema,
ObInnerTableSchema::all_virtual_timestamp_service_ora_schema,
ObInnerTableSchema::all_virtual_table_real_agent_ora_idx_data_table_id_real_agent_schema,
ObInnerTableSchema::all_virtual_table_real_agent_ora_idx_db_tb_name_real_agent_schema,
@ -3649,6 +3661,10 @@ const schema_create_func sys_view_schema_creators [] = {
ObInnerTableSchema::v_ob_timestamp_service_schema,
ObInnerTableSchema::dba_ob_external_table_files_schema,
ObInnerTableSchema::all_ob_external_table_files_schema,
ObInnerTableSchema::gv_ob_px_p2p_datahub_schema,
ObInnerTableSchema::v_ob_px_p2p_datahub_schema,
ObInnerTableSchema::gv_sql_join_filter_schema,
ObInnerTableSchema::v_sql_join_filter_schema,
ObInnerTableSchema::dba_ob_table_stat_stale_info_schema,
ObInnerTableSchema::cdb_ob_external_table_files_schema,
ObInnerTableSchema::dba_synonyms_schema,
@ -3877,6 +3893,10 @@ const schema_create_func sys_view_schema_creators [] = {
ObInnerTableSchema::dba_ob_table_opt_stat_gather_history_ora_schema,
ObInnerTableSchema::dba_ob_external_table_files_ora_schema,
ObInnerTableSchema::all_ob_external_table_files_ora_schema,
ObInnerTableSchema::gv_ob_px_p2p_datahub_ora_schema,
ObInnerTableSchema::v_ob_px_p2p_datahub_ora_schema,
ObInnerTableSchema::gv_sql_join_filter_ora_schema,
ObInnerTableSchema::v_sql_join_filter_ora_schema,
ObInnerTableSchema::dba_ob_table_stat_stale_info_ora_schema,
ObInnerTableSchema::gv_ob_sql_audit_ora_schema,
ObInnerTableSchema::v_ob_sql_audit_ora_schema,
@ -4523,6 +4543,7 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_VIRTUAL_THREAD_TID,
OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TID,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_TID,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TID,
OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID,
@ -4745,6 +4766,7 @@ const uint64_t tenant_space_tables [] = {
OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TID,
OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TID,
OB_ALL_VIRTUAL_THREAD_ORA_TID,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TID,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA_TID,
OB_GV_OB_PLAN_CACHE_STAT_TID,
OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID,
@ -4967,6 +4989,10 @@ const uint64_t tenant_space_tables [] = {
OB_V_OB_TIMESTAMP_SERVICE_TID,
OB_DBA_OB_EXTERNAL_TABLE_FILES_TID,
OB_ALL_OB_EXTERNAL_TABLE_FILES_TID,
OB_GV_OB_PX_P2P_DATAHUB_TID,
OB_V_OB_PX_P2P_DATAHUB_TID,
OB_GV_SQL_JOIN_FILTER_TID,
OB_V_SQL_JOIN_FILTER_TID,
OB_DBA_OB_TABLE_STAT_STALE_INFO_TID,
OB_DBA_SYNONYMS_TID,
OB_DBA_OBJECTS_ORA_TID,
@ -5194,6 +5220,10 @@ const uint64_t tenant_space_tables [] = {
OB_DBA_OB_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TID,
OB_DBA_OB_EXTERNAL_TABLE_FILES_ORA_TID,
OB_ALL_OB_EXTERNAL_TABLE_FILES_ORA_TID,
OB_GV_OB_PX_P2P_DATAHUB_ORA_TID,
OB_V_OB_PX_P2P_DATAHUB_ORA_TID,
OB_GV_SQL_JOIN_FILTER_ORA_TID,
OB_V_SQL_JOIN_FILTER_ORA_TID,
OB_DBA_OB_TABLE_STAT_STALE_INFO_ORA_TID,
OB_GV_OB_SQL_AUDIT_ORA_TID,
OB_V_OB_SQL_AUDIT_ORA_TID,
@ -6057,6 +6087,7 @@ const uint64_t all_ora_mapping_virtual_table_org_tables [] = {
OB_ALL_VIRTUAL_TABLE_OPT_STAT_GATHER_HISTORY_TID,
OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_TID,
OB_ALL_VIRTUAL_THREAD_TID,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TID,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_TID, };
const uint64_t all_ora_mapping_virtual_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID
@ -6174,6 +6205,7 @@ const uint64_t all_ora_mapping_virtual_tables [] = { OB_ALL_VIRTUAL_SQL_AUDIT_O
, OB_ALL_VIRTUAL_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TID
, OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TID
, OB_ALL_VIRTUAL_THREAD_ORA_TID
, OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TID
, OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA_TID
, };
@ -6568,6 +6600,7 @@ const char* const tenant_space_table_names [] = {
OB_ALL_VIRTUAL_THREAD_TNAME,
OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TNAME,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_TNAME,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TNAME,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TNAME,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TNAME,
OB_ALL_VIRTUAL_PLAN_STAT_ORA_TNAME,
@ -6790,6 +6823,7 @@ const char* const tenant_space_table_names [] = {
OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TNAME,
OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TNAME,
OB_ALL_VIRTUAL_THREAD_ORA_TNAME,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TNAME,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA_TNAME,
OB_GV_OB_PLAN_CACHE_STAT_TNAME,
OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME,
@ -7012,6 +7046,10 @@ const char* const tenant_space_table_names [] = {
OB_V_OB_TIMESTAMP_SERVICE_TNAME,
OB_DBA_OB_EXTERNAL_TABLE_FILES_TNAME,
OB_ALL_OB_EXTERNAL_TABLE_FILES_TNAME,
OB_GV_OB_PX_P2P_DATAHUB_TNAME,
OB_V_OB_PX_P2P_DATAHUB_TNAME,
OB_GV_SQL_JOIN_FILTER_TNAME,
OB_V_SQL_JOIN_FILTER_TNAME,
OB_DBA_OB_TABLE_STAT_STALE_INFO_TNAME,
OB_DBA_SYNONYMS_TNAME,
OB_DBA_OBJECTS_ORA_TNAME,
@ -7239,6 +7277,10 @@ const char* const tenant_space_table_names [] = {
OB_DBA_OB_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TNAME,
OB_DBA_OB_EXTERNAL_TABLE_FILES_ORA_TNAME,
OB_ALL_OB_EXTERNAL_TABLE_FILES_ORA_TNAME,
OB_GV_OB_PX_P2P_DATAHUB_ORA_TNAME,
OB_V_OB_PX_P2P_DATAHUB_ORA_TNAME,
OB_GV_SQL_JOIN_FILTER_ORA_TNAME,
OB_V_SQL_JOIN_FILTER_ORA_TNAME,
OB_DBA_OB_TABLE_STAT_STALE_INFO_ORA_TNAME,
OB_GV_OB_SQL_AUDIT_ORA_TNAME,
OB_V_OB_SQL_AUDIT_ORA_TNAME,
@ -8116,6 +8158,7 @@ const uint64_t tenant_distributed_vtables [] = {
OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_TID,
OB_ALL_VIRTUAL_THREAD_TID,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_TID,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID,
OB_ALL_VIRTUAL_SQL_AUDIT_ORA_ALL_VIRTUAL_SQL_AUDIT_I1_TID,
OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID,
@ -8175,6 +8218,7 @@ const uint64_t tenant_distributed_vtables [] = {
OB_ALL_VIRTUAL_TRANS_SCHEDULER_ORA_TID,
OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TID,
OB_ALL_VIRTUAL_THREAD_ORA_TID,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TID,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA_TID, };
const uint64_t restrict_access_virtual_tables[] = {
@ -8267,6 +8311,7 @@ const uint64_t restrict_access_virtual_tables[] = {
OB_ALL_VIRTUAL_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TID,
OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TID,
OB_ALL_VIRTUAL_THREAD_ORA_TID,
OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TID,
OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA_TID };
@ -10340,11 +10385,11 @@ static inline int get_sys_table_lob_aux_schema(const uint64_t tid,
const int64_t OB_CORE_TABLE_COUNT = 4;
const int64_t OB_SYS_TABLE_COUNT = 233;
const int64_t OB_VIRTUAL_TABLE_COUNT = 661;
const int64_t OB_SYS_VIEW_COUNT = 684;
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1583;
const int64_t OB_VIRTUAL_TABLE_COUNT = 663;
const int64_t OB_SYS_VIEW_COUNT = 692;
const int64_t OB_SYS_TENANT_TABLE_COUNT = 1593;
const int64_t OB_CORE_SCHEMA_VERSION = 1;
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1586;
const int64_t OB_BOOTSTRAP_SCHEMA_VERSION = 1596;
} // end namespace share
} // end namespace oceanbase

View File

@ -623,6 +623,7 @@ const uint64_t OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_TID = 12383; // "__all_vir
const uint64_t OB_ALL_VIRTUAL_THREAD_TID = 12384; // "__all_virtual_thread"
const uint64_t OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TID = 12393; // "__all_virtual_virtual_long_ops_status_mysql_sys_agent"
const uint64_t OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_TID = 12395; // "__all_virtual_timestamp_service"
const uint64_t OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TID = 12397; // "__all_virtual_px_p2p_datahub"
const uint64_t OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TID = 15009; // "ALL_VIRTUAL_SQL_AUDIT_ORA"
const uint64_t OB_ALL_VIRTUAL_PLAN_STAT_ORA_TID = 15010; // "ALL_VIRTUAL_PLAN_STAT_ORA"
const uint64_t OB_ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA_TID = 15012; // "ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA"
@ -837,6 +838,7 @@ const uint64_t OB_ALL_VIRTUAL_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TID = 15295; //
const uint64_t OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TID = 15296; // "ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA"
const uint64_t OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TID = 15297; // "ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT"
const uint64_t OB_ALL_VIRTUAL_THREAD_ORA_TID = 15298; // "ALL_VIRTUAL_THREAD_ORA"
const uint64_t OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TID = 15384; // "ALL_VIRTUAL_PX_P2P_DATAHUB_ORA"
const uint64_t OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA_TID = 15385; // "ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA"
const uint64_t OB_GV_OB_PLAN_CACHE_STAT_TID = 20001; // "GV$OB_PLAN_CACHE_STAT"
const uint64_t OB_GV_OB_PLAN_CACHE_PLAN_STAT_TID = 20002; // "GV$OB_PLAN_CACHE_PLAN_STAT"
@ -1136,6 +1138,10 @@ const uint64_t OB_V_OB_THREAD_TID = 21381; // "V$OB_THREAD"
const uint64_t OB_V_OB_TIMESTAMP_SERVICE_TID = 21404; // "V$OB_TIMESTAMP_SERVICE"
const uint64_t OB_DBA_OB_EXTERNAL_TABLE_FILES_TID = 21417; // "DBA_OB_EXTERNAL_TABLE_FILES"
const uint64_t OB_ALL_OB_EXTERNAL_TABLE_FILES_TID = 21418; // "ALL_OB_EXTERNAL_TABLE_FILES"
const uint64_t OB_GV_OB_PX_P2P_DATAHUB_TID = 21419; // "GV$OB_PX_P2P_DATAHUB"
const uint64_t OB_V_OB_PX_P2P_DATAHUB_TID = 21420; // "V$OB_PX_P2P_DATAHUB"
const uint64_t OB_GV_SQL_JOIN_FILTER_TID = 21421; // "GV$SQL_JOIN_FILTER"
const uint64_t OB_V_SQL_JOIN_FILTER_TID = 21422; // "V$SQL_JOIN_FILTER"
const uint64_t OB_DBA_OB_TABLE_STAT_STALE_INFO_TID = 21423; // "DBA_OB_TABLE_STAT_STALE_INFO"
const uint64_t OB_CDB_OB_EXTERNAL_TABLE_FILES_TID = 21425; // "CDB_OB_EXTERNAL_TABLE_FILES"
const uint64_t OB_DBA_SYNONYMS_TID = 25001; // "DBA_SYNONYMS"
@ -1364,6 +1370,10 @@ const uint64_t OB_DBA_OB_TASK_OPT_STAT_GATHER_HISTORY_ORA_TID = 25227; // "DBA_O
const uint64_t OB_DBA_OB_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TID = 25228; // "DBA_OB_TABLE_OPT_STAT_GATHER_HISTORY_ORA"
const uint64_t OB_DBA_OB_EXTERNAL_TABLE_FILES_ORA_TID = 25234; // "DBA_OB_EXTERNAL_TABLE_FILES_ORA"
const uint64_t OB_ALL_OB_EXTERNAL_TABLE_FILES_ORA_TID = 25235; // "ALL_OB_EXTERNAL_TABLE_FILES_ORA"
const uint64_t OB_GV_OB_PX_P2P_DATAHUB_ORA_TID = 25243; // "GV$OB_PX_P2P_DATAHUB_ORA"
const uint64_t OB_V_OB_PX_P2P_DATAHUB_ORA_TID = 25244; // "V$OB_PX_P2P_DATAHUB_ORA"
const uint64_t OB_GV_SQL_JOIN_FILTER_ORA_TID = 25245; // "GV$SQL_JOIN_FILTER_ORA"
const uint64_t OB_V_SQL_JOIN_FILTER_ORA_TID = 25246; // "V$SQL_JOIN_FILTER_ORA"
const uint64_t OB_DBA_OB_TABLE_STAT_STALE_INFO_ORA_TID = 25247; // "DBA_OB_TABLE_STAT_STALE_INFO_ORA"
const uint64_t OB_GV_OB_SQL_AUDIT_ORA_TID = 28002; // "GV$OB_SQL_AUDIT_ORA"
const uint64_t OB_V_OB_SQL_AUDIT_ORA_TID = 28003; // "V$OB_SQL_AUDIT_ORA"
@ -2772,6 +2782,7 @@ const char *const OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_TNAME = "__all_virtual_
const char *const OB_ALL_VIRTUAL_THREAD_TNAME = "__all_virtual_thread";
const char *const OB_ALL_VIRTUAL_VIRTUAL_LONG_OPS_STATUS_MYSQL_SYS_AGENT_TNAME = "__all_virtual_virtual_long_ops_status_mysql_sys_agent";
const char *const OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_TNAME = "__all_virtual_timestamp_service";
const char *const OB_ALL_VIRTUAL_PX_P2P_DATAHUB_TNAME = "__all_virtual_px_p2p_datahub";
const char *const OB_ALL_VIRTUAL_SQL_AUDIT_ORA_TNAME = "ALL_VIRTUAL_SQL_AUDIT";
const char *const OB_ALL_VIRTUAL_PLAN_STAT_ORA_TNAME = "ALL_VIRTUAL_PLAN_STAT";
const char *const OB_ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN_ORA_TNAME = "ALL_VIRTUAL_PLAN_CACHE_PLAN_EXPLAIN";
@ -2986,6 +2997,7 @@ const char *const OB_ALL_VIRTUAL_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TNAME = "ALL_
const char *const OB_ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR_ORA_TNAME = "ALL_VIRTUAL_OPT_STAT_GATHER_MONITOR";
const char *const OB_ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT_TNAME = "ALL_VIRTUAL_LONG_OPS_STATUS_SYS_AGENT";
const char *const OB_ALL_VIRTUAL_THREAD_ORA_TNAME = "ALL_VIRTUAL_THREAD";
const char *const OB_ALL_VIRTUAL_PX_P2P_DATAHUB_ORA_TNAME = "ALL_VIRTUAL_PX_P2P_DATAHUB";
const char *const OB_ALL_VIRTUAL_TIMESTAMP_SERVICE_ORA_TNAME = "ALL_VIRTUAL_TIMESTAMP_SERVICE";
const char *const OB_GV_OB_PLAN_CACHE_STAT_TNAME = "GV$OB_PLAN_CACHE_STAT";
const char *const OB_GV_OB_PLAN_CACHE_PLAN_STAT_TNAME = "GV$OB_PLAN_CACHE_PLAN_STAT";
@ -3285,6 +3297,10 @@ const char *const OB_V_OB_THREAD_TNAME = "V$OB_THREAD";
const char *const OB_V_OB_TIMESTAMP_SERVICE_TNAME = "V$OB_TIMESTAMP_SERVICE";
const char *const OB_DBA_OB_EXTERNAL_TABLE_FILES_TNAME = "DBA_OB_EXTERNAL_TABLE_FILES";
const char *const OB_ALL_OB_EXTERNAL_TABLE_FILES_TNAME = "ALL_OB_EXTERNAL_TABLE_FILES";
const char *const OB_GV_OB_PX_P2P_DATAHUB_TNAME = "GV$OB_PX_P2P_DATAHUB";
const char *const OB_V_OB_PX_P2P_DATAHUB_TNAME = "V$OB_PX_P2P_DATAHUB";
const char *const OB_GV_SQL_JOIN_FILTER_TNAME = "GV$SQL_JOIN_FILTER";
const char *const OB_V_SQL_JOIN_FILTER_TNAME = "V$SQL_JOIN_FILTER";
const char *const OB_DBA_OB_TABLE_STAT_STALE_INFO_TNAME = "DBA_OB_TABLE_STAT_STALE_INFO";
const char *const OB_CDB_OB_EXTERNAL_TABLE_FILES_TNAME = "CDB_OB_EXTERNAL_TABLE_FILES";
const char *const OB_DBA_SYNONYMS_TNAME = "DBA_SYNONYMS";
@ -3513,6 +3529,10 @@ const char *const OB_DBA_OB_TASK_OPT_STAT_GATHER_HISTORY_ORA_TNAME = "DBA_OB_TAS
const char *const OB_DBA_OB_TABLE_OPT_STAT_GATHER_HISTORY_ORA_TNAME = "DBA_OB_TABLE_OPT_STAT_GATHER_HISTORY";
const char *const OB_DBA_OB_EXTERNAL_TABLE_FILES_ORA_TNAME = "DBA_OB_EXTERNAL_TABLE_FILES";
const char *const OB_ALL_OB_EXTERNAL_TABLE_FILES_ORA_TNAME = "ALL_OB_EXTERNAL_TABLE_FILES";
const char *const OB_GV_OB_PX_P2P_DATAHUB_ORA_TNAME = "GV$OB_PX_P2P_DATAHUB";
const char *const OB_V_OB_PX_P2P_DATAHUB_ORA_TNAME = "V$OB_PX_P2P_DATAHUB";
const char *const OB_GV_SQL_JOIN_FILTER_ORA_TNAME = "GV$SQL_JOIN_FILTER";
const char *const OB_V_SQL_JOIN_FILTER_ORA_TNAME = "V$SQL_JOIN_FILTER";
const char *const OB_DBA_OB_TABLE_STAT_STALE_INFO_ORA_TNAME = "DBA_OB_TABLE_STAT_STALE_INFO";
const char *const OB_GV_OB_SQL_AUDIT_ORA_TNAME = "GV$OB_SQL_AUDIT";
const char *const OB_V_OB_SQL_AUDIT_ORA_TNAME = "V$OB_SQL_AUDIT";

View File

@ -8151,9 +8151,6 @@ def_table_schema(
# table_id = 11117: used for __all_virtual_tablet_stat on column_store branch
################################################################
################################################################
# INFORMATION SCHEMA
@ -11701,6 +11698,30 @@ def_table_schema(
# 12400 __all_virtual_ls_log_restore_status
# 12401: __all_virtual_tenant_parameter
#
def_table_schema(
owner = 'mingdou.tmd',
table_name = '__all_virtual_px_p2p_datahub',
table_id = '12397',
table_type = 'VIRTUAL_TABLE',
in_tenant_space = True,
gm_columns = [],
rowkey_columns = [],
normal_columns = [
('svr_ip', 'varchar:MAX_IP_ADDR_LENGTH'),
('svr_port', 'int'),
('trace_id', 'varchar:OB_MAX_TRACE_ID_BUFFER_SIZE'),
('datahub_id', 'bigint'),
('message_type', 'varchar:256'),
('tenant_id', 'int'),
('hold_size', 'bigint'),
('timeout_ts', 'timestamp'),
('start_time', 'timestamp')
],
partition_columns = ['svr_ip', 'svr_port'],
vtable_route_policy = 'distributed',
)
# 余留位置
#
@ -12083,6 +12104,7 @@ def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15298'
# 15382: __all_virtual_transfer_task_history
# 15383: __all_virtual_resource_pool_sys_agent
# 15384: __all_virtual_px_p2p_datahub
def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15384', all_def_keywords['__all_virtual_px_p2p_datahub'])))
def_table_schema(**no_direct_access(gen_oracle_mapping_virtual_table_def('15385', all_def_keywords['__all_virtual_timestamp_service'])))
# 15386: __all_virtual_column_group
# 15387: __all_virtual_ls_log_restore_status
@ -26336,10 +26358,6 @@ def_table_schema(
# 21414: CDB_OB_TRANSFER_TASKS
# 21415: DBA_OB_TRANSFER_TASK_HISTORY
# 21416: CDB_OB_TRANSFER_TASK_HISTORY
# 21419: GV$OB_PX_P2P_DATAHUB
# 21420: V$OB_PX_P2P_DATAHUB
# 21421: GV$SQL_JOIN_FILTER
# 21422: V$SQL_JOIN_FILTER
def_table_schema(
owner = 'jim.wjh',
@ -26391,6 +26409,97 @@ def_table_schema(
""".replace("\n", " ")
)
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'GV$OB_PX_P2P_DATAHUB',
table_id = '21419',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
view_definition = """
SELECT
SVR_IP,
SVR_PORT,
CAST(TRACE_ID AS CHAR(64)) AS TRACE_ID,
CAST(DATAHUB_ID AS SIGNED) AS DATAHUB_ID,
CAST(MESSAGE_TYPE AS CHAR(256)) AS MESSAGE_TYPE,
CAST(TENANT_ID AS SIGNED) as TENANT_ID,
CAST(HOLD_SIZE AS SIGNED) as HOLD_SIZE,
CAST(TIMEOUT_TS AS DATETIME) as TIMEOUT_TS,
CAST(START_TIME AS DATETIME) as START_TIME
FROM oceanbase.__all_virtual_px_p2p_datahub
""".replace("\n", " "),
normal_columns = [
],
)
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'V$OB_PX_P2P_DATAHUB',
table_id = '21420',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
view_definition = """
SELECT * FROM OCEANBASE.GV$OB_PX_P2P_DATAHUB
WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT()
""".replace("\n", " "),
normal_columns = [
],
)
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'GV$SQL_JOIN_FILTER',
table_id = '21421',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
view_definition = """
SELECT
SVR_IP,
SVR_PORT,
CAST(NULL AS SIGNED) AS QC_SESSION_ID,
CAST(NULL AS SIGNED) AS QC_INSTANCE_ID,
CAST(NULL AS SIGNED) AS SQL_PLAN_HASH_VALUE,
CAST(OTHERSTAT_5_VALUE AS SIGNED) as FILTER_ID,
CAST(NULL AS SIGNED) as BITS_SET,
CAST(OTHERSTAT_1_VALUE AS SIGNED) as FILTERED,
CAST(OTHERSTAT_3_VALUE AS SIGNED) as PROBED,
CAST(NULL AS SIGNED) as ACTIVE,
CAST(TENANT_ID AS SIGNED) as CON_ID,
CAST(TRACE_ID AS CHAR(64)) as TRACE_ID
FROM oceanbase.__all_virtual_sql_plan_monitor
WHERE plan_operation = 'PHY_JOIN_FILTER'
""".replace("\n", " "),
normal_columns = [
],
)
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'V$SQL_JOIN_FILTER',
table_id = '21422',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
view_definition = """
SELECT * FROM OCEANBASE.GV$SQL_JOIN_FILTER
WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT()
""".replace("\n", " "),
normal_columns = [
],
)
def_table_schema(
owner = 'yibo.tyf',
@ -26537,6 +26646,7 @@ JOIN OCEANBASE.__ALL_OPTSTAT_GLOBAL_PREFS GP
ON GP.SNAME = 'STALE_PERCENT'
""".replace("\n", " ")
)
# 21424: V$OB_LS_LOG_RESTORE_STATUS
def_table_schema(
@ -26563,7 +26673,6 @@ def_table_schema(
""".replace("\n", " ")
)
################################################################################
# Oracle System View (25000, 30000]
# Data Dictionary View (25000, 28000]
@ -44082,19 +44191,6 @@ def_table_schema(
# 25231: DBA_WR_STAT_NAME
# 25232: DBA_WR_SYSSTAT
# 25233: DBA_OB_LOG_RESTORE_SOURCE
# 25234: DBA_OB_EXTERNAL_TABLE_FILES
# 25235: ALL_OB_ETERNAL_TABLE_FILES
# 25237: DBA_OB_BALANCE_JOBS
# 25238: DBA_OB_BALANCE_JOB_HISTORY
# 25239: DBA_OB_BALANCE_TASKS
# 25240: DBA_OB_BALANCE_TASK_HISTORY
# 25241: DBA_OB_TRANSFER_TASKS
# 25242: DBA_OB_TRANSFER_TASK_HISTORY
# 25243: GV$OB_PX_P2P_DATAHUB
# 25244: V$OB_PX_P2P_DATAHUB
# 25245: GV$SQL_JOIN_FILTER
# 25246: V$SQL_JOIN_FILTER
# 25247: DBA_OB_TABLE_STAT_STALE_INFO
def_table_schema(
owner = 'jim.wjh',
@ -44151,6 +44247,107 @@ def_table_schema(
""".replace("\n", " ")
)
# 25237: DBA_OB_BALANCE_JOBS
# 25238: DBA_OB_BALANCE_JOB_HISTORY
# 25239: DBA_OB_BALANCE_TASKS
# 25240: DBA_OB_BALANCE_TASK_HISTORY
# 25241: DBA_OB_TRANSFER_TASKS
# 25242: DBA_OB_TRANSFER_TASK_HISTORY
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'GV$OB_PX_P2P_DATAHUB',
name_postfix = '_ORA',
database_id = 'OB_ORA_SYS_DATABASE_ID',
table_id = '25243',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
normal_columns = [],
view_definition = """
SELECT
SVR_IP,
SVR_PORT,
CAST(TRACE_ID AS CHAR(64)) AS TRACE_ID,
CAST(DATAHUB_ID AS NUMBER) AS DATAHUB_ID,
CAST(MESSAGE_TYPE AS VARCHAR2(256)) AS MESSAGE_TYPE,
CAST(TENANT_ID AS NUMBER) as TENANT_ID,
CAST(HOLD_SIZE AS NUMBER) as HOLD_SIZE,
CAST(TIMEOUT_TS AS TIMESTAMP) as TIMEOUT_TS,
CAST(START_TIME AS TIMESTAMP) as START_TIME
FROM SYS.ALL_VIRTUAL_PX_P2P_DATAHUB
""".replace("\n", " "),
)
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'V$OB_PX_P2P_DATAHUB',
name_postfix = '_ORA',
database_id = 'OB_ORA_SYS_DATABASE_ID',
table_id = '25244',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
normal_columns = [],
view_definition = """
SELECT * FROM SYS.GV$OB_PX_P2P_DATAHUB
WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT()
""".replace("\n", " "),
)
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'GV$SQL_JOIN_FILTER',
name_postfix = '_ORA',
database_id = 'OB_ORA_SYS_DATABASE_ID',
table_id = '25245',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
normal_columns = [],
view_definition = """
SELECT
SVR_IP,
SVR_PORT,
CAST(NULL AS NUMBER) AS QC_SESSION_ID,
CAST(NULL AS NUMBER) AS QC_INSTANCE_ID,
CAST(NULL AS NUMBER) AS SQL_PLAN_HASH_VALUE,
CAST(OTHERSTAT_5_VALUE AS NUMBER) as FILTER_ID,
CAST(NULL AS NUMBER) as BITS_SET,
CAST(OTHERSTAT_1_VALUE AS NUMBER) as FILTERED,
CAST(OTHERSTAT_3_VALUE AS NUMBER) as PROBED,
CAST(NULL AS NUMBER) as ACTIVE,
CAST(TENANT_ID AS NUMBER) as CON_ID,
CAST(TRACE_ID AS CHAR(64)) as TRACE_ID
FROM SYS.ALL_VIRTUAL_SQL_PLAN_MONITOR
WHERE plan_operation = 'PHY_JOIN_FILTER'
""".replace("\n", " "),
)
def_table_schema(
owner = 'mingdou.tmd',
table_name = 'V$SQL_JOIN_FILTER',
name_postfix = '_ORA',
database_id = 'OB_ORA_SYS_DATABASE_ID',
table_id = '25246',
table_type = 'SYSTEM_VIEW',
gm_columns = [],
in_tenant_space = True,
rowkey_columns = [],
normal_columns = [],
view_definition = """
SELECT * FROM SYS.GV$SQL_JOIN_FILTER
WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT()
""".replace("\n", " "),
)
# 25247: DBA_OB_TABLE_STAT_STALE_INFO
def_table_schema(
owner = 'yibo.tyf',
table_name = 'DBA_OB_TABLE_STAT_STALE_INFO',

View File

@ -132,4 +132,5 @@ TG_DEF(MemDumpTimer, MemDumpTimer, "", TG_STATIC, TIMER)
TG_DEF(SSTableDefragment, SSTableDefragment, "", TG_STATIC, TIMER)
TG_DEF(TenantMetaMemMgr, TenantMetaMemMgr, "", TG_STATIC, TIMER)
TG_DEF(HeartbeatService, HeartbeatService, "", TG_STATIC, REENTRANT_THREAD_POOL, ThreadCountPair(2, 2))
TG_DEF(DetectManager, DetectManager, "", TG_STATIC, OB_THREAD_POOL, ThreadCountPair(1, 1))
#endif

View File

@ -1425,6 +1425,9 @@ DEF_INT(server_id, OB_CLUSTER_PARAMETER, "0", "[1, 65536]",
DEF_INT(_pipelined_table_function_memory_limit, OB_TENANT_PARAMETER, "524288000", "[1024,18446744073709551615]",
"pipeline table function result set memory size limit. default 524288000 (500M), Range: [1024,18446744073709551615]",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_BOOL(_enable_px_fast_reclaim, OB_CLUSTER_PARAMETER, "True",
"Enable the fast reclaim function through PX tasks deteting for survival by detect manager. The default value is True.",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));
DEF_BOOL(_enable_reserved_user_dcl_restriction, OB_CLUSTER_PARAMETER, "False",
"specifies whether to forbid non-reserved user to modify reserved users",
ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE));

View File

@ -29,6 +29,7 @@ namespace common {
class ObLDHandle;
class ObTenantIOManager;
template<typename T> class ObServerObjectPool;
class ObDetectManager;
}
namespace omt {
class ObPxPools;
@ -247,7 +248,8 @@ using ObPartTransCtxObjPool = common::ObServerObjectPool<transaction::ObPartTran
sql::ObFLTSpanMgr*, \
ObTestModule*, \
oceanbase::common::sqlclient::ObTenantOciEnvs*, \
rootserver::ObHeartbeatService* \
rootserver::ObHeartbeatService*, \
oceanbase::common::ObDetectManager* \
)

View File

@ -249,6 +249,10 @@ enum ObSysVarClassType
SYS_VAR_PARALLEL_DEGREE_LIMIT = 10143,
SYS_VAR_PARALLEL_MIN_SCAN_TIME_THRESHOLD = 10144,
SYS_VAR_OPTIMIZER_DYNAMIC_SAMPLING = 10145,
SYS_VAR_RUNTIME_FILTER_TYPE = 10146,
SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS = 10147,
SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM = 10148,
SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE = 10149,
};
}

View File

@ -2342,6 +2342,30 @@ int ObSysVarOnCheckFuncs::get_string(const ObObj &val, ObString &str)
return ret;
}
int ObSysVarOnCheckFuncs::check_runtime_filter_type_is_valid(
sql::ObExecContext &ctx,
const ObSetVar &set_var,
const ObBasicSysVar &sys_var,
const common::ObObj &in_val,
common::ObObj &out_val)
{
int ret = OB_SUCCESS;
ObString str_val;
if (OB_FAIL(in_val.get_varchar(str_val))) {
LOG_WARN("fail to get varchar", K(ret), K(in_val));
} else {
int64_t rf_type = ObConfigRuntimeFilterChecker::get_runtime_filter_type(str_val.ptr(),
str_val.length());
if (rf_type >= 0) {
out_val = in_val;
} else {
ret = OB_ERR_WRONG_VALUE_FOR_VAR;
LOG_USER_ERROR(OB_ERR_WRONG_VALUE_FOR_VAR, str_val.length(), str_val.ptr(), str_val.length(), str_val.ptr());
}
}
return ret;
}
int ObSysVarOnUpdateFuncs::update_tx_isolation(ObExecContext &ctx,
const ObSetVar &set_var,
const ObBasicSysVar &sys_var,

View File

@ -893,6 +893,11 @@ public:
const ObBasicSysVar &sys_var,
const common::ObObj &in_val,
common::ObObj &out_val);
static int check_runtime_filter_type_is_valid(sql::ObExecContext &ctx,
const ObSetVar &set_var,
const ObBasicSysVar &sys_var,
const common::ObObj &in_val,
common::ObObj &out_val);
private:
static int check_session_readonly(sql::ObExecContext &ctx,
const ObSetVar &set_var,

View File

@ -244,6 +244,10 @@ namespace share
static const char* const OB_SV_PARALLEL_DEGREE_LIMIT = "parallel_degree_limit";
static const char* const OB_SV_PARALLEL_MIN_SCAN_TIME_THRESHOLD = "parallel_min_scan_time_threshold";
static const char* const OB_SV_OPTIMIZER_DYNAMIC_SAMPLING = "optimizer_dynamic_sampling";
static const char* const OB_SV_RUNTIME_FILTER_TYPE = "runtime_filter_type";
static const char* const OB_SV_RUNTIME_FILTER_WAIT_TIME_MS = "runtime_filter_wait_time_ms";
static const char* const OB_SV_RUNTIME_FILTER_MAX_IN_NUM = "runtime_filter_max_in_num";
static const char* const OB_SV_RUNTIME_BLOOM_FILTER_MAX_SIZE = "runtime_bloom_filter_max_size";
}
}

View File

@ -299,6 +299,10 @@ const char *ObSysVarFactory::SYS_VAR_NAMES_SORTED_BY_NAME[] = {
"regexp_stack_limit",
"regexp_time_limit",
"resource_manager_plan",
"runtime_bloom_filter_max_size",
"runtime_filter_max_in_num",
"runtime_filter_type",
"runtime_filter_wait_time_ms",
"secure_file_priv",
"server_id",
"server_uuid",
@ -530,6 +534,10 @@ const ObSysVarClassType ObSysVarFactory::SYS_VAR_IDS_SORTED_BY_NAME[] = {
SYS_VAR_REGEXP_STACK_LIMIT,
SYS_VAR_REGEXP_TIME_LIMIT,
SYS_VAR_RESOURCE_MANAGER_PLAN,
SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE,
SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM,
SYS_VAR_RUNTIME_FILTER_TYPE,
SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS,
SYS_VAR_SECURE_FILE_PRIV,
SYS_VAR_SERVER_ID,
SYS_VAR_SERVER_UUID,
@ -812,7 +820,11 @@ const char *ObSysVarFactory::SYS_VAR_NAMES_SORTED_BY_ID[] = {
"parallel_degree_policy",
"parallel_degree_limit",
"parallel_min_scan_time_threshold",
"optimizer_dynamic_sampling"
"optimizer_dynamic_sampling",
"runtime_filter_type",
"runtime_filter_wait_time_ms",
"runtime_filter_max_in_num",
"runtime_bloom_filter_max_size"
};
bool ObSysVarFactory::sys_var_name_case_cmp(const char *name1, const ObString &name2)
@ -1208,6 +1220,10 @@ int ObSysVarFactory::create_all_sys_vars()
+ sizeof(ObSysVarParallelDegreeLimit)
+ sizeof(ObSysVarParallelMinScanTimeThreshold)
+ sizeof(ObSysVarOptimizerDynamicSampling)
+ sizeof(ObSysVarRuntimeFilterType)
+ sizeof(ObSysVarRuntimeFilterWaitTimeMs)
+ sizeof(ObSysVarRuntimeFilterMaxInNum)
+ sizeof(ObSysVarRuntimeBloomFilterMaxSize)
;
void *ptr = NULL;
if (OB_ISNULL(ptr = allocator_.alloc(total_mem_size))) {
@ -3268,6 +3284,42 @@ int ObSysVarFactory::create_all_sys_vars()
ptr = (void *)((char *)ptr + sizeof(ObSysVarOptimizerDynamicSampling));
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeFilterType())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeFilterType", K(ret));
} else {
store_buf_[ObSysVarsToIdxMap::get_store_idx(static_cast<int64_t>(SYS_VAR_RUNTIME_FILTER_TYPE))] = sys_var_ptr;
ptr = (void *)((char *)ptr + sizeof(ObSysVarRuntimeFilterType));
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeFilterWaitTimeMs())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeFilterWaitTimeMs", K(ret));
} else {
store_buf_[ObSysVarsToIdxMap::get_store_idx(static_cast<int64_t>(SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS))] = sys_var_ptr;
ptr = (void *)((char *)ptr + sizeof(ObSysVarRuntimeFilterWaitTimeMs));
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeFilterMaxInNum())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeFilterMaxInNum", K(ret));
} else {
store_buf_[ObSysVarsToIdxMap::get_store_idx(static_cast<int64_t>(SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM))] = sys_var_ptr;
ptr = (void *)((char *)ptr + sizeof(ObSysVarRuntimeFilterMaxInNum));
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeBloomFilterMaxSize())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeBloomFilterMaxSize", K(ret));
} else {
store_buf_[ObSysVarsToIdxMap::get_store_idx(static_cast<int64_t>(SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE))] = sys_var_ptr;
ptr = (void *)((char *)ptr + sizeof(ObSysVarRuntimeBloomFilterMaxSize));
}
}
}
return ret;
@ -5786,6 +5838,50 @@ int ObSysVarFactory::create_sys_var(ObIAllocator &allocator_, ObSysVarClassType
}
break;
}
case SYS_VAR_RUNTIME_FILTER_TYPE: {
void *ptr = NULL;
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObSysVarRuntimeFilterType)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc memory", K(ret), K(sizeof(ObSysVarRuntimeFilterType)));
} else if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeFilterType())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeFilterType", K(ret));
}
break;
}
case SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS: {
void *ptr = NULL;
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObSysVarRuntimeFilterWaitTimeMs)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc memory", K(ret), K(sizeof(ObSysVarRuntimeFilterWaitTimeMs)));
} else if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeFilterWaitTimeMs())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeFilterWaitTimeMs", K(ret));
}
break;
}
case SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM: {
void *ptr = NULL;
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObSysVarRuntimeFilterMaxInNum)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc memory", K(ret), K(sizeof(ObSysVarRuntimeFilterMaxInNum)));
} else if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeFilterMaxInNum())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeFilterMaxInNum", K(ret));
}
break;
}
case SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE: {
void *ptr = NULL;
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObSysVarRuntimeBloomFilterMaxSize)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to alloc memory", K(ret), K(sizeof(ObSysVarRuntimeBloomFilterMaxSize)));
} else if (OB_ISNULL(sys_var_ptr = new (ptr)ObSysVarRuntimeBloomFilterMaxSize())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("fail to new ObSysVarRuntimeBloomFilterMaxSize", K(ret));
}
break;
}
default: {
ret = OB_ERR_UNEXPECTED;

View File

@ -1649,6 +1649,34 @@ public:
inline virtual ObSysVarClassType get_type() const { return SYS_VAR_OPTIMIZER_DYNAMIC_SAMPLING; }
inline virtual const common::ObObj &get_global_default_value() const { return ObSysVariables::get_default_value(227); }
};
class ObSysVarRuntimeFilterType : public ObVarcharSysVar
{
public:
ObSysVarRuntimeFilterType() : ObVarcharSysVar(ObSysVarOnCheckFuncs::check_runtime_filter_type_is_valid, NULL, NULL, NULL, NULL) {}
inline virtual ObSysVarClassType get_type() const { return SYS_VAR_RUNTIME_FILTER_TYPE; }
inline virtual const common::ObObj &get_global_default_value() const { return ObSysVariables::get_default_value(228); }
};
class ObSysVarRuntimeFilterWaitTimeMs : public ObIntSysVar
{
public:
ObSysVarRuntimeFilterWaitTimeMs() : ObIntSysVar(NULL, NULL, NULL, NULL, NULL) {}
inline virtual ObSysVarClassType get_type() const { return SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS; }
inline virtual const common::ObObj &get_global_default_value() const { return ObSysVariables::get_default_value(229); }
};
class ObSysVarRuntimeFilterMaxInNum : public ObIntSysVar
{
public:
ObSysVarRuntimeFilterMaxInNum() : ObIntSysVar(NULL, NULL, NULL, NULL, NULL) {}
inline virtual ObSysVarClassType get_type() const { return SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM; }
inline virtual const common::ObObj &get_global_default_value() const { return ObSysVariables::get_default_value(230); }
};
class ObSysVarRuntimeBloomFilterMaxSize : public ObIntSysVar
{
public:
ObSysVarRuntimeBloomFilterMaxSize() : ObIntSysVar(NULL, NULL, NULL, NULL, NULL) {}
inline virtual ObSysVarClassType get_type() const { return SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE; }
inline virtual const common::ObObj &get_global_default_value() const { return ObSysVariables::get_default_value(231); }
};
class ObSysVarFactory
@ -1669,7 +1697,7 @@ public:
static const common::ObString get_sys_var_name_by_id(ObSysVarClassType sys_var_id);
const static int64_t MYSQL_SYS_VARS_COUNT = 97;
const static int64_t OB_SYS_VARS_COUNT = 131;
const static int64_t OB_SYS_VARS_COUNT = 135;
const static int64_t ALL_SYS_VARS_COUNT = MYSQL_SYS_VARS_COUNT + OB_SYS_VARS_COUNT;
const static int16_t OB_SPECIFIC_SYS_VAR_ID_OFFSET = 10000;

View File

@ -2994,13 +2994,64 @@ static struct VarsInit{
ObSysVars[227].alias_ = "OB_SV_OPTIMIZER_DYNAMIC_SAMPLING" ;
}();
[&] (){
ObSysVars[228].info_ = "set runtime filter type, including the bloom_filter/range/in filter" ;
ObSysVars[228].name_ = "runtime_filter_type" ;
ObSysVars[228].data_type_ = ObVarcharType ;
ObSysVars[228].value_ = "BLOOM_FILTER,RANGE,IN" ;
ObSysVars[228].flags_ = ObSysVarFlag::GLOBAL_SCOPE | ObSysVarFlag::SESSION_SCOPE | ObSysVarFlag::INFLUENCE_PLAN ;
ObSysVars[228].on_check_and_convert_func_ = "ObSysVarOnCheckFuncs::check_runtime_filter_type_is_valid" ;
ObSysVars[228].id_ = SYS_VAR_RUNTIME_FILTER_TYPE ;
cur_max_var_id = MAX(cur_max_var_id, static_cast<int64_t>(SYS_VAR_RUNTIME_FILTER_TYPE)) ;
ObSysVarsIdToArrayIdx[SYS_VAR_RUNTIME_FILTER_TYPE] = 228 ;
ObSysVars[228].alias_ = "OB_SV_RUNTIME_FILTER_TYPE" ;
}();
[&] (){
ObSysVars[229].info_ = "set default wait time ms for runtime filter, default is 10ms" ;
ObSysVars[229].name_ = "runtime_filter_wait_time_ms" ;
ObSysVars[229].data_type_ = ObIntType ;
ObSysVars[229].value_ = "10" ;
ObSysVars[229].flags_ = ObSysVarFlag::GLOBAL_SCOPE | ObSysVarFlag::SESSION_SCOPE | ObSysVarFlag::NEED_SERIALIZE ;
ObSysVars[229].id_ = SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS ;
cur_max_var_id = MAX(cur_max_var_id, static_cast<int64_t>(SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS)) ;
ObSysVarsIdToArrayIdx[SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS] = 229 ;
ObSysVars[229].alias_ = "OB_SV_RUNTIME_FILTER_WAIT_TIME_MS" ;
}();
[&] (){
ObSysVars[230].info_ = "set max in number for runtime in filter, default is 1024" ;
ObSysVars[230].name_ = "runtime_filter_max_in_num" ;
ObSysVars[230].data_type_ = ObIntType ;
ObSysVars[230].value_ = "1024" ;
ObSysVars[230].min_val_ = "0" ;
ObSysVars[230].max_val_ = "10240" ;
ObSysVars[230].flags_ = ObSysVarFlag::GLOBAL_SCOPE | ObSysVarFlag::SESSION_SCOPE | ObSysVarFlag::NEED_SERIALIZE ;
ObSysVars[230].id_ = SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM ;
cur_max_var_id = MAX(cur_max_var_id, static_cast<int64_t>(SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM)) ;
ObSysVarsIdToArrayIdx[SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM] = 230 ;
ObSysVars[230].alias_ = "OB_SV_RUNTIME_FILTER_MAX_IN_NUM" ;
}();
[&] (){
ObSysVars[231].info_ = "set max size for single runtime bloom filter, default is 2GB" ;
ObSysVars[231].name_ = "runtime_bloom_filter_max_size" ;
ObSysVars[231].data_type_ = ObIntType ;
ObSysVars[231].value_ = "2147483648" ;
ObSysVars[231].flags_ = ObSysVarFlag::GLOBAL_SCOPE | ObSysVarFlag::SESSION_SCOPE | ObSysVarFlag::NEED_SERIALIZE ;
ObSysVars[231].id_ = SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE ;
cur_max_var_id = MAX(cur_max_var_id, static_cast<int64_t>(SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE)) ;
ObSysVarsIdToArrayIdx[SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE] = 231 ;
ObSysVars[231].alias_ = "OB_SV_RUNTIME_BLOOM_FILTER_MAX_SIZE" ;
}();
if (cur_max_var_id >= ObSysVarFactory::OB_MAX_SYS_VAR_ID) {
HasInvalidSysVar = true;
}
}
}vars_init;
static int64_t var_amount = 228;
static int64_t var_amount = 232;
int64_t ObSysVariables::get_all_sys_var_count(){ return ObSysVarFactory::ALL_SYS_VARS_COUNT;}
ObSysVarClassType ObSysVariables::get_sys_var_id(int64_t i){ return ObSysVars[i].id_;}

View File

@ -3026,5 +3026,56 @@
"info_cn": "",
"background_cn": "",
"ref_url": "?singleDoc#"
},
"runtime_filter_type": {
"id": 10146,
"name": "runtime_filter_type",
"value": "BLOOM_FILTER,RANGE,IN",
"data_type": "varchar",
"info": "set runtime filter type, including the bloom_filter/range/in filter",
"flags": "GLOBAL | SESSION | INFLUENCE_PLAN",
"on_check_and_convert_func": "ObSysVarOnCheckFuncs::check_runtime_filter_type_is_valid",
"publish_version": "420",
"info_cn": "",
"background_cn": "",
"ref_url": ""
},
"runtime_filter_wait_time_ms": {
"id": 10147,
"name": "runtime_filter_wait_time_ms",
"value": "10",
"data_type": "int",
"info": "set default wait time ms for runtime filter, default is 10ms",
"flags": "GLOBAL | SESSION | NEED_SERIALIZE",
"publish_version": "420",
"info_cn": "",
"background_cn": "",
"ref_url": ""
},
"runtime_filter_max_in_num": {
"id": 10148,
"name": "runtime_filter_max_in_num",
"value": "1024",
"data_type": "int",
"info": "set max in number for runtime in filter, default is 1024",
"flags": "GLOBAL | SESSION | NEED_SERIALIZE",
"min_val": "0",
"max_val": "10240",
"publish_version": "420",
"info_cn": "",
"background_cn": "",
"ref_url": ""
},
"runtime_bloom_filter_max_size": {
"id": 10149,
"name": "runtime_bloom_filter_max_size",
"value": "2147483648",
"data_type": "int",
"info": "set max size for single runtime bloom filter, default is 2GB",
"flags": "GLOBAL | SESSION | NEED_SERIALIZE",
"publish_version": "420",
"info_cn": "",
"background_cn": "",
"ref_url": ""
}
}

View File

@ -720,6 +720,11 @@ ob_set_subtarget(ob_sql engine_px
engine/px/datahub/components/ob_dh_init_channel.cpp
engine/px/datahub/components/ob_dh_second_stage_reporting_wf.cpp
engine/px/datahub/components/ob_dh_opt_stats_gather.cpp
engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp
engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.cpp
engine/px/p2p_datahub/ob_p2p_dh_msg.cpp
engine/px/p2p_datahub/ob_p2p_dh_rpc_process.cpp
engine/px/p2p_datahub/ob_runtime_filter_msg.cpp
)
ob_set_subtarget(ob_sql engine_set

View File

@ -141,6 +141,7 @@
#include "observer/table_load/ob_table_load_struct.h"
#include "sql/resolver/cmd/ob_load_data_stmt.h"
#include "share/stat/ob_stat_define.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
namespace oceanbase
{
@ -2504,10 +2505,9 @@ int ObStaticEngineCG::generate_spec(ObLogJoinFilter &op, ObJoinFilterSpec &spec,
UNUSED(in_root_job);
spec.set_mode(op.is_create_filter() ? JoinFilterMode::CREATE : JoinFilterMode::USE);
spec.set_filter_id(op.get_filter_id());
spec.set_server_id(GCTX.server_id_);
spec.set_filter_length(op.get_filter_length());
spec.set_is_shuffle(op.is_use_filter_shuffle());
spec.set_filter_type(op.get_filter_type());
spec.set_shared_filter_type(op.get_filter_type());
spec.is_shuffle_ = op.is_use_filter_shuffle();
if (OB_FAIL(spec.join_keys_.init(op.get_join_exprs().count()))) {
LOG_WARN("failed to init join keys", K(ret));
} else if (OB_NOT_NULL(op.get_tablet_id_expr()) &&
@ -2515,8 +2515,14 @@ int ObStaticEngineCG::generate_spec(ObLogJoinFilter &op, ObJoinFilterSpec &spec,
LOG_WARN("fail to generate calc part id expr", K(ret), KP(op.get_tablet_id_expr()));
} else if (OB_FAIL(spec.hash_funcs_.init(op.get_join_exprs().count()))) {
LOG_WARN("failed to init join keys", K(ret));
} else if (OB_FAIL(spec.null_first_cmp_funcs_.init(op.get_join_exprs().count()))) {
LOG_WARN("failed to init cmp funcs", K(ret));
} else if (OB_FAIL(spec.null_last_cmp_funcs_.init(op.get_join_exprs().count()))) {
LOG_WARN("failed to init cmp funcs", K(ret));
} else if (OB_FAIL(generate_rt_exprs(op.get_join_exprs(), spec.join_keys_))) {
LOG_WARN("failed to generate rt exprs", K(ret));
} else if (OB_FAIL(spec.need_null_cmp_flags_.assign(op.get_is_null_safe_cmps()))) {
LOG_WARN("fail to assign cml flags", K(ret));
} else {
if (OB_NOT_NULL(spec.calc_tablet_id_expr_)) {
ObHashFunc hash_func;
@ -2528,28 +2534,80 @@ int ObStaticEngineCG::generate_spec(ObLogJoinFilter &op, ObJoinFilterSpec &spec,
for (int64_t i = 0; i < spec.join_keys_.count() && OB_SUCC(ret); ++i) {
ObExpr *join_expr = spec.join_keys_.at(i);
ObHashFunc hash_func;
ObCmpFunc null_first_cmp;
ObCmpFunc null_last_cmp;
null_first_cmp.cmp_func_ = join_expr->basic_funcs_->null_first_cmp_;
null_last_cmp.cmp_func_ = join_expr->basic_funcs_->null_last_cmp_;
set_murmur_hash_func(hash_func, join_expr->basic_funcs_);
if (OB_ISNULL(hash_func.hash_func_) || OB_ISNULL(hash_func.batch_hash_func_)) {
if (OB_ISNULL(hash_func.hash_func_) || OB_ISNULL(hash_func.batch_hash_func_) ||
OB_ISNULL(null_first_cmp.cmp_func_) ||
OB_ISNULL(null_last_cmp.cmp_func_ )) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("hash func is null, check datatype is valid", K(ret));
} else if (OB_FAIL(spec.hash_funcs_.push_back(hash_func))) {
LOG_WARN("failed to push back hash func", K(ret));
} else if (OB_FAIL(spec.null_first_cmp_funcs_.push_back(null_first_cmp))) {
LOG_WARN("failed to push back null first cmp func", K(ret));
} else if (OB_FAIL(spec.null_last_cmp_funcs_.push_back(null_last_cmp))) {
LOG_WARN("failed to push back null last cmp func", K(ret));
}
}
}
}
if (OB_SUCC(ret) && !op.is_create_filter()) {
ObExpr *join_filter_expr = nullptr;
if (OB_ISNULL(op.get_join_filter_expr())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("join filter expr is null", K(ret));
} else if (OB_ISNULL(join_filter_expr =
reinterpret_cast<ObExpr *>(
ObStaticEngineExprCG::get_left_value_rt_expr(*op.get_join_filter_expr())))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("join filter rt_expr_ is null", K(ret));
if (OB_SUCC(ret)) {
int64_t tenant_id =
op.get_plan()->get_optimizer_context().get_session_info()->get_effective_tenant_id();
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id));
if (tenant_config.is_valid()) {
const char *ptr = NULL;
if (OB_ISNULL(ptr = tenant_config->_px_bloom_filter_group_size.get_value())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("each group size ptr is null", K(ret));
} else if (0 == ObString::make_string("auto").case_compare(ptr)) {
spec.each_group_size_ = -1;
} else {
char *end_ptr = nullptr;
spec.each_group_size_ = strtoull(ptr, &end_ptr, 10); // get group size from tenant config
if (*end_ptr != '\0') {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("each group size ptr is unexpected", K(ret));
}
}
} else {
spec.filter_expr_id_ = join_filter_expr->expr_ctx_id_;
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected tenant config", K(ret));
}
}
if (OB_SUCC(ret)) {
// construct runtime filter exec info
ObRuntimeFilterInfo rf_info;
const common::ObIArray<int64_t> &p2p_sequence_ids = op.get_p2p_sequence_ids();
const common::ObIArray<RuntimeFilterType> &rf_types =
op.get_join_filter_types();
CK(p2p_sequence_ids.count() > 0 && p2p_sequence_ids.count() == rf_types.count());
OZ(spec.rf_infos_.init(rf_types.count()));
ObExpr *join_filter_expr = nullptr;
for (int i = 0; i < rf_types.count() && OB_SUCC(ret); ++i) {
rf_info.reset();
join_filter_expr = nullptr;
rf_info.p2p_datahub_id_ = p2p_sequence_ids.at(i);
rf_info.filter_shared_type_ = op.get_filter_type();
rf_info.dh_msg_type_ = static_cast<ObP2PDatahubMsgBase::ObP2PDatahubMsgType>(rf_types.at(i));
if (!op.is_create_filter()) {
const common::ObIArray<ObRawExpr *> &join_filter_exprs =
op.get_join_filter_exprs();
if (OB_ISNULL(join_filter_expr =
reinterpret_cast<ObExpr *>(
ObStaticEngineExprCG::get_left_value_rt_expr(*join_filter_exprs.at(i))))) {
ret = OB_ERR_UNEXPECTED;
} else {
rf_info.filter_expr_id_ = join_filter_expr->expr_ctx_id_;
}
}
if (OB_SUCC(ret) && OB_FAIL(spec.rf_infos_.push_back(rf_info))) {
LOG_WARN("fail to push back rf info", K(ret));
}
}
}
return ret;
@ -6479,6 +6537,10 @@ int ObStaticEngineCG::set_other_properties(const ObLogPlan &log_plan, ObPhysical
phy_plan.set_minimal_worker_count(log_plan.get_optimizer_context().get_minimal_worker_count());
phy_plan.set_is_batched_multi_stmt(log_plan.get_optimizer_context().is_batched_multi_stmt());
phy_plan.set_need_consistent_snapshot(log_plan.need_consistent_read());
// only if all servers's version >= CLUSTER_VERSION_4_2_0_0
if (GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_2_0_0) {
phy_plan.set_enable_px_fast_reclaim(GCONF._enable_px_fast_reclaim);
}
if (OB_FAIL(phy_plan.set_expected_worker_map(log_plan.get_optimizer_context().get_expected_worker_map()))) {
LOG_WARN("set expected worker map", K(ret));
} else if (OB_FAIL(phy_plan.set_minimal_worker_map(log_plan.get_optimizer_context().get_minimal_worker_map()))) {

View File

@ -1125,6 +1125,9 @@ int ObDtlBasicChannel::switch_buffer(const int64_t min_size, const bool is_eof,
msg_writer_->write_msg_type(write_buffer_);
write_buffer_->set_data_msg(is_data_msg_);
write_buffer_->is_eof() = is_eof;
if (is_data_msg_ && register_dm_info_.is_valid()) {
write_buffer_->set_register_dm_info(register_dm_info_);
}
LOG_TRACE("trace new buffer", K(is_data_msg_), K(is_eof), KP(id_), KP(peer_id_));
}
}

View File

@ -43,6 +43,7 @@ ObDtlChannel::ObDtlChannel(uint64_t id, const common::ObAddr &peer)
batch_id_(0),
is_px_channel_(false),
ignore_error_(false),
register_dm_info_(),
loop_idx_(OB_INVALID_INDEX_INT64),
compressor_type_(common::ObCompressorType::NONE_COMPRESSOR),
owner_mod_(DTLChannelOwner::INVALID_OWNER),

View File

@ -27,6 +27,7 @@
#include "sql/dtl/ob_op_metric.h"
#include "observer/ob_server_struct.h"
#include "lib/compress/ob_compress_util.h"
#include "share/detect/ob_detectable_id.h"
namespace oceanbase {
@ -251,6 +252,10 @@ public:
bool ignore_error() { return ignore_error_; }
void set_ignore_error(bool flag) { ignore_error_ = flag; }
void set_register_dm_info(common::ObRegisterDmInfo &register_dm_info) { register_dm_info_ = register_dm_info; }
const common::ObRegisterDmInfo &get_register_dm_info() { return register_dm_info_; }
virtual int push_buffer_batch_info() = 0;
protected:
common::ObThreadCond cond_;
@ -278,6 +283,9 @@ protected:
int64_t batch_id_;
bool is_px_channel_;
bool ignore_error_;
// for single dfo dispatch scene, the process of using intermediate results is at the rpc processor end
// the add the ObRegisterDmInfo in dtl channel and send to processor for register check item into dm
common::ObRegisterDmInfo register_dm_info_;
int64_t loop_idx_;

View File

@ -16,6 +16,7 @@
#include "observer/virtual_table/ob_all_virtual_dtl_interm_result_monitor.h"
#include "sql/dtl/ob_dtl_linked_buffer.h"
#include "sql/dtl/ob_dtl_msg_type.h"
#include "share/detect/ob_detect_manager_utils.h"
using namespace oceanbase;
using namespace common;
@ -300,13 +301,17 @@ void ObDTLIntermResultManager::free_interm_result_info(ObDTLIntermResultInfo *re
}
}
int ObDTLIntermResultManager::erase_interm_result_info(ObDTLIntermResultKey &key)
int ObDTLIntermResultManager::erase_interm_result_info(ObDTLIntermResultKey &key,
bool need_unregister_check_item_from_dm)
{
int ret = OB_SUCCESS;
ObDTLIntermResultInfo *result_info = NULL;
if (OB_FAIL(map_.erase_refactored(key, &result_info))) {
LOG_TRACE("fail to get row store in result manager", K(key), K(ret));
} else {
if (need_unregister_check_item_from_dm) {
ObDetectManagerUtils::intern_result_unregister_check_item_from_dm(result_info);
}
dec_interm_result_ref_count(result_info);
}
return ret;
@ -492,6 +497,16 @@ int ObDTLIntermResultManager::process_interm_result_inner(ObDtlLinkedBuffer &buf
LOG_WARN("fail to init buffer", K(ret));
} else if (OB_FAIL(ObDTLIntermResultManager::getInstance().insert_interm_result_info(key, result_info_guard.result_info_))) {
LOG_WARN("fail to insert row store", K(ret));
} else {
int reg_dm_ret = ObDetectManagerUtils::single_dfo_register_check_item_into_dm(
buffer.get_register_dm_info(), key, result_info_guard.result_info_);
if (OB_SUCCESS != reg_dm_ret) {
LOG_WARN("[DM] single dfo fail to register_check_item_into_dm",
K(reg_dm_ret), K(buffer.get_register_dm_info()), K(key));
}
LOG_TRACE("register_check_item_into_dm", K(reg_dm_ret),
K(buffer.get_register_dm_info()), K(key),
K(result_info_guard.result_info_->unregister_dm_info_.node_sequence_id_));
}
}
}

View File

@ -10,6 +10,9 @@
* See the Mulan PubL v2 for more details.
*/
#ifndef __OB_DTL_INTERM_RESULT_MANAGER_H__
#define __OB_DTL_INTERM_RESULT_MANAGER_H__
#include "lib/hash/ob_hashmap.h"
#include "lib/ob_define.h"
#include "common/sql_mode/ob_sql_mode.h"
@ -17,8 +20,7 @@
#include "sql/engine/basic/ob_chunk_row_store.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
#include "lib/allocator/ob_allocator.h"
#ifndef __OB_DTL_INTERM_RESULT_MANAGER_H__
#define __OB_DTL_INTERM_RESULT_MANAGER_H__
#include "share/detect/ob_detectable_id.h"
namespace oceanbase
{
@ -76,7 +78,7 @@ struct ObDTLIntermResultInfo
ObDTLIntermResultInfo()
: datum_store_(NULL), ret_(common::OB_SUCCESS),
is_read_(false), is_eof_(false), ref_count_(0),
trace_id_(), dump_time_(0), dump_cost_(0)
trace_id_(), dump_time_(0), dump_cost_(0), unregister_dm_info_()
{}
~ObDTLIntermResultInfo() {}
bool is_store_valid() const { return NULL != datum_store_; }
@ -105,6 +107,7 @@ public:
common::ObCurTraceId::TraceId trace_id_;
int64_t dump_time_;
int64_t dump_cost_;
common::ObUnregisterDmInfo unregister_dm_info_;
ObDTLIntermResultMonitorInfo monitor_info_;
uint64_t tenant_id_;
};
@ -237,7 +240,7 @@ public:
int get_interm_result_info(ObDTLIntermResultKey &key, ObDTLIntermResultInfo &result_info);
int create_interm_result_info(ObMemAttr &attr, ObDTLIntermResultInfoGuard &result_info_guard,
const ObDTLIntermResultMonitorInfo &monitor_info);
int erase_interm_result_info(ObDTLIntermResultKey &key);
int erase_interm_result_info(ObDTLIntermResultKey &key, bool need_unregister_check_item_from_dm=true);
int insert_interm_result_info(ObDTLIntermResultKey &key, ObDTLIntermResultInfo *&result_info);
// 以下两个接口会持有bucket读锁.
int clear_timeout_result_info(ObDTLIntermResultGC &gc);

View File

@ -121,6 +121,9 @@ OB_DEF_SERIALIZE(ObDtlLinkedBuffer)
if (OB_SUCC(ret)) {
LST_DO_CODE(OB_UNIS_ENCODE, enable_channel_sync_);
}
if (OB_SUCC(ret)) {
LST_DO_CODE(OB_UNIS_ENCODE, register_dm_info_);
}
}
}
return ret;
@ -156,6 +159,9 @@ OB_DEF_DESERIALIZE(ObDtlLinkedBuffer)
enable_channel_sync_ = false;
LST_DO_CODE(OB_UNIS_DECODE, enable_channel_sync_);
}
if (OB_SUCC(ret)) {
LST_DO_CODE(OB_UNIS_DECODE, register_dm_info_);
}
}
if (OB_SUCC(ret)) {
(void)ObSQLUtils::adjust_time_by_ntp_offset(timeout_ts_);
@ -185,6 +191,7 @@ OB_DEF_SERIALIZE_SIZE(ObDtlLinkedBuffer)
}
LST_DO_CODE(OB_UNIS_ADD_LEN, dfo_id_, sqc_id_);
LST_DO_CODE(OB_UNIS_ADD_LEN, enable_channel_sync_);
LST_DO_CODE(OB_UNIS_ADD_LEN, register_dm_info_);
return len;
}

View File

@ -16,6 +16,7 @@
#include "lib/queue/ob_link.h"
#include "sql/dtl/ob_dtl_msg_type.h"
#include "lib/container/ob_array_serialization.h"
#include "share/detect/ob_detectable_id.h"
namespace oceanbase {
namespace sql {
@ -135,7 +136,8 @@ public:
rows_cnt_(0), batch_info_(),
dfo_id_(common::OB_INVALID_ID),
sqc_id_(common::OB_INVALID_ID),
enable_channel_sync_(false)
enable_channel_sync_(false),
register_dm_info_()
{}
ObDtlLinkedBuffer(char * buf, int64_t size)
: buf_(buf), size_(size), pos_(), is_data_msg_(false), seq_no_(0), tenant_id_(0),
@ -144,7 +146,8 @@ public:
rows_cnt_(0), batch_info_(),
dfo_id_(common::OB_INVALID_ID),
sqc_id_(common::OB_INVALID_ID),
enable_channel_sync_(false)
enable_channel_sync_(false),
register_dm_info_()
{}
~ObDtlLinkedBuffer() { reset_batch_info(); }
TO_STRING_KV(K_(size), K_(pos), K_(is_data_msg), K_(seq_no), K_(tenant_id), K_(allocated_chid),
@ -264,6 +267,9 @@ public:
uint64_t enable_channel_sync() const { return enable_channel_sync_; }
void set_enable_channel_sync(const bool enable_channel_sync) { enable_channel_sync_ = enable_channel_sync; }
const common::ObRegisterDmInfo &get_register_dm_info() const { return register_dm_info_; }
void set_register_dm_info(const common::ObRegisterDmInfo &register_dm_info) { register_dm_info_ = register_dm_info; }
//不包含allocated_chid_ copy,谁申请谁释放
static void assign(const ObDtlLinkedBuffer &src, ObDtlLinkedBuffer *dst) {
MEMCPY(dst->buf_, src.buf_, src.size_);
@ -281,6 +287,7 @@ public:
dst->dfo_id_ = src.dfo_id_;
dst->sqc_id_ = src.sqc_id_;
dst->enable_channel_sync_ = src.enable_channel_sync_;
dst->register_dm_info_ = src.register_dm_info_;
}
void shallow_copy(const ObDtlLinkedBuffer &src)
@ -299,6 +306,7 @@ public:
dfo_id_ = src.dfo_id_;
sqc_id_ = src.sqc_id_;
enable_channel_sync_ = src.enable_channel_sync_;
register_dm_info_ = src.register_dm_info_;
}
OB_INLINE ObDtlDfoKey &get_dfo_key() {
@ -417,6 +425,7 @@ The memory layout is as below:
int64_t dfo_id_;
int64_t sqc_id_;
bool enable_channel_sync_;
common::ObRegisterDmInfo register_dm_info_;
};
} // dtl

View File

@ -176,7 +176,7 @@ ObDtlRpcChannel::ObDtlRpcChannel(
const uint64_t tenant_id,
const uint64_t id,
const ObAddr &peer)
: ObDtlBasicChannel(tenant_id, id, peer)
: ObDtlBasicChannel(tenant_id, id, peer), recv_sqc_fin_res_(false)
{}
ObDtlRpcChannel::ObDtlRpcChannel(
@ -184,7 +184,7 @@ ObDtlRpcChannel::ObDtlRpcChannel(
const uint64_t id,
const ObAddr &peer,
const int64_t hash_val)
: ObDtlBasicChannel(tenant_id, id, peer, hash_val)
: ObDtlBasicChannel(tenant_id, id, peer, hash_val), recv_sqc_fin_res_(false)
{}
ObDtlRpcChannel::~ObDtlRpcChannel()
@ -204,6 +204,7 @@ int ObDtlRpcChannel::init()
void ObDtlRpcChannel::destroy()
{
recv_sqc_fin_res_ = false;
}
int ObDtlRpcChannel::feedup(ObDtlLinkedBuffer *&buffer)
@ -249,6 +250,9 @@ int ObDtlRpcChannel::feedup(ObDtlLinkedBuffer *&buffer)
linked_buffer = nullptr;
} else if (FALSE_IT(inc_recv_buffer_cnt())) {
} else {
if (static_cast<uint16_t>(ObDtlMsgType::FINISH_SQC_RESULT) == header.type_) {
recv_sqc_fin_res_ = true;
}
if (buffer->is_data_msg()) {
metric_.mark_first_in();
if (buffer->is_eof()) {

View File

@ -101,6 +101,9 @@ public:
virtual int feedup(ObDtlLinkedBuffer *&buffer) override;
virtual int send_message(ObDtlLinkedBuffer *&buf);
bool recv_sqc_fin_res() { return recv_sqc_fin_res_; }
private:
bool recv_sqc_fin_res_;
};
} // dtl

View File

@ -20,7 +20,8 @@
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/px/ob_dfo.h"
#include "sql/engine/px/ob_px_sqc_handler.h"
#include "share/detect/ob_detect_manager_utils.h"
namespace oceanbase
{
using namespace common;
@ -341,6 +342,19 @@ int ObTempTableInsertOp::insert_chunk_row_store()
dtl::ObDTLIntermResultManager::getInstance().erase_interm_result_info(dtl_int_key);
} else {
row_store->datum_store_->reset_callback();
ObPxSqcHandler *handler = ctx_.get_sqc_handler();
if (MY_SPEC.is_distributed_ && OB_NOT_NULL(handler) && handler->get_phy_plan().is_enable_px_fast_reclaim()) {
int reg_dm_ret = ObDetectManagerUtils::temp_table_register_check_item_into_dm(
handler->get_sqc_init_arg().sqc_.get_px_detectable_ids().qc_detectable_id_,
handler->get_sqc_init_arg().sqc_.get_qc_addr(),
dtl_int_key, row_store);
if (OB_SUCCESS != reg_dm_ret) {
LOG_WARN("[DM] failed register_check_item_into_dm", K(dtl_int_key), K(dtl_int_key));
}
LOG_TRACE("[DM] temptable register a check item", K(reg_dm_ret), K(dtl_int_key),
K(handler->get_sqc_init_arg().sqc_.get_px_detectable_ids().qc_detectable_id_),
K(handler->get_sqc_init_arg().sqc_.get_qc_addr()));
}
}
}
}

View File

@ -164,6 +164,17 @@ int ObTempTableTransformationOp::destory_interm_results()
}
}
}
#ifdef ERRSIM
ObSQLSessionInfo *session = ctx_.get_my_session();
int64_t query_timeout = 0;
session->get_query_timeout(query_timeout);
if (OB_FAIL(OB_E(EventTable::EN_PX_TEMP_TABLE_NOT_DESTROY_REMOTE_INTERM_RESULT) OB_SUCCESS)) {
LOG_WARN("ObTempTableTransformationOp not destory_remote_interm_results by design", K(ret), K(query_timeout));
return OB_SUCCESS;
}
#endif
if (OB_SUCC(ret) && !svrs.empty() &&
OB_FAIL(destory_remote_interm_results(svrs, args))) {
LOG_WARN("failed to destory interm results", K(ret));

View File

@ -1029,6 +1029,8 @@ static ObExpr::EvalFunc g_expr_eval_functions[] = {
ObExprXmlSerialize::eval_xml_serialize, /* 601 */
ObExprXmlcast::eval_xmlcast, /* 602 */
ObExprUpdateXml::eval_update_xml, /* 603 */
ObExprJoinFilter::eval_range_filter, /* 604 */
ObExprJoinFilter::eval_in_filter /* 605 */
};
static ObExpr::EvalBatchFunc g_expr_eval_batch_functions[] = {
@ -1144,6 +1146,8 @@ static ObExpr::EvalBatchFunc g_expr_eval_batch_functions[] = {
ObExprCoalesce::calc_batch_coalesce_expr, /* 109 */
ObExprIsNot::calc_batch_is_not_null, /* 110 */
NULL, //ObExprNlsInitCap::calc_nls_initcap_batch /* 111 */
ObExprJoinFilter::eval_range_filter_batch, /* 112 */
ObExprJoinFilter::eval_in_filter_batch, /* 113 */
calc_sqrt_expr_mysql_in_batch, /* 114 */
calc_sqrt_expr_oracle_double_in_batch, /* 115 */
calc_sqrt_expr_oracle_number_in_batch /* 116 */

View File

@ -15,6 +15,8 @@
#include "ob_expr_extract.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
using namespace oceanbase::share;
using namespace oceanbase::common;
@ -31,13 +33,27 @@ namespace sql
int ret = OB_SUCCESS;\
eval_flags.set(idx);\
results[idx].set_int(is_match);\
if (OB_FAIL(collect_sample_info(join_filter_ctx, is_match))) {\
LOG_WARN("fail to collect sample info", K(ret));\
} else {\
++join_filter_ctx->total_count_;\
}\
collect_sample_info(join_filter_ctx, is_match);\
++join_filter_ctx->total_count_;\
return ret;\
}))) {}
#define CHECK_MAX_WAIT_TIME() \
int64_t cur_time = ObTimeUtility::current_time();\
if (cur_time - join_filter_ctx->start_time_ >\
join_filter_ctx->max_wait_time_ms_ * 1000) {\
join_filter_ctx->need_wait_rf_ = false;\
break;\
} else {\
ob_usleep(1000);\
}
ObExprJoinFilter::ObExprJoinFilterContext::~ObExprJoinFilterContext()
{
if (OB_NOT_NULL(rf_msg_)) {
// rf_msg_ is got from PX_P2P_DH map
// do not destroy it, because other worker threads may not start yet
rf_msg_->dec_ref_count();
}
}
void ObExprJoinFilter::ObExprJoinFilterContext::reset_monitor_info()
{
@ -52,7 +68,7 @@ void ObExprJoinFilter::ObExprJoinFilterContext::reset_monitor_info()
ObExprJoinFilter::ObExprJoinFilter(ObIAllocator& alloc)
: ObExprOperator(alloc,
T_OP_JOIN_BLOOM_FILTER,
T_OP_RUNTIME_FILTER,
"JOIN_BLOOM_FILTER",
MORE_THAN_ZERO,
VALID_FOR_GENERATED_COL,
@ -77,254 +93,123 @@ int ObExprJoinFilter::calc_result_typeN(ObExprResType& type,
return ret;
}
int ObExprJoinFilter::eval_bloom_filter(const ObExpr &expr, ObEvalCtx &ctx,
ObDatum &res)
{
int ret = OB_SUCCESS;
bool is_match = true;
uint64_t op_id = expr.expr_ctx_id_;
ObExecContext &exec_ctx = ctx.exec_ctx_;
ObExprJoinFilterContext *join_filter_ctx = NULL;
// 在exec_ctx中获取expr_ctx
if (OB_ISNULL(join_filter_ctx = static_cast<ObExprJoinFilterContext *>(
exec_ctx.get_expr_op_ctx(op_id)))) {
// join filter ctx may be null in das.
res.set_int(1);
} else {
// 获取join bloom filter
ObPxBloomFilter *&bloom_filter_ptr_ = join_filter_ctx->bloom_filter_ptr_;
if (OB_ISNULL(bloom_filter_ptr_) && (join_filter_ctx->n_times_ & CHECK_TIMES) == 0) {
if (OB_FAIL(ObPxBloomFilterManager::instance().get_px_bloom_filter(join_filter_ctx->bf_key_,
bloom_filter_ptr_))) {
ret = OB_SUCCESS;
}
}
if (OB_NOT_NULL(bloom_filter_ptr_)) {
if (OB_FAIL(check_bf_ready(exec_ctx, join_filter_ctx))) {
LOG_WARN("fail to check bf ready", K(ret));
} else if (!join_filter_ctx->is_ready() || join_filter_ctx->dynamic_disable()) {
} else if (expr.arg_cnt_ <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the expr of arg cnt is invalid", K(ret));
} else {
uint64_t hash_val = JOIN_FILTER_SEED;
ObDatum *datum = nullptr;
ObHashFunc hash_func;
for (int i = 0; OB_SUCC(ret) && i < expr.arg_cnt_; ++i) {
if (OB_FAIL(expr.args_[i]->eval(ctx, datum))) {
LOG_WARN("failed to eval datum", K(ret));
} else {
if (OB_ISNULL(expr.inner_functions_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr.inner_functions_ is null", K(ret));
} else {
hash_func.hash_func_ = reinterpret_cast<ObDatumHashFuncType>(expr.inner_functions_[i * 2]);
if (OB_FAIL(hash_func.hash_func_(*datum, hash_val, hash_val))) {
LOG_WARN("failed to do hash", K(ret));
}
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(bloom_filter_ptr_->might_contain(hash_val, is_match))) {
LOG_WARN("fail to check filter might contain value", K(ret), K(hash_val));
} else {
join_filter_ctx->check_count_++;
}
}
}
}
if (OB_SUCC(ret)) {
++join_filter_ctx->n_times_;
if (!is_match) {
join_filter_ctx->filter_count_++;
}
res.set_int(is_match ? 1 : 0);
if (OB_FAIL(collect_sample_info(join_filter_ctx, is_match))) {
LOG_WARN("fail to collect sample info", K(ret));
} else {
join_filter_ctx->total_count_++;
}
}
}
return ret;
}
int ObExprJoinFilter::eval_bloom_filter_batch(
const ObExpr &expr,
ObEvalCtx &ctx,
const ObBitVector &skip,
const int64_t batch_size)
{
LOG_DEBUG("eval expr bloom filter in batch mode", K(batch_size));
int ret = OB_SUCCESS;
bool is_match = true;
uint64_t op_id = expr.expr_ctx_id_;
ObExecContext &exec_ctx = ctx.exec_ctx_;
ObExprJoinFilterContext *join_filter_ctx = NULL;
ObDatum *results = expr.locate_batch_datums(ctx); // for batch
ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); // for batch
if (OB_ISNULL(join_filter_ctx = static_cast<ObExprJoinFilterContext *>(
exec_ctx.get_expr_op_ctx(op_id)))) { // get expr_ctx from exec_ctx
// join filter ctx may be null in das.
if (OB_FAIL(ObBitVector::flip_foreach(skip, batch_size,
[&](int64_t idx) __attribute__((always_inline)) {
eval_flags.set(idx);
results[idx].set_int(is_match); // all results are true when join_filter_ctx is not ready.
return OB_SUCCESS;
}))) { /* do nothing*/ }
} else {
ObPxBloomFilter *&bloom_filter_ptr_ = join_filter_ctx->bloom_filter_ptr_; // get join bloom filter
if (OB_ISNULL(bloom_filter_ptr_) && (join_filter_ctx->n_times_ & CHECK_TIMES) == 0) {
if (OB_FAIL(ObPxBloomFilterManager::instance().get_px_bloom_filter(join_filter_ctx->bf_key_,
bloom_filter_ptr_))) {
ret = OB_SUCCESS;
}
}
if (OB_NOT_NULL(bloom_filter_ptr_)) {
if (OB_FAIL(check_bf_ready(exec_ctx, join_filter_ctx))) {
LOG_WARN("fail to check bf ready", K(ret));
} else if (!join_filter_ctx->is_ready() || join_filter_ctx->dynamic_disable()) {
FILL_BATCH_RESULT();
} else if (expr.arg_cnt_ <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the expr of arg cnt is invalid", K(ret));
} else {
uint64_t seed = JOIN_FILTER_SEED;
uint64_t *hash_values = reinterpret_cast<uint64_t *>(
ctx.frames_[expr.frame_idx_] + expr.res_buf_off_);
for (int64_t i = 0; OB_SUCC(ret) && i < expr.arg_cnt_; ++i) {
ObExpr *e = expr.args_[i];
if (OB_FAIL(e->eval_batch(ctx, skip, batch_size))) {
LOG_WARN("evaluate batch failed", K(ret), K(*e));
} else {
const bool is_batch_seed = (i > 0);
if (OB_ISNULL(expr.inner_functions_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the inner_functions_ of expr is null", K(ret));
} else {
ObBatchDatumHashFunc hash_func_batch = reinterpret_cast<ObBatchDatumHashFunc>(expr.inner_functions_[i * 2 + 1]);
hash_func_batch(hash_values,
e->locate_batch_datums(ctx), e->is_batch_result(),
skip, batch_size,
is_batch_seed ? hash_values : &seed,
is_batch_seed);
}
}
}
if (OB_FAIL(ObBitVector::flip_foreach(skip, batch_size,
[&](int64_t idx) __attribute__((always_inline)) {
bloom_filter_ptr_->prefetch_bits_block(hash_values[idx]); return OB_SUCCESS;
}))) {
} else if (OB_FAIL(ObBitVector::flip_foreach(skip, batch_size,
[&](int64_t idx) __attribute__((always_inline)) {
ret = bloom_filter_ptr_->might_contain(hash_values[idx], is_match);
if (OB_SUCC(ret)) {
join_filter_ctx->filter_count_ += !is_match;
eval_flags.set(idx);
results[idx].set_int(is_match);
if (OB_FAIL(collect_sample_info(join_filter_ctx, is_match))) {
LOG_WARN("fail to collect sample info", K(ret));
} else {
++join_filter_ctx->check_count_;
++join_filter_ctx->total_count_;
}
}
return ret;
}))) {
LOG_WARN("failed to process prefetch block", K(ret));
}
}
} else { // bloom_filter_ptr_ is null
LOG_DEBUG("the bloom_filter_ptr_ is null in batch mode", K(ret));
FILL_BATCH_RESULT();
}
}
return ret;
}
int ObExprJoinFilter::cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_expr,
ObExpr &rt_expr) const
{
int ret = OB_SUCCESS;
UNUSED(expr_cg_ctx);
UNUSED(raw_expr);
rt_expr.eval_func_ = eval_bloom_filter;
rt_expr.eval_batch_func_ = eval_bloom_filter_batch;
rt_expr.inner_func_cnt_ = rt_expr.arg_cnt_ * 2;
switch(raw_expr.get_runtime_filter_type()) {
case RuntimeFilterType::BLOOM_FILTER: {
rt_expr.eval_func_ = eval_bloom_filter;
rt_expr.eval_batch_func_ = eval_bloom_filter_batch;
break;
}
case RuntimeFilterType::RANGE: {
rt_expr.eval_func_ = eval_range_filter;
rt_expr.eval_batch_func_ = eval_range_filter_batch;
break;
}
case RuntimeFilterType::IN: {
rt_expr.eval_func_ = eval_in_filter;
rt_expr.eval_batch_func_ = eval_in_filter_batch;
break;
}
default: {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected raw expr type", K(ret));
}
}
rt_expr.inner_func_cnt_ = rt_expr.arg_cnt_ * FUNCTION_CNT;
if (0 == rt_expr.inner_func_cnt_) {
// do nothing
} else if (OB_FAIL(ret)) {
} else if (OB_ISNULL(rt_expr.inner_functions_ = reinterpret_cast<void**>(expr_cg_ctx.allocator_->
alloc(sizeof(ObExpr::EvalFunc) * rt_expr.arg_cnt_ * 2)))) {
alloc(sizeof(ObExpr::EvalFunc) * rt_expr.arg_cnt_ * FUNCTION_CNT)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc memory for inner_functions_ failed", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < rt_expr.arg_cnt_; ++i) {
bool is_murmur_hash_v2_ = expr_cg_ctx.cur_cluster_version_ >= CLUSTER_VERSION_4_1_0_0;
rt_expr.inner_functions_[i * 2] = is_murmur_hash_v2_ ?
reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_)
: reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->murmur_hash_);
rt_expr.inner_functions_[i * 2 + 1] = is_murmur_hash_v2_ ?
reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_batch_)
: reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->murmur_hash_batch_);
rt_expr.inner_functions_[GET_FUNC(i, HASH_ROW)] =
reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_);
rt_expr.inner_functions_[GET_FUNC(i, HASH_BATCH)] =
reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_batch_);
rt_expr.inner_functions_[GET_FUNC(i, NULL_FIRST_COMPARE)] =
reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->null_first_cmp_);
rt_expr.inner_functions_[GET_FUNC(i, NULL_LAST_COMPARE)] =
reinterpret_cast<void*>(rt_expr.args_[i]->basic_funcs_->null_last_cmp_);
}
}
return ret;
}
int ObExprJoinFilter::check_bf_ready(
int ObExprJoinFilter::check_rf_ready(
ObExecContext &exec_ctx,
ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(join_filter_ctx)) {
} else if (!join_filter_ctx->is_ready()) {
ObPxBloomFilter *&bloom_filter_ptr_ = join_filter_ctx->bloom_filter_ptr_;
ObP2PDatahubMsgBase *&rf_msg = join_filter_ctx->rf_msg_;
if (join_filter_ctx->is_ready()) {
} else if (OB_ISNULL(rf_msg)) {
if (join_filter_ctx->need_wait_ready()) {
while (!join_filter_ctx->is_ready() && OB_SUCC(exec_ctx.fast_check_status())) {
if (bloom_filter_ptr_->check_ready()) {
join_filter_ctx->ready_ts_ = ObTimeUtility::current_time();
join_filter_ctx->is_ready_ = true;
} else {
ob_usleep(100);
if (OB_NOT_NULL(rf_msg)) {
if (rf_msg->check_ready()) {
break;
}
CHECK_MAX_WAIT_TIME();
} else if (OB_FAIL(PX_P2P_DH.atomic_get_msg(join_filter_ctx->rf_key_, rf_msg))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
CHECK_MAX_WAIT_TIME();
} else {
LOG_WARN("fail to get msg", K(ret));
}
}
}
} else {
if ((join_filter_ctx->n_times_ & CHECK_TIMES) == 0 && bloom_filter_ptr_->check_ready()) {
join_filter_ctx->ready_ts_ = ObTimeUtility::current_time();
join_filter_ctx->is_ready_ = true;
} else if ((join_filter_ctx->n_times_ & CHECK_TIMES) == 0) {
if (OB_FAIL(PX_P2P_DH.atomic_get_msg(join_filter_ctx->rf_key_, rf_msg))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to get msg", K(ret));
}
}
}
if (OB_SUCC(ret)) {
if (OB_NOT_NULL(rf_msg) && rf_msg->check_ready()) {
join_filter_ctx->is_ready_ = true;
join_filter_ctx->ready_ts_ = ObTimeUtility::current_time();
}
}
} else if ((join_filter_ctx->n_times_ & CHECK_TIMES) == 0 &&
rf_msg->check_ready()) {
join_filter_ctx->ready_ts_ = ObTimeUtility::current_time();
join_filter_ctx->is_ready_ = true;
}
return ret;
}
int ObExprJoinFilter::collect_sample_info(
void ObExprJoinFilter::collect_sample_info(
ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx,
bool is_match)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(join_filter_ctx)) {
} else if (OB_FAIL(check_need_dynamic_diable_bf(join_filter_ctx))) {
LOG_WARN("fail to check need dynamic disable bf", K(ret));
} else if (!join_filter_ctx->dynamic_disable()) {
if (!is_match) {
join_filter_ctx->partial_filter_count_++;
if (OB_NOT_NULL(join_filter_ctx)) {
check_need_dynamic_diable_bf(join_filter_ctx);
if (!join_filter_ctx->dynamic_disable()) {
if (!is_match) {
join_filter_ctx->partial_filter_count_++;
}
join_filter_ctx->partial_total_count_++;
}
join_filter_ctx->partial_total_count_++;
}
return ret;
}
int ObExprJoinFilter::check_need_dynamic_diable_bf(
void ObExprJoinFilter::check_need_dynamic_diable_bf(
ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(join_filter_ctx)) {
} else if (join_filter_ctx->cur_pos_ == join_filter_ctx->next_check_start_pos_) {
join_filter_ctx->partial_total_count_ = 0;
@ -353,6 +238,120 @@ int ObExprJoinFilter::check_need_dynamic_diable_bf(
join_filter_ctx->dynamic_disable_ = true;
}
}
}
int ObExprJoinFilter::eval_bloom_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res)
{
return eval_filter_internal(expr, ctx, res);
}
int ObExprJoinFilter::eval_range_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res)
{
return eval_filter_internal(expr, ctx, res);
}
int ObExprJoinFilter::eval_in_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res)
{
return eval_filter_internal(expr, ctx, res);
}
int ObExprJoinFilter::eval_filter_internal(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res)
{
int ret = OB_SUCCESS;
bool is_match = true;
uint64_t op_id = expr.expr_ctx_id_;
ObExecContext &exec_ctx = ctx.exec_ctx_;
ObExprJoinFilterContext *join_filter_ctx = NULL;
// get expr ctx from exec ctx
if (OB_ISNULL(join_filter_ctx = static_cast<ObExprJoinFilterContext *>(
exec_ctx.get_expr_op_ctx(op_id)))) {
// join filter ctx may be null in das.
res.set_int(1);
} else {
if (join_filter_ctx->is_first_) {
join_filter_ctx->start_time_ = ObTimeUtility::current_time();
join_filter_ctx->is_first_ = false;
}
if (OB_FAIL(check_rf_ready(exec_ctx, join_filter_ctx))) {
LOG_WARN("fail to check bf ready", K(ret));
} else if (OB_ISNULL(join_filter_ctx->rf_msg_)) {
res.set_int(1);
} else if (!join_filter_ctx->is_ready() || join_filter_ctx->dynamic_disable()) {
res.set_int(1);
} else if (OB_FAIL(join_filter_ctx->rf_msg_->might_contain(expr, ctx, *join_filter_ctx, res))) {
LOG_WARN("fail to check contain row", K(ret));
}
if (OB_SUCC(ret)) {
join_filter_ctx->n_times_++;
join_filter_ctx->total_count_++;
}
}
return ret;
}
int ObExprJoinFilter::eval_bloom_filter_batch(
const ObExpr &expr,
ObEvalCtx &ctx,
const ObBitVector &skip,
const int64_t batch_size)
{
return eval_filter_batch_internal(expr, ctx, skip, batch_size);
}
int ObExprJoinFilter::eval_range_filter_batch(
const ObExpr &expr,
ObEvalCtx &ctx,
const ObBitVector &skip,
const int64_t batch_size)
{
return eval_filter_batch_internal(expr, ctx, skip, batch_size);
}
int ObExprJoinFilter::eval_in_filter_batch(
const ObExpr &expr,
ObEvalCtx &ctx,
const ObBitVector &skip,
const int64_t batch_size)
{
return eval_filter_batch_internal(expr, ctx, skip, batch_size);
}
int ObExprJoinFilter::eval_filter_batch_internal(
const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const int64_t batch_size)
{
int ret = OB_SUCCESS;
bool is_match = true;
uint64_t op_id = expr.expr_ctx_id_;
ObExecContext &exec_ctx = ctx.exec_ctx_;
ObExprJoinFilterContext *join_filter_ctx = NULL;
ObDatum *results = expr.locate_batch_datums(ctx); // for batch
ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); // for batch
if (OB_ISNULL(join_filter_ctx = static_cast<ObExprJoinFilterContext *>(
exec_ctx.get_expr_op_ctx(op_id)))) { // get expr_ctx from exec_ctx
// join filter ctx may be null in das.
if (OB_FAIL(ObBitVector::flip_foreach(skip, batch_size,
[&](int64_t idx) __attribute__((always_inline)) {
eval_flags.set(idx);
results[idx].set_int(is_match); // all results are true when join_filter_ctx is not ready.
return OB_SUCCESS;
}))) { /* do nothing*/ }
} else {
if (join_filter_ctx->is_first_) {
join_filter_ctx->start_time_ = ObTimeUtility::current_time();
join_filter_ctx->is_first_ = false;
}
if (OB_FAIL(check_rf_ready(exec_ctx, join_filter_ctx))) {
LOG_WARN("fail to check bf ready", K(ret));
} else if (OB_ISNULL(join_filter_ctx->rf_msg_)) {
FILL_BATCH_RESULT();
} else if (!join_filter_ctx->is_ready() || join_filter_ctx->dynamic_disable()) {
FILL_BATCH_RESULT();
} else if (OB_FAIL(join_filter_ctx->rf_msg_->might_contain_batch(
expr, ctx, skip, batch_size, *join_filter_ctx))) {
LOG_WARN("fail to might contain batch");
}
}
return ret;
}

View File

@ -14,31 +14,55 @@
#define SRC_SQL_ENGINE_EXPR_OB_EXPR_JOIN_FILTER_H_
#include "sql/engine/expr/ob_expr_operator.h"
#include "sql/engine/px/ob_px_bloom_filter.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h"
namespace oceanbase
{
namespace sql
{
class ObP2PDatahubMsgBase;
enum RuntimeFilterType
{
NOT_INIT_RUNTIME_FILTER_TYPE = 0,
BLOOM_FILTER = 1,
RANGE = 2,
IN = 3
};
class ObExprJoinFilter : public ObExprOperator
{
public:
#define FUNCTION_CNT 4
#define GET_FUNC(i, j) (((i) * (FUNCTION_CNT)) + (j))
enum FunctionIndex{
HASH_ROW = 0,
HASH_BATCH = 1,
NULL_FIRST_COMPARE = 2,
NULL_LAST_COMPARE = 3
};
class ObExprJoinFilterContext : public ObExprOperatorCtx
{
public:
ObExprJoinFilterContext() : ObExprOperatorCtx(),
bloom_filter_ptr_(NULL), bf_key_(), filter_count_(0), total_count_(0), check_count_(0),
n_times_(0), ready_ts_(0), next_check_start_pos_(0), window_cnt_(0), window_size_(0),
rf_msg_(nullptr), rf_key_(), start_time_(0),
filter_count_(0), total_count_(0), check_count_(0),
n_times_(0), ready_ts_(0), next_check_start_pos_(0),
window_cnt_(0), window_size_(0),
partial_filter_count_(0), partial_total_count_(0),
cur_pos_(total_count_), flag_(0) {}
virtual ~ObExprJoinFilterContext() {}
cur_pos_(total_count_), flag_(0)
{
need_wait_rf_ = true;
is_first_ = true;
}
virtual ~ObExprJoinFilterContext();
public:
bool is_ready() { return is_ready_; }
bool need_wait_ready() { return need_wait_bf_; }
bool need_wait_ready() { return need_wait_rf_; }
bool dynamic_disable() { return dynamic_disable_; }
void reset_monitor_info();
public:
ObPxBloomFilter *bloom_filter_ptr_;
ObPXBloomFilterHashWrapper bf_key_;
ObP2PDatahubMsgBase *rf_msg_;
ObP2PDhKey rf_key_;
int64_t start_time_;
int64_t filter_count_;
int64_t total_count_;
int64_t check_count_;
@ -55,10 +79,12 @@ public:
union {
uint64_t flag_;
struct {
bool need_wait_bf_:1;
bool need_wait_rf_:1;
bool is_ready_:1;
bool dynamic_disable_:1;
uint64_t reserved_:61;
bool is_first_:1;
int32_t max_wait_time_ms_:32;
int32_t reserved_:28;
};
};
};
@ -71,24 +97,43 @@ public:
common::ObExprTypeCtx& type_ctx)
const override;
static int eval_bloom_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res);
static int eval_range_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res);
static int eval_in_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res);
static int eval_bloom_filter_batch(
const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const int64_t batch_size);
static int eval_range_filter_batch(
const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const int64_t batch_size);
static int eval_in_filter_batch(
const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const int64_t batch_size);
static int eval_filter_internal(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res);
static int eval_filter_batch_internal(
const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const int64_t batch_size);
virtual int cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_expr,
ObExpr &rt_expr) const override;
virtual bool need_rt_ctx() const override { return true; }
// hard code seed, 32 bit max prime number
static const int64_t JOIN_FILTER_SEED = 4294967279;
private:
static int check_bf_ready(
ObExecContext &exec_ctx,
ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx);
static int collect_sample_info(
static void collect_sample_info(
ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx,
bool is_match);
static int check_need_dynamic_diable_bf(
private:
static int check_rf_ready(
ObExecContext &exec_ctx,
ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx);
static void check_need_dynamic_diable_bf(
ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx);
private:
static const int64_t CHECK_TIMES = 127;
private:
DISALLOW_COPY_AND_ASSIGN(ObExprJoinFilter);
};

File diff suppressed because it is too large Load Diff

View File

@ -23,6 +23,8 @@
#include "sql/dtl/ob_dtl_flow_control.h"
#include "sql/dtl/ob_dtl_channel_loop.h"
#include "sql/dtl/ob_op_metric.h"
#include "sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h"
#include "share/detect/ob_detectable_id.h"
namespace oceanbase
@ -39,9 +41,34 @@ struct ObJoinFilterShareInfo
uint64_t ch_provider_ptr_; // sqc_proxy, 由于序列化需要, 使用指针表示.
uint64_t release_ref_ptr_; // 释放内存引用计数, 初始值为worker个数.
uint64_t filter_ptr_; //此指针将作为PX JOIN FILTER CREATE算子共享内存.
uint64_t shared_msgs_; //sqc-shared dh msgs
OB_UNIS_VERSION_V(1);
};
struct ObJoinFilterRuntimeConfig
{
OB_UNIS_VERSION_V(1);
public:
TO_STRING_KV(K_(bloom_filter_ratio), K_(each_group_size), K_(bf_piece_size),
K_(runtime_filter_wait_time_ms), K_(runtime_filter_max_in_num),
K_(runtime_bloom_filter_max_size), K_(px_message_compression));
public:
ObJoinFilterRuntimeConfig() :
bloom_filter_ratio_(0.0),
each_group_size_(OB_INVALID_ID),
bf_piece_size_(0),
runtime_filter_wait_time_ms_(0),
runtime_filter_max_in_num_(0),
runtime_bloom_filter_max_size_(0),
px_message_compression_(false) {}
double bloom_filter_ratio_;
int64_t each_group_size_;
int64_t bf_piece_size_;
int64_t runtime_filter_wait_time_ms_;
int64_t runtime_filter_max_in_num_;
int64_t runtime_bloom_filter_max_size_;
bool px_message_compression_;
};
class ObJoinFilterOpInput : public ObOpInput
{
OB_UNIS_VERSION_V(1);
@ -49,10 +76,11 @@ public:
ObJoinFilterOpInput(ObExecContext &ctx, const ObOpSpec &spec)
: ObOpInput(ctx, spec),
share_info_(),
is_local_create_(true),
task_id_(0),
px_sequence_id_(OB_INVALID_ID),
bf_idx_at_sqc_proxy_(-1)
bf_idx_at_sqc_proxy_(-1),
config_(),
register_dm_info_()
{}
virtual ~ObJoinFilterOpInput() {}
@ -65,19 +93,29 @@ public:
this->~ObJoinFilterOpInput();
new (ptr) ObJoinFilterOpInput(ctx, spec);
}
int check_finish(bool &is_end, bool is_shared);
bool check_release(bool is_shared);
bool is_finish();
bool check_release();
// 每个worker共享同一块sqc_proxy
void set_sqc_proxy(ObPxSQCProxy &sqc_proxy)
{
share_info_.ch_provider_ptr_ = reinterpret_cast<uint64_t>(&sqc_proxy);
}
bool is_local_create() { return is_local_create_; }
ObJoinFilterOp *get_filter()
{
return reinterpret_cast<ObJoinFilterOp *>(share_info_.filter_ptr_);
}
int init_share_info(common::ObIAllocator &allocator, int64_t task_count);
int init_share_info(
const ObJoinFilterSpec &spec,
ObExecContext &ctx,
int64_t task_count,
int64_t sqc_count);
int init_shared_msgs(const ObJoinFilterSpec &spec,
ObExecContext &ctx,
int64_t sqc_count);
static int construct_msg_details(const ObJoinFilterSpec &spec,
ObPxSQCProxy *sqc_proxy,
ObJoinFilterRuntimeConfig &config,
ObP2PDatahubMsgBase &msg, int64_t sqc_count);
void set_task_id(int64_t task_id) { task_id_ = task_id; }
inline void set_bf_idx_at_sqc_proxy(int64_t idx) { bf_idx_at_sqc_proxy_ = idx; }
@ -85,60 +123,95 @@ public:
inline int64_t get_bf_idx_at_sqc_proxy() { return bf_idx_at_sqc_proxy_; }
void set_px_sequence_id(int64_t id) { px_sequence_id_ = id; }
int64_t get_px_sequence_id() { return px_sequence_id_; }
int load_runtime_config(const ObJoinFilterSpec &spec, ObExecContext &ctx);
void init_register_dm_info(ObDetectableId id, common::ObAddr addr)
{
register_dm_info_.detectable_id_ = id;
register_dm_info_.addr_ = addr;
}
public:
ObJoinFilterShareInfo share_info_; //bloom filter共享内存
bool is_local_create_; //用于标记create算子是否是local的.
int64_t task_id_; //在pwj join场景中会用到此task_id作为bf_key
int64_t px_sequence_id_;
int64_t bf_idx_at_sqc_proxy_;
ObJoinFilterRuntimeConfig config_;
common::ObRegisterDmInfo register_dm_info_;
DISALLOW_COPY_AND_ASSIGN(ObJoinFilterOpInput);
};
struct ObRuntimeFilterInfo
{
OB_UNIS_VERSION_V(1);
public:
TO_STRING_KV(K_(filter_expr_id), K_(p2p_datahub_id), K_(filter_shared_type));
public:
ObRuntimeFilterInfo() :
filter_expr_id_(OB_INVALID_ID),
p2p_datahub_id_(OB_INVALID_ID),
filter_shared_type_(INVALID_TYPE),
dh_msg_type_(ObP2PDatahubMsgBase::ObP2PDatahubMsgType::NOT_INIT)
{}
virtual ~ObRuntimeFilterInfo() = default;
void reset () {
filter_expr_id_ = OB_INVALID_ID;
p2p_datahub_id_ = OB_INVALID_ID;
dh_msg_type_ = ObP2PDatahubMsgBase::ObP2PDatahubMsgType::NOT_INIT;
}
int64_t filter_expr_id_;
int64_t p2p_datahub_id_;
JoinFilterSharedType filter_shared_type_;
ObP2PDatahubMsgBase::ObP2PDatahubMsgType dh_msg_type_;
};
class ObJoinFilterSpec : public ObOpSpec
{
OB_UNIS_VERSION_V(1);
OB_UNIS_VERSION_V(2);
public:
ObJoinFilterSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type);
INHERIT_TO_STRING_KV("op_spec", ObOpSpec,
K_(mode), K_(filter_id), K_(server_id), K_(filter_len),
K_(is_shuffle), K_(filter_expr_id));
K_(mode), K_(filter_id), K_(filter_len), K_(rf_infos));
inline void set_mode(JoinFilterMode mode) { mode_ = mode; }
inline JoinFilterMode get_mode() const { return mode_; }
inline void set_filter_id(int64_t id) { filter_id_ = id; }
inline int64_t get_filter_id() const { return filter_id_; }
inline void set_server_id(int64_t id) { server_id_ = id; }
inline int64_t get_server_id() const { return server_id_; }
inline void set_filter_length(int64_t len) { filter_len_ = len; }
inline int64_t get_filter_length() const { return filter_len_; }
inline ObIArray<ObExpr*> &get_exprs() { return join_keys_; }
inline bool is_create_mode() const { return JoinFilterMode::CREATE == mode_; }
inline bool is_use_mode() const { return JoinFilterMode::USE == mode_; }
inline bool is_shuffle() const { return is_shuffle_; }
inline void set_is_shuffle(bool flag) { is_shuffle_ = flag; }
inline bool is_partition_filter() const
{ return filter_type_ == JoinFilterType::NONSHARED_PARTITION_JOIN_FILTER ||
filter_type_ == JoinFilterType::SHARED_PARTITION_JOIN_FILTER; };
inline void set_filter_type(JoinFilterType type) { filter_type_ = type; }
{ return filter_shared_type_ == JoinFilterSharedType::NONSHARED_PARTITION_JOIN_FILTER ||
filter_shared_type_ == JoinFilterSharedType::SHARED_PARTITION_JOIN_FILTER; };
inline void set_shared_filter_type(JoinFilterSharedType type) { filter_shared_type_ = type; }
inline bool is_shared_join_filter() const
{ return filter_type_ == JoinFilterType::SHARED_JOIN_FILTER ||
filter_type_ == JoinFilterType::SHARED_PARTITION_JOIN_FILTER; }
{ return filter_shared_type_ == JoinFilterSharedType::SHARED_JOIN_FILTER ||
filter_shared_type_ == JoinFilterSharedType::SHARED_PARTITION_JOIN_FILTER; }
JoinFilterMode mode_;
int64_t filter_id_;
int64_t server_id_;
int64_t filter_len_;
bool is_shuffle_; //filter create端检查filter use端是否需要做shuffle
ExprFixedArray join_keys_;
common::ObHashFuncs hash_funcs_;
int64_t filter_expr_id_;
JoinFilterType filter_type_;
ObCmpFuncs null_first_cmp_funcs_;
ObCmpFuncs null_last_cmp_funcs_;
JoinFilterSharedType filter_shared_type_;
ObExpr *calc_tablet_id_expr_;
common::ObFixedArray<ObRuntimeFilterInfo, common::ObIAllocator> rf_infos_;
common::ObFixedArray<bool, common::ObIAllocator> need_null_cmp_flags_;
bool is_shuffle_;
int64_t each_group_size_;
};
class ObJoinFilterOp : public ObOperator
{
struct ObJoinFilterMsg {
ObJoinFilterMsg() : bf_msg_(nullptr), range_msg_(nullptr), in_msg_(nullptr) {}
ObRFBloomFilterMsg *bf_msg_;
ObRFRangeFilterMsg *range_msg_;
ObRFInFilterMsg *in_msg_;
};
public:
ObJoinFilterOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input);
virtual ~ObJoinFilterOp();
@ -148,34 +221,44 @@ public:
virtual int inner_rescan() override;
virtual int inner_get_next_row() override;
virtual int inner_get_next_batch(const int64_t max_row_cnt) override; // for batch
virtual void destroy() override { ObOperator::destroy(); }
virtual void destroy() override {
lucky_devil_champions_.reset();
local_rf_msgs_.reset();
shared_rf_msgs_.reset();
ObOperator::destroy();
}
static int link_ch_sets(ObPxBloomFilterChSets &ch_sets,
common::ObIArray<dtl::ObDtlChannel *> &channels);
private:
bool is_valid();
bool is_acceptable_filter();
int destroy_filter();
int send_filter();
int send_local_filter();
int mark_rpc_filter();
int insert_by_row();
int insert_by_row_batch(const ObBatchRows *child_brs);
int check_contain_row(bool &match);
int calc_hash_value(uint64_t &hash_value, bool &ignore);
int calc_hash_value(uint64_t &hash_value);
int calc_expr_values(ObDatum *&datum);
int do_create_filter_rescan();
int do_use_filter_rescan();
int try_send_join_filter();
int try_merge_join_filter();
int calc_each_bf_group_size(int64_t &);
int update_plan_monitor_info();
int prepre_bloom_filter_ctx(ObBloomFilterSendCtx *bf_ctx);
int open_join_filter_create();
int open_join_filter_use();
int close_join_filter_create();
int close_join_filter_use();
int init_shared_msgs_from_input();
int init_local_msg_from_shared_msg(ObP2PDatahubMsgBase &msg);
int release_local_msg();
int release_shared_msg();
private:
static const int64_t ADAPTIVE_BF_WINDOW_ORG_SIZE = 4096;
static constexpr double ACCEPTABLE_FILTER_RATE = 0.98;
public:
ObPXBloomFilterHashWrapper bf_key_;
ObPxBloomFilter *filter_use_;
ObPxBloomFilter *filter_create_;
ObPxBloomFilterChSets *bf_ch_sets_;
ObJoinFilterMsg *filter_create_msg_;
ObArray<ObP2PDatahubMsgBase *> shared_rf_msgs_; // sqc level share
ObArray<ObP2PDatahubMsgBase *> local_rf_msgs_;
uint64_t *batch_hash_values_;
ObArray<bool> lucky_devil_champions_;
};
}

View File

@ -128,7 +128,8 @@ ObPhysicalPlan::ObPhysicalPlan(MemoryContext &mem_context /* = CURRENT_CONTEXT *
need_record_plan_info_(false),
enable_append_(false),
append_table_id_(0),
logical_plan_()
logical_plan_(),
is_enable_px_fast_reclaim_(false)
{
}
@ -219,6 +220,7 @@ void ObPhysicalPlan::reset()
tm_sessid_ = -1;
need_record_plan_info_ = false;
logical_plan_.reset();
is_enable_px_fast_reclaim_ = false;
}
void ObPhysicalPlan::destroy()
@ -766,7 +768,8 @@ OB_SERIALIZE_MEMBER(ObPhysicalPlan,
min_cluster_version_,
need_record_plan_info_,
enable_append_,
append_table_id_);
append_table_id_,
is_enable_px_fast_reclaim_);
int ObPhysicalPlan::set_table_locations(const ObTablePartitionInfoArray &infos,
ObSchemaGetterGuard &schema_guard)

View File

@ -467,10 +467,14 @@ public:
min_cluster_version_ = curr_cluster_version;
}
}
int set_logical_plan(ObLogicalPlanRawData &logical_plan);
inline ObLogicalPlanRawData& get_logical_plan() { return logical_plan_; }
inline const ObLogicalPlanRawData& get_logical_plan()const { return logical_plan_; }
int set_feedback_info(ObExecContext &ctx);
void set_enable_px_fast_reclaim(bool value) { is_enable_px_fast_reclaim_ = value; }
bool is_enable_px_fast_reclaim() const { return is_enable_px_fast_reclaim_; }
public:
static const int64_t MAX_PRINTABLE_SIZE = 2 * 1024 * 1024;
private:
@ -640,6 +644,8 @@ public:
bool enable_append_; // for APPEND hint
uint64_t append_table_id_;
ObLogicalPlanRawData logical_plan_;
// for detecor manager
bool is_enable_px_fast_reclaim_;
};
inline void ObPhysicalPlan::set_affected_last_insert_id(bool affected_last_insert_id)

View File

@ -138,6 +138,17 @@ int ObPxFifoCoordOp::fetch_rows(const int64_t row_cnt)
first_row_fetched_ = true; // 控制不再主动调用 startup_msg_loop,后继 loop 都消息触发
}
#ifdef ERRSIM
ObSQLSessionInfo *session = ctx_.get_my_session();
int64_t query_timeout = 0;
session->get_query_timeout(query_timeout);
if (OB_FAIL(OB_E(EventTable::EN_PX_QC_EARLY_TERMINATE, query_timeout) OB_SUCCESS)) {
LOG_WARN("fifo qc not interrupt qc by design", K(ret), K(query_timeout));
sleep(14);
return ret;
}
#endif
while (OB_SUCC(ret)) {
// rows must received by coord op instead of receive op, otherwise we will
// trap in receive op and lose control.

View File

@ -608,8 +608,6 @@ int ObPxMSReceiveOp::inner_get_next_row()
LOG_ERROR("Get operator context failed", K(ret), K(MY_SPEC.id_));
} else if (OB_FAIL(try_link_channel())) {
LOG_WARN("failed to init channel", K(ret));
} else if (!ctx_.get_bloom_filter_ctx_array().empty() && OB_FAIL(prepare_send_bloom_filter())) {
LOG_WARN("fail to prepare send bloom filter", K(ret));
} else if (OB_FAIL(try_send_bloom_filter())) {
LOG_WARN("fail to send bloom filter", K(ret));
}

View File

@ -538,6 +538,16 @@ int ObPxReceiveOp::wrap_get_next_batch(const int64_t max_row_cnt)
int ObPxReceiveOp::erase_dtl_interm_result()
{
int ret = OB_SUCCESS;
#ifdef ERRSIM
ObSQLSessionInfo *session = ctx_.get_my_session();
int64_t query_timeout = 0;
session->get_query_timeout(query_timeout);
if (OB_FAIL(OB_E(EventTable::EN_PX_SINGLE_DFO_NOT_ERASE_DTL_INTERM_RESULT) OB_SUCCESS)) {
LOG_WARN("ObPxCoordOp not erase_dtl_interm_result by design", K(ret), K(query_timeout));
return OB_SUCCESS;
}
#endif
dtl::ObDtlChannelInfo ci;
ObDTLIntermResultKey key;
for (int i = 0; i < get_ch_set().count(); ++i) {
@ -713,33 +723,10 @@ int ObPxFifoReceiveOp::inner_get_next_batch(const int64_t max_row_cnt)
return ret;
}
int ObPxReceiveOp::prepare_send_bloom_filter()
{
int ret = OB_SUCCESS;
ObJoinFilterDataCtx temp_bf_ctx; // just for declare a reference, don't use temp_bf_ctx to do anything else
ObJoinFilterDataCtx &bf_ctx = temp_bf_ctx;
const ObPxReceiveSpec &spec = static_cast<const ObPxReceiveSpec &>(get_spec());
while(OB_SUCC(ret) && bf_ctx_idx_ < spec.bloom_filter_id_array_.count()) {
if (OB_FAIL(get_bf_ctx(bf_ctx_idx_++, bf_ctx))) {
// get_bf_ctx will find a valid bf_ctx, if not will report -4016
// bf_ctx_idx_++ means it need get next one bf_ctx at next fetch_rows
LOG_WARN("failed to get bloom filter context", K(bf_ctx_idx_), K(ret));
} else if (bf_ctx.filter_ready_ && OB_FAIL(ObPxMsgProc::mark_rpc_filter(ctx_, // bf_ctx.filter_ready_ promise that only one thread will run mark_rpc_filter()
bf_ctx,
each_group_size_))) {
LOG_WARN("fail to send rpc bloom filter", K(bf_ctx_idx_), K(each_group_size_), K(ret));
} else {
LOG_DEBUG("succ to mark rpc filter", K(bf_ctx_idx_), K(spec.bloom_filter_id_array_.count()), K(each_group_size_),
K(bf_ctx.filter_ready_));
}
}
return ret;
}
int ObPxReceiveOp::try_send_bloom_filter()
{
int ret = OB_SUCCESS;
ObPxReceiveOpInput *recv_input = reinterpret_cast<ObPxReceiveOpInput*>(input_);
/*ObPxReceiveOpInput *recv_input = reinterpret_cast<ObPxReceiveOpInput*>(input_);
ObPxSQCProxy *sqc_proxy = reinterpret_cast<ObPxSQCProxy *>(recv_input->get_ch_provider());
ObPhysicalPlanCtx *phy_plan_ctx = GET_PHY_PLAN_CTX(ctx_);
common::ObIArray<ObBloomFilterSendCtx> &bf_send_ctx_array = sqc_proxy->get_bf_send_ctx_array();
@ -802,7 +789,7 @@ int ObPxReceiveOp::try_send_bloom_filter()
}
}
++bf_send_idx_;
}
}*/
return ret;
}
@ -829,10 +816,6 @@ int ObPxFifoReceiveOp::fetch_rows(const int64_t row_cnt)
ObPhysicalPlanCtx *phy_plan_ctx = GET_PHY_PLAN_CTX(ctx_);
if (OB_FAIL(try_link_channel())) {
LOG_WARN("failed to init channel", K(ret));
} else if (!ctx_.get_bloom_filter_ctx_array().empty() && OB_FAIL(prepare_send_bloom_filter())) {
LOG_WARN("fail to prepare send bloom filter", K(ret));
} else if (OB_FAIL(try_send_bloom_filter())) {
LOG_WARN("fail to try send bloom filter", K(ret));
}
if (OB_SUCC(ret)) {
@ -935,9 +918,6 @@ int ObPxFifoReceiveOp::get_rows_from_channels(const int64_t row_cnt, int64_t tim
brs_.size_ = read_rows;
}
}
if (OB_SUCC(ret) && OB_FAIL(try_send_bloom_filter())) {
LOG_WARN("fail to try send bloom filter", K(ret));
}
break;
}
if (msg_loop_.all_eof(task_channels_.count())) {
@ -948,8 +928,6 @@ int ObPxFifoReceiveOp::get_rows_from_channels(const int64_t row_cnt, int64_t tim
if (OB_FAIL(msg_loop_.process_any())) {
if (OB_EAGAIN != ret) {
LOG_WARN("fail pop sqc execution result from channel", K(ret));
} else if (OB_FAIL(try_send_bloom_filter())) {
LOG_WARN("fail to try send bloom filter", K(ret));
} else {
ret = OB_EAGAIN;
}

View File

@ -22,6 +22,7 @@
#include "sql/dtl/ob_dtl_utils.h"
#include "sql/engine/px/ob_px_sqc_handler.h"
#include "sql/engine/aggregate/ob_merge_groupby_op.h"
#include "share/detect/ob_detect_manager_utils.h"
namespace oceanbase
{
@ -345,6 +346,14 @@ int ObPxTransmitOp::init_channel(ObPxTransmitOpInput &trans_input)
}
loop_.set_interm_result(use_interm_result);
int64_t thread_id = GETTID();
ObPxSqcHandler *handler = ctx_.get_sqc_handler();
bool should_reg_dm = use_interm_result && OB_NOT_NULL(handler) && handler->get_phy_plan().is_enable_px_fast_reclaim();
common::ObRegisterDmInfo register_dm_info;
if (should_reg_dm) {
ObDetectManagerUtils::prepare_register_dm_info(register_dm_info, handler);
}
ARRAY_FOREACH_X(channels, idx, cnt, OB_SUCC(ret)) {
dtl::ObDtlChannel *ch = channels.at(idx);
if (OB_ISNULL(ch)) {
@ -353,6 +362,11 @@ int ObPxTransmitOp::init_channel(ObPxTransmitOpInput &trans_input)
} else {
ch->set_audit(enable_audit);
ch->set_interm_result(use_interm_result);
// if use_interm_result, set register_dm_info in dtl channel
// so that the peer rpc processor can use this information to register check item into dm
if (should_reg_dm) {
ch->set_register_dm_info(register_dm_info);
}
ch->set_enable_channel_sync(min_cluster_version >= CLUSTER_VERSION_4_1_0_0);
ch->set_batch_id(px_batch_id);
ch->set_compression_type(dfc_.get_compressor_type());

View File

@ -21,6 +21,14 @@ using namespace oceanbase::common;
using namespace oceanbase::sql;
using namespace oceanbase::sql::dtl;
OB_SERIALIZE_MEMBER(ObPxDetectableIds,
qc_detectable_id_,
sqc_detectable_id_);
OB_SERIALIZE_MEMBER(ObP2PDhMapInfo,
p2p_sequence_ids_,
target_addrs_);
OB_SERIALIZE_MEMBER(ObPxSqcMeta,
execution_id_,
qc_id_,
@ -52,7 +60,10 @@ OB_SERIALIZE_MEMBER(ObPxSqcMeta,
access_table_location_keys_,
adjoining_root_dfo_,
is_single_tsc_leaf_dfo_,
access_external_table_files_);
access_external_table_files_,
px_detectable_ids_,
p2p_dh_map_info_,
sqc_count_);
OB_SERIALIZE_MEMBER(ObPxTask,
qc_id_,
dfo_id_,
@ -108,6 +119,8 @@ int ObPxSqcMeta::assign(const ObPxSqcMeta &other)
LOG_WARN("failed to assgin to table location keys.", K(ret));
} else if (OB_FAIL(access_table_location_indexes_.assign(other.access_table_location_indexes_))) {
LOG_WARN("failed to assgin to table location keys.", K(ret));
} else if (OB_FAIL(p2p_dh_map_info_.assign(other.p2p_dh_map_info_))) {
LOG_WARN("fail to assign p2p dh map info", K(ret));
} else {
execution_id_ = other.execution_id_;
qc_id_ = other.qc_id_;
@ -138,6 +151,9 @@ int ObPxSqcMeta::assign(const ObPxSqcMeta &other)
server_not_alive_ = other.server_not_alive_;
adjoining_root_dfo_ = other.adjoining_root_dfo_;
is_single_tsc_leaf_dfo_ = other.is_single_tsc_leaf_dfo_;
px_detectable_ids_ = other.px_detectable_ids_;
interrupt_by_dm_ = other.interrupt_by_dm_;
sqc_count_ = other.sqc_count_;
}
access_external_table_files_.reuse();
for (int i = 0; OB_SUCC(ret) && i < other.access_external_table_files_.count(); i++) {
@ -372,40 +388,6 @@ int ObDfo::alloc_data_xchg_ch()
}
return ret;
}
int ObDfo::alloc_bloom_filter_ch()
{
int ret = OB_SUCCESS;
if (is_px_create_bloom_filter()) {
} else if (is_px_use_bloom_filter()) {
use_filter_ch_map_.set_filter_id(px_bf_id_);
}
return ret;
}
int ObDfo::condition_push_back(ObPxBloomFilterChSet &ch_set, ObPxBloomFilterChSets &ch_sets)
{
int ret = OB_SUCCESS;
bool repeat = false;
for (int i = 0;i < ch_sets.count(); ++i) {
if (ch_set.get_exec_addr() == ch_sets.at(i).get_exec_addr()) {
repeat = true;
break;
}
}
if (!repeat && OB_FAIL(ch_sets.push_back(ch_set))) {
LOG_WARN("fail to push back ch_set", K(ret));
}
return ret;
}
int ObDfo::get_use_filter_chs(ObPxBloomFilterChInfo &create_filter_ch_map)
{
int ret = OB_SUCCESS;
ret = create_filter_ch_map.assign(use_filter_ch_map_);
return ret;
}
int ObDfo::get_task_transmit_chs_for_update(ObIArray<ObPxTaskChSet *> &ch_sets)
{
@ -1002,6 +984,7 @@ void ObDfo::reset_resource(ObDfo *dfo)
}
}
dfo->transmit_ch_sets_.reset();
dfo->p2p_dh_map_info_.destroy();
dfo->~ObDfo();
dfo = nullptr;
}
@ -1023,4 +1006,4 @@ const ObPhysicalPlan* ObDfo::get_plan_by_root()
plan = root_op_spec_->get_phy_plan();
}
return plan;
}
}

View File

@ -34,8 +34,14 @@
#include "sql/das/ob_das_define.h"
#include "lib/string/ob_strings.h"
#include "share/external_table/ob_external_table_file_mgr.h"
#include "share/detect/ob_detect_callback.h"
namespace oceanbase
{
namespace common
{
class ObIDetectCallback;
}
namespace sql
{
@ -55,6 +61,7 @@ class ObPhysicalPlan;
class ObSqcTaskMgr;
class ObPxSqcHandler;
class ObJoinFilter;
class ObPxCoordInfo;
// 在 PX 端描述每个 SQC 的 task
// 通过 exec_addr 区分 SQC
@ -121,6 +128,60 @@ public:
int64_t location_end_pos_;
TO_STRING_KV(K_(table_location_key), K_(location_start_pos), K_(location_end_pos));
};
struct ObPxDetectableIds
{
OB_UNIS_VERSION(1);
public:
ObPxDetectableIds() : qc_detectable_id_(), sqc_detectable_id_() {}
ObPxDetectableIds(const common::ObDetectableId &qc_detectable_id, const common::ObDetectableId &sqc_detectable_id)
: qc_detectable_id_(qc_detectable_id), sqc_detectable_id_(sqc_detectable_id) {}
void operator=(const ObPxDetectableIds &other)
{
qc_detectable_id_ = other.qc_detectable_id_;
sqc_detectable_id_ = other.sqc_detectable_id_;
}
bool operator==(const ObPxDetectableIds &other) const
{
return qc_detectable_id_ == other.qc_detectable_id_ &&
sqc_detectable_id_ == other.sqc_detectable_id_;
}
TO_STRING_KV(K_(qc_detectable_id), K_(sqc_detectable_id));
common::ObDetectableId qc_detectable_id_;
common::ObDetectableId sqc_detectable_id_;
};
struct ObP2PDhMapInfo
{
OB_UNIS_VERSION(1);
public:
ObP2PDhMapInfo() : p2p_sequence_ids_(),
target_addrs_() {}
~ObP2PDhMapInfo() {
p2p_sequence_ids_.reset();
target_addrs_.reset();
}
void destroy() {
p2p_sequence_ids_.reset();
target_addrs_.reset();
}
int assign(const ObP2PDhMapInfo &other) {
int ret = OB_SUCCESS;
if (OB_FAIL(p2p_sequence_ids_.assign(other.p2p_sequence_ids_))) {
OB_LOG(WARN, "fail to assign other p2p seq id", K(ret));
} else if (OB_FAIL(target_addrs_.assign(other.target_addrs_))) {
OB_LOG(WARN, "fail to assign other target_addrs_", K(ret));
}
return ret;
}
bool is_empty() { return p2p_sequence_ids_.empty(); }
public:
common::ObSArray<int64_t> p2p_sequence_ids_;
common::ObSArray<ObSArray<ObAddr>> target_addrs_;
TO_STRING_KV(K_(p2p_sequence_ids), K_(target_addrs));
};
// PX 端描述每个 SQC 的数据结构
class ObPxSqcMeta
{
@ -165,7 +226,10 @@ public:
adjoining_root_dfo_(false),
is_single_tsc_leaf_dfo_(false),
allocator_("PxSqcMetaInner"),
access_external_table_files_()
access_external_table_files_(),
px_detectable_ids_(),
interrupt_by_dm_(false),
p2p_dh_map_info_()
{}
~ObPxSqcMeta() = default;
int assign(const ObPxSqcMeta &other);
@ -189,6 +253,10 @@ public:
inline ObPxInterruptID get_interrupt_id() { return px_int_id_; }
inline void set_interrupt_id(const ObPxInterruptID &px_int_id)
{ px_int_id_ = px_int_id; }
void set_px_detectable_ids(const ObPxDetectableIds &px_detectable_ids) { px_detectable_ids_ = px_detectable_ids; }
const ObPxDetectableIds &get_px_detectable_ids() { return px_detectable_ids_; }
void set_interrupt_by_dm(bool val) { interrupt_by_dm_ = val; }
bool is_interrupt_by_dm() { return interrupt_by_dm_; }
void set_exec_addr(const common::ObAddr &addr) { exec_addr_ = addr; }
void set_qc_addr(const common::ObAddr &addr) { qc_addr_ = addr; }
void set_qc_channel(dtl::ObDtlChannel *ch) { qc_channel_ = ch; }
@ -276,6 +344,9 @@ public:
bool adjoining_root_dfo() const { return adjoining_root_dfo_; }
void set_single_tsc_leaf_dfo(bool flag) { is_single_tsc_leaf_dfo_ = flag; }
bool is_single_tsc_leaf_dfo() { return is_single_tsc_leaf_dfo_; }
ObP2PDhMapInfo &get_p2p_dh_map_info() { return p2p_dh_map_info_;};
void set_sqc_count(int64_t sqc_cnt) { sqc_count_ = sqc_cnt; }
int64_t get_sqc_count() const { return sqc_count_;}
TO_STRING_KV(K_(need_report), K_(execution_id), K_(qc_id), K_(sqc_id), K_(dfo_id), K_(exec_addr), K_(qc_addr),
K_(qc_ch_info), K_(sqc_ch_info),
K_(task_count), K_(max_task_count), K_(min_task_count),
@ -342,6 +413,12 @@ private:
bool is_single_tsc_leaf_dfo_;
ObArenaAllocator allocator_;
ObSEArray<share::ObExternalFileInfo, 8> access_external_table_files_;
ObPxDetectableIds px_detectable_ids_;
bool interrupt_by_dm_;
// for p2p dh msg
ObP2PDhMapInfo p2p_dh_map_info_;
int64_t sqc_count_;
};
class ObDfo
@ -390,13 +467,19 @@ public:
temp_table_id_(0),
slave_mapping_type_(SlaveMappingType::SM_NONE),
part_ch_map_(),
px_bloom_filter_mode_(JoinFilterMode::NOT_INIT),
px_bf_id_(OB_INVALID_ID),
use_filter_ch_map_(),
total_task_cnt_(0),
pkey_table_loc_id_(0),
tsc_op_cnt_(0),
external_table_files_()
external_table_files_(),
px_detectable_ids_(),
detect_cb_(nullptr),
node_sequence_id_(0),
p2p_dh_ids_(),
p2p_dh_addrs_(),
p2p_dh_loc_(nullptr),
need_p2p_info_(false),
p2p_dh_map_info_(),
coord_info_ptr_(nullptr)
{
}
@ -452,7 +535,6 @@ public:
int64_t get_sqcs_count() { return sqcs_.count(); }
int build_tasks();
int alloc_data_xchg_ch();
int alloc_bloom_filter_ch();
/* 获取 qc 端的 channel 端口 */
int get_qc_channels(common::ObIArray<dtl::ObDtlChannel *> &sqc_chs);
@ -497,6 +579,13 @@ public:
void set_px_sequence_id(uint64_t px_sequence_id) { px_sequence_id_ = px_sequence_id; }
int64_t get_px_sequence_id() const { return px_sequence_id_; }
void set_px_detectable_ids(const ObPxDetectableIds &px_detectable_ids) { px_detectable_ids_ = px_detectable_ids; }
const ObPxDetectableIds &get_px_detectable_ids() { return px_detectable_ids_; }
common::ObIDetectCallback *get_detect_cb() { return detect_cb_; }
void set_detect_cb(common::ObIDetectCallback *cb) { detect_cb_ = cb; }
void set_node_sequence_id(uint64_t node_sequence_id) { node_sequence_id_ = node_sequence_id; }
uint64_t get_node_sequence_id() { return node_sequence_id_; }
void set_temp_table_id(uint64_t temp_table_id) { temp_table_id_ = temp_table_id; }
uint64_t get_temp_table_id() const { return temp_table_id_; }
@ -529,21 +618,6 @@ public:
void set_dist_method(ObPQDistributeMethod::Type dist_method) { dist_method_ = dist_method; }
ObPQDistributeMethod::Type get_dist_method() { return dist_method_; }
void set_px_bloom_filter_mode(JoinFilterMode mode) { px_bloom_filter_mode_ = mode; }
JoinFilterMode get_is_px_bloom_filter() { return px_bloom_filter_mode_; }
bool is_px_use_bloom_filter()
{ return JoinFilterMode::USE == px_bloom_filter_mode_; }
bool is_px_create_bloom_filter()
{ return JoinFilterMode::CREATE == px_bloom_filter_mode_; }
bool have_px_bloom_filter()
{ return JoinFilterMode::USE == px_bloom_filter_mode_ ||
JoinFilterMode::CREATE == px_bloom_filter_mode_; }
void set_px_bf_id(int64_t id) { px_bf_id_ = id; }
int64_t get_px_bf_id() const { return px_bf_id_; }
int get_use_filter_chs(ObPxBloomFilterChInfo &create_filter_ch_map);
ObPxBloomFilterChInfo &get_use_filter_ch_info() { return use_filter_ch_map_; }
ObPxChTotalInfos &get_dfo_ch_total_infos() { return dfo_ch_infos_; }
int64_t get_total_task_count() { return total_task_cnt_; }
int get_dfo_ch_info(int64_t sqc_idx, dtl::ObDtlChTotalInfo *&ch_info);
@ -566,6 +640,16 @@ public:
bool is_leaf_dfo() { return child_dfos_.empty(); }
bool is_single_tsc_leaf_dfo() { return is_leaf_dfo() && 1 == tsc_op_cnt_; }
common::ObIArray<share::ObExternalFileInfo> &get_external_table_files() { return external_table_files_; }
int add_p2p_dh_ids(int64_t id) { return p2p_dh_ids_.push_back(id); }
common::ObIArray<int64_t> &get_p2p_dh_ids() { return p2p_dh_ids_; }
void set_p2p_dh_loc(ObDASTableLoc *p2p_dh_loc) { p2p_dh_loc_ = p2p_dh_loc; }
ObDASTableLoc *get_p2p_dh_loc() { return p2p_dh_loc_; }
common::ObIArray<ObAddr> &get_p2p_dh_addrs() { return p2p_dh_addrs_; }
void set_need_p2p_info(bool flag) { need_p2p_info_ = flag; }
bool need_p2p_info() { return need_p2p_info_; }
void set_coord_info_ptr(ObPxCoordInfo *ptr) { coord_info_ptr_ = ptr; }
ObPxCoordInfo *get_coord_info_ptr() { return coord_info_ptr_; }
ObP2PDhMapInfo &get_p2p_dh_map_info() { return p2p_dh_map_info_;};
TO_STRING_KV(K_(execution_id),
K_(dfo_id),
K_(is_active),
@ -586,18 +670,18 @@ public:
"child", get_child_count(),
K_(slave_mapping_type),
K_(dist_method),
K_(px_bloom_filter_mode),
K_(px_bf_id),
K_(pkey_table_loc_id),
K_(tsc_op_cnt),
K_(transmit_ch_sets),
K_(receive_ch_sets_map));
K_(receive_ch_sets_map),
K_(p2p_dh_ids),
K_(p2p_dh_addrs),
K_(need_p2p_info));
private:
DISALLOW_COPY_AND_ASSIGN(ObDfo);
private:
int calc_total_task_count();
int condition_push_back(ObPxBloomFilterChSet &ch_set, ObPxBloomFilterChSets &ch_sets);
private:
common::ObIAllocator &allocator_;
uint64_t execution_id_;
@ -653,13 +737,23 @@ private:
SlaveMappingType slave_mapping_type_;
ObPxPartChMapArray part_ch_map_;
ObPQDistributeMethod::Type dist_method_;
JoinFilterMode px_bloom_filter_mode_; //标记dfo中的px bloom filter
int64_t px_bf_id_; //记录px_bloom_filter_id
ObPxBloomFilterChInfo use_filter_ch_map_; // use and create channel info is same
int64_t total_task_cnt_; // the task total count of dfo start worker
int64_t pkey_table_loc_id_; // record pkey table loc id for child dfo
int64_t tsc_op_cnt_;
common::ObArray<share::ObExternalFileInfo> external_table_files_;
// for dm
ObPxDetectableIds px_detectable_ids_;
common::ObIDetectCallback *detect_cb_;
uint64_t node_sequence_id_;
// ---------------
// for p2p dh mgr
common::ObArray<int64_t>p2p_dh_ids_; //for dh create
common::ObArray<ObAddr>p2p_dh_addrs_; //for dh use
ObDASTableLoc *p2p_dh_loc_;
bool need_p2p_info_;
ObP2PDhMapInfo p2p_dh_map_info_;
// ---------------
ObPxCoordInfo *coord_info_ptr_;
};

View File

@ -23,6 +23,7 @@
#include "sql/engine/px/exchange/ob_px_repart_transmit_op.h"
#include "sql/optimizer/ob_px_resource_analyzer.h"
#include "sql/engine/px/ob_px_scheduler.h"
#include "share/detect/ob_detect_manager_utils.h"
#include "sql/engine/px/ob_px_coord_op.h"
using namespace oceanbase::common;
@ -472,6 +473,26 @@ int ObDfoMgr::do_split(ObExecContext &exec_ctx,
} else {
px_coord_info.table_access_type_ = TableAccessType::PURE_VIRTUAL_TABLE;
}
if (parent_dfo->need_p2p_info_ && parent_dfo->get_p2p_dh_addrs().empty()) {
ObDASTableLoc *table_loc = nullptr;
if (OB_ISNULL(table_loc = DAS_CTX(exec_ctx).get_table_loc_by_id(
tsc_op->get_table_loc_id(), tsc_op->get_loc_ref_table_id()))) {
OZ(ObTableLocation::get_full_leader_table_loc(exec_ctx.get_allocator(),
exec_ctx.get_my_session()->get_effective_tenant_id(),
tsc_op->get_table_loc_id(),
tsc_op->get_loc_ref_table_id(),
table_loc));
}
if (OB_FAIL(ret)) {
} else {
const DASTabletLocList &locations = table_loc->get_tablet_locs();
parent_dfo->set_p2p_dh_loc(table_loc);
if (OB_FAIL(get_location_addrs<DASTabletLocList>(locations,
parent_dfo->get_p2p_dh_addrs()))) {
LOG_WARN("fail get location addrs", K(ret));
}
}
}
} else if (phy_op->is_dml_operator() && NULL != parent_dfo) {
// 当前op是一个dml算子,需要设置dfo的属性
parent_dfo->set_dml_op(true);
@ -479,11 +500,43 @@ int ObDfoMgr::do_split(ObExecContext &exec_ctx,
parent_dfo->set_temp_table_scan(true);
const ObTempTableAccessOpSpec *access = static_cast<const ObTempTableAccessOpSpec*>(phy_op);
parent_dfo->set_temp_table_id(access->get_table_id());
} else if (IS_PX_BLOOM_FILTER(phy_op->get_type()) && NULL != parent_dfo) {
if (parent_dfo->need_p2p_info_ && parent_dfo->get_p2p_dh_addrs().empty()) {
OZ(px_coord_info.p2p_temp_table_info_.temp_access_ops_.push_back(phy_op));
OZ(px_coord_info.p2p_temp_table_info_.dfos_.push_back(parent_dfo));
}
} else if (IS_PX_GI(phy_op->get_type()) && NULL != parent_dfo) {
const ObGranuleIteratorSpec *gi_spec =
static_cast<const ObGranuleIteratorSpec *>(phy_op);
if (gi_spec->bf_info_.is_inited_) {
ObP2PDfoMapNode node;
node.target_dfo_id_ = parent_dfo->get_dfo_id();
if (OB_FAIL(px_coord_info.p2p_dfo_map_.set_refactored(
gi_spec->bf_info_.p2p_dh_id_,
node))) {
LOG_WARN("fail to set p2p dh id to map", K(ret));
} else {
parent_dfo->set_need_p2p_info(true);
}
}
} else if (IS_PX_JOIN_FILTER(phy_op->get_type()) && NULL != parent_dfo) {
const ObJoinFilterSpec *filter_spec = static_cast<const ObJoinFilterSpec *>(phy_op);
if(filter_spec->is_shuffle()) {
parent_dfo->set_px_bf_id(filter_spec->get_filter_id());
parent_dfo->set_px_bloom_filter_mode(filter_spec->get_mode());
if(filter_spec->is_shared_join_filter() && filter_spec->is_shuffle_) {
ObP2PDfoMapNode node;
node.target_dfo_id_ = parent_dfo->get_dfo_id();
for (int i = 0; i < filter_spec->rf_infos_.count() && OB_SUCC(ret); ++i) {
if (filter_spec->is_create_mode()) {
if (OB_FAIL(parent_dfo->add_p2p_dh_ids(
filter_spec->rf_infos_.at(i).p2p_datahub_id_))) {
LOG_WARN("fail to add p2p dh ids", K(ret));
}
} else if (OB_FAIL(px_coord_info.p2p_dfo_map_.set_refactored(
filter_spec->rf_infos_.at(i).p2p_datahub_id_,
node))) {
LOG_WARN("fail to set p2p dh id to map", K(ret));
} else {
parent_dfo->set_need_p2p_info(true);
}
}
}
} else if (IS_PX_COORD(phy_op->type_)) {
if (top_px) {
@ -511,63 +564,89 @@ int ObDfoMgr::do_split(ObExecContext &exec_ctx,
if (OB_SUCC(ret) && nullptr != dfo) {
if (IS_PX_COORD(phy_op->type_)) {
dfo->set_coord_info_ptr(&px_coord_info);
dfo->set_root_dfo(true);
dfo->set_single(true);
dfo->set_dop(1);
dfo->set_execution_id(exec_ctx.get_my_session()->get_current_execution_id());
dfo->set_px_sequence_id(dfo_int_gen.get_px_sequence_id());
// 存在嵌套情况,则dfo可能已经被设置过一些信息,所以这里不会覆盖
if (OB_INVALID_ID == dfo->get_dfo_id()) {
//只有顶层的dfo的receive才没有设置dfo id,即使嵌套dfo,也会设置,因为会根据transmit进行设置
dfo->set_dfo_id(ObDfo::MAX_DFO_ID);
}
if (OB_INVALID_ID == dfo->get_qc_id()) {
// receive的px记录在了transmit上
const ObTransmitSpec *transmit = static_cast<const ObTransmitSpec *>(phy_op->get_child());
if (OB_INVALID_ID != transmit->get_px_id()) {
dfo->set_qc_id(transmit->get_px_id());
if (OB_NOT_NULL(phy_op->get_phy_plan()) && phy_op->get_phy_plan()->is_enable_px_fast_reclaim()) {
ObDetectableId sqc_detectable_id;
// if generate_detectable_id failed, means that server id is not ready
if (OB_FAIL(ObDetectManagerUtils::generate_detectable_id(sqc_detectable_id, GET_TENANT_ID()))) {
LOG_WARN("[DM] failed to generate_detectable_id for sqc");
} else {
ObPxDetectableIds px_detectable_ids(px_coord_info.qc_detectable_id_, sqc_detectable_id);
dfo->set_px_detectable_ids(px_detectable_ids);
}
}
// 对于 root dfo 来说,它并不是一个真实的 dfo,没有分配 id
// 所以使用 ObDfo::MAX_DFO_ID表示
if (OB_FAIL(dfo_int_gen.gen_id(dfo->get_dfo_id(), dfo->get_interrupt_id()))) {
LOG_WARN("fail gen dfo int id", K(ret));
if (OB_SUCC(ret)) {
// 存在嵌套情况,则dfo可能已经被设置过一些信息,所以这里不会覆盖
if (OB_INVALID_ID == dfo->get_dfo_id()) {
//只有顶层的dfo的receive才没有设置dfo id,即使嵌套dfo,也会设置,因为会根据transmit进行设置
dfo->set_dfo_id(ObDfo::MAX_DFO_ID);
}
if (OB_INVALID_ID == dfo->get_qc_id()) {
// receive的px记录在了transmit上
const ObTransmitSpec *transmit = static_cast<const ObTransmitSpec *>(phy_op->get_child());
if (OB_INVALID_ID != transmit->get_px_id()) {
dfo->set_qc_id(transmit->get_px_id());
}
}
// 对于 root dfo 来说,它并不是一个真实的 dfo,没有分配 id
// 所以使用 ObDfo::MAX_DFO_ID表示
if (OB_FAIL(dfo_int_gen.gen_id(dfo->get_dfo_id(), dfo->get_interrupt_id()))) {
LOG_WARN("fail gen dfo int id", K(ret));
}
LOG_TRACE("cur dfo info", K(dfo->get_qc_id()), K(dfo->get_dfo_id()), K(dfo->get_dop()));
}
LOG_TRACE("cur dfo info", K(dfo->get_qc_id()), K(dfo->get_dfo_id()), K(dfo->get_dop()));
} else {
const ObTransmitSpec *transmit = static_cast<const ObTransmitSpec *>(phy_op);
// 如果 transmit 下面的子树里包含 px coord 算子,那么下面这些设置都会被
// 修改成 is_local = true, dop = 1
dfo->set_coord_info_ptr(&px_coord_info);
dfo->set_single(transmit->is_px_single());
dfo->set_dop(transmit->get_px_dop());
dfo->set_qc_id(transmit->get_px_id());
dfo->set_dfo_id(transmit->get_dfo_id());
dfo->set_execution_id(exec_ctx.get_my_session()->get_current_execution_id());
dfo->set_px_sequence_id(dfo_int_gen.get_px_sequence_id());
dfo->set_dist_method(transmit->dist_method_);
dfo->set_slave_mapping_type(transmit->get_slave_mapping_type());
parent_dfo->set_slave_mapping_type(transmit->get_slave_mapping_type());
dfo->set_pkey_table_loc_id(
(reinterpret_cast<const ObPxTransmitSpec *>(transmit))->repartition_table_id_);
if (OB_ISNULL(parent_dfo)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("parent dfo should not be null", K(ret));
} else if (transmit->get_px_dop() <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("should have dop set by optimizer", K(ret), K(transmit->get_px_dop()));
} else if (OB_FAIL(dfo_int_gen.gen_id(transmit->get_dfo_id(),
dfo->get_interrupt_id()))) {
LOG_WARN("fail gen dfo int id", K(ret));
} else {
dfo->set_qc_server_id(GCTX.server_id_);
dfo->set_parent_dfo_id(parent_dfo->get_dfo_id());
LOG_TRACE("cur dfo dop",
"dfo_id", dfo->get_dfo_id(),
"is_local", transmit->is_px_single(),
"dop", transmit->get_px_dop(),
K(dfo->get_qc_id()),
"parent dfo_id", parent_dfo->get_dfo_id(),
"slave mapping", transmit->is_slave_mapping());
if (OB_NOT_NULL(phy_op->get_phy_plan()) && phy_op->get_phy_plan()->is_enable_px_fast_reclaim()) {
ObDetectableId sqc_detectable_id;
// if generate_detectable_id failed, means that server id is not ready
if (OB_FAIL(ObDetectManagerUtils::generate_detectable_id(sqc_detectable_id, GET_TENANT_ID()))) {
LOG_WARN("[DM] failed to generate_detectable_id for sqc");
} else {
ObPxDetectableIds px_detectable_ids(px_coord_info.qc_detectable_id_, sqc_detectable_id);
dfo->set_px_detectable_ids(px_detectable_ids);
}
}
if (OB_SUCC(ret)) {
dfo->set_dist_method(transmit->dist_method_);
dfo->set_slave_mapping_type(transmit->get_slave_mapping_type());
parent_dfo->set_slave_mapping_type(transmit->get_slave_mapping_type());
dfo->set_pkey_table_loc_id(
(reinterpret_cast<const ObPxTransmitSpec *>(transmit))->repartition_table_id_);
if (OB_ISNULL(parent_dfo)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("parent dfo should not be null", K(ret));
} else if (transmit->get_px_dop() <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("should have dop set by optimizer", K(ret), K(transmit->get_px_dop()));
} else if (OB_FAIL(dfo_int_gen.gen_id(transmit->get_dfo_id(),
dfo->get_interrupt_id()))) {
LOG_WARN("fail gen dfo int id", K(ret));
} else {
dfo->set_qc_server_id(GCTX.server_id_);
dfo->set_parent_dfo_id(parent_dfo->get_dfo_id());
LOG_TRACE("cur dfo dop",
"dfo_id", dfo->get_dfo_id(),
"is_local", transmit->is_px_single(),
"dop", transmit->get_px_dop(),
K(dfo->get_qc_id()),
"parent dfo_id", parent_dfo->get_dfo_id(),
"slave mapping", transmit->is_slave_mapping());
}
}
}
}

View File

@ -21,6 +21,9 @@
#include "sql/engine/px/ob_px_dtl_msg.h"
#include "sql/engine/px/ob_px_rpc_processor.h"
#include "sql/engine/px/ob_px_sqc_async_proxy.h"
#include "share/ob_server_blacklist.h"
#include "share/detect/ob_detect_manager_utils.h"
#include "ob_px_coord_op.h"
using namespace oceanbase::common;
using namespace oceanbase::share;
@ -64,6 +67,37 @@ int ObDfoSchedulerBasic::init_all_dfo_channel(ObExecContext &ctx) const
return ret;
}
int ObDfoSchedulerBasic::prepare_schedule_info(ObExecContext &exec_ctx)
{
int ret = OB_SUCCESS;
// for temp table
if (!coord_info_.p2p_temp_table_info_.temp_access_ops_.empty()) {
ObIArray<ObSqlTempTableCtx>& temp_ctx = exec_ctx.get_temp_table_ctx();
ObSqlTempTableCtx *ctx = nullptr;
CK(coord_info_.p2p_temp_table_info_.temp_access_ops_.count() ==
coord_info_.p2p_temp_table_info_.dfos_.count());
for (int i = 0; i < coord_info_.p2p_temp_table_info_.dfos_.count() && OB_SUCC(ret); ++i) {
ObDfo *parent_dfo = coord_info_.p2p_temp_table_info_.dfos_.at(i);
ctx = nullptr;
if (parent_dfo->need_p2p_info() && parent_dfo->get_p2p_dh_addrs().empty()) {
for (int64_t j = 0; nullptr == ctx && j < temp_ctx.count(); j++) {
if (parent_dfo->get_temp_table_id() == temp_ctx.at(j).temp_table_id_) {
ctx = &temp_ctx.at(j);
}
}
if (OB_NOT_NULL(ctx) && !ctx->interm_result_infos_.empty()) {
for (int j = 0; OB_SUCC(ret) && j < ctx->interm_result_infos_.count(); ++j) {
if (OB_FAIL(parent_dfo->get_p2p_dh_addrs().push_back(ctx->interm_result_infos_.at(j).addr_))) {
LOG_WARN("fail to push back p2p dh addrs", K(ret));
}
}
}
}
}
}
return ret;
}
int ObDfoSchedulerBasic::on_sqc_threads_inited(ObExecContext &ctx, ObDfo &dfo) const
{
int ret = OB_SUCCESS;
@ -71,11 +105,6 @@ int ObDfoSchedulerBasic::on_sqc_threads_inited(ObExecContext &ctx, ObDfo &dfo) c
if (OB_FAIL(dfo.prepare_channel_info())) {
LOG_WARN("failed to prepare channel info", K(ret));
}
if (OB_SUCC(ret)) {
if (dfo.have_px_bloom_filter() && OB_FAIL(dfo.alloc_bloom_filter_ch())) {
LOG_WARN("fail to init px bloom filter channel", K(ret));
}
}
LOG_TRACE("on_sqc_threads_inited: dfo data xchg ch allocated", K(ret));
return ret;
}
@ -239,80 +268,6 @@ int ObDfoSchedulerBasic::dispatch_transmit_channel_info_via_sqc(ObExecContext &c
return ret;
}
int ObDfoSchedulerBasic::dispatch_bf_channel_info(ObExecContext &ctx,
ObDfo &child, ObDfo &parent) const
{
int ret = OB_SUCCESS;
UNUSED(ctx);
UNUSED(child);
ObPhysicalPlanCtx *phy_plan_ctx = NULL;
if (OB_ISNULL(phy_plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("phy plan ctx NULL", K(ret));
} else if (parent.is_root_dfo()) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("not support root dfo send bloom filter", K(ret));
LOG_USER_ERROR(OB_NOT_SUPPORTED, "root dfo send bloom filter");
} else {
// send to dfo with receive operator
ObArray<ObPxSqcMeta *> sqcs;
if (OB_FAIL(parent.get_sqcs(sqcs))) {
LOG_WARN("fail get sqcs", K(parent), K(ret));
} else {
ARRAY_FOREACH_X(sqcs, idx, cnt, OB_SUCC(ret)) {
ObDtlChannel *ch = sqcs.at(idx)->get_qc_channel();
int64_t sqc_id = sqcs.at(idx)->get_sqc_id();
ObPxCreateBloomFilterChannelMsg bloom_filter_channel_msg;
bloom_filter_channel_msg.sqc_count_ = sqcs.count();
bloom_filter_channel_msg.sqc_id_ = sqc_id;
if (OB_ISNULL(ch) || OB_INVALID_INDEX == sqc_id) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected param", KP(ch), K(parent), K(sqc_id), K(ret));
} else if (OB_FAIL(child.get_use_filter_chs(bloom_filter_channel_msg.ch_set_info_))) {
LOG_WARN("fail get parent channel", K(ret));
} else if (OB_FAIL(ch->send(bloom_filter_channel_msg,
phy_plan_ctx->get_timeout_timestamp()))) {
LOG_WARN("fail push data to channel", K(ret));
} else if (OB_FAIL(ch->flush(true, false))) {
LOG_WARN("fail flush dtl data", K(ret));
} else {
LOG_TRACE("dispatched bf ch",
K(idx), K(cnt), K(*ch), K(sqc_id));
}
}
if (OB_SUCC(ret) && OB_FAIL(ObPxChannelUtil::sqcs_channles_asyn_wait(sqcs))) {
LOG_WARN("failed to wait for sqcs", K(ret));
}
}
}
return ret;
}
int ObDfoSchedulerBasic::set_bloom_filter_ch_for_root_dfo(ObExecContext &ctx,
ObDfo &dfo) const
{
int ret = OB_SUCCESS;
UNUSED(ctx);
dfo.set_px_bloom_filter_mode(JoinFilterMode::CREATE);
if (OB_FAIL(dfo.alloc_bloom_filter_ch())) {
LOG_WARN("fail to alloc bloom filter channel", K(ret));
}
return ret;
}
int ObDfoSchedulerBasic::build_bloom_filter_ch(ObExecContext &ctx,
ObDfo &child, ObDfo &parent) const
{
int ret = OB_SUCCESS;
ObPxBloomFilterChInfo &use_filter_ch_info = child.get_use_filter_ch_info();
uint64_t tenant_id = OB_INVALID_ID;
if (OB_FAIL(get_tenant_id(ctx, tenant_id))) {
} else if (OB_FAIL(ObSlaveMapUtil::build_bf_mn_channel(
use_filter_ch_info, child, parent, tenant_id))) {
LOG_WARN("failed to build mn channel", K(ret));
}
return ret;
}
// -------------分割线-----------
int ObSerialDfoScheduler::init_all_dfo_channel(ObExecContext &ctx) const
{
@ -770,6 +725,7 @@ int ObParallelDfoScheduler::do_schedule_dfo(ObExecContext &exec_ctx, ObDfo &dfo)
ch->set_thread_id(thread_id);
(void)coord_info_.msg_loop_.register_channel(*ch);
sqc.set_qc_channel(ch);
sqc.set_sqc_count(sqcs.count());
LOG_TRACE("link qc-sqc channel and registered to qc msg loop. ready to receive sqc ctrl msg",
K(idx), K(cnt), K(*ch), K(dfo), K(sqc));
}
@ -958,8 +914,6 @@ int ObParallelDfoScheduler::check_if_can_prealloc_xchg_ch(ObDfo &child,
if (child.is_scheduled() || parent.is_scheduled()) {
bret = false;
} else if (parent.is_px_create_bloom_filter() && child.is_px_use_bloom_filter()) {
bret = false;
} else if (OB_FAIL(child.get_sqcs(sqcs))) {
LOG_WARN("fail to get child sqcs", K(ret));
} else {
@ -1174,6 +1128,8 @@ int ObParallelDfoScheduler::dispatch_sqc(ObExecContext &exec_ctx,
sqc.set_adjoining_root_dfo(true);
}
}
ObArray<ObPeerTaskState> peer_states;
ObArray<dtl::ObDtlChannel *> dtl_channels;
// 分发 sqc 可能需要重试,
// 分发 sqc 的 rpc 成功,但 sqc 上无法分配最小个数的 worker 线程,`dispatch_sqc`内部进行重试,
@ -1198,6 +1154,14 @@ int ObParallelDfoScheduler::dispatch_sqc(ObExecContext &exec_ctx,
OB_SUCCESS == cb->get_ret_code().rcode_ &&
OB_SUCCESS == cb->get_result().rc_) {
sqc.set_need_report(true);
ObPeerTaskState peer_state(sqc.get_sqc_addr());
if (OB_FAIL(peer_states.push_back(peer_state))) {
LOG_WARN("[DM] fail to push back", K(ret), K(sqc.get_sqc_addr()),
K(dfo.get_px_detectable_ids().sqc_detectable_id_));
} else if (OB_FAIL(dtl_channels.push_back(sqc.get_qc_channel()))) {
LOG_WARN("[DM] fail to push back dtl channels", K(ret), K(sqc.get_sqc_addr()),
K(dfo.get_px_detectable_ids().sqc_detectable_id_));
}
} else if (!cb->is_processed()) {
// if init_sqc_msg is not processed and the msg may be sent successfully, set server not alive.
// then when qc waiting_all_dfo_exit, it will push sqc.access_table_locations into trans_result,
@ -1225,6 +1189,15 @@ int ObParallelDfoScheduler::dispatch_sqc(ObExecContext &exec_ctx,
const ObPxRpcInitSqcResponse &resp = (*cb).get_result();
ObPxSqcMeta &sqc = *sqcs.at(idx);
sqc.set_need_report(true);
ObPeerTaskState peer_state(sqc.get_sqc_addr());
if (OB_FAIL(peer_states.push_back(peer_state))) {
LOG_WARN("[DM] fail to push back", K(ret), K(sqc.get_sqc_addr()),
K(dfo.get_px_detectable_ids().sqc_detectable_id_));
} else if (OB_FAIL(dtl_channels.push_back(sqc.get_qc_channel()))) {
LOG_WARN("[DM] fail to push back dtl channels", K(ret), K(sqc.get_sqc_addr()),
K(dfo.get_px_detectable_ids().sqc_detectable_id_));
}
if (!fast_sqc) {
ObPxInitSqcResultMsg pkt;
pkt.dfo_id_ = sqc.get_dfo_id();
@ -1243,6 +1216,15 @@ int ObParallelDfoScheduler::dispatch_sqc(ObExecContext &exec_ctx,
}
}
}
if (OB_NOT_NULL(phy_plan) && phy_plan->is_enable_px_fast_reclaim() && !peer_states.empty()
&& peer_states.count() == dtl_channels.count()) {
int reg_ret = ObDetectManagerUtils::qc_register_check_item_into_dm(dfo, peer_states, dtl_channels);
if (OB_SUCCESS != reg_ret) {
LOG_WARN("[DM] qc failed to register_check_item_into_dm", K(reg_ret),
K(dfo.get_px_detectable_ids().qc_detectable_id_));
}
}
return ret;
}
@ -1475,40 +1457,22 @@ int ObParallelDfoScheduler::schedule_pair(ObExecContext &exec_ctx,
// 那么它的 is_schedule 状态会被更新为 true,下面的 schedule_dfo
// 显然就不会被调度。
//
if (child.is_px_use_bloom_filter() && parent.is_px_create_bloom_filter()) {
// if with bloom filter, it is better to schedule parent first
// because child dfo depends on parent dfo's bloom filter creation
if (OB_SUCC(ret)) {
if (!parent.is_scheduled()) {
if (OB_FAIL(schedule_dfo(exec_ctx, parent))) { // 发送 DFO 到各个 server
LOG_WARN("fail schedule dfo", K(parent), K(ret));
}
}
}
if (OB_SUCC(ret)) {
if (!child.is_scheduled()) {
if (OB_FAIL(schedule_dfo(exec_ctx, child))) { // 发送 DFO 到各个 server
LOG_WARN("fail schedule dfo", K(child), K(ret));
}
}
}
} else {
// otherwise I'd like to schedule child first
// schedule child first
// because child can do some useful (e.g. scan) work while parent is scheduling
if (OB_SUCC(ret)) {
if (!child.is_scheduled()) {
if (OB_FAIL(schedule_dfo(exec_ctx, child))) { // 发送 DFO 到各个 server
LOG_WARN("fail schedule dfo", K(child), K(ret));
}
}
}
if (OB_SUCC(ret)) {
if (!parent.is_scheduled()) {
if (OB_FAIL(schedule_dfo(exec_ctx, parent))) { // 发送 DFO 到各个 server
LOG_WARN("fail schedule dfo", K(parent), K(ret));
}
if (OB_SUCC(ret)) {
if (!child.is_scheduled()) {
if (OB_FAIL(schedule_dfo(exec_ctx, child))) { // 发送 DFO 到各个 server
LOG_WARN("fail schedule dfo", K(child), K(ret));
}
}
}
if (OB_SUCC(ret)) {
if (!parent.is_scheduled()) {
if (OB_FAIL(schedule_dfo(exec_ctx, parent))) { // 发送 DFO 到各个 server
LOG_WARN("fail schedule dfo", K(parent), K(ret));
}
}
}
return ret;
}

View File

@ -47,13 +47,8 @@ public:
virtual int init_all_dfo_channel(ObExecContext &ctx) const;
virtual int on_sqc_threads_inited(ObExecContext &ctx, ObDfo &dfo) const;
virtual int dispatch_root_dfo_channel_info(ObExecContext &ctx, ObDfo &child, ObDfo &parent) const;
int dispatch_bf_channel_info(ObExecContext &ctx,
ObDfo &child, ObDfo &parent) const;
int set_bloom_filter_ch_for_root_dfo(ObExecContext &ctx,
ObDfo &dfo) const;
int build_bloom_filter_ch(ObExecContext &ctx,
ObDfo &child, ObDfo &parent) const;
int get_tenant_id(ObExecContext &ctx, uint64_t &tenant_id) const;
int prepare_schedule_info(ObExecContext &ctx);
private:
DISALLOW_COPY_AND_ASSIGN(ObDfoSchedulerBasic);
protected:

View File

@ -19,9 +19,13 @@
#include "sql/executor/ob_task_spliter.h"
#include "sql/engine/dml/ob_table_insert_op.h"
#include "sql/engine/expr/ob_expr_join_filter.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h"
#include "sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
#include "share/schema/ob_schema_struct.h"
#include "share/schema/ob_part_mgr_util.h"
namespace oceanbase
{
using namespace common;
@ -170,8 +174,8 @@ ObGranuleIteratorOp::ObGranuleIteratorOp(ObExecContext &exec_ctx, const ObOpSpec
pwj_rescan_task_infos_(),
filter_count_(0),
total_count_(0),
bf_key_(),
bloom_filter_ptr_(NULL),
rf_msg_(NULL),
rf_key_(),
tablet2part_id_map_(),
real_child_(NULL),
is_parallel_runtime_filtered_(false)
@ -411,11 +415,9 @@ int ObGranuleIteratorOp::inner_open()
LOG_WARN("child_op is null", K(ret));
} else if (MY_SPEC.bf_info_.is_inited_) {
ObGIOpInput *input = static_cast<ObGIOpInput*>(input_);
bf_key_.init(MY_SPEC.bf_info_.tenant_id_,
MY_SPEC.bf_info_.filter_id_,
MY_SPEC.bf_info_.server_id_,
input->get_px_sequence_id(),
MY_SPEC.bf_info_.is_shared_? 0 : worker_id_);
rf_key_.task_id_ = MY_SPEC.bf_info_.is_shared_? 0 : worker_id_;
rf_key_.px_sequence_id_ = input->px_sequence_id_;
rf_key_.p2p_datahub_id_ = MY_SPEC.bf_info_.p2p_dh_id_;
} else if (OB_FAIL(prepare_table_scan())) {
LOG_WARN("prepare table scan failed", K(ret));
}
@ -425,6 +427,11 @@ int ObGranuleIteratorOp::inner_open()
int ObGranuleIteratorOp::inner_close()
{
if (OB_NOT_NULL(rf_msg_)) {
// rf_msg_ is got from PX_P2P_DH map
// do not destroy it, because other worker thread may not start yet
rf_msg_->dec_ref_count();
}
return OB_SUCCESS;
}
@ -663,23 +670,30 @@ int ObGranuleIteratorOp::do_join_filter_partition_pruning(
{
int ret = OB_SUCCESS;
bool is_match = false;
while (OB_SUCC(ret) && OB_ISNULL(bloom_filter_ptr_)) {
if (OB_FAIL(ObPxBloomFilterManager::instance().get_px_bloom_filter(bf_key_,
bloom_filter_ptr_))) {
while (OB_SUCC(ret) && (OB_ISNULL(rf_msg_) || !rf_msg_->check_ready())) {
if (OB_ISNULL(rf_msg_) && OB_FAIL(PX_P2P_DH.atomic_get_msg(rf_key_, rf_msg_))) {
if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
} else {
LOG_WARN("fail to get px bloom filter", K(ret), K(bf_key_));
LOG_WARN("fail to get px bloom filter", K(ret), K(rf_key_));
}
if (OB_SUCC(ret)) {
if (OB_FAIL(ctx_.fast_check_status())) {
LOG_WARN("fail to check status", K(ret));
}
if (OB_SUCC(ret)) {
if (OB_FAIL(ctx_.fast_check_status())) {
LOG_WARN("fail to check status", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_ISNULL(rf_msg_) || !rf_msg_->check_ready()) {
if (MY_SPEC.bf_info_.is_shuffle_) {
partition_pruning = false;
break;
}
}
}
}
if (OB_SUCC(ret) && OB_NOT_NULL(bloom_filter_ptr_)) {
if (OB_SUCC(ret) && OB_NOT_NULL(rf_msg_) && rf_msg_->check_ready()) {
uint64_t hash_val = ObExprJoinFilter::JOIN_FILTER_SEED;
ObDatum &datum = MY_SPEC.tablet_id_expr_->locate_expr_datum(eval_ctx_);
if (MY_SPEC.bf_info_.skip_subpart_) {
@ -695,9 +709,10 @@ int ObGranuleIteratorOp::do_join_filter_partition_pruning(
if (OB_SUCC(ret) && !is_match) {
datum.int_ = &tablet_id;
datum.len_ = sizeof(tablet_id);
ObRFBloomFilterMsg *bf_msg = static_cast<ObRFBloomFilterMsg *>(rf_msg_);
if (OB_FAIL(MY_SPEC.hash_func_.hash_func_(datum, hash_val, hash_val))) {
LOG_WARN("fail to calc hash value", K(ret));
} else if (OB_FAIL(bloom_filter_ptr_->might_contain(hash_val, is_match))) {
} else if (OB_FAIL(bf_msg->bloom_filter_.might_contain(hash_val, is_match))) {
LOG_WARN("fail to check filter might contain value", K(ret), K(hash_val));
} else {
partition_pruning = !is_match;

View File

@ -21,12 +21,12 @@
#include "sql/engine/table/ob_table_scan_op.h"
#include "sql/rewrite/ob_query_range.h"
#include "sql/engine/px/ob_granule_pump.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h"
namespace oceanbase
{
namespace sql
{
class ObP2PDatahubMsgBase;
class ObPartitionIdHashFunc
{
public:
@ -238,8 +238,8 @@ private:
//for partition pruning
int64_t filter_count_; // filtered part count when part pruning activated
int64_t total_count_; // total partition count or block count processed, rescan included
ObPXBloomFilterHashWrapper bf_key_;
ObPxBloomFilter *bloom_filter_ptr_;
ObP2PDatahubMsgBase *rf_msg_;
ObP2PDhKey rf_key_;
ObPxTablet2PartIdMap tablet2part_id_map_;
ObOperator *real_child_;
bool is_parallel_runtime_filtered_;

View File

@ -16,13 +16,13 @@
#include "lib/allocator/page_arena.h"
#include "lib/string/ob_string.h"
#include "lib/lock/ob_spin_lock.h"
#include "share/external_table/ob_external_table_file_mgr.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/table/ob_table_scan_op.h"
#include "sql/engine/px/ob_px_dtl_msg.h"
#include "sql/engine/px/ob_granule_util.h"
#include "sql/engine/ob_engine_op_traits.h"
namespace oceanbase
{
namespace share {

View File

@ -41,7 +41,7 @@ namespace sql
#define IS_PX_GI(type) \
((type) == PHY_GRANULE_ITERATOR)
#define IS_PX_BLOOM_FILTER(type) \
#define IS_PX_JOIN_FILTER(type) \
((type) == PHY_JOIN_FILTER)
@ -98,7 +98,7 @@ enum JoinFilterMode
CREATE,
USE,
};
enum JoinFilterType
enum JoinFilterSharedType
{
INVALID_TYPE,
SHARED_JOIN_FILTER, // broadcast/hash

View File

@ -46,8 +46,9 @@ ObPxBloomFilter::ObPxBloomFilter() : data_length_(0), bits_count_(0), fpp_(0.0),
int ObPxBloomFilter::init(int64_t data_length, ObIAllocator &allocator, double fpp /*= 0.01 */)
{
int ret = OB_SUCCESS;
if (data_length <= 0 || fpp <= 0) {
ret = OB_INVALID_ARGUMENT;
data_length = max(data_length, 1);
if (fpp <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to init px bloom filter", K(ret), K(data_length), K(fpp));
} else {
data_length_ = data_length;
@ -77,7 +78,31 @@ int ObPxBloomFilter::init(int64_t data_length, ObIAllocator &allocator, double f
return ret;
}
int ObPxBloomFilter::init(ObPxBloomFilter *filter)
int ObPxBloomFilter::assign(const ObPxBloomFilter &filter)
{
int ret = OB_SUCCESS;
data_length_ = filter.data_length_;
bits_count_ = filter.bits_count_;
fpp_ = filter.fpp_;
hash_func_count_ = filter.hash_func_count_;
is_inited_ = filter.is_inited_;
bits_array_length_ = filter.bits_array_length_;
true_count_ = filter.true_count_;
might_contain_ = filter.might_contain_;
void *bits_array_buf = NULL;
if (OB_ISNULL(bits_array_buf = allocator_.alloc((bits_array_length_ + CACHE_LINE_SIZE)* sizeof(int64_t)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc filter", K(bits_array_length_), K(begin_idx_), K(end_idx_), K(ret));
} else {
int64_t align_addr = ((reinterpret_cast<int64_t>(bits_array_buf)
+ CACHE_LINE_SIZE - 1) >> LOG_CACHE_LINE_SIZE) << LOG_CACHE_LINE_SIZE;
bits_array_ = reinterpret_cast<int64_t *>(align_addr);
MEMCPY(bits_array_, filter.bits_array_, sizeof(int64_t) * bits_array_length_);
}
return ret;
}
int ObPxBloomFilter::init(const ObPxBloomFilter *filter)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(filter)) {
@ -273,14 +298,13 @@ int ObPxBloomFilter::process_first_phase_recieve_count(int64_t whole_expect_size
return ret;
}
int ObPxBloomFilter::generate_receive_count_array()
int ObPxBloomFilter::generate_receive_count_array(int64_t piece_size)
{
int ret = OB_SUCCESS;
int64_t send_size = GCONF._send_bloom_filter_size * 125;
int64_t count = ceil(bits_array_length_ / (double)send_size);
int64_t count = ceil(bits_array_length_ / (double)piece_size);
int64_t begin_idx = 0;
for (int i = 0; OB_SUCC(ret) && i < count; ++i) {
begin_idx = i * send_size;
begin_idx = i * piece_size;
if (begin_idx >= bits_array_length_) {
begin_idx = bits_array_length_ - 1;
}
@ -386,7 +410,9 @@ OB_DEF_SERIALIZE_SIZE(ObPxBloomFilter)
}
//-------------------------------------分割线----------------------------
int ObPxBFStaticInfo::init(int64_t tenant_id, int64_t filter_id,
int64_t server_id, bool is_shared, bool skip_subpart)
int64_t server_id, bool is_shared,
bool skip_subpart, int64_t p2p_dh_id,
bool is_shuffle)
{
int ret = OB_SUCCESS;
if (is_inited_){
@ -398,6 +424,8 @@ int ObPxBFStaticInfo::init(int64_t tenant_id, int64_t filter_id,
server_id_ = server_id;
is_shared_ = is_shared;
skip_subpart_ = skip_subpart;
p2p_dh_id_ = p2p_dh_id;
is_shuffle_ = is_shuffle;
is_inited_ = true;
}
return ret;
@ -514,7 +542,7 @@ int ObPxBloomFilterManager::get_px_bf_for_merge_filter(ObPXBloomFilterHashWrappe
}
OB_SERIALIZE_MEMBER(ObPxBFStaticInfo, is_inited_, tenant_id_, filter_id_,
server_id_, is_shared_, skip_subpart_);
server_id_, is_shared_, skip_subpart_, p2p_dh_id_, is_shuffle_);
OB_SERIALIZE_MEMBER(ObPXBloomFilterHashWrapper, tenant_id_, filter_id_,
server_id_, px_sequence_id_, task_id_)
OB_SERIALIZE_MEMBER(ObPxBFSendBloomFilterArgs, bf_key_, bloom_filter_,

View File

@ -59,7 +59,7 @@ public:
ObPxBloomFilter();
virtual ~ObPxBloomFilter() {};
int init(int64_t data_length, common::ObIAllocator &allocator, double fpp = 0.01);
int init(ObPxBloomFilter *filter);
int init(const ObPxBloomFilter *filter);
void reset_filter();
inline int might_contain(uint64_t hash, bool &is_match) {
return (this->*might_contain_)(hash, is_match);
@ -87,8 +87,9 @@ public:
int64_t get_end_idx() { return end_idx_; }
void prefetch_bits_block(uint64_t hash);
typedef int (ObPxBloomFilter::*GetFunc)(uint64_t hash, bool &is_match);
int generate_receive_count_array();
int generate_receive_count_array(int64_t piece_size);
void reset();
int assign(const ObPxBloomFilter &filter);
TO_STRING_KV(K_(data_length), K_(bits_count), K_(fpp), K_(hash_func_count), K_(is_inited),
K_(bits_array_length), K_(true_count));
private:
@ -129,18 +130,24 @@ public:
ObPxBFStaticInfo()
: is_inited_(false), tenant_id_(common::OB_INVALID_TENANT_ID),
filter_id_(common::OB_INVALID_ID), server_id_(common::OB_INVALID_ID),
is_shared_(false), skip_subpart_(false)
is_shared_(false), skip_subpart_(false),
p2p_dh_id_(OB_INVALID_ID), is_shuffle_(false)
{}
int init(int64_t tenant_id, int64_t filter_id,
int64_t server_id, bool is_shared, bool skip_subpart);
int64_t server_id, bool is_shared,
bool skip_subpart, int64_t p2p_dh_id,
bool is_shuffle);
bool is_inited_;
int64_t tenant_id_;
int64_t filter_id_;
int64_t server_id_;
bool is_shared_; // 执行期join filter内存是否共享, false代表线程级, true代表sqc级.
bool skip_subpart_; // 是否忽略二级分区
int64_t p2p_dh_id_;
bool is_shuffle_;
TO_STRING_KV(K(is_inited_), K(tenant_id_), K(filter_id_),
K(server_id_), K(is_shared_), K(skip_subpart_));
K(server_id_), K(is_shared_), K(skip_subpart_),
K(is_shuffle_), K(p2p_dh_id_));
};
class ObPXBloomFilterHashWrapper

View File

@ -43,6 +43,8 @@
#include "sql/dtl/ob_dtl_interm_result_manager.h"
#include "sql/engine/px/exchange/ob_px_ms_coord_op.h"
#include "sql/engine/px/datahub/components/ob_dh_init_channel.h"
#include "share/detect/ob_detect_manager_utils.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
namespace oceanbase
{
@ -129,6 +131,8 @@ ObPxCoordOp::ObPxCoordOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInpu
register_interrupted_(false),
px_sequence_id_(0),
interrupt_id_(0),
register_detectable_id_(false),
detectable_id_(),
px_dop_(1),
time_recorder_(0),
batch_rescan_param_version_(0),
@ -246,6 +250,7 @@ int ObPxCoordOp::rescan()
}
}
if (OB_FAIL(ret)) {
} else if (FALSE_IT(ObDetectManagerUtils::qc_unregister_detectable_id_from_dm(detectable_id_, register_detectable_id_))) {
} else if (FALSE_IT(clear_interrupt())) {
} else if (OB_FAIL(destroy_all_channel())) {
LOG_WARN("release dtl channel failed", K(ret));
@ -260,6 +265,9 @@ int ObPxCoordOp::rescan()
} else if (FALSE_IT(px_sequence_id_ = GCTX.sql_engine_->get_px_sequence_id())) {
} else if (OB_FAIL(register_interrupt())) {
LOG_WARN("fail to register interrupt", K(ret));
} else if (OB_NOT_NULL(get_spec().get_phy_plan()) && get_spec().get_phy_plan()->is_enable_px_fast_reclaim()
&& OB_FAIL(ObDetectManagerUtils::qc_register_detectable_id_into_dm(detectable_id_, register_detectable_id_, GET_TENANT_ID()))) {
LOG_WARN("fail to register detectable_id", K(ret));
} else if (OB_FAIL(init_dfo_mgr(
ObDfoInterruptIdGen(interrupt_id_,
(uint32_t)GCTX.server_id_,
@ -348,10 +356,15 @@ int ObPxCoordOp::inner_open()
LOG_WARN("Server is initializing", K(ret), K(GCTX.server_id_));
} else if (OB_FAIL(post_init_op_ctx())) {
LOG_WARN("init operator context failed", K(ret));
} else if (OB_FAIL(coord_info_.init())) {
LOG_WARN("fail to init coord info", K(ret));
} else if (FALSE_IT(px_sequence_id_ = GCTX.sql_engine_->get_px_sequence_id())) {
LOG_WARN("fail to get px sequence id", K(ret));
} else if (OB_FAIL(register_interrupt())) {
LOG_WARN("fail to register interrupt", K(ret));
} else if (OB_NOT_NULL(get_spec().get_phy_plan()) && get_spec().get_phy_plan()->is_enable_px_fast_reclaim()
&& OB_FAIL(ObDetectManagerUtils::qc_register_detectable_id_into_dm(detectable_id_, register_detectable_id_, GET_TENANT_ID()))) {
LOG_WARN("fail to register detectable_id", K(ret));
} else if (OB_FAIL(init_dfo_mgr(
ObDfoInterruptIdGen(interrupt_id_,
(uint32_t)GCTX.server_id_,
@ -422,6 +435,9 @@ int ObPxCoordOp::terminate_running_dfos(ObDfoMgr &dfo_mgr)
} else if (!dfos.empty() && OB_FAIL(wait_all_running_dfos_exit())) {
LOG_WARN("fail to exit dfo", K(ret));
}
if (OB_NOT_NULL(get_spec().get_phy_plan()) && get_spec().get_phy_plan()->is_enable_px_fast_reclaim()) {
(void)ObDetectManagerUtils::qc_unregister_all_check_items_from_dm(dfos);
}
return ret;
}
@ -471,21 +487,113 @@ int ObPxCoordOp::setup_op_input(ObDfo &root)
return ret;
}
int ObPxCoordOp::try_clear_p2p_dh_info()
{
int ret = OB_SUCCESS;
#ifdef ERRSIM
ObSQLSessionInfo *session = ctx_.get_my_session();
int64_t query_timeout = 0;
session->get_query_timeout(query_timeout);
if (OB_FAIL(OB_E(EventTable::EN_PX_NOT_ERASE_P2P_DH_MSG) OB_SUCCESS)) {
LOG_WARN("qc not clear p2p dh info by design", K(ret), K(query_timeout));
return OB_SUCCESS;
}
#endif
if (!coord_info_.p2p_dfo_map_.empty()) {
hash::ObHashMap<ObAddr, ObSArray<int64_t> *, hash::NoPthreadDefendMode> dh_map;
if (OB_FAIL(dh_map.create(coord_info_.p2p_dfo_map_.size() * 2,
"ClearP2PDhMap",
"ClearP2PDhMap"))) {
LOG_WARN("fail to create dh map", K(ret));
}
ObSArray<int64_t> *p2p_ids = nullptr;
void *ptr = nullptr;
common::ObArenaAllocator allocator;
int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id();
allocator.set_tenant_id(tenant_id);
FOREACH_X(entry, coord_info_.p2p_dfo_map_, OB_SUCC(ret)) {
for (int i = 0; OB_SUCC(ret) && i < entry->second.addrs_.count(); ++i) {
ptr = nullptr;
p2p_ids = nullptr;
if (OB_FAIL(dh_map.get_refactored(entry->second.addrs_.at(i), p2p_ids))) {
if (OB_HASH_NOT_EXIST == ret) {
if (OB_ISNULL(ptr = allocator_.alloc(sizeof(ObSArray<int64_t>)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc memory", K(ret));
} else {
p2p_ids = new(ptr) ObSArray<int64_t>();
ret = OB_SUCCESS;
}
} else {
LOG_WARN("fail to get array", K(ret));
}
}
if (OB_SUCC(ret) && OB_NOT_NULL(p2p_ids)) {
if (OB_FAIL(p2p_ids->push_back(entry->first))) {
LOG_WARN("fail to push back array ptr", K(ret));
} else if (OB_FAIL(dh_map.set_refactored(entry->second.addrs_.at(i), p2p_ids, 1))) {
LOG_WARN("fail to set p2p sequence ids", K(ret));
}
}
}
}
FOREACH_X(entry, dh_map, true) {
ObPxP2PClearMsgArg arg;
arg.px_seq_id_ = px_sequence_id_;
int tmp_ret = arg.p2p_dh_ids_.assign(*entry->second);
if (OB_SUCCESS == tmp_ret && !arg.p2p_dh_ids_.empty()) {
if (OB_FAIL(PX_P2P_DH.get_proxy().to(entry->first).
by(tenant_id).
clear_dh_msg(arg, nullptr))) {
LOG_WARN("fail to clear dh msg", K(ret));
ret = OB_SUCCESS;
}
}
entry->second->reset();
}
allocator.reset();
}
return ret;
}
int ObPxCoordOp::inner_close()
{
int ret = OB_SUCCESS;
// close过程中忽略terminate错误码
int terminate_ret = OB_SUCCESS;
if (OB_SUCCESS != (terminate_ret = terminate_running_dfos(coord_info_.dfo_mgr_))) {
// #issue/44180396
if (OB_NOT_NULL(ctx_.get_my_session()) &&
ctx_.get_my_session()->get_trans_result().is_incomplete()) {
ret = terminate_ret;
} else {
LOG_WARN("fail to terminate running dfo, ignore ret", K(terminate_ret));
bool should_terminate_running_dfos = true;
#ifdef ERRSIM
ObSQLSessionInfo *session = ctx_.get_my_session();
int64_t query_timeout = 0;
session->get_query_timeout(query_timeout);
if (OB_FAIL(OB_E(EventTable::EN_PX_QC_EARLY_TERMINATE, query_timeout) OB_SUCCESS)) {
LOG_WARN("qc not interrupt qc by design", K(ret), K(query_timeout));
should_terminate_running_dfos = false;
}
#endif
if (should_terminate_running_dfos) {
if (OB_SUCCESS != (terminate_ret = terminate_running_dfos(coord_info_.dfo_mgr_))) {
// #issue/44180396
if (OB_NOT_NULL(ctx_.get_my_session()) &&
ctx_.get_my_session()->get_trans_result().is_incomplete()) {
ret = terminate_ret;
} else {
LOG_WARN("fail to terminate running dfo, ignore ret", K(terminate_ret));
}
}
}
unregister_first_buffer_cache();
(void)ObDetectManagerUtils::qc_unregister_detectable_id_from_dm(detectable_id_, register_detectable_id_);
const ObIArray<ObDfo *> &dfos = coord_info_.dfo_mgr_.get_all_dfos();
if (OB_NOT_NULL(get_spec().get_phy_plan()) && get_spec().get_phy_plan()->is_enable_px_fast_reclaim()) {
(void)ObDetectManagerUtils::qc_unregister_all_check_items_from_dm(dfos);
}
(void)try_clear_p2p_dh_info();
(void)clear_interrupt();
int release_channel_ret = OB_SUCCESS;
if (OB_SUCCESS != (release_channel_ret = destroy_all_channel())) {
@ -733,8 +841,9 @@ int ObPxCoordOp::check_all_sqc(ObIArray<ObDfo *> &active_dfos,
}
all_dfo_terminate = false;
break;
} else if (sqc->is_server_not_alive()) {
} else if (sqc->is_server_not_alive() || sqc->is_interrupt_by_dm()) {
sqc->set_server_not_alive(false);
sqc->set_interrupt_by_dm(false);
const DASTabletLocIArray &access_locations = sqc->get_access_table_locations();
for (int64_t i = 0; i < access_locations.count() && OB_SUCC(ret); i++) {
if (OB_FAIL(ctx_.get_my_session()->get_trans_result().add_touched_ls(access_locations.at(i)->ls_id_))) {
@ -969,6 +1078,15 @@ int ObPxCoordOp::batch_rescan()
int ObPxCoordOp::erase_dtl_interm_result()
{
int ret = OB_SUCCESS;
#ifdef ERRSIM
ObSQLSessionInfo *session = ctx_.get_my_session();
int64_t query_timeout = 0;
session->get_query_timeout(query_timeout);
if (OB_FAIL(OB_E(EventTable::EN_PX_SINGLE_DFO_NOT_ERASE_DTL_INTERM_RESULT) OB_SUCCESS)) {
LOG_WARN("ObPxCoordOp not erase_dtl_interm_result by design", K(ret), K(query_timeout));
return OB_SUCCESS;
}
#endif
if (static_cast<const ObPxCoordSpec&>(get_spec()).batch_op_info_.is_inited()) {
ObDTLIntermResultKey key;
ObDtlChannelInfo ci;

View File

@ -132,6 +132,7 @@ protected:
int erase_dtl_interm_result();
// send rpc to clean dtl interm result of not scheduled dfos.
virtual void clean_dfos_dtl_interm_result() = 0;
int try_clear_p2p_dh_info();
protected:
common::ObArenaAllocator allocator_;
common::ObArenaAllocator row_allocator_;
@ -152,6 +153,8 @@ protected:
* */
uint64_t px_sequence_id_;
ObInterruptibleTaskID interrupt_id_;
bool register_detectable_id_;
ObDetectableId detectable_id_;
int px_dop_;
int64_t time_recorder_;
int64_t batch_rescan_param_version_;

View File

@ -23,6 +23,8 @@
#include "sql/engine/px/ob_px_target_mgr.h"
#include "sql/engine/px/ob_px_sqc_handler.h"
#include "sql/dtl/ob_dtl_basic_channel.h"
#include "share/detect/ob_detect_callback.h"
#include "share/detect/ob_detect_manager_utils.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
@ -90,6 +92,9 @@ int ObInitSqcP::process()
LOG_WARN("Worker thread res not enough", K_(result));
} else if (OB_FAIL(sqc_handler->link_qc_sqc_channel())) {
LOG_WARN("Failed to link qc sqc channel", K(ret));
} else if (sqc_handler->get_phy_plan().is_enable_px_fast_reclaim() &&
OB_FAIL(ObDetectManagerUtils::sqc_register_into_dm(sqc_handler, sqc_handler->get_sqc_init_arg().sqc_))) {
LOG_WARN("[DM] sqc failed to register_into_dm");
} else {
/*do nothing*/
}
@ -395,6 +400,9 @@ int ObInitFastSqcP::process()
LOG_WARN("Session can't be null", K(ret));
} else if (OB_FAIL(sqc_handler->link_qc_sqc_channel())) {
LOG_WARN("fail to link qc sqc channel", K(ret));
} else if (sqc_handler->get_phy_plan().is_enable_px_fast_reclaim() &&
OB_FAIL(ObDetectManagerUtils::sqc_register_into_dm(sqc_handler, sqc_handler->get_sqc_init_arg().sqc_))) {
LOG_WARN("[DM] sqc failed to register_into_dm");
} else {
ObPxRpcInitSqcArgs &arg = sqc_handler->get_sqc_init_arg();
arg.sqc_.set_task_count(1);

View File

@ -39,6 +39,7 @@
#include "sql/engine/px/datahub/components/ob_dh_sample.h"
#include "sql/engine/px/ob_px_sqc_proxy.h"
#include "storage/tx/ob_trans_service.h"
#include "share/detect/ob_detect_manager_utils.h"
namespace oceanbase
{
@ -120,7 +121,9 @@ int ObPxMsgProc::startup_msg_loop(ObExecContext &ctx)
LOG_TRACE("TIMERECORD ",
"reserve:=-1 name:=QC dfoid:=-1 sqcid:=-1 taskid:=-1 start:",
ObTimeUtility::current_time());
if (OB_FAIL(scheduler_->init_all_dfo_channel(ctx))) {
if (OB_FAIL(scheduler_->prepare_schedule_info(ctx))) {
LOG_WARN("fail to prepare schedule info", K(ret));
} else if (OB_FAIL(scheduler_->init_all_dfo_channel(ctx))) {
LOG_WARN("fail to init all dfo channel", K(ret));
} else if (OB_FAIL(scheduler_->try_schedule_next_dfo(ctx))) {
LOG_WARN("fail to sched next one dfo", K(ret));
@ -311,6 +314,19 @@ int ObPxMsgProc::on_sqc_finish_msg(ObExecContext &ctx,
} else { /*do nothing.*/ }
} else { /*do nothing.*/ }
if (OB_SUCC(ret)) {
if (OB_NOT_NULL(edge->get_detect_cb())) {
#ifdef ERRSIM
if (OB_FAIL(OB_E(EventTable::EN_PX_SLOW_PROCESS_SQC_FINISH_MSG) OB_SUCCESS)) {
LOG_WARN("qc slow process sqc finish msg by desgin", K(ret));
usleep(100 * 1000L);
ret = OB_SUCCESS;
}
#endif
int set_finish_ret = edge->get_detect_cb()->atomic_set_finished(sqc->get_sqc_addr());
if (OB_SUCCESS != set_finish_ret) {
LOG_WARN("[DM] failed to atomic_set_finished", K(set_finish_ret), K(sqc->get_sqc_addr()));
}
}
sqc->set_thread_finish(true);
if (sqc->is_ignore_vtable_error() && OB_SUCCESS != pkt.rc_
&& ObVirtualTableErrorWhitelist::should_ignore_vtable_error(pkt.rc_)) {
@ -383,6 +399,10 @@ int ObPxMsgProc::on_sqc_finish_msg(ObExecContext &ctx,
if (OB_SUCC(ret)) {
if (edge->is_thread_finish()) {
if (OB_NOT_NULL(ctx.get_physical_plan_ctx()->get_phy_plan()) &&
ctx.get_physical_plan_ctx()->get_phy_plan()->is_enable_px_fast_reclaim()) {
(void)ObDetectManagerUtils::qc_unregister_check_item_from_dm(edge);
}
ret = scheduler_->try_schedule_next_dfo(ctx);
if (OB_ITER_END == ret) {
coord_info_.all_threads_finish_ = true;
@ -494,94 +514,7 @@ int ObPxMsgProc::on_dfo_pair_thread_inited(ObExecContext &ctx, ObDfo &child, ObD
LOG_TRACE("dispatch dtl data channel for pair ok", K(parent), K(child));
}
}
//如果执行计划包含px bloom filter, 判断子dfo是否包含use_filter算子,
//如果有, 则为它们建立channel信息, 并dispach给parent dfo
if (OB_SUCC(ret) && child.is_px_use_bloom_filter()) {
if (parent.is_root_dfo()
&& OB_FAIL(scheduler_->set_bloom_filter_ch_for_root_dfo(ctx, parent))) {
LOG_WARN("fail to set bloom filter ch for root dfo", K(ret));
} else if (OB_FAIL(scheduler_->build_bloom_filter_ch(ctx, child, parent))) {
LOG_WARN("fail to setup bloom filter channel", K(ret));
} else if (OB_FAIL(scheduler_->dispatch_bf_channel_info(ctx, child, parent))) {
LOG_WARN("fail setup bloom filter data channel for child-parent pair", K(ret));
} else {
LOG_TRACE("dispatch px bloom filter channel for pair ok", K(parent), K(child));
}
}
return ret;
}
int ObPxMsgProc::mark_rpc_filter(ObExecContext &ctx,
ObJoinFilterDataCtx &bf_ctx,
int64_t &each_group_size)
{
int ret = OB_SUCCESS;
ObPhysicalPlanCtx *phy_plan_ctx = GET_PHY_PLAN_CTX(ctx);
bf_ctx.filter_ready_ = false; // make sure there is only one thread of this sqc can mark_rpc_filter
if (OB_ISNULL(bf_ctx.filter_data_) ||
OB_ISNULL(phy_plan_ctx) ||
OB_ISNULL(ctx.get_my_session())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the filter data or phy plan ctx is null", K(ret));
} else if (0 == bf_ctx.ch_provider_ptr_) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("not support root dfo send bloom filter", K(ret));
LOG_USER_ERROR(OB_NOT_SUPPORTED, "root dfo send bloom filter");
} else {
int64_t sqc_count = 0;
int64_t tenant_id = ctx.get_my_session()->get_effective_tenant_id();
ObPxSQCProxy *ch_provider = reinterpret_cast<ObPxSQCProxy *>(bf_ctx.ch_provider_ptr_);
// get channel info(addr) between receive op that will send bloom filter and child dfo sqc thread
if (OB_FAIL(ch_provider->get_bloom_filter_ch(bf_ctx.ch_set_,
sqc_count, phy_plan_ctx->get_timeout_timestamp(), false))) {
LOG_WARN("fail get data ch sets from provider", K(ret));
} else if (bf_ctx.ch_set_.count() <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ch_info_set count is unexpected", K(ret));
} else {
if (0 == each_group_size) { // only need calc once
omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id));
if (OB_LIKELY(tenant_config.is_valid())) {
const char *ptr = NULL;
if (OB_ISNULL(ptr = tenant_config->_px_bloom_filter_group_size.get_value())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("each group size ptr is null", K(ret));
} else if (0 == ObString::make_string("auto").case_compare(ptr)) {
each_group_size = sqrt(bf_ctx.ch_set_.count()); // auto calc group size
} else {
char *end_ptr = nullptr;
each_group_size = strtoull(ptr, &end_ptr, 10); // get group size from tenant config
if (*end_ptr != '\0') {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("each group size ptr is unexpected", K(ret));
}
}
}
each_group_size = (each_group_size <= 0 ? 1 : each_group_size);
}
int64_t send_size = GCONF._send_bloom_filter_size * 125; // 125 = 1000/8, send size means how many byte of partial bloom filter will be send at once
int64_t send_count = ceil(bf_ctx.filter_data_->filter_.get_bits_array_length() / (double)send_size); // how many piece of partial bloom filter will be send by all threads(sqc level)
bf_ctx.filter_data_->bloom_filter_count_ = sqc_count * send_count; // count of piece of partial bloom filter that child dfo will get
common::ObIArray<ObBloomFilterSendCtx> &sqc_bf_send_ctx_array = ch_provider->get_bf_send_ctx_array();
int64_t bf_idx_at_sqc_proxy = bf_ctx.bf_idx_at_sqc_proxy_;
if (0 > bf_idx_at_sqc_proxy || bf_idx_at_sqc_proxy >= sqc_bf_send_ctx_array.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid bf_idx_at_sqc_proxy", K(bf_idx_at_sqc_proxy), K(sqc_bf_send_ctx_array.count()), K(ret));
} else {
ObBloomFilterSendCtx &bf_send_ctx = sqc_bf_send_ctx_array.at(bf_idx_at_sqc_proxy);
bf_send_ctx.set_filter_data(bf_ctx.filter_data_);
if (OB_FAIL(bf_send_ctx.assign_bf_ch_set(bf_ctx.ch_set_))) {
LOG_WARN("failed to assign bloom filter ch_set", K(ret));
} else if (OB_FAIL(bf_send_ctx.generate_filter_indexes(each_group_size, bf_ctx.ch_set_.count()))) {
LOG_WARN("failed to generate filter indexs", K(ret));
} else {
bf_send_ctx.set_bf_compress_type(bf_ctx.compressor_type_);
bf_send_ctx.set_per_channel_bf_count(send_count);
bf_send_ctx.set_bloom_filter_ready(true); // means bloom filter is ready to be sent
}
}
}
}
return ret;
}
@ -851,5 +784,17 @@ int ObPxTerminateMsgProc::on_piece_msg(
return common::OB_SUCCESS;
}
int ObPxCoordInfo::init()
{
int ret = OB_SUCCESS;
const int64_t bucket_num = 32;
if (OB_FAIL(p2p_dfo_map_.create(bucket_num,
"PxDfoMapKey",
"PxDfoMapNode"))) {
LOG_WARN("create hash table failed", K(ret));
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -54,6 +54,35 @@ enum class TableAccessType {
PURE_VIRTUAL_TABLE,
HAS_USER_TABLE
};
struct ObP2PDfoMapNode
{
ObP2PDfoMapNode() : target_dfo_id_(OB_INVALID_ID), addrs_() {}
~ObP2PDfoMapNode() { addrs_.reset(); }
int assign(const ObP2PDfoMapNode &other) {
target_dfo_id_ = other.target_dfo_id_;
return addrs_.assign(other.addrs_);
}
void reset() {
target_dfo_id_ = OB_INVALID_ID;
addrs_.reset();
}
int64_t target_dfo_id_;
common::ObSArray<ObAddr>addrs_;
TO_STRING_KV(K(target_dfo_id_), K(addrs_));
};
struct ObTempTableP2PInfo
{
ObTempTableP2PInfo() : temp_access_ops_(), dfos_() {}
~ObTempTableP2PInfo() { reset(); }
void reset() {
temp_access_ops_.reset();
dfos_.reset();
}
ObSEArray<const ObOpSpec *, 4> temp_access_ops_;
ObSEArray<ObDfo *, 4> dfos_;
TO_STRING_KV(K(temp_access_ops_), K(dfos_));
};
// 这些信息是调度时候需要用的变量,暂时统一叫做CoordInfo
class ObPxCoordInfo
{
@ -71,13 +100,18 @@ public:
coord_(coord),
batch_rescan_ctl_(NULL),
pruning_table_location_(NULL),
table_access_type_(TableAccessType::NO_TABLE)
table_access_type_(TableAccessType::NO_TABLE),
qc_detectable_id_(),
p2p_dfo_map_(),
p2p_temp_table_info_()
{}
virtual ~ObPxCoordInfo() {}
virtual void destroy()
{
dfo_mgr_.destroy();
piece_msg_ctx_mgr_.reset();
p2p_dfo_map_.destroy();
p2p_temp_table_info_.reset();
}
void reset_for_rescan()
{
@ -85,7 +119,10 @@ public:
dfo_mgr_.destroy();
piece_msg_ctx_mgr_.reset();
batch_rescan_ctl_ = NULL;
p2p_dfo_map_.reuse();
p2p_temp_table_info_.reset();
}
int init();
bool enable_px_batch_rescan() { return get_rescan_param_count() > 0; }
int64_t get_rescan_param_count()
{
@ -112,6 +149,10 @@ public:
ObBatchRescanCtl *batch_rescan_ctl_;
const common::ObIArray<ObTableLocation> *pruning_table_location_;
TableAccessType table_access_type_;
ObDetectableId qc_detectable_id_;
// key = p2p_dh_id value = dfo_id + target_addrs
hash::ObHashMap<int64_t, ObP2PDfoMapNode, hash::NoPthreadDefendMode> p2p_dfo_map_;
ObTempTableP2PInfo p2p_temp_table_info_;
};
class ObDfoSchedulerBasic;

View File

@ -20,6 +20,7 @@
#include "sql/optimizer/ob_storage_estimator.h"
#include "sql/ob_sql_trans_control.h"
#include "storage/tx/ob_trans_service.h"
#include "share/detect/ob_detect_manager_utils.h"
using namespace oceanbase::sql;
using namespace oceanbase::common;
@ -316,7 +317,10 @@ int ObPxSqcHandler::destroy_sqc()
LOG_WARN("end ddl failed", K(tmp_ret));
end_ret = OB_SUCCESS == end_ret ? tmp_ret : end_ret;
}
if (OB_NOT_NULL(des_phy_plan_) && des_phy_plan_->is_enable_px_fast_reclaim()) {
(void) ObDetectManagerUtils::sqc_unregister_check_item_from_dm(
sqc_init_args_->sqc_.get_px_detectable_ids().qc_detectable_id_, node_sequence_id_);
}
if (has_flag(OB_SQC_HANDLER_QC_SQC_LINKED)) {
/**
* sqc-qc通道的连接是rpc中process的最后一步link成功就会有这个flag
@ -328,6 +332,10 @@ int ObPxSqcHandler::destroy_sqc()
if (OB_FAIL(sub_coord_->report_sqc_finish(end_ret))) {
LOG_WARN("fail report sqc to qc", K(ret));
}
if (OB_NOT_NULL(des_phy_plan_) && des_phy_plan_->is_enable_px_fast_reclaim()) {
(void) ObDetectManagerUtils::sqc_unregister_detectable_id_from_dm(
sqc_init_args_->sqc_.get_px_detectable_ids().sqc_detectable_id_);
}
ObPxSqcMeta &sqc = sqc_init_args_->sqc_;
LOG_TRACE("sqc send report to qc", K(sqc));
}
@ -343,9 +351,6 @@ int ObPxSqcHandler::destroy_sqc()
LOG_WARN("Failed to unlink channel", K(ret));
}
}
if (has_flag(OB_SQC_HANDLER_BLOOM_FILTER_NEED_CLEAR)) {
sub_coord_->destroy_bloom_filter();
}
return ret;
}

View File

@ -26,7 +26,6 @@ namespace sql
#define OB_SQC_HANDLER_TRAN_STARTED (1ULL)
#define OB_SQC_HANDLER_QC_SQC_LINKED (1ULL << 1)
#define OB_SQC_HANDLER_BLOOM_FILTER_NEED_CLEAR (1ULL << 2)
class ObPxWorkNotifier
{
@ -64,7 +63,8 @@ public:
ObPxSqcHandler() :
mem_context_(NULL), tenant_id_(UINT64_MAX), reserved_px_thread_count_(0), process_flags_(0),
end_ret_(OB_SUCCESS), reference_count_(1), notifier_(nullptr), exec_ctx_(nullptr),
des_phy_plan_(nullptr), sqc_init_args_(nullptr), sub_coord_(nullptr), rpc_level_(INT32_MAX) {
des_phy_plan_(nullptr), sqc_init_args_(nullptr), sub_coord_(nullptr), rpc_level_(INT32_MAX),
node_sequence_id_(0) {
}
~ObPxSqcHandler() = default;
static constexpr const char *OP_LABEL = ObModIds::ObModIds::OB_SQL_SQC_HANDLER;
@ -123,6 +123,7 @@ public:
bool all_task_success();
int64_t get_rpc_level() { return rpc_level_; }
void set_rpc_level(int64_t level) { rpc_level_ = level; }
void set_node_sequence_id(uint64_t node_sequence_id) { node_sequence_id_ = node_sequence_id; }
int thread_count_auto_scaling(int64_t &reserved_px_thread_count);
TO_STRING_KV(K_(tenant_id), K_(reserved_px_thread_count), KP_(notifier),
K_(exec_ctx), K_(des_phy_plan), K_(sqc_init_args), KP_(sub_coord), K_(rpc_level));
@ -145,6 +146,7 @@ private:
ObPxSubCoord *sub_coord_;
trace::FltTransCtx flt_ctx_;
int64_t rpc_level_;
uint64_t node_sequence_id_;
};
}

View File

@ -87,7 +87,8 @@ ObPxSQCProxy::ObPxSQCProxy(ObSqcCtx &sqc_ctx,
first_buffer_cache_(nullptr),
bf_send_ctx_array_(),
sample_msg_(),
init_channel_msg_()
init_channel_msg_(),
p2p_dh_map_()
{
}
@ -514,6 +515,21 @@ int ObPxSQCProxy::report(int end_ret) const
}
}
#ifdef ERRSIM
if (OB_SUCC(ret)) {
int64_t query_timeout = 0;
session->get_query_timeout(query_timeout);
if (OB_FAIL(OB_E(EventTable::EN_PX_SQC_NOT_REPORT_TO_QC, query_timeout) OB_SUCCESS)) {
static bool errsim = false;
errsim = !errsim;
if (errsim) {
LOG_WARN("sqc report to qc by design", K(ret), K(query_timeout));
return OB_SUCCESS;
}
}
}
#endif
ObDtlChannel *ch = sqc.get_sqc_channel();
// overwrite ret
if (OB_ISNULL(ch)) {
@ -681,3 +697,47 @@ int ObPxSQCProxy::sync_wait_all(ObPxDatahubDataProvider &provider)
return ret;
}
int ObPxSQCProxy::construct_p2p_dh_map(ObP2PDhMapInfo &map_info)
{
int ret = OB_SUCCESS;
int bucket_size = 2 * map_info.p2p_sequence_ids_.count();
if (0 == bucket_size) {
} else if (OB_FAIL(p2p_dh_map_.create(bucket_size,
"SQCDHMapKey",
"SQCDHMAPNode"))) {
LOG_WARN("create hash table failed", K(ret));
} else if (map_info.p2p_sequence_ids_.count() !=
map_info.target_addrs_.count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected map info count", K(ret));
} else {
for (int i = 0; OB_SUCC(ret) &&
i < map_info.p2p_sequence_ids_.count(); ++i) {
if (OB_FAIL(p2p_dh_map_.set_refactored(map_info.p2p_sequence_ids_.at(i),
&map_info.target_addrs_.at(i)))) {
LOG_WARN("fail to set p2p dh map", K(ret));
}
}
}
return ret;
}
int ObPxSQCProxy::check_is_local_dh(int64_t p2p_dh_id, bool &is_local, int64_t msg_cnt)
{
int ret = OB_SUCCESS;
ObSArray<ObAddr> *target_addrs = nullptr;
if (OB_FAIL(p2p_dh_map_.get_refactored(p2p_dh_id, target_addrs))) {
LOG_WARN("fail to get dh map", K(ret));
} else if (OB_ISNULL(target_addrs) || target_addrs->empty()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected target addrs", K(ret));
} else if (target_addrs->count() == 1 &&
GCTX.self_addr() == target_addrs->at(0) &&
1 == msg_cnt) {
is_local = true;
} else {
is_local = false;
}
return ret;
}

View File

@ -14,6 +14,7 @@
#define __OB_SQL_PX_SQC_PROXY_H__
#include "lib/lock/ob_spin_lock.h"
#include "lib/hash/ob_hashmap.h"
#include "sql/engine/px/ob_px_dtl_msg.h"
#include "sql/dtl/ob_dtl_linked_buffer.h"
#include "sql/dtl/ob_dtl_task.h"
@ -24,6 +25,7 @@
#include "sql/engine/px/datahub/components/ob_dh_sample.h"
#include "sql/engine/px/datahub/components/ob_dh_init_channel.h"
namespace oceanbase
{
namespace sql
@ -62,41 +64,43 @@ class ObBloomFilterSendCtx
public:
ObBloomFilterSendCtx() :
bloom_filter_ready_(false),
bf_ch_set_(),
filter_data_(NULL),
filter_indexes_(),
per_channel_bf_count_(0),
filter_channel_idx_(0),
bf_compressor_type_(common::ObCompressorType::NONE_COMPRESSOR)
per_addr_bf_count_(0),
filter_addr_idx_(0),
bf_compressor_type_(common::ObCompressorType::NONE_COMPRESSOR),
each_group_size_(0)
{}
~ObBloomFilterSendCtx() {}
bool bloom_filter_ready() const { return bloom_filter_ready_; }
void set_bloom_filter_ready(bool flag) { bloom_filter_ready_ = flag; }
int64_t &get_filter_channel_idx() { return filter_channel_idx_; }
int64_t &get_filter_addr_idx() { return filter_addr_idx_; }
common::ObIArray<BloomFilterIndex> &get_filter_indexes() { return filter_indexes_; }
void set_filter_data(ObPxBloomFilterData *data) { filter_data_ = data; }
ObPxBloomFilterData *get_filter_data() { return filter_data_; }
int generate_filter_indexes(int64_t each_group_size, int64_t channel_count);
void set_per_channel_bf_count(int64_t count) { per_channel_bf_count_ = count; }
int64_t get_per_channel_bf_count() { return per_channel_bf_count_; }
int generate_filter_indexes(int64_t each_group_size, int64_t addr_count);
void set_per_addr_bf_count(int64_t count) { per_addr_bf_count_ = count; }
int64_t get_per_addr_bf_count() { return per_addr_bf_count_; }
void set_bf_compress_type(common::ObCompressorType type)
{ bf_compressor_type_ = type; }
common::ObCompressorType get_bf_compress_type() { return bf_compressor_type_; }
int assign_bf_ch_set(ObPxBloomFilterChSet &bf_ch_set) { return bf_ch_set_.assign(bf_ch_set); }
ObPxBloomFilterChSet &get_bf_ch_set() { return bf_ch_set_; }
int64_t &get_each_group_size() { return each_group_size_; }
TO_STRING_KV(K_(bloom_filter_ready));
private:
bool bloom_filter_ready_;
ObPxBloomFilterChSet bf_ch_set_;
ObPxBloomFilterData *filter_data_;
common::ObArray<BloomFilterIndex> filter_indexes_;
int64_t per_channel_bf_count_;
int64_t filter_channel_idx_;
int64_t per_addr_bf_count_;
int64_t filter_addr_idx_;
common::ObCompressorType bf_compressor_type_;
int64_t each_group_size_;
};
class ObPxSQCProxy
{
public:
typedef hash::ObHashMap<int64_t, common::ObSArray<ObAddr> *,
hash::NoPthreadDefendMode> SQCP2PDhMap;
public:
ObPxSQCProxy(ObSqcCtx &sqc_ctx, ObPxRpcInitSqcArgs &arg);
virtual ~ObPxSQCProxy();
@ -179,6 +183,9 @@ public:
int64_t get_dh_msg_cnt() const;
void atomic_inc_dh_msg_cnt();
int64_t atomic_add_and_fetch_dh_msg_cnt();
int construct_p2p_dh_map(ObP2PDhMapInfo &map_info);
SQCP2PDhMap &get_p2p_dh_map() { return p2p_dh_map_; }
int check_is_local_dh(int64_t p2p_dh_id, bool &is_local_dh, int64_t msg_cnt);
private:
/* functions */
int setup_loop_proc(ObSqcCtx &sqc_ctx);
@ -204,9 +211,9 @@ private:
bool send_piece,
bool need_wait_whole_msg);
/* variables */
public:
public:
ObSqcCtx &sqc_ctx_;
private:
private:
ObPxRpcInitSqcArgs &sqc_arg_;
// 所有 worker 都抢这个锁,抢到者为 leader,负责推进 msg loop
common::ObSpinLock leader_token_lock_;
@ -218,7 +225,9 @@ private:
common::ObArray<ObBloomFilterSendCtx> bf_send_ctx_array_; // record bloom filters ready to be sent
ObDynamicSamplePieceMsg sample_msg_;
ObInitChannelPieceMsg init_channel_msg_;
common::ObThreadCond msg_ready_cond_; // msg cond is shared by transmit && rescive && bloom filter
// msg cond is shared by transmit && rescive && bloom filter
common::ObThreadCond msg_ready_cond_;
SQCP2PDhMap p2p_dh_map_;
DISALLOW_COPY_AND_ASSIGN(ObPxSQCProxy);
};

View File

@ -42,6 +42,7 @@
#include "storage/ddl/ob_direct_insert_sstable_ctx.h"
#include "sql/engine/px/ob_granule_pump.h"
#include "sql/das/ob_das_utils.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
@ -86,6 +87,8 @@ int ObPxSubCoord::pre_process()
LOG_WARN("unexpected status: op root is null", K(ret));
} else if (OB_FAIL(rebuild_sqc_access_table_locations())) {
LOG_WARN("fail to rebuild locations and tsc ops", K(ret));
} else if (OB_FAIL(construct_p2p_dh_map())) {
LOG_WARN("fail to construct p2p dh map", K(ret));
} else if (OB_FAIL(setup_op_input(*sqc_arg_.exec_ctx_,
*sqc_arg_.op_spec_root_,
sqc_ctx_,
@ -263,27 +266,7 @@ int ObPxSubCoord::pre_setup_op_input(ObExecContext &ctx,
const ObIArray<ObSqcTableLocationKey> &tsc_location_keys)
{
int ret = OB_SUCCESS;
if (IS_PX_BLOOM_FILTER(root.get_type())) {
ObJoinFilterSpec &filter_op = reinterpret_cast<ObJoinFilterSpec &>(root);
if (filter_op.is_use_mode() && filter_op.is_shuffle()) {
ObPxBloomFilter *filter_use = NULL;
ObPXBloomFilterHashWrapper bf_key;
bf_key.init(ctx.get_my_session()->get_effective_tenant_id(), filter_op.get_filter_id(),
filter_op.get_server_id(), sqc_arg_.sqc_.get_px_sequence_id());
if (OB_FAIL(ObPxBloomFilterManager::init_px_bloom_filter(filter_op.get_filter_length(),
ctx.get_allocator(),
filter_use))) {
LOG_WARN("fail to init px bloom filter", K(ret));
} else if (OB_FAIL(filter_use->generate_receive_count_array())) {
LOG_WARN("fail to generate receive count array", K(ret));
} else if (OB_FAIL(ObPxBloomFilterManager::instance().set_px_bloom_filter(bf_key, filter_use))) {
LOG_WARN("fail to set bloom filter to bloom filter manager", K(ret));
} else {
sqc_arg_.sqc_handler_->add_flag(OB_SQC_HANDLER_BLOOM_FILTER_NEED_CLEAR);
bf_key_ = bf_key;
}
}
} else if (IS_PX_GI(root.get_type())) {
if (IS_PX_GI(root.get_type())) {
// if it's not single tsc leaf dfo,
// setup_gi_op_input will be called later by subcoord preprocess func
if (is_single_tsc_leaf_dfo_ &&
@ -368,42 +351,42 @@ int ObPxSubCoord::setup_op_input(ObExecContext &ctx,
} else {
gi_input->set_parallelism(sqc.get_task_count());
}
} else if (IS_PX_BLOOM_FILTER(root.get_type())) {
} else if (IS_PX_JOIN_FILTER(root.get_type())) {
ObPxSqcMeta &sqc = sqc_arg_.sqc_;
ObJoinFilterSpec *filter_spec = reinterpret_cast<ObJoinFilterSpec *>(&root);
ObJoinFilterOpInput *filter_input = NULL;
ObPxBloomFilter *filter_create = NULL;
ObPxBloomFilter *filter_use = NULL;
int64_t tenant_id = ctx.get_my_session()->get_effective_tenant_id();
ObOperatorKit *kit = ctx.get_operator_kit(root.id_);
if (OB_ISNULL(kit) || OB_ISNULL(kit->input_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("operator is NULL", K(ret), KP(kit));
} else if (FALSE_IT(filter_input = static_cast<ObJoinFilterOpInput*>(kit->input_))) {
} else if (OB_FAIL(filter_input->init_share_info(ctx.get_allocator(), sqc.get_task_count()))) {
} else if (FALSE_IT(filter_input->set_px_sequence_id(
sqc.get_interrupt_id().px_interrupt_id_.first_))) {
} else if (OB_FAIL(filter_input->load_runtime_config(*filter_spec, ctx))) {
LOG_WARN("fail to load runtime config", K(ret));
} else if (FALSE_IT(filter_input->init_register_dm_info(
sqc.get_px_detectable_ids().qc_detectable_id_, sqc.get_qc_addr()))) {
} else if (filter_spec->is_create_mode()) {
int64_t filter_len = filter_spec->get_filter_length();
filter_input->is_local_create_ = false;
filter_input->set_sqc_proxy(sqc_ctx.sqc_proxy_);
if (!filter_spec->is_shared_join_filter()) {
/*do nothing*/
} else if (OB_FAIL(ObPxBloomFilterManager::init_px_bloom_filter(filter_len,
ctx.get_allocator(), filter_create))) {
LOG_WARN("fail to init px bloom filter", K(ret));
} else {
filter_input->share_info_.filter_ptr_ = reinterpret_cast<uint64_t>(filter_create);
if (OB_NOT_NULL(filter_create) && filter_spec->is_shuffle()) {
int64_t bf_idx_at_sqc_proxy = -1;
if (OB_FAIL(sqc_ctx.sqc_proxy_.append_bf_send_ctx(bf_idx_at_sqc_proxy))) {
LOG_WARN("failed to pre alloc bloom filter send ctx", K(ret));
} else {
filter_input->set_bf_idx_at_sqc_proxy(bf_idx_at_sqc_proxy);
} else if (OB_FAIL(filter_input->init_share_info(*filter_spec,
ctx, sqc.get_task_count(),
filter_spec->is_shuffle_? sqc.get_sqc_count() : 1))) {
LOG_WARN("fail to init share info", K(ret));
} else if (filter_spec->is_shuffle_) {
ObArray<ObP2PDatahubMsgBase *> *array_ptr =
reinterpret_cast<ObArray<ObP2PDatahubMsgBase *> *>(filter_input->share_info_.shared_msgs_);
for (int i = 0; OB_SUCC(ret) && i < array_ptr->count(); ++i) {
if (OB_FAIL(rf_msgs_.push_back(array_ptr->at(i)))) {
LOG_WARN("fail to push back rf msgs", K(ret));
}
}
}
}
if (OB_SUCC(ret)) {
filter_input->set_sqc_proxy(sqc_ctx.sqc_proxy_);
}
} else if (root.get_type() == PHY_TEMP_TABLE_ACCESS) {
ObPxSqcMeta &sqc = sqc_arg_.sqc_;
ObTempTableAccessOpInput *access_input = NULL;
@ -736,7 +719,13 @@ int ObPxSubCoord::end_process()
LOG_WARN("fail check task finish status", K(ret));
}
}
for (int i = 0; i < rf_msgs_.count(); ++i) {
rf_msgs_.at(i)->destroy();
rf_msgs_.at(i) = nullptr;
}
if (!rf_msgs_.empty()) {
rf_msgs_.reset();
}
NG_TRACE(tag3);
LOG_TRACE("exit ObPxSubCoord process", K(ret));
@ -789,29 +778,6 @@ void ObPxSubCoord::destroy_first_buffer_cache()
LOG_TRACE("trace unregister first buffer cache", K(ret), K(dfo_key));
}
void ObPxSubCoord::destroy_bloom_filter()
{
int ret = OB_SUCCESS;
ObPxBloomFilter *filter = NULL;
if (OB_FAIL(ObPxBloomFilterManager::instance()
.erase_px_bloom_filter(bf_key_, filter))) {
LOG_TRACE("fail to erase sqc px bloom filter", K(ret));
} else if (OB_NOT_NULL(filter)) {
int count = 0;
while (!filter->is_merge_filter_finish()) {
// A cumbersome but safe waiting behavior.
// When the worker thread exits,
// it is necessary to ensure that
// no one in the DTL holds the filter pointer or writing.
if (0 == (count++ % 1000)) { // one log per second
LOG_TRACE("wait dtl holds bloom filter end");
}
ob_usleep(1000);
}
filter->reset();
}
}
int ObPxSubCoord::check_need_start_ddl(bool &need_start_ddl)
{
int ret = OB_SUCCESS;

View File

@ -26,9 +26,11 @@
#include "sql/engine/ob_engine_op_traits.h"
#include "sql/dtl/ob_dtl_channel_loop.h"
#include "sql/dtl/ob_dtl_local_first_buffer_manager.h"
#include "sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h"
#include "sql/ob_sql_trans_control.h"
#include "lib/allocator/ob_safe_arena.h"
namespace oceanbase
{
namespace observer
@ -39,7 +41,6 @@ struct ObGlobalContext;
namespace sql
{
class ObPxSQCHandler;
class ObPxSubCoord
{
public:
@ -52,8 +53,8 @@ public:
local_worker_factory_(gctx, allocator_),
thread_worker_factory_(gctx, allocator_),
first_buffer_cache_(allocator_),
bf_key_(),
is_single_tsc_leaf_dfo_(false)
is_single_tsc_leaf_dfo_(false),
rf_msgs_()
{}
virtual ~ObPxSubCoord() = default;
int pre_process();
@ -71,7 +72,6 @@ public:
}
int init_first_buffer_cache(int64_t dop);
void destroy_first_buffer_cache();
void destroy_bloom_filter();
// for ddl insert sstable
// using start and end pair function to control the life cycle of ddl context
@ -132,6 +132,10 @@ private:
const int64_t table_id,
ObIArray<std::pair<share::ObLSID, ObTabletID>> &ls_tablet_ids) const;
void try_get_dml_op(ObOpSpec &root, ObTableModifySpec *&dml_op);
int construct_p2p_dh_map() {
return sqc_ctx_.sqc_proxy_.construct_p2p_dh_map(
sqc_arg_.sqc_.get_p2p_dh_map_info());
}
private:
const observer::ObGlobalContext &gctx_;
ObPxRpcInitSqcArgs &sqc_arg_;
@ -143,8 +147,8 @@ private:
ObPxThreadWorkerFactory thread_worker_factory_; // 超过1个task的部分,使用thread 构造 worker
int64_t reserved_thread_count_;
dtl::ObDtlLocalFirstBufferCache first_buffer_cache_;
ObPXBloomFilterHashWrapper bf_key_; // for bloom_filter_use op
bool is_single_tsc_leaf_dfo_;
ObArray<ObP2PDatahubMsgBase *> rf_msgs_; // for clear
DISALLOW_COPY_AND_ASSIGN(ObPxSubCoord);
};
}

View File

@ -605,14 +605,13 @@ int ObPxTaskProcess::OpPreparation::apply(ObExecContext &ctx,
input->set_sqc_id(sqc_id_);
input->set_dfo_id(dfo_id_);
}
} else if (IS_PX_BLOOM_FILTER(op.get_type())) {
} else if (IS_PX_JOIN_FILTER(op.get_type())) {
ObJoinFilterSpec *filter_spec = reinterpret_cast<ObJoinFilterSpec *>(&op);
if (OB_ISNULL(kit->input_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("operator is NULL", K(ret), K(op.id_), KP(kit));
} else {
ObJoinFilterOpInput *input = static_cast<ObJoinFilterOpInput *>(kit->input_);
input->set_px_sequence_id(task_->px_int_id_.px_interrupt_id_.first_);
if (!filter_spec->is_shared_join_filter()) {
input->set_task_id(task_id_);
}

View File

@ -18,6 +18,7 @@
#include "sql/dtl/ob_dtl_channel_group.h"
#include "sql/dtl/ob_dtl.h"
#include "sql/engine/px/ob_px_util.h"
#include "sql/engine/px/ob_px_scheduler.h"
#include "sql/executor/ob_task_spliter.h"
#include "observer/ob_server_struct.h"
#include "sql/engine/px/exchange/ob_receive_op.h"
@ -34,6 +35,7 @@
#include "rpc/obrpc/ob_net_keepalive.h"
#include "share/external_table/ob_external_table_utils.h"
using namespace oceanbase::common;
using namespace oceanbase::sql;
using namespace oceanbase::sql::dtl;
@ -382,30 +384,45 @@ int ObPXServerAddrUtil::find_dml_ops_inner(common::ObIArray<const ObTableModifyS
return ret;
}
template<class T>
static int get_location_addrs(const T &locations,
ObIArray<ObAddr> &addrs)
int ObPXServerAddrUtil::generate_dh_map_info(ObDfo &dfo)
{
int ret = OB_SUCCESS;
hash::ObHashSet<ObAddr> addr_set;
if (OB_FAIL(addr_set.create(locations.size()))) {
LOG_WARN("fail creat addr set", "size", locations.size(), K(ret));
}
for (auto iter = locations.begin(); OB_SUCC(ret) && iter != locations.end(); ++iter) {
ret = addr_set.exist_refactored((*iter)->server_);
if (OB_HASH_EXIST == ret) {
ret = OB_SUCCESS;
} else if (OB_HASH_NOT_EXIST == ret) {
if (OB_FAIL(addrs.push_back((*iter)->server_))) {
LOG_WARN("fail push back server", K(ret));
} else if (OB_FAIL(addr_set.set_refactored((*iter)->server_))) {
LOG_WARN("fail set addr to addr_set", K(ret));
ObP2PDhMapInfo &p2p_map_info = dfo.get_p2p_dh_map_info();
if (OB_ISNULL(dfo.get_coord_info_ptr())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected coord info ptr", K(ret));
} else if (!dfo.get_p2p_dh_ids().empty() && p2p_map_info.is_empty()) {
ObP2PDfoMapNode node;
for (int i = 0; i < dfo.get_p2p_dh_ids().count() && OB_SUCC(ret); ++i) {
node.reset();
if (OB_FAIL(dfo.get_coord_info_ptr()->p2p_dfo_map_.get_refactored(
dfo.get_p2p_dh_ids().at(i), node))) {
LOG_WARN("fail to get target dfo id", K(ret));
} else if (node.addrs_.empty()) {
ObDfo *target_dfo_ptr = nullptr;
if (OB_FAIL(dfo.get_coord_info_ptr()->dfo_mgr_.find_dfo_edge(
node.target_dfo_id_, target_dfo_ptr))) {
LOG_WARN("fail to find dfo edge", K(ret));
} else if (target_dfo_ptr->get_p2p_dh_addrs().empty()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("fail to get p2p dh addrs", K(ret), K(dfo.get_p2p_dh_ids().at(i)));
} else if (OB_FAIL(node.addrs_.assign(target_dfo_ptr->get_p2p_dh_addrs()))) {
LOG_WARN("fail to assign p2p dh addrs", K(ret));
} else if (OB_FAIL(dfo.get_coord_info_ptr()->p2p_dfo_map_.set_refactored(
dfo.get_p2p_dh_ids().at(i), node, 1/*over_write*/))) {
LOG_WARN("fail to set p2p dh addrs", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(p2p_map_info.p2p_sequence_ids_.push_back(dfo.get_p2p_dh_ids().at(i)))) {
LOG_WARN("fail to push back p2p map info", K(ret));
} else if (OB_FAIL(p2p_map_info.target_addrs_.push_back(node.addrs_))) {
LOG_WARN("fail to push back addrs", K(ret));
}
}
} else {
LOG_WARN("fail check server exist in addr_set", K(ret));
}
}
(void)addr_set.destroy();
return ret;
}
@ -435,6 +452,13 @@ int ObPXServerAddrUtil::build_dfo_sqc(ObExecContext &ctx,
LOG_TRACE("parallel not set in query hint. set default to 1");
}
}
// generate dh map info
if (OB_SUCC(ret)) {
if (OB_FAIL(generate_dh_map_info(dfo))) {
LOG_WARN("fail to generate dh map info", K(ret));
}
}
if (OB_SUCC(ret) && addrs.count() > 0) {
common::ObArray<ObPxSqcMeta *> sqcs;
int64_t total_part_cnt = 0;
@ -450,10 +474,18 @@ int ObPXServerAddrUtil::build_dfo_sqc(ObExecContext &ctx,
sqc.set_px_sequence_id(dfo.get_px_sequence_id());
sqc.set_qc_id(dfo.get_qc_id());
sqc.set_interrupt_id(dfo.get_interrupt_id());
sqc.set_px_detectable_ids(dfo.get_px_detectable_ids());
sqc.set_fulltree(dfo.is_fulltree());
sqc.set_qc_server_id(dfo.get_qc_server_id());
sqc.set_parent_dfo_id(dfo.get_parent_dfo_id());
sqc.set_single_tsc_leaf_dfo(dfo.is_single_tsc_leaf_dfo());
if (OB_SUCC(ret)) {
if (!dfo.get_p2p_dh_map_info().is_empty()) {
if (OB_FAIL(sqc.get_p2p_dh_map_info().assign(dfo.get_p2p_dh_map_info()))) {
LOG_WARN("fail to assign p2p dh map info", K(ret));
}
}
}
for (auto iter = locations.begin(); OB_SUCC(ret) && iter != locations.end(); ++iter) {
if (addrs.at(i) == (*iter)->server_) {
if (OB_FAIL(sqc_locations.push_back(*iter))) {
@ -518,7 +550,7 @@ int ObPXServerAddrUtil::alloc_by_temp_child_distribution(ObExecContext &exec_ctx
}
int ObPXServerAddrUtil::alloc_by_temp_child_distribution_inner(ObExecContext &exec_ctx,
ObDfo &child)
ObDfo &child)
{
int ret = OB_SUCCESS;
ObSEArray<const ObTableScanSpec *, 2> scan_ops;
@ -544,6 +576,8 @@ int ObPXServerAddrUtil::alloc_by_temp_child_distribution_inner(ObExecContext &ex
ObArray<int64_t> sqc_result_count;
if (OB_FAIL(sqc_result_count.prepare_allocate(interm_result_infos.count()))) {
LOG_WARN("Failed to pre allocate sqc part count");
} else if (OB_FAIL(generate_dh_map_info(child))) {
LOG_WARN("fail to generate dh map info", K(ret));
}
for (int64_t j = 0; OB_SUCC(ret) && j < interm_result_infos.count(); j++) {
SMART_VAR(ObPxSqcMeta, sqc) {
@ -556,10 +590,19 @@ int ObPXServerAddrUtil::alloc_by_temp_child_distribution_inner(ObExecContext &ex
sqc.set_px_sequence_id(child.get_px_sequence_id());
sqc.set_qc_id(child.get_qc_id());
sqc.set_interrupt_id(child.get_interrupt_id());
sqc.set_px_detectable_ids(child.get_px_detectable_ids());
sqc.set_fulltree(child.is_fulltree());
sqc.set_qc_server_id(child.get_qc_server_id());
sqc.set_parent_dfo_id(child.get_parent_dfo_id());
if (OB_FAIL(child.add_sqc(sqc))) {
if (OB_SUCC(ret)) {
if (!child.get_p2p_dh_map_info().is_empty()) {
if (OB_FAIL(sqc.get_p2p_dh_map_info().assign(child.get_p2p_dh_map_info()))) {
LOG_WARN("fail to assign p2p dh map info", K(ret));
}
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(child.add_sqc(sqc))) {
LOG_WARN("fail add sqc", K(sqc), K(ret));
}
}
@ -632,6 +675,7 @@ int ObPXServerAddrUtil::alloc_by_child_distribution(const ObDfo &child, ObDfo &p
sqc.set_px_sequence_id(parent.get_px_sequence_id());
sqc.set_qc_id(parent.get_qc_id());
sqc.set_interrupt_id(parent.get_interrupt_id());
sqc.set_px_detectable_ids(parent.get_px_detectable_ids());
sqc.set_fulltree(parent.is_fulltree());
sqc.set_qc_server_id(parent.get_qc_server_id());
sqc.set_parent_dfo_id(parent.get_parent_dfo_id());
@ -699,6 +743,12 @@ int ObPXServerAddrUtil::alloc_by_random_distribution(ObExecContext &exec_ctx,
total_task_count += sqc_max_task_counts.at(i);
}
}
// generate dh map info
if (OB_SUCC(ret)) {
if (OB_FAIL(generate_dh_map_info(parent))) {
LOG_WARN("fail to generate dh map info", K(ret));
}
}
for (int64_t i = 0; i < addrs.count() && OB_SUCC(ret); ++i) {
SMART_VAR(ObPxSqcMeta, sqc) {
sqc.set_exec_addr(addrs.at(i));
@ -713,10 +763,19 @@ int ObPXServerAddrUtil::alloc_by_random_distribution(ObExecContext &exec_ctx,
sqc.set_px_sequence_id(parent.get_px_sequence_id());
sqc.set_qc_id(parent.get_qc_id());
sqc.set_interrupt_id(parent.get_interrupt_id());
sqc.set_px_detectable_ids(parent.get_px_detectable_ids());
sqc.set_fulltree(parent.is_fulltree());
sqc.set_qc_server_id(parent.get_qc_server_id());
sqc.set_parent_dfo_id(parent.get_parent_dfo_id());
if (OB_FAIL(parent.add_sqc(sqc))) {
if (OB_SUCC(ret)) {
if (!parent.get_p2p_dh_map_info().is_empty()) {
if (OB_FAIL(sqc.get_p2p_dh_map_info().assign(parent.get_p2p_dh_map_info()))) {
LOG_WARN("fail to assign p2p dh map info", K(ret));
}
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(parent.add_sqc(sqc))) {
LOG_WARN("fail add sqc", K(sqc), K(ret));
}
}
@ -730,7 +789,14 @@ int ObPXServerAddrUtil::alloc_by_local_distribution(ObExecContext &exec_ctx,
{
int ret = OB_SUCCESS;
ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(exec_ctx);
if (OB_ISNULL(plan_ctx)) {
// generate dh map info
if (OB_SUCC(ret)) {
if (OB_FAIL(generate_dh_map_info(dfo))) {
LOG_WARN("fail to generate dh map info", K(ret));
}
}
if (OB_FAIL(ret)) {
} else if (OB_ISNULL(plan_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("NULL phy plan ctx", K(ret));
} else {
@ -745,12 +811,14 @@ int ObPXServerAddrUtil::alloc_by_local_distribution(ObExecContext &exec_ctx,
sqc.set_px_sequence_id(dfo.get_px_sequence_id());
sqc.set_qc_id(dfo.get_qc_id());
sqc.set_interrupt_id(dfo.get_interrupt_id());
sqc.set_px_detectable_ids(dfo.get_px_detectable_ids());
sqc.set_fulltree(dfo.is_fulltree());
sqc.set_parent_dfo_id(dfo.get_parent_dfo_id());
sqc.set_qc_server_id(dfo.get_qc_server_id());
if (OB_FAIL(dfo.add_sqc(sqc))) {
LOG_WARN("fail add sqc", K(sqc), K(ret));
if (!dfo.get_p2p_dh_map_info().is_empty()) {
OZ(sqc.get_p2p_dh_map_info().assign(dfo.get_p2p_dh_map_info()));
}
OZ(dfo.add_sqc(sqc));
}
}
return ret;

View File

@ -247,6 +247,8 @@ private:
static int assign_external_files_to_sqc(const common::ObIArray<share::ObExternalFileInfo> &files,
bool is_file_on_disk,
common::ObIArray<ObPxSqcMeta *> &sqcs);
private:
static int generate_dh_map_info(ObDfo &dfo);
DISALLOW_COPY_AND_ASSIGN(ObPXServerAddrUtil);
};
@ -568,6 +570,33 @@ public:
static bool is_in_blacklist(const common::ObAddr &addr, int64_t server_start_time);
};
template<class T>
static int get_location_addrs(const T &locations,
ObIArray<ObAddr> &addrs)
{
int ret = OB_SUCCESS;
hash::ObHashSet<ObAddr> addr_set;
if (OB_FAIL(addr_set.create(locations.size()))) {
SQL_LOG(WARN, "fail create addr set", K(locations.size()), K(ret));
}
for (auto iter = locations.begin(); OB_SUCC(ret) && iter != locations.end(); ++iter) {
ret = addr_set.exist_refactored((*iter)->server_);
if (OB_HASH_EXIST == ret) {
ret = OB_SUCCESS;
} else if (OB_HASH_NOT_EXIST == ret) {
if (OB_FAIL(addrs.push_back((*iter)->server_))) {
SQL_LOG(WARN, "fail push back server", K(ret));
} else if (OB_FAIL(addr_set.set_refactored((*iter)->server_))) {
SQL_LOG(WARN, "fail set addr to addr_set", K(ret));
}
} else {
SQL_LOG(WARN, "fail check server exist in addr_set", K(ret));
}
}
(void)addr_set.destroy();
return ret;
}
}
}

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