diff --git a/deps/oblib/src/lib/hash/ob_hashmap.h b/deps/oblib/src/lib/hash/ob_hashmap.h index e40949e09..d69581887 100644 --- a/deps/oblib/src/lib/hash/ob_hashmap.h +++ b/deps/oblib/src/lib/hash/ob_hashmap.h @@ -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 + 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 int serialization(_archive &archive) { diff --git a/deps/oblib/src/lib/hash/ob_hashtable.h b/deps/oblib/src/lib/hash/ob_hashtable.h index ebc1cd0d8..0bcf29250 100644 --- a/deps/oblib/src/lib/hash/ob_hashtable.h +++ b/deps/oblib/src/lib/hash/ob_hashtable.h @@ -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 + 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 + 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 diff --git a/deps/oblib/src/lib/utility/ob_tracepoint.h b/deps/oblib/src/lib/utility/ob_tracepoint.h index 88dedd21f..738af8ff1 100644 --- a/deps/oblib/src/lib/utility/ob_tracepoint.h +++ b/deps/oblib/src/lib/utility/ob_tracepoint.h @@ -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 diff --git a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h index 0dd13ed8f..47dc5ae47 100644 --- a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h +++ b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h @@ -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) diff --git a/mittest/simple_server/CMakeLists.txt b/mittest/simple_server/CMakeLists.txt index 6482714f8..4f39109b9 100644 --- a/mittest/simple_server/CMakeLists.txt +++ b/mittest/simple_server/CMakeLists.txt @@ -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) diff --git a/mittest/simple_server/test_ob_detect_manager_in_simple_server.cpp b/mittest/simple_server/test_ob_detect_manager_in_simple_server.cpp new file mode 100644 index 000000000..cc263348f --- /dev/null +++ b/mittest/simple_server/test_ob_detect_manager_in_simple_server.cpp @@ -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 +#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 &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 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 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; +} diff --git a/src/objit/include/objit/common/ob_item_type.h b/src/objit/include/objit/common/ob_item_type.h index afe2335b8..8a9fa43e5 100755 --- a/src/objit/include/objit/common/ob_item_type.h +++ b/src/objit/include/objit/common/ob_item_type.h @@ -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. diff --git a/src/observer/CMakeLists.txt b/src/observer/CMakeLists.txt index 4d7799ce1..854748bb8 100644 --- a/src/observer/CMakeLists.txt +++ b/src/observer/CMakeLists.txt @@ -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 diff --git a/src/observer/ob_server.cpp b/src/observer/ob_server.cpp index f845a0a7c..963a6c7b5 100644 --- a/src/observer/ob_server.cpp +++ b/src/observer/ob_server.cpp @@ -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"); diff --git a/src/observer/ob_srv_xlator_partition.cpp b/src/observer/ob_srv_xlator_partition.cpp index 963ce0764..3400eb985 100644 --- a/src/observer/ob_srv_xlator_partition.cpp +++ b/src/observer/ob_srv_xlator_partition.cpp @@ -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 diff --git a/src/observer/ob_srv_xlator_primary.cpp b/src/observer/ob_srv_xlator_primary.cpp index 8c2f69c9b..0b1159f45 100644 --- a/src/observer/ob_srv_xlator_primary.cpp +++ b/src/observer/ob_srv_xlator_primary.cpp @@ -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_); diff --git a/src/observer/omt/ob_multi_tenant.cpp b/src/observer/omt/ob_multi_tenant.cpp index 51cd40df9..ba4d12a91 100644 --- a/src/observer/omt/ob_multi_tenant.cpp +++ b/src/observer/omt/ob_multi_tenant.cpp @@ -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, nullptr, nullptr, nullptr, nullptr, server_obj_pool_mtl_destroy); + 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); diff --git a/src/observer/virtual_table/ob_all_virtual_px_p2p_datahub.cpp b/src/observer/virtual_table/ob_all_virtual_px_p2p_datahub.cpp new file mode 100644 index 000000000..1f2995989 --- /dev/null +++ b/src/observer/virtual_table/ob_all_virtual_px_p2p_datahub.cpp @@ -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 &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 */ diff --git a/src/observer/virtual_table/ob_all_virtual_px_p2p_datahub.h b/src/observer/virtual_table/ob_all_virtual_px_p2p_datahub.h new file mode 100644 index 000000000..d36073f3c --- /dev/null +++ b/src/observer/virtual_table/ob_all_virtual_px_p2p_datahub.h @@ -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 &node_array, int64_t tenant_id) : + node_array_(node_array), tenant_id_(tenant_id) {}; + ~P2PMsgTraverseCall() = default; + int operator() (common::hash::HashMapPair &entry); + common::ObArray &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 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_ diff --git a/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp b/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp index 99ef38c74..3b77aacd3 100644 --- a/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp +++ b/src/observer/virtual_table/ob_virtual_table_iterator_factory.cpp @@ -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 ¶ms, } 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(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))) { diff --git a/src/share/CMakeLists.txt b/src/share/CMakeLists.txt index 8363c798f..dfc9e1ce5 100644 --- a/src/share/CMakeLists.txt +++ b/src/share/CMakeLists.txt @@ -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 diff --git a/src/share/config/ob_config_helper.cpp b/src/share/config/ob_config_helper.cpp index e845fb29f..047b65db2 100644 --- a/src/share/config/ob_config_helper.cpp +++ b/src/share/config/ob_config_helper.cpp @@ -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 diff --git a/src/share/config/ob_config_helper.h b/src/share/config/ob_config_helper.h index d2057c8ba..da8ac5f1a 100644 --- a/src/share/config/ob_config_helper.h +++ b/src/share/config/ob_config_helper.h @@ -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 { diff --git a/src/share/detect/ob_detect_callback.cpp b/src/share/detect/ob_detect_callback.cpp new file mode 100644 index 000000000..3e682051a --- /dev/null +++ b/src/share/detect/ob_detect_callback.cpp @@ -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 &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 &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 &entry) +{ + UNUSED(entry); + common::ObIArray &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( + 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(finish_msg.get_serialize_size()); + header.type_ = static_cast(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 &peer_states, const ObInterruptibleTaskID &tid, sql::ObDfo &dfo, + const ObArray &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(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 diff --git a/src/share/detect/ob_detect_callback.h b/src/share/detect/ob_detect_callback.h new file mode 100644 index 000000000..e6640c39e --- /dev/null +++ b/src/share/detect/ob_detect_callback.h @@ -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 &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 &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 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 &peer_states, const ObInterruptibleTaskID &tid, sql::ObDfo &dfo, + const ObArray &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 dtl_channels_; +}; + +class ObSqcDetectCB : public ObIDetectCallback +{ +public: + ObSqcDetectCB(const ObArray &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 &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 &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 &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 diff --git a/src/share/detect/ob_detect_manager.cpp b/src/share/detect/ob_detect_manager.cpp new file mode 100644 index 000000000..c97f5fc96 --- /dev/null +++ b/src/share/detect/ob_detect_manager.cpp @@ -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(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(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(DEFAULT_CHECK_MAP_BUCKETS_COUNT * mem_factor); + int64_t detectable_ids_bucket_count = static_cast(DEFAULT_CHECK_MAP_BUCKETS_COUNT * mem_factor); + int64_t still_need_check_id_bucket_count = static_cast(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(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 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 &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 &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 &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 &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 &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 &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 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(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 diff --git a/src/share/detect/ob_detect_manager.h b/src/share/detect/ob_detect_manager.h new file mode 100644 index 000000000..85abc59c6 --- /dev/null +++ b/src/share/detect/ob_detect_manager.h @@ -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, + hash::equal_to> 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, + hash::equal_to> CHECK_MAP; + + /// Atomic insertion callback + class ObDetectCallbackNodeAddCall + { + public: + void operator()(hash::HashMapPair &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 &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 &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 &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 &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 + 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 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 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 rpc_statuses_; + + ObAddr self_; + bool is_inited_; + ObThreadCond cond_; +}; + +} // end namespace common +} // end namespace oceanbase diff --git a/src/share/detect/ob_detect_manager_utils.cpp b/src/share/detect/ob_detect_manager_utils.cpp new file mode 100644 index 000000000..30d340703 --- /dev/null +++ b/src/share/detect/ob_detect_manager_utils.cpp @@ -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 ®ister_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 ®ister_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 ®ister_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 &peer_states, + const ObArray &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 &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 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 ®ister_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 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 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 ®ister_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 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 diff --git a/src/share/detect/ob_detect_manager_utils.h b/src/share/detect/ob_detect_manager_utils.h new file mode 100644 index 000000000..f62c7b595 --- /dev/null +++ b/src/share/detect/ob_detect_manager_utils.h @@ -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 ®ister_dm_info, sql::ObPxSqcHandler *handler); + + static int qc_register_detectable_id_into_dm(common::ObDetectableId &detectable_id, + bool ®ister_detectable_id, uint64_t tenant_id); + static void qc_unregister_detectable_id_from_dm(const common::ObDetectableId &detectable_id, + bool ®ister_detectable_id); + static int qc_register_check_item_into_dm(sql::ObDfo &dfo, + const ObArray &peer_states, + const ObArray &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 &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 ®ister_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 ®ister_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 diff --git a/src/share/detect/ob_detect_rpc_processor.cpp b/src/share/detect/ob_detect_rpc_processor.cpp new file mode 100644 index 000000000..a26a6248f --- /dev/null +++ b/src/share/detect/ob_detect_rpc_processor.cpp @@ -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 diff --git a/src/share/detect/ob_detect_rpc_processor.h b/src/share/detect/ob_detect_rpc_processor.h new file mode 100644 index 000000000..57ae67cc5 --- /dev/null +++ b/src/share/detect/ob_detect_rpc_processor.h @@ -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> +{ +public: + ObDetectRpcP() {} + virtual ~ObDetectRpcP() = default; +protected: + int process(); +private: + DISALLOW_COPY_AND_ASSIGN(ObDetectRpcP); +}; + +} // end namespace obrpc +} // end namespace oceanbase diff --git a/src/share/detect/ob_detect_rpc_proxy.cpp b/src/share/detect/ob_detect_rpc_proxy.cpp new file mode 100644 index 000000000..17b32b5dd --- /dev/null +++ b/src/share/detect/ob_detect_rpc_proxy.cpp @@ -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(&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 diff --git a/src/share/detect/ob_detect_rpc_proxy.h b/src/share/detect/ob_detect_rpc_proxy.h new file mode 100644 index 000000000..6acd0bea5 --- /dev/null +++ b/src/share/detect/ob_detect_rpc_proxy.h @@ -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 peer_ids_; // serialize from hashset + common::ObSEArray 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 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 +{ +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 diff --git a/src/share/detect/ob_detectable_id.cpp b/src/share/detect/ob_detectable_id.cpp new file mode 100644 index 000000000..8eec847f2 --- /dev/null +++ b/src/share/detect/ob_detectable_id.cpp @@ -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 diff --git a/src/share/detect/ob_detectable_id.h b/src/share/detect/ob_detectable_id.h new file mode 100644 index 000000000..6fc19cc40 --- /dev/null +++ b/src/share/detect/ob_detectable_id.h @@ -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 diff --git a/src/share/diagnosis/ob_sql_monitor_statname.h b/src/share/diagnosis/ob_sql_monitor_statname.h index 620d728ef..f1e70eea0 100644 --- a/src/share/diagnosis/ob_sql_monitor_statname.h +++ b/src/share/diagnosis/ob_sql_monitor_statname.h @@ -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") diff --git a/src/share/inner_table/ob_inner_table_schema.12351_12400.cpp b/src/share/inner_table/ob_inner_table_schema.12351_12400.cpp index f7681efe2..d79a8eb87 100644 --- a/src/share/inner_table/ob_inner_table_schema.12351_12400.cpp +++ b/src/share/inner_table/ob_inner_table_schema.12351_12400.cpp @@ -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 diff --git a/src/share/inner_table/ob_inner_table_schema.15351_15400.cpp b/src/share/inner_table/ob_inner_table_schema.15351_15400.cpp index fba14d283..c67494fef 100644 --- a/src/share/inner_table/ob_inner_table_schema.15351_15400.cpp +++ b/src/share/inner_table/ob_inner_table_schema.15351_15400.cpp @@ -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; diff --git a/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp b/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp index 095cd81c1..91b827c83 100644 --- a/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp +++ b/src/share/inner_table/ob_inner_table_schema.21301_21350.cpp @@ -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)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.21401_21450.cpp b/src/share/inner_table/ob_inner_table_schema.21401_21450.cpp index 7e843adee..bd370560f 100644 --- a/src/share/inner_table/ob_inner_table_schema.21401_21450.cpp +++ b/src/share/inner_table/ob_inner_table_schema.21401_21450.cpp @@ -110,7 +110,7 @@ int ObInnerTableSchema::dba_ob_external_table_files_schema(ObTableSchema &table_ 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 B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS TABLE_SCHEMA, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM OCEANBASE.__ALL_EXTERNAL_TABLE_FILE A INNER JOIN OCEANBASE.__ALL_TABLE B ON A.TABLE_ID = B.TABLE_ID AND B.TENANT_ID = 0 INNER JOIN OCEANBASE.__ALL_DATABASE C ON B.DATABASE_ID = C.DATABASE_ID AND C.TENANT_ID = 0 WHERE B.TABLE_TYPE = 14 AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS TABLE_SCHEMA, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM OCEANBASE.__ALL_EXTERNAL_TABLE_FILE A INNER JOIN OCEANBASE.__ALL_TABLE B ON A.TABLE_ID = B.TABLE_ID AND B.TENANT_ID = 0 INNER JOIN OCEANBASE.__ALL_DATABASE C ON B.DATABASE_ID = C.DATABASE_ID AND C.TENANT_ID = 0 WHERE B.TABLE_TYPE = 14 AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -160,7 +160,207 @@ int ObInnerTableSchema::all_ob_external_table_files_schema(ObTableSchema &table_ 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 B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS TABLE_SCHEMA, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM OCEANBASE.__ALL_EXTERNAL_TABLE_FILE A INNER JOIN OCEANBASE.__ALL_TABLE B ON A.TABLE_ID = B.TABLE_ID AND B.TENANT_ID = 0 INNER JOIN OCEANBASE.__ALL_DATABASE C ON B.DATABASE_ID = C.DATABASE_ID AND C.TENANT_ID = 0 WHERE B.TABLE_TYPE = 14 AND 0 = sys_privilege_check('table_acc', EFFECTIVE_TENANT_ID(), C.DATABASE_NAME, B.TABLE_NAME) AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS TABLE_SCHEMA, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM OCEANBASE.__ALL_EXTERNAL_TABLE_FILE A INNER JOIN OCEANBASE.__ALL_TABLE B ON A.TABLE_ID = B.TABLE_ID AND B.TENANT_ID = 0 INNER JOIN OCEANBASE.__ALL_DATABASE C ON B.DATABASE_ID = C.DATABASE_ID AND C.TENANT_ID = 0 WHERE B.TABLE_TYPE = 14 AND 0 = sys_privilege_check('table_acc', EFFECTIVE_TENANT_ID(), C.DATABASE_NAME, B.TABLE_NAME) AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::gv_ob_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_GV_OB_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(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_GV_OB_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)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( 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 )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::v_ob_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_V_OB_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(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_V_OB_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)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT * FROM OCEANBASE.GV$OB_PX_P2P_DATAHUB WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::gv_sql_join_filter_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_GV_SQL_JOIN_FILTER_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_GV_SQL_JOIN_FILTER_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT SVR_IP, 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' )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::v_sql_join_filter_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_SYS_DATABASE_ID); + table_schema.set_table_id(OB_V_SQL_JOIN_FILTER_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_V_SQL_JOIN_FILTER_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT * FROM OCEANBASE.GV$SQL_JOIN_FILTER WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -210,7 +410,7 @@ int ObInnerTableSchema::dba_ob_table_stat_stale_info_schema(ObTableSchema &table 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"__( WITH V AS (SELECT NVL(T.TENANT_ID, 0) AS TENANT_ID, NVL(T.TABLE_ID, VT.TABLE_ID) AS TABLE_ID, NVL(T.TABLET_ID, VT.TABLET_ID) AS TABLET_ID, NVL(T.INSERTS, 0) + NVL(VT.INSERT_ROW_COUNT, 0) - NVL(T.LAST_INSERTS, 0) AS INSERTS, NVL(T.UPDATES, 0) + NVL(VT.UPDATE_ROW_COUNT, 0) - NVL(T.LAST_UPDATES, 0) AS UPDATES, NVL(T.DELETES, 0) + NVL(VT.DELETE_ROW_COUNT, 0) - NVL(T.LAST_DELETES, 0) AS DELETES FROM OCEANBASE.__ALL_MONITOR_MODIFIED T FULL JOIN OCEANBASE.__ALL_VIRTUAL_DML_STATS VT ON T.TABLE_ID = VT.TABLE_ID AND T.TABLET_ID = VT.TABLET_ID AND VT.TENANT_ID = EFFECTIVE_TENANT_ID() ) SELECT CAST(TM.DATABASE_NAME AS CHAR(128)) AS DATABASE_NAME, CAST(TM.TABLE_NAME AS CHAR(128)) AS TABLE_NAME, CAST(TM.PART_NAME AS CHAR(128)) AS PARTITION_NAME, CAST(TM.SUB_PART_NAME AS CHAR(128)) AS SUBPARTITION_NAME, CAST(TS.ROW_CNT AS SIGNED) AS LAST_ANALYZED_ROWS, TS.LAST_ANALYZED AS LAST_ANALYZED_TIME, CAST(TM.INSERTS AS SIGNED) AS INSERTS, CAST(TM.UPDATES AS SIGNED) AS UPDATES, CAST(TM.DELETES AS SIGNED) AS DELETES, CAST(NVL(CAST(UP.VALCHAR AS SIGNED), CAST(GP.SPARE4 AS SIGNED)) AS SIGNED) STALE_PERCENT, CAST(CASE NVL((TM.INSERTS + TM.UPDATES + TM.DELETES) > TS.ROW_CNT * NVL(CAST(UP.VALCHAR AS SIGNED), CAST(GP.SPARE4 AS SIGNED)) / 100, (TM.INSERTS + TM.UPDATES + TM.DELETES) > 0) WHEN 0 THEN 'NO' WHEN 1 THEN 'YES' END AS CHAR(3)) AS IS_STALE FROM (SELECT T.TENANT_ID, T.TABLE_ID, CASE T.PART_LEVEL WHEN 0 THEN T.TABLE_ID WHEN 1 THEN P.PART_ID WHEN 2 THEN SP.SUB_PART_ID END AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, SP.SUB_PART_NAME, NVL(V.INSERTS, 0) AS INSERTS, NVL(V.UPDATES, 0) AS UPDATES, NVL(V.DELETES, 0) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID LEFT JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID LEFT JOIN OCEANBASE.__ALL_SUB_PART SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = CASE T.PART_LEVEL WHEN 0 THEN T.TABLET_ID WHEN 1 THEN P.TABLET_ID WHEN 2 THEN SP.TABLET_ID END WHERE T.TABLE_TYPE IN (0, 3, 6) UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = P.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 6) AND T.PART_LEVEL = 1 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), MIN(P.PART_ID) AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID JOIN OCEANBASE.__ALL_SUB_PART SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 6) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID JOIN OCEANBASE.__ALL_SUB_PART SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 6) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME ) TM LEFT JOIN OCEANBASE.__ALL_TABLE_STAT TS ON TM.TENANT_ID = TS.TENANT_ID AND TM.TABLE_ID = TS.TABLE_ID AND TM.PARTITION_ID = TS.PARTITION_ID LEFT JOIN OCEANBASE.__ALL_OPTSTAT_USER_PREFS UP ON TM.TENANT_ID = UP.TENANT_ID AND TM.TABLE_ID = UP.TABLE_ID AND UP.PNAME = 'STALE_PERCENT' JOIN OCEANBASE.__ALL_OPTSTAT_GLOBAL_PREFS GP ON GP.SNAME = 'STALE_PERCENT' )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( WITH V AS (SELECT NVL(T.TENANT_ID, 0) AS TENANT_ID, NVL(T.TABLE_ID, VT.TABLE_ID) AS TABLE_ID, NVL(T.TABLET_ID, VT.TABLET_ID) AS TABLET_ID, NVL(T.INSERTS, 0) + NVL(VT.INSERT_ROW_COUNT, 0) - NVL(T.LAST_INSERTS, 0) AS INSERTS, NVL(T.UPDATES, 0) + NVL(VT.UPDATE_ROW_COUNT, 0) - NVL(T.LAST_UPDATES, 0) AS UPDATES, NVL(T.DELETES, 0) + NVL(VT.DELETE_ROW_COUNT, 0) - NVL(T.LAST_DELETES, 0) AS DELETES FROM OCEANBASE.__ALL_MONITOR_MODIFIED T FULL JOIN OCEANBASE.__ALL_VIRTUAL_DML_STATS VT ON T.TABLE_ID = VT.TABLE_ID AND T.TABLET_ID = VT.TABLET_ID AND VT.TENANT_ID = EFFECTIVE_TENANT_ID() ) SELECT CAST(TM.DATABASE_NAME AS CHAR(128)) AS DATABASE_NAME, CAST(TM.TABLE_NAME AS CHAR(128)) AS TABLE_NAME, CAST(TM.PART_NAME AS CHAR(128)) AS PARTITION_NAME, CAST(TM.SUB_PART_NAME AS CHAR(128)) AS SUBPARTITION_NAME, CAST(TS.ROW_CNT AS SIGNED) AS LAST_ANALYZED_ROWS, TS.LAST_ANALYZED AS LAST_ANALYZED_TIME, CAST(TM.INSERTS AS SIGNED) AS INSERTS, CAST(TM.UPDATES AS SIGNED) AS UPDATES, CAST(TM.DELETES AS SIGNED) AS DELETES, CAST(NVL(CAST(UP.VALCHAR AS SIGNED), CAST(GP.SPARE4 AS SIGNED)) AS SIGNED) STALE_PERCENT, CAST(CASE NVL((TM.INSERTS + TM.UPDATES + TM.DELETES) > TS.ROW_CNT * NVL(CAST(UP.VALCHAR AS SIGNED), CAST(GP.SPARE4 AS SIGNED)) / 100, (TM.INSERTS + TM.UPDATES + TM.DELETES) > 0) WHEN 0 THEN 'NO' WHEN 1 THEN 'YES' END AS CHAR(3)) AS IS_STALE FROM (SELECT T.TENANT_ID, T.TABLE_ID, CASE T.PART_LEVEL WHEN 0 THEN T.TABLE_ID WHEN 1 THEN P.PART_ID WHEN 2 THEN SP.SUB_PART_ID END AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, SP.SUB_PART_NAME, NVL(V.INSERTS, 0) AS INSERTS, NVL(V.UPDATES, 0) AS UPDATES, NVL(V.DELETES, 0) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID LEFT JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID LEFT JOIN OCEANBASE.__ALL_SUB_PART SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = CASE T.PART_LEVEL WHEN 0 THEN T.TABLET_ID WHEN 1 THEN P.TABLET_ID WHEN 2 THEN SP.TABLET_ID END WHERE T.TABLE_TYPE IN (0, 3, 6) UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = P.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 6) AND T.PART_LEVEL = 1 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), MIN(P.PART_ID) AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID JOIN OCEANBASE.__ALL_SUB_PART SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 6) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM OCEANBASE.__ALL_TABLE T JOIN OCEANBASE.__ALL_DATABASE DB ON T.TENANT_ID = DB.TENANT_ID AND DB.DATABASE_ID = T.DATABASE_ID JOIN OCEANBASE.__ALL_PART P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID JOIN OCEANBASE.__ALL_SUB_PART SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TENANT_ID = V.TENANT_ID AND T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 6) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME ) TM LEFT JOIN OCEANBASE.__ALL_TABLE_STAT TS ON TM.TENANT_ID = TS.TENANT_ID AND TM.TABLE_ID = TS.TABLE_ID AND TM.PARTITION_ID = TS.PARTITION_ID LEFT JOIN OCEANBASE.__ALL_OPTSTAT_USER_PREFS UP ON TM.TENANT_ID = UP.TENANT_ID AND TM.TABLE_ID = UP.TABLE_ID AND UP.PNAME = 'STALE_PERCENT' JOIN OCEANBASE.__ALL_OPTSTAT_GLOBAL_PREFS GP ON GP.SNAME = 'STALE_PERCENT' )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -260,7 +460,7 @@ int ObInnerTableSchema::cdb_ob_external_table_files_schema(ObTableSchema &table_ 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 A.TENANT_ID AS TENANT_ID, B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS TABLE_SCHEMA, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM OCEANBASE.__ALL_VIRTUAL_EXTERNAL_TABLE_FILE A INNER JOIN OCEANBASE.__ALL_VIRTUAL_TABLE B ON A.TABLE_ID = B.TABLE_ID AND A.TENANT_ID=B.TENANT_ID INNER JOIN OCEANBASE.__ALL_VIRTUAL_DATABASE C ON B.DATABASE_ID = C.DATABASE_ID AND B.TENANT_ID=C.TENANT_ID WHERE B.TABLE_TYPE = 14 AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT A.TENANT_ID AS TENANT_ID, B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS TABLE_SCHEMA, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM OCEANBASE.__ALL_VIRTUAL_EXTERNAL_TABLE_FILE A INNER JOIN OCEANBASE.__ALL_VIRTUAL_TABLE B ON A.TABLE_ID = B.TABLE_ID AND A.TENANT_ID=B.TENANT_ID INNER JOIN OCEANBASE.__ALL_VIRTUAL_DATABASE C ON B.DATABASE_ID = C.DATABASE_ID AND B.TENANT_ID=C.TENANT_ID WHERE B.TABLE_TYPE = 14 AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.25201_25250.cpp b/src/share/inner_table/ob_inner_table_schema.25201_25250.cpp index 0d80283af..1a8def1cc 100644 --- a/src/share/inner_table/ob_inner_table_schema.25201_25250.cpp +++ b/src/share/inner_table/ob_inner_table_schema.25201_25250.cpp @@ -1160,7 +1160,7 @@ int ObInnerTableSchema::all_db_links_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 B.USER_NAME AS OWNER, A.DBLINK_NAME AS DB_LINK, A.USER_NAME AS USERNAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_NAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_OWNER, CAST(CASE DRIVER_PROTO WHEN 1 THEN A.CONN_STRING ELSE (A.HOST_IP || ':' || TO_CHAR(A.HOST_PORT)) END AS VARCHAR2(2000))AS HOST, CAST(A.GMT_CREATE AS DATE) AS CREATED, CAST('' AS VARCHAR2(3)) AS HIDDEN, CAST('' AS VARCHAR2(3)) AS SHARD_INTERNAL, CAST('YES' AS VARCHAR2(3)) AS VALID, CAST('' AS VARCHAR2(3)) AS INTRA_CDB, A.TENANT_NAME AS TENANT_NAME, A.REVERSE_TENANT_NAME AS REVERSE_TENANT_NAME, A.CLUSTER_NAME AS CLUSTER_NAME, A.REVERSE_CLUSTER_NAME AS REVERSE_CLUSTER_NAME, A.REVERSE_HOST_IP AS REVERSE_HOST, A.REVERSE_HOST_PORT AS REVERSE_PORT, A.REVERSE_USER_NAME AS REVERSE_USERNAME FROM SYS.ALL_VIRTUAL_DBLINK_REAL_AGENT A, SYS.ALL_VIRTUAL_USER_REAL_AGENT B, SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB WHERE A.TENANT_ID = EFFECTIVE_TENANT_ID() AND A.OWNER_ID = B.USER_ID AND B.USER_NAME = DB.DATABASE_NAME AND (DB.DATABASE_ID = USERENV('SCHEMAID') OR USER_CAN_ACCESS_OBJ(1, A.DBLINK_ID, DB.DATABASE_ID) = 1) )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT B.USER_NAME AS OWNER, A.DBLINK_NAME AS DB_LINK, A.USER_NAME AS USERNAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_NAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_OWNER, CAST(CASE DRIVER_PROTO WHEN 1 THEN A.CONN_STRING ELSE (A.HOST_IP || ':' || TO_CHAR(A.HOST_PORT)) END AS VARCHAR2(2000))AS HOST, CAST(A.GMT_CREATE AS DATE) AS CREATED, CAST('' AS VARCHAR2(3)) AS HIDDEN, CAST('' AS VARCHAR2(3)) AS SHARD_INTERNAL, CAST('YES' AS VARCHAR2(3)) AS VALID, CAST('' AS VARCHAR2(3)) AS INTRA_CDB, A.TENANT_NAME AS TENANT_NAME, A.REVERSE_TENANT_NAME AS REVERSE_TENANT_NAME, A.CLUSTER_NAME AS CLUSTER_NAME, A.REVERSE_CLUSTER_NAME AS REVERSE_CLUSTER_NAME, A.REVERSE_HOST_IP AS REVERSE_HOST, A.REVERSE_HOST_PORT AS REVERSE_PORT, A.REVERSE_USER_NAME AS REVERSE_USERNAME FROM SYS.ALL_VIRTUAL_DBLINK_REAL_AGENT A, SYS.ALL_VIRTUAL_USER_REAL_AGENT B, SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB WHERE A.TENANT_ID = EFFECTIVE_TENANT_ID() AND A.OWNER_ID = B.USER_ID AND B.USER_NAME = DB.DATABASE_NAME AND (DB.DATABASE_ID = USERENV('SCHEMAID') OR USER_CAN_ACCESS_OBJ(1, A.DBLINK_ID, DB.DATABASE_ID) = 1) )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -1210,7 +1210,7 @@ int ObInnerTableSchema::dba_db_links_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 B.USER_NAME AS OWNER, A.DBLINK_NAME AS DB_LINK, A.USER_NAME AS USERNAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_NAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_OWNER, CAST(CASE DRIVER_PROTO WHEN 1 THEN A.CONN_STRING ELSE (A.HOST_IP || ':' || TO_CHAR(A.HOST_PORT)) END AS VARCHAR2(2000))AS HOST, CAST(A.GMT_CREATE AS DATE) AS CREATED, CAST('' AS VARCHAR2(3)) AS HIDDEN, CAST('' AS VARCHAR2(3)) AS SHARD_INTERNAL, CAST('YES' AS VARCHAR2(3)) AS VALID, CAST('' AS VARCHAR2(3)) AS INTRA_CDB, A.TENANT_NAME AS TENANT_NAME, A.REVERSE_TENANT_NAME AS REVERSE_TENANT_NAME, A.CLUSTER_NAME AS CLUSTER_NAME, A.REVERSE_CLUSTER_NAME AS REVERSE_CLUSTER_NAME, A.REVERSE_HOST_IP AS REVERSE_HOST, A.REVERSE_HOST_PORT AS REVERSE_PORT, A.REVERSE_USER_NAME AS REVERSE_USERNAME FROM SYS.ALL_VIRTUAL_DBLINK_REAL_AGENT A, SYS.ALL_VIRTUAL_USER_REAL_AGENT B WHERE A.TENANT_ID = EFFECTIVE_TENANT_ID() AND A.OWNER_ID = B.USER_ID; )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT B.USER_NAME AS OWNER, A.DBLINK_NAME AS DB_LINK, A.USER_NAME AS USERNAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_NAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_OWNER, CAST(CASE DRIVER_PROTO WHEN 1 THEN A.CONN_STRING ELSE (A.HOST_IP || ':' || TO_CHAR(A.HOST_PORT)) END AS VARCHAR2(2000))AS HOST, CAST(A.GMT_CREATE AS DATE) AS CREATED, CAST('' AS VARCHAR2(3)) AS HIDDEN, CAST('' AS VARCHAR2(3)) AS SHARD_INTERNAL, CAST('YES' AS VARCHAR2(3)) AS VALID, CAST('' AS VARCHAR2(3)) AS INTRA_CDB, A.TENANT_NAME AS TENANT_NAME, A.REVERSE_TENANT_NAME AS REVERSE_TENANT_NAME, A.CLUSTER_NAME AS CLUSTER_NAME, A.REVERSE_CLUSTER_NAME AS REVERSE_CLUSTER_NAME, A.REVERSE_HOST_IP AS REVERSE_HOST, A.REVERSE_HOST_PORT AS REVERSE_PORT, A.REVERSE_USER_NAME AS REVERSE_USERNAME FROM SYS.ALL_VIRTUAL_DBLINK_REAL_AGENT A, SYS.ALL_VIRTUAL_USER_REAL_AGENT B WHERE A.TENANT_ID = EFFECTIVE_TENANT_ID() AND A.OWNER_ID = B.USER_ID; )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -1260,7 +1260,7 @@ int ObInnerTableSchema::user_db_links_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 A.DBLINK_NAME AS DB_LINK, A.USER_NAME AS USERNAME, CAST('' AS VARCHAR2(30)) AS PASSWORD, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_NAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_OWNER, CAST(CASE DRIVER_PROTO WHEN 1 THEN A.CONN_STRING ELSE (A.HOST_IP || ':' || TO_CHAR(A.HOST_PORT)) END AS VARCHAR2(2000))AS HOST, CAST(A.GMT_CREATE AS DATE) AS CREATED, CAST('' AS VARCHAR2(3)) AS HIDDEN, CAST('' AS VARCHAR2(3)) AS SHARD_INTERNAL, CAST('YES' AS VARCHAR2(3)) AS VALID, CAST('' AS VARCHAR2(3)) AS INTRA_CDB, A.TENANT_NAME AS TENANT_NAME, A.REVERSE_TENANT_NAME AS REVERSE_TENANT_NAME, A.CLUSTER_NAME AS CLUSTER_NAME, A.REVERSE_CLUSTER_NAME AS REVERSE_CLUSTER_NAME, A.REVERSE_HOST_IP AS REVERSE_HOST, A.REVERSE_HOST_PORT AS REVERSE_PORT, A.REVERSE_USER_NAME AS REVERSE_USERNAME FROM SYS.ALL_VIRTUAL_DBLINK_REAL_AGENT A, SYS.ALL_VIRTUAL_USER_REAL_AGENT B WHERE A.TENANT_ID = EFFECTIVE_TENANT_ID() AND A.OWNER_ID = B.USER_ID AND B.USER_NAME = SYS_CONTEXT('USERENV','CURRENT_USER'); )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT A.DBLINK_NAME AS DB_LINK, A.USER_NAME AS USERNAME, CAST('' AS VARCHAR2(30)) AS PASSWORD, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_NAME, CAST('' AS VARCHAR2(128)) AS CREDENTIAL_OWNER, CAST(CASE DRIVER_PROTO WHEN 1 THEN A.CONN_STRING ELSE (A.HOST_IP || ':' || TO_CHAR(A.HOST_PORT)) END AS VARCHAR2(2000))AS HOST, CAST(A.GMT_CREATE AS DATE) AS CREATED, CAST('' AS VARCHAR2(3)) AS HIDDEN, CAST('' AS VARCHAR2(3)) AS SHARD_INTERNAL, CAST('YES' AS VARCHAR2(3)) AS VALID, CAST('' AS VARCHAR2(3)) AS INTRA_CDB, A.TENANT_NAME AS TENANT_NAME, A.REVERSE_TENANT_NAME AS REVERSE_TENANT_NAME, A.CLUSTER_NAME AS CLUSTER_NAME, A.REVERSE_CLUSTER_NAME AS REVERSE_CLUSTER_NAME, A.REVERSE_HOST_IP AS REVERSE_HOST, A.REVERSE_HOST_PORT AS REVERSE_PORT, A.REVERSE_USER_NAME AS REVERSE_USERNAME FROM SYS.ALL_VIRTUAL_DBLINK_REAL_AGENT A, SYS.ALL_VIRTUAL_USER_REAL_AGENT B WHERE A.TENANT_ID = EFFECTIVE_TENANT_ID() AND A.OWNER_ID = B.USER_ID AND B.USER_NAME = SYS_CONTEXT('USERENV','CURRENT_USER'); )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -1410,7 +1410,7 @@ int ObInnerTableSchema::dba_ob_external_table_files_ora_schema(ObTableSchema &ta 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 B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS OWNER, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM SYS.ALL_VIRTUAL_EXTERNAL_TABLE_FILE_REAL_AGENT A INNER JOIN SYS.ALL_VIRTUAL_TABLE_REAL_AGENT B ON A.TABLE_ID = B.TABLE_ID INNER JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT C ON B.DATABASE_ID = C.DATABASE_ID AND B.TENANT_ID = C.TENANT_ID WHERE B.TENANT_ID = EFFECTIVE_TENANT_ID() AND B.TABLE_TYPE = 14 AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS OWNER, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM SYS.ALL_VIRTUAL_EXTERNAL_TABLE_FILE_REAL_AGENT A INNER JOIN SYS.ALL_VIRTUAL_TABLE_REAL_AGENT B ON A.TABLE_ID = B.TABLE_ID INNER JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT C ON B.DATABASE_ID = C.DATABASE_ID AND B.TENANT_ID = C.TENANT_ID WHERE B.TENANT_ID = EFFECTIVE_TENANT_ID() AND B.TABLE_TYPE = 14 AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -1460,7 +1460,207 @@ int ObInnerTableSchema::all_ob_external_table_files_ora_schema(ObTableSchema &ta 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 B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS OWNER, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM SYS.ALL_VIRTUAL_EXTERNAL_TABLE_FILE_REAL_AGENT A INNER JOIN SYS.ALL_VIRTUAL_TABLE_REAL_AGENT B ON A.TABLE_ID = B.TABLE_ID INNER JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT C ON B.DATABASE_ID = C.DATABASE_ID AND B.TENANT_ID = C.TENANT_ID WHERE B.TENANT_ID = EFFECTIVE_TENANT_ID() AND B.TABLE_TYPE = 14 AND (C.DATABASE_ID = USERENV('SCHEMAID') OR USER_CAN_ACCESS_OBJ(1, B.TABLE_ID, C.DATABASE_ID) = 1) AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT B.TABLE_NAME AS TABLE_NAME, C.DATABASE_NAME AS OWNER, 'P0' AS PARTITION_NAME, A.FILE_URL AS FILE_URL, A.FILE_SIZE AS FILE_SIZE FROM SYS.ALL_VIRTUAL_EXTERNAL_TABLE_FILE_REAL_AGENT A INNER JOIN SYS.ALL_VIRTUAL_TABLE_REAL_AGENT B ON A.TABLE_ID = B.TABLE_ID INNER JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT C ON B.DATABASE_ID = C.DATABASE_ID AND B.TENANT_ID = C.TENANT_ID WHERE B.TENANT_ID = EFFECTIVE_TENANT_ID() AND B.TABLE_TYPE = 14 AND (C.DATABASE_ID = USERENV('SCHEMAID') OR USER_CAN_ACCESS_OBJ(1, B.TABLE_ID, C.DATABASE_ID) = 1) AND (A.DELETE_VERSION = 9223372036854775807 OR A.DELETE_VERSION < A.CREATE_VERSION) )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::gv_ob_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_GV_OB_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(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_GV_OB_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(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( 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 )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::v_ob_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_V_OB_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(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_V_OB_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(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT * FROM SYS.GV$OB_PX_P2P_DATAHUB WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::gv_sql_join_filter_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_GV_SQL_JOIN_FILTER_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_GV_SQL_JOIN_FILTER_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT SVR_IP, 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' )__"))) { + LOG_ERROR("fail to set view_definition", K(ret)); + } + } + table_schema.set_index_using_type(USING_BTREE); + table_schema.set_row_store_type(ENCODING_ROW_STORE); + table_schema.set_store_format(OB_STORE_FORMAT_DYNAMIC_MYSQL); + table_schema.set_progressive_merge_round(1); + table_schema.set_storage_format_version(3); + table_schema.set_tablet_id(0); + + table_schema.set_max_used_column_id(column_id); + return ret; +} + +int ObInnerTableSchema::v_sql_join_filter_ora_schema(ObTableSchema &table_schema) +{ + int ret = OB_SUCCESS; + uint64_t column_id = OB_APP_MIN_COLUMN_ID - 1; + + //generated fields: + table_schema.set_tenant_id(OB_SYS_TENANT_ID); + table_schema.set_tablegroup_id(OB_INVALID_ID); + table_schema.set_database_id(OB_ORA_SYS_DATABASE_ID); + table_schema.set_table_id(OB_V_SQL_JOIN_FILTER_ORA_TID); + table_schema.set_rowkey_split_pos(0); + table_schema.set_is_use_bloomfilter(false); + table_schema.set_progressive_merge_num(0); + table_schema.set_rowkey_column_num(0); + table_schema.set_load_type(TABLE_LOAD_TYPE_IN_DISK); + table_schema.set_table_type(SYSTEM_VIEW); + table_schema.set_index_type(INDEX_TYPE_IS_NOT); + table_schema.set_def_type(TABLE_DEF_TYPE_INTERNAL); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_table_name(OB_V_SQL_JOIN_FILTER_ORA_TNAME))) { + LOG_ERROR("fail to set table_name", K(ret)); + } + } + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_compress_func_name(OB_DEFAULT_COMPRESS_FUNC_NAME))) { + LOG_ERROR("fail to set compress_func_name", K(ret)); + } + } + table_schema.set_part_level(PARTITION_LEVEL_ZERO); + table_schema.set_charset_type(ObCharset::get_default_charset()); + table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); + + if (OB_SUCC(ret)) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT * FROM SYS.GV$SQL_JOIN_FILTER WHERE SVR_IP=HOST_IP() AND SVR_PORT=RPC_PORT() )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } @@ -1510,7 +1710,7 @@ int ObInnerTableSchema::dba_ob_table_stat_stale_info_ora_schema(ObTableSchema &t 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"__( WITH V AS (SELECT NVL(T.TABLE_ID, VT.TABLE_ID) AS TABLE_ID, NVL(T.TABLET_ID, VT.TABLET_ID) AS TABLET_ID, NVL(T.INSERTS, 0) + NVL(VT.INSERT_ROW_COUNT, 0) - NVL(T.LAST_INSERTS, 0) AS INSERTS, NVL(T.UPDATES, 0) + NVL(VT.UPDATE_ROW_COUNT, 0) - NVL(T.LAST_UPDATES, 0) AS UPDATES, NVL(T.DELETES, 0) + NVL(VT.DELETE_ROW_COUNT, 0) - NVL(T.LAST_DELETES, 0) AS DELETES FROM SYS.ALL_VIRTUAL_MONITOR_MODIFIED_REAL_AGENT T FULL JOIN SYS.ALL_VIRTUAL_DML_STATS VT ON T.TABLE_ID = VT.TABLE_ID AND T.TABLET_ID = VT.TABLET_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND VT.TENANT_ID = EFFECTIVE_TENANT_ID() ) SELECT CAST(TM.DATABASE_NAME AS VARCHAR2(128)) AS OWNER, CAST(TM.TABLE_NAME AS VARCHAR2(128)) AS TABLE_NAME, CAST(TM.PART_NAME AS VARCHAR2(128)) AS PARTITION_NAME, CAST(TM.SUB_PART_NAME AS VARCHAR2(128)) AS SUBPARTITION_NAME, CAST(TS.ROW_CNT AS NUMBER) AS LAST_ANALYZED_ROWS, TS.LAST_ANALYZED AS LAST_ANALYZED_TIME, CAST(TM.INSERTS AS NUMBER) AS INSERTS, CAST(TM.UPDATES AS NUMBER) AS UPDATES, CAST(TM.DELETES AS NUMBER) AS DELETES, CAST(NVL(CAST(UP.VALCHAR AS NUMBER), CAST(GP.SPARE4 AS NUMBER)) AS NUMBER) STALE_PERCENT, CAST(CASE WHEN TS.ROW_CNT IS NOT NULL THEN CASE WHEN (TM.INSERTS + TM.UPDATES + TM.DELETES) > TS.ROW_CNT * NVL(CAST(UP.VALCHAR AS NUMBER), CAST(GP.SPARE4 AS NUMBER)) / 100 THEN 'YES' ELSE 'NO' END ELSE CASE WHEN (TM.INSERTS + TM.UPDATES + TM.DELETES) > 0 THEN 'YES' ELSE 'NO' END END AS VARCHAR2(3)) AS IS_STALE FROM (SELECT T.TENANT_ID, T.TABLE_ID, CASE T.PART_LEVEL WHEN 0 THEN T.TABLE_ID WHEN 1 THEN P.PART_ID WHEN 2 THEN SP.SUB_PART_ID END AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, SP.SUB_PART_NAME, NVL(V.INSERTS, 0) AS INSERTS, NVL(V.UPDATES, 0) AS UPDATES, NVL(V.DELETES, 0) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TABLE_ID = P.TABLE_ID AND P.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN SYS.ALL_VIRTUAL_SUB_PART_REAL_AGENT SP ON T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID AND SP.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = CASE T.PART_LEVEL WHEN 0 THEN T.TABLET_ID WHEN 1 THEN P.TABLET_ID WHEN 2 THEN SP.TABLET_ID END WHERE T.TABLE_TYPE IN (0, 3, 8, 9) UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TABLE_ID = P.TABLE_ID AND P.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = P.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 8, 9) AND T.PART_LEVEL = 1 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), MIN(P.PART_ID) AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID JOIN SYS.ALL_VIRTUAL_SUB_PART_REAL_AGENT SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 8, 9) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TABLE_ID = P.TABLE_ID AND P.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_SUB_PART_REAL_AGENT SP ON T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID AND SP.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 8, 9) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME ) TM LEFT JOIN SYS.ALL_VIRTUAL_TABLE_STAT_REAL_AGENT TS ON TM.TABLE_ID = TS.TABLE_ID AND TM.PARTITION_ID = TS.PARTITION_ID AND TM.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN SYS.ALL_VIRTUAL_OPTSTAT_USER_PREFS_REAL_AGENT UP ON TM.TABLE_ID = UP.TABLE_ID AND UP.PNAME = 'STALE_PERCENT' AND UP.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_OPTSTAT_GLOBAL_PREFS_REAL_AGENT GP ON GP.SNAME = 'STALE_PERCENT' )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( WITH V AS (SELECT NVL(T.TABLE_ID, VT.TABLE_ID) AS TABLE_ID, NVL(T.TABLET_ID, VT.TABLET_ID) AS TABLET_ID, NVL(T.INSERTS, 0) + NVL(VT.INSERT_ROW_COUNT, 0) - NVL(T.LAST_INSERTS, 0) AS INSERTS, NVL(T.UPDATES, 0) + NVL(VT.UPDATE_ROW_COUNT, 0) - NVL(T.LAST_UPDATES, 0) AS UPDATES, NVL(T.DELETES, 0) + NVL(VT.DELETE_ROW_COUNT, 0) - NVL(T.LAST_DELETES, 0) AS DELETES FROM SYS.ALL_VIRTUAL_MONITOR_MODIFIED_REAL_AGENT T FULL JOIN SYS.ALL_VIRTUAL_DML_STATS VT ON T.TABLE_ID = VT.TABLE_ID AND T.TABLET_ID = VT.TABLET_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND VT.TENANT_ID = EFFECTIVE_TENANT_ID() ) SELECT CAST(TM.DATABASE_NAME AS VARCHAR2(128)) AS OWNER, CAST(TM.TABLE_NAME AS VARCHAR2(128)) AS TABLE_NAME, CAST(TM.PART_NAME AS VARCHAR2(128)) AS PARTITION_NAME, CAST(TM.SUB_PART_NAME AS VARCHAR2(128)) AS SUBPARTITION_NAME, CAST(TS.ROW_CNT AS NUMBER) AS LAST_ANALYZED_ROWS, TS.LAST_ANALYZED AS LAST_ANALYZED_TIME, CAST(TM.INSERTS AS NUMBER) AS INSERTS, CAST(TM.UPDATES AS NUMBER) AS UPDATES, CAST(TM.DELETES AS NUMBER) AS DELETES, CAST(NVL(CAST(UP.VALCHAR AS NUMBER), CAST(GP.SPARE4 AS NUMBER)) AS NUMBER) STALE_PERCENT, CAST(CASE WHEN TS.ROW_CNT IS NOT NULL THEN CASE WHEN (TM.INSERTS + TM.UPDATES + TM.DELETES) > TS.ROW_CNT * NVL(CAST(UP.VALCHAR AS NUMBER), CAST(GP.SPARE4 AS NUMBER)) / 100 THEN 'YES' ELSE 'NO' END ELSE CASE WHEN (TM.INSERTS + TM.UPDATES + TM.DELETES) > 0 THEN 'YES' ELSE 'NO' END END AS VARCHAR2(3)) AS IS_STALE FROM (SELECT T.TENANT_ID, T.TABLE_ID, CASE T.PART_LEVEL WHEN 0 THEN T.TABLE_ID WHEN 1 THEN P.PART_ID WHEN 2 THEN SP.SUB_PART_ID END AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, SP.SUB_PART_NAME, NVL(V.INSERTS, 0) AS INSERTS, NVL(V.UPDATES, 0) AS UPDATES, NVL(V.DELETES, 0) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TABLE_ID = P.TABLE_ID AND P.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN SYS.ALL_VIRTUAL_SUB_PART_REAL_AGENT SP ON T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID AND SP.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = CASE T.PART_LEVEL WHEN 0 THEN T.TABLET_ID WHEN 1 THEN P.TABLET_ID WHEN 2 THEN SP.TABLET_ID END WHERE T.TABLE_TYPE IN (0, 3, 8, 9) UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TABLE_ID = P.TABLE_ID AND P.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = P.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 8, 9) AND T.PART_LEVEL = 1 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), MIN(P.PART_ID) AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TENANT_ID = P.TENANT_ID AND T.TABLE_ID = P.TABLE_ID JOIN SYS.ALL_VIRTUAL_SUB_PART_REAL_AGENT SP ON T.TENANT_ID = SP.TENANT_ID AND T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 8, 9) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME, P.PART_NAME UNION ALL SELECT MIN(T.TENANT_ID), MIN(T.TABLE_ID), -1 AS PARTITION_ID, DB.DATABASE_NAME, T.TABLE_NAME, NULL AS PART_NAME, NULL AS SUB_PART_NAME, SUM(NVL(V.INSERTS, 0)) AS INSERTS, SUM(NVL(V.UPDATES, 0)) AS UPDATES, SUM(NVL(V.DELETES, 0)) AS DELETES FROM SYS.ALL_VIRTUAL_TABLE_REAL_AGENT T JOIN SYS.ALL_VIRTUAL_DATABASE_REAL_AGENT DB ON DB.DATABASE_ID = T.DATABASE_ID AND T.TENANT_ID = EFFECTIVE_TENANT_ID() AND DB.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_PART_REAL_AGENT P ON T.TABLE_ID = P.TABLE_ID AND P.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_SUB_PART_REAL_AGENT SP ON T.TABLE_ID = SP.TABLE_ID AND P.PART_ID = SP.PART_ID AND SP.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN V ON T.TABLE_ID = V.TABLE_ID AND V.TABLET_ID = SP.TABLET_ID WHERE T.TABLE_TYPE IN (0, 3, 8, 9) AND T.PART_LEVEL = 2 GROUP BY DB.DATABASE_NAME, T.TABLE_NAME ) TM LEFT JOIN SYS.ALL_VIRTUAL_TABLE_STAT_REAL_AGENT TS ON TM.TABLE_ID = TS.TABLE_ID AND TM.PARTITION_ID = TS.PARTITION_ID AND TM.TENANT_ID = EFFECTIVE_TENANT_ID() LEFT JOIN SYS.ALL_VIRTUAL_OPTSTAT_USER_PREFS_REAL_AGENT UP ON TM.TABLE_ID = UP.TABLE_ID AND UP.PNAME = 'STALE_PERCENT' AND UP.TENANT_ID = EFFECTIVE_TENANT_ID() JOIN SYS.ALL_VIRTUAL_OPTSTAT_GLOBAL_PREFS_REAL_AGENT GP ON GP.SNAME = 'STALE_PERCENT' )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp b/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp index 77a5b937a..cd8fdf728 100644 --- a/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp +++ b/src/share/inner_table/ob_inner_table_schema.28151_28200.cpp @@ -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)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.h b/src/share/inner_table/ob_inner_table_schema.h index 55e3122ff..5ea36cc46 100644 --- a/src/share/inner_table/ob_inner_table_schema.h +++ b/src/share/inner_table/ob_inner_table_schema.h @@ -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 diff --git a/src/share/inner_table/ob_inner_table_schema_constants.h b/src/share/inner_table/ob_inner_table_schema_constants.h index 417cb0909..862f1ab30 100644 --- a/src/share/inner_table/ob_inner_table_schema_constants.h +++ b/src/share/inner_table/ob_inner_table_schema_constants.h @@ -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"; diff --git a/src/share/inner_table/ob_inner_table_schema_def.py b/src/share/inner_table/ob_inner_table_schema_def.py index af6becfca..f2784e2c9 100644 --- a/src/share/inner_table/ob_inner_table_schema_def.py +++ b/src/share/inner_table/ob_inner_table_schema_def.py @@ -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', diff --git a/src/share/ob_thread_define.h b/src/share/ob_thread_define.h index 9f695108b..3b3c124e2 100644 --- a/src/share/ob_thread_define.h +++ b/src/share/ob_thread_define.h @@ -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 diff --git a/src/share/parameter/ob_parameter_seed.ipp b/src/share/parameter/ob_parameter_seed.ipp index 4eb8a3dbc..2657d24a9 100644 --- a/src/share/parameter/ob_parameter_seed.ipp +++ b/src/share/parameter/ob_parameter_seed.ipp @@ -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)); diff --git a/src/share/rc/ob_tenant_base.h b/src/share/rc/ob_tenant_base.h index 6b02f4185..9209205c7 100644 --- a/src/share/rc/ob_tenant_base.h +++ b/src/share/rc/ob_tenant_base.h @@ -29,6 +29,7 @@ namespace common { class ObLDHandle; class ObTenantIOManager; template class ObServerObjectPool; + class ObDetectManager; } namespace omt { class ObPxPools; @@ -247,7 +248,8 @@ using ObPartTransCtxObjPool = common::ObServerObjectPool= 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, diff --git a/src/share/system_variable/ob_system_variable.h b/src/share/system_variable/ob_system_variable.h index fb7746fde..f19d61569 100644 --- a/src/share/system_variable/ob_system_variable.h +++ b/src/share/system_variable/ob_system_variable.h @@ -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, diff --git a/src/share/system_variable/ob_system_variable_alias.h b/src/share/system_variable/ob_system_variable_alias.h index 81eccf2f2..cad679dd0 100644 --- a/src/share/system_variable/ob_system_variable_alias.h +++ b/src/share/system_variable/ob_system_variable_alias.h @@ -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"; } } diff --git a/src/share/system_variable/ob_system_variable_factory.cpp b/src/share/system_variable/ob_system_variable_factory.cpp index 831ccb968..d17be4e19 100644 --- a/src/share/system_variable/ob_system_variable_factory.cpp +++ b/src/share/system_variable/ob_system_variable_factory.cpp @@ -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(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(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(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(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; diff --git a/src/share/system_variable/ob_system_variable_factory.h b/src/share/system_variable/ob_system_variable_factory.h index 4bf3123c6..119544a57 100644 --- a/src/share/system_variable/ob_system_variable_factory.h +++ b/src/share/system_variable/ob_system_variable_factory.h @@ -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; diff --git a/src/share/system_variable/ob_system_variable_init.cpp b/src/share/system_variable/ob_system_variable_init.cpp index 905652261..b1040ef93 100644 --- a/src/share/system_variable/ob_system_variable_init.cpp +++ b/src/share/system_variable/ob_system_variable_init.cpp @@ -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(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(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(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(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_;} diff --git a/src/share/system_variable/ob_system_variable_init.json b/src/share/system_variable/ob_system_variable_init.json index c4bded336..7fd0cd6ab 100644 --- a/src/share/system_variable/ob_system_variable_init.json +++ b/src/share/system_variable/ob_system_variable_init.json @@ -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": "" } } diff --git a/src/sql/CMakeLists.txt b/src/sql/CMakeLists.txt index 5e0db58f2..e6bd939b7 100644 --- a/src/sql/CMakeLists.txt +++ b/src/sql/CMakeLists.txt @@ -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 diff --git a/src/sql/code_generator/ob_static_engine_cg.cpp b/src/sql/code_generator/ob_static_engine_cg.cpp index 78bd36a33..b4c8ee961 100644 --- a/src/sql/code_generator/ob_static_engine_cg.cpp +++ b/src/sql/code_generator/ob_static_engine_cg.cpp @@ -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( - 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 &p2p_sequence_ids = op.get_p2p_sequence_ids(); + const common::ObIArray &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(rf_types.at(i)); + if (!op.is_create_filter()) { + const common::ObIArray &join_filter_exprs = + op.get_join_filter_exprs(); + if (OB_ISNULL(join_filter_expr = + reinterpret_cast( + 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()))) { diff --git a/src/sql/dtl/ob_dtl_basic_channel.cpp b/src/sql/dtl/ob_dtl_basic_channel.cpp index 25d1fd2a8..c2a2e32cf 100644 --- a/src/sql/dtl/ob_dtl_basic_channel.cpp +++ b/src/sql/dtl/ob_dtl_basic_channel.cpp @@ -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_)); } } diff --git a/src/sql/dtl/ob_dtl_channel.cpp b/src/sql/dtl/ob_dtl_channel.cpp index 037cfe1f5..5b30353b8 100644 --- a/src/sql/dtl/ob_dtl_channel.cpp +++ b/src/sql/dtl/ob_dtl_channel.cpp @@ -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), diff --git a/src/sql/dtl/ob_dtl_channel.h b/src/sql/dtl/ob_dtl_channel.h index 4c1418c83..8b91db2ae 100644 --- a/src/sql/dtl/ob_dtl_channel.h +++ b/src/sql/dtl/ob_dtl_channel.h @@ -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 ®ister_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_; diff --git a/src/sql/dtl/ob_dtl_interm_result_manager.cpp b/src/sql/dtl/ob_dtl_interm_result_manager.cpp index 047d6361e..2afb66fb9 100644 --- a/src/sql/dtl/ob_dtl_interm_result_manager.cpp +++ b/src/sql/dtl/ob_dtl_interm_result_manager.cpp @@ -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_)); } } } diff --git a/src/sql/dtl/ob_dtl_interm_result_manager.h b/src/sql/dtl/ob_dtl_interm_result_manager.h index b2b292f9f..cd3d7f890 100644 --- a/src/sql/dtl/ob_dtl_interm_result_manager.h +++ b/src/sql/dtl/ob_dtl_interm_result_manager.h @@ -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); diff --git a/src/sql/dtl/ob_dtl_linked_buffer.cpp b/src/sql/dtl/ob_dtl_linked_buffer.cpp index dfcd247e5..50f440cb8 100644 --- a/src/sql/dtl/ob_dtl_linked_buffer.cpp +++ b/src/sql/dtl/ob_dtl_linked_buffer.cpp @@ -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; } diff --git a/src/sql/dtl/ob_dtl_linked_buffer.h b/src/sql/dtl/ob_dtl_linked_buffer.h index 27f4448f0..3a5b269a2 100644 --- a/src/sql/dtl/ob_dtl_linked_buffer.h +++ b/src/sql/dtl/ob_dtl_linked_buffer.h @@ -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 ®ister_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 diff --git a/src/sql/dtl/ob_dtl_rpc_channel.cpp b/src/sql/dtl/ob_dtl_rpc_channel.cpp index ba0fe9fcd..33564d833 100644 --- a/src/sql/dtl/ob_dtl_rpc_channel.cpp +++ b/src/sql/dtl/ob_dtl_rpc_channel.cpp @@ -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(ObDtlMsgType::FINISH_SQC_RESULT) == header.type_) { + recv_sqc_fin_res_ = true; + } if (buffer->is_data_msg()) { metric_.mark_first_in(); if (buffer->is_eof()) { diff --git a/src/sql/dtl/ob_dtl_rpc_channel.h b/src/sql/dtl/ob_dtl_rpc_channel.h index 1b58bdd05..a39858892 100644 --- a/src/sql/dtl/ob_dtl_rpc_channel.h +++ b/src/sql/dtl/ob_dtl_rpc_channel.h @@ -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 diff --git a/src/sql/engine/basic/ob_temp_table_insert_op.cpp b/src/sql/engine/basic/ob_temp_table_insert_op.cpp index 8bd6a6490..5db837b62 100644 --- a/src/sql/engine/basic/ob_temp_table_insert_op.cpp +++ b/src/sql/engine/basic/ob_temp_table_insert_op.cpp @@ -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())); + } } } } diff --git a/src/sql/engine/basic/ob_temp_table_transformation_op.cpp b/src/sql/engine/basic/ob_temp_table_transformation_op.cpp index 77d20aab2..6a32963e4 100644 --- a/src/sql/engine/basic/ob_temp_table_transformation_op.cpp +++ b/src/sql/engine/basic/ob_temp_table_transformation_op.cpp @@ -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)); diff --git a/src/sql/engine/expr/ob_expr_eval_functions.cpp b/src/sql/engine/expr/ob_expr_eval_functions.cpp index 1678f307c..82159828a 100644 --- a/src/sql/engine/expr/ob_expr_eval_functions.cpp +++ b/src/sql/engine/expr/ob_expr_eval_functions.cpp @@ -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 */ diff --git a/src/sql/engine/expr/ob_expr_join_filter.cpp b/src/sql/engine/expr/ob_expr_join_filter.cpp index f5d8f0c03..6632eb8aa 100644 --- a/src/sql/engine/expr/ob_expr_join_filter.cpp +++ b/src/sql/engine/expr/ob_expr_join_filter.cpp @@ -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( - 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(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( - 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( - 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(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(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(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_) - : reinterpret_cast(rt_expr.args_[i]->basic_funcs_->murmur_hash_); - rt_expr.inner_functions_[i * 2 + 1] = is_murmur_hash_v2_ ? - reinterpret_cast(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_batch_) - : reinterpret_cast(rt_expr.args_[i]->basic_funcs_->murmur_hash_batch_); + rt_expr.inner_functions_[GET_FUNC(i, HASH_ROW)] = + reinterpret_cast(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_); + rt_expr.inner_functions_[GET_FUNC(i, HASH_BATCH)] = + reinterpret_cast(rt_expr.args_[i]->basic_funcs_->murmur_hash_v2_batch_); + rt_expr.inner_functions_[GET_FUNC(i, NULL_FIRST_COMPARE)] = + reinterpret_cast(rt_expr.args_[i]->basic_funcs_->null_first_cmp_); + rt_expr.inner_functions_[GET_FUNC(i, NULL_LAST_COMPARE)] = + reinterpret_cast(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( + 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( + 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; } diff --git a/src/sql/engine/expr/ob_expr_join_filter.h b/src/sql/engine/expr/ob_expr_join_filter.h index 99fd36936..ca015ef77 100644 --- a/src/sql/engine/expr/ob_expr_join_filter.h +++ b/src/sql/engine/expr/ob_expr_join_filter.h @@ -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); }; diff --git a/src/sql/engine/join/ob_join_filter_op.cpp b/src/sql/engine/join/ob_join_filter_op.cpp index 0578d3665..765303636 100644 --- a/src/sql/engine/join/ob_join_filter_op.cpp +++ b/src/sql/engine/join/ob_join_filter_op.cpp @@ -24,8 +24,10 @@ #include "sql/dtl/ob_dtl.h" #include "sql/dtl/ob_dtl_channel_group.h" #include "sql/engine/px/ob_px_util.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" #include "observer/omt/ob_tenant_config_mgr.h" + using namespace oceanbase; using namespace common; using namespace omt; @@ -36,11 +38,44 @@ OB_SERIALIZE_MEMBER(ObJoinFilterShareInfo, unfinished_count_ptr_, ch_provider_ptr_, release_ref_ptr_, - filter_ptr_); + filter_ptr_, + shared_msgs_); + +OB_SERIALIZE_MEMBER(ObJoinFilterRuntimeConfig, + bloom_filter_ratio_, + each_group_size_, + bf_piece_size_, + runtime_filter_wait_time_ms_, + runtime_filter_max_in_num_, + runtime_bloom_filter_max_size_, + px_message_compression_); + +OB_SERIALIZE_MEMBER(ObRuntimeFilterInfo, + filter_expr_id_, + p2p_datahub_id_, + filter_shared_type_, + dh_msg_type_); + +OB_SERIALIZE_MEMBER((ObJoinFilterSpec, ObOpSpec), + mode_, + filter_id_, + filter_len_, + join_keys_, + hash_funcs_, + null_first_cmp_funcs_, + null_last_cmp_funcs_, + filter_shared_type_, + calc_tablet_id_expr_, + rf_infos_, + need_null_cmp_flags_, + is_shuffle_, + each_group_size_); + OB_SERIALIZE_MEMBER(ObJoinFilterOpInput, share_info_, - is_local_create_, - bf_idx_at_sqc_proxy_); + bf_idx_at_sqc_proxy_, + px_sequence_id_, + config_); int ObJoinFilterOpInput::init(ObTaskInfo &task_info) { @@ -52,26 +87,24 @@ int ObJoinFilterOpInput::init(ObTaskInfo &task_info) return ret; } -int ObJoinFilterOpInput::check_finish(bool &is_end, bool is_shared) +bool ObJoinFilterOpInput::is_finish() { - int ret = OB_SUCCESS; + bool ret = true; uint64_t *count_ptr = reinterpret_cast(share_info_.unfinished_count_ptr_); - if (OB_ISNULL(count_ptr) || !is_shared) { - is_end = true; - } else if (0 == ATOMIC_AAF(count_ptr, -1)) { - is_end = true; - } else { - is_end = false; + if (OB_NOT_NULL(count_ptr)) { + if (0 != *count_ptr) { + ret = false; + } } return ret; } -bool ObJoinFilterOpInput::check_release(bool is_shared) +bool ObJoinFilterOpInput::check_release() { bool res = false; uint64_t *release_ref_ptr = reinterpret_cast(share_info_.release_ref_ptr_); - if (OB_ISNULL(release_ref_ptr) || !is_shared) { - res = true; + if (OB_ISNULL(release_ref_ptr)) { + res = false; } else if (0 == ATOMIC_AAF(release_ref_ptr, -1)) { res = true; } else { @@ -80,79 +113,222 @@ bool ObJoinFilterOpInput::check_release(bool is_shared) return res; } -int ObJoinFilterOpInput::init_share_info(common::ObIAllocator &allocator, int64_t task_count) +int ObJoinFilterOpInput::load_runtime_config(const ObJoinFilterSpec &spec, ObExecContext &ctx) { int ret = OB_SUCCESS; - uint64_t *send_count_ptr = NULL; - uint64_t *close_count_ptr = NULL; - uint64_t *count_ptr = NULL; - if (OB_ISNULL(send_count_ptr = (uint64_t *)allocator.alloc(sizeof(uint64_t)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to alloc ptr", K(ret)); - } else if (OB_ISNULL(close_count_ptr = (uint64_t *)allocator.alloc(sizeof(uint64_t)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to alloc ptr", K(ret)); - } else if (OB_ISNULL(count_ptr = (uint64_t *)allocator.alloc(sizeof(uint64_t)))) { + config_.bloom_filter_ratio_ = ((double)GCONF._bloom_filter_ratio / 100.0); + config_.bf_piece_size_ = GCONF._send_bloom_filter_size; + config_.each_group_size_ = spec.each_group_size_; + config_.runtime_filter_wait_time_ms_ = ctx.get_my_session()-> + get_runtime_filter_wait_time_ms(); + config_.runtime_filter_max_in_num_ = ctx.get_my_session()-> + get_runtime_filter_max_in_num(); + config_.runtime_bloom_filter_max_size_ = ctx.get_my_session()-> + get_runtime_bloom_filter_max_size(); + config_.px_message_compression_ = true; + LOG_TRACE("load runtime filter conifg", K(config_)); + return ret; +} + +int ObJoinFilterOpInput::init_share_info( + const ObJoinFilterSpec &spec, + ObExecContext &ctx, + int64_t task_count, + int64_t sqc_count) +{ + int ret = OB_SUCCESS; + uint64_t *ptr = NULL; + common::ObIAllocator &allocator = ctx.get_allocator(); + if (OB_ISNULL(ptr = (uint64_t *)allocator.alloc(sizeof(uint64_t) * 2))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("fail to alloc ptr", K(ret)); + } else if (OB_FAIL(init_shared_msgs(spec, ctx, sqc_count))) { + LOG_WARN("fail to init shared msgs", K(ret)); } else { - *send_count_ptr = task_count; - *close_count_ptr = task_count; - *count_ptr = task_count; - share_info_.release_ref_ptr_ = reinterpret_cast(close_count_ptr); - share_info_.unfinished_count_ptr_ = reinterpret_cast(send_count_ptr); + ptr[0] = task_count; + ptr[1] = task_count; + share_info_.release_ref_ptr_ = reinterpret_cast(&ptr[0]); + share_info_.unfinished_count_ptr_ = reinterpret_cast(&ptr[1]); } return ret; } -OB_SERIALIZE_MEMBER((ObJoinFilterSpec, ObOpSpec), - mode_, - filter_id_, - server_id_, - filter_len_, - is_shuffle_, - join_keys_, - hash_funcs_, - filter_expr_id_, - filter_type_, - calc_tablet_id_expr_); +int ObJoinFilterOpInput::init_shared_msgs( + const ObJoinFilterSpec &spec, + ObExecContext &ctx, + int64_t sqc_count) +{ + int ret = OB_SUCCESS; + const int64_t tenant_id = ctx.get_my_session()->get_effective_tenant_id(); + const int64_t timeout_ts = GET_PHY_PLAN_CTX(ctx)->get_timeout_timestamp(); + common::ObIAllocator &allocator = ctx.get_allocator(); + ObArray *array_ptr = nullptr; + ObPxSQCProxy *sqc_proxy = reinterpret_cast(share_info_.ch_provider_ptr_); + void *ptr = nullptr; + if (OB_ISNULL(ptr = allocator.alloc(sizeof(ObArray)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to allocate memory", K(ret)); + } else { + array_ptr = new(ptr) ObArray(); + ObP2PDatahubMsgBase *msg_ptr = nullptr; + for (int i = 0; i < spec.rf_infos_.count() && OB_SUCC(ret); ++i) { + msg_ptr = nullptr; + if (OB_FAIL(PX_P2P_DH.alloc_msg(allocator, spec.rf_infos_.at(i).dh_msg_type_, msg_ptr))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else if (OB_FAIL(msg_ptr->init(spec.rf_infos_.at(i).p2p_datahub_id_, + px_sequence_id_, 0/*task_id*/, tenant_id, timeout_ts, register_dm_info_))) { + LOG_WARN("fail to init msg", K(ret)); + } else if (OB_FAIL(construct_msg_details(spec, sqc_proxy, config_, *msg_ptr, sqc_count))) { + LOG_WARN("fail to construct msg details", K(ret)); + } else if (OB_FAIL(array_ptr->push_back(msg_ptr))) { + LOG_WARN("fail to push back array ptr", K(ret)); + } + } + } + if (OB_SUCC(ret) && OB_NOT_NULL(array_ptr)) { + share_info_.shared_msgs_ = reinterpret_cast(array_ptr); + } + + if (OB_FAIL(ret) && OB_NOT_NULL(array_ptr)) { + array_ptr->reset(); + array_ptr = nullptr; + } + + return ret; +} + +int ObJoinFilterOpInput::construct_msg_details( + const ObJoinFilterSpec &spec, + ObPxSQCProxy *sqc_proxy, + ObJoinFilterRuntimeConfig &config, + ObP2PDatahubMsgBase &msg, + int64_t sqc_count) +{ + int ret = OB_SUCCESS; + switch(msg.get_msg_type()) { + case ObP2PDatahubMsgBase::BLOOM_FILTER_MSG: { + ObSArray *target_addrs = nullptr; + ObRFBloomFilterMsg &bf_msg = static_cast(msg); + ObPxSQCProxy::SQCP2PDhMap &dh_map = sqc_proxy->get_p2p_dh_map(); + if (OB_FAIL(bf_msg.bloom_filter_.init(spec.filter_len_, + bf_msg.get_allocator(), + config.bloom_filter_ratio_))) { + LOG_WARN("failed to init bloom filter", K(ret)); + } else if (!spec.is_shared_join_filter() || !spec.is_shuffle_) { + bf_msg.set_msg_expect_cnt(1); + bf_msg.set_msg_cur_cnt(1); + } else if (OB_FAIL(dh_map.get_refactored(bf_msg.get_p2p_datahub_id(), target_addrs))) { + LOG_WARN("fail to get dh map", K(ret)); + } else if (target_addrs->count() == 1 && + GCTX.self_addr() == target_addrs->at(0) && + sqc_count == 1) { + bf_msg.set_msg_expect_cnt(1); + bf_msg.set_msg_cur_cnt(1); + } else { + int64_t target_cnt = target_addrs->count(); + int64_t each_group_size = (OB_INVALID_ID == config.each_group_size_) ? + sqrt(target_cnt) : config.each_group_size_; + int64_t *filter_idx_ptr = nullptr; + bool *create_finish = nullptr; + if (OB_FAIL(bf_msg.generate_filter_indexes(each_group_size, target_cnt, config.bf_piece_size_))) { + LOG_WARN("fail to generate filter indexes", K(ret)); + } else if (OB_ISNULL(filter_idx_ptr = static_cast( + msg.get_allocator().alloc(sizeof(int64_t))))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate memory", K(ret)); + } else if (OB_ISNULL(create_finish = static_cast( + msg.get_allocator().alloc(sizeof(bool))))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate memory", K(ret)); + } else { + *filter_idx_ptr = 0; + *create_finish = false; + bf_msg.filter_idx_ = filter_idx_ptr; + bf_msg.create_finish_ = create_finish; + int64_t piece_cnt = ceil(bf_msg.bloom_filter_.get_bits_array_length() / + (double)config.bf_piece_size_); + bf_msg.set_msg_expect_cnt(sqc_count * piece_cnt); + bf_msg.set_msg_cur_cnt(1); + bf_msg.expect_first_phase_count_ = sqc_count; + bf_msg.piece_size_ = config.bf_piece_size_; + } + } + break; + } + case ObP2PDatahubMsgBase::RANGE_FILTER_MSG: { + ObRFRangeFilterMsg &range_msg = static_cast(msg); + int64_t col_cnt = spec.join_keys_.count(); + if (OB_FAIL(range_msg.lower_bounds_.prepare_allocate(col_cnt))) { + LOG_WARN("fail to prepare allocate col cnt", K(ret)); + } else if (OB_FAIL(range_msg.upper_bounds_.prepare_allocate(col_cnt))) { + LOG_WARN("fail to prepare allocate col cnt", K(ret)); + } else if (OB_FAIL(range_msg.cells_size_.prepare_allocate(col_cnt))) { + LOG_WARN("fail to prepare allocate col cnt", K(ret)); + } else if (OB_FAIL(range_msg.null_first_cmp_funcs_.assign(spec.null_first_cmp_funcs_))) { + LOG_WARN("fail to init cmp funcs", K(ret)); + } else if (OB_FAIL(range_msg.null_last_cmp_funcs_.assign(spec.null_last_cmp_funcs_))) { + LOG_WARN("fail to init cmp funcs", K(ret)); + } else if (OB_FAIL(range_msg.need_null_cmp_flags_.assign(spec.need_null_cmp_flags_))) { + LOG_WARN("fail to init cmp flags", K(ret)); + } else { + range_msg.set_msg_expect_cnt(sqc_count); + range_msg.set_msg_cur_cnt(1); + } + break; + } + case ObP2PDatahubMsgBase::IN_FILTER_MSG: { + ObRFInFilterMsg &in_msg = static_cast(msg); + int64_t col_cnt = spec.join_keys_.count(); + if (OB_FAIL(in_msg.rows_set_.create(config.runtime_filter_max_in_num_ * 2, + "RFInFilter", + "RFInFilter"))) { + LOG_WARN("fail to init in hash set", K(ret)); + } else if (OB_FAIL(in_msg.cur_row_.prepare_allocate(col_cnt))) { + LOG_WARN("fail to prepare allocate col cnt", K(ret)); + } else if (OB_FAIL(in_msg.cmp_funcs_.assign(spec.null_first_cmp_funcs_))) { + LOG_WARN("fail to init cmp funcs", K(ret)); + } else if (OB_FAIL(in_msg.hash_funcs_.assign(spec.hash_funcs_))) { + LOG_WARN("fail to init cmp funcs", K(ret)); + } else if (OB_FAIL(in_msg.need_null_cmp_flags_.assign(spec.need_null_cmp_flags_))) { + LOG_WARN("fail to init cmp flags", K(ret)); + } else { + in_msg.set_msg_expect_cnt(sqc_count); + in_msg.set_msg_cur_cnt(1); + in_msg.col_cnt_ = col_cnt; + in_msg.max_in_num_ = config.runtime_filter_max_in_num_; + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected msg type", K(ret)); + } + } + return ret; +} ObJoinFilterSpec::ObJoinFilterSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type) : ObOpSpec(alloc, type), mode_(NOT_INIT), filter_id_(OB_INVALID_ID), - server_id_(OB_INVALID_ID), filter_len_(0), - is_shuffle_(false), join_keys_(alloc), hash_funcs_(alloc), - filter_expr_id_(OB_INVALID_ID), - filter_type_(JoinFilterType::INVALID_TYPE), - calc_tablet_id_expr_(NULL) + null_first_cmp_funcs_(alloc), + null_last_cmp_funcs_(alloc), + filter_shared_type_(JoinFilterSharedType::INVALID_TYPE), + calc_tablet_id_expr_(NULL), + rf_infos_(alloc), + need_null_cmp_flags_(alloc), + is_shuffle_(false), + each_group_size_(OB_INVALID_ID) { - } //------------------------------------------ ObJoinFilterOp -------------------------------- int ObJoinFilterOp::destroy_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)) { - while (!filter->is_merge_filter_finish()) { - // 一个繁琐但安全的等待行为. - // 工作线程退出需要确保DTL已经没人持有filter指针在做写入. - ob_usleep(1000); - } - filter->reset(); - } - if (ret == OB_HASH_NOT_EXIST) { - //尽力而为 如果manager里不存在的话 说明create端提前结束没有发送filter. - ret = OB_SUCCESS; - } return ret; } @@ -186,16 +362,14 @@ int ObJoinFilterOp::link_ch_sets(ObPxBloomFilterChSets &ch_sets, bool ObJoinFilterOp::is_valid() { return child_ != NULL && MY_SPEC.mode_ != JoinFilterMode::NOT_INIT && - MY_SPEC.filter_type_ != JoinFilterType::INVALID_TYPE; + MY_SPEC.filter_shared_type_ != JoinFilterSharedType::INVALID_TYPE; } ObJoinFilterOp::ObJoinFilterOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input) : ObOperator(exec_ctx, spec, input), - bf_key_(), - filter_use_(NULL), - filter_create_(NULL), - bf_ch_sets_(NULL), - batch_hash_values_(NULL) + filter_create_msg_(nullptr), + batch_hash_values_(NULL), + lucky_devil_champions_() { } @@ -207,67 +381,13 @@ ObJoinFilterOp::~ObJoinFilterOp() int ObJoinFilterOp::inner_open() { int ret = OB_SUCCESS; - ObJoinFilterOpInput *op_input = static_cast(input_); if (OB_UNLIKELY(!is_valid())) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("builder operator is invalid", K(ret), K(MY_SPEC.filter_type_), K(MY_SPEC.mode_)); - } else if (MY_SPEC.is_create_mode()) { - int64_t filter_len = MY_SPEC.filter_len_; - if (!MY_SPEC.is_shared_join_filter() && OB_FAIL(ObPxEstimateSizeUtil::get_px_size( - &ctx_, MY_SPEC.px_est_size_factor_, filter_len, filter_len))) { - LOG_WARN("failed to get px size", K(ret)); - } else if (op_input->is_local_create_ || !MY_SPEC.is_shared_join_filter()) { - 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 { - LOG_TRACE("join filter buffer length", - K(filter_len), - K(MY_SPEC.filter_len_)); - } - } else if (OB_ISNULL(filter_create_ = reinterpret_cast( - op_input->share_info_.filter_ptr_))) { - ret = OB_NOT_INIT; - LOG_WARN("the bloom filter is not init", K(ret)); - } - if (OB_SUCC(ret) && MY_SPEC.max_batch_size_ > 0) { - if (OB_ISNULL(batch_hash_values_ = - (uint64_t *)ctx_.get_allocator().alloc(sizeof(uint64_t) * MY_SPEC.max_batch_size_))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to alloc batch_hash_values_", K(ret), K(MY_SPEC.max_batch_size_)); - } - } - } - if (OB_SUCC(ret)) { - bf_key_.init(ctx_.get_my_session()->get_effective_tenant_id(), - MY_SPEC.filter_id_, - MY_SPEC.server_id_, - op_input->get_px_sequence_id(), - op_input->task_id_); - if (MY_SPEC.is_use_mode()) { - //在ctx中创建expr ctx, 并初始化bloom filter key - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = NULL; - if (OB_ISNULL(join_filter_ctx = static_cast( - ctx_.get_expr_op_ctx(MY_SPEC.filter_expr_id_)))) { - if (OB_FAIL(ctx_.create_expr_op_ctx(MY_SPEC.filter_expr_id_, join_filter_ctx))) { - LOG_WARN("failed to create operator ctx", K(ret), K(MY_SPEC.filter_expr_id_)); - } else { - join_filter_ctx->bf_key_ = bf_key_; - int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); - bool wait_bloom_filter_ready = false; - omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); - if (OB_LIKELY(tenant_config.is_valid())) { - wait_bloom_filter_ready = tenant_config->_enable_px_bloom_filter_sync; - } - join_filter_ctx->need_wait_bf_ = wait_bloom_filter_ready; - join_filter_ctx->window_size_ = ADAPTIVE_BF_WINDOW_ORG_SIZE; - } - } else { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("join filter ctx is unexpected", K(ret)); - } - } + LOG_WARN("builder operator is invalid", K(ret), K(MY_SPEC.filter_shared_type_), K(MY_SPEC.mode_)); + } else if (MY_SPEC.is_create_mode() && OB_FAIL(open_join_filter_create())) { + LOG_WARN("fail to open join filter create op", K(ret)); + } else if ((MY_SPEC.is_use_mode() && OB_FAIL(open_join_filter_use()))) { + LOG_WARN("fail to open join filter use op", K(ret)); } return ret; } @@ -275,11 +395,10 @@ int ObJoinFilterOp::inner_open() int ObJoinFilterOp::do_create_filter_rescan() { int ret = OB_SUCCESS; - if (OB_ISNULL(filter_create_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("filter create is unexpected", K(ret)); - } else { - filter_create_->reset_filter(); + for (int i = 0; i < local_rf_msgs_.count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(local_rf_msgs_.at(i)->reuse())) { + LOG_WARN("fail to reuse local rf msgs", K(ret)); + } } return ret; } @@ -287,13 +406,6 @@ int ObJoinFilterOp::do_create_filter_rescan() int ObJoinFilterOp::do_use_filter_rescan() { int ret = OB_SUCCESS; - ObExprJoinFilter::ObExprJoinFilterContext *filter_expr_ctx = NULL; - if (OB_ISNULL(filter_expr_ctx = static_cast( - ctx_.get_expr_op_ctx(MY_SPEC.filter_expr_id_)))) { - LOG_TRACE("join filter expr ctx is null"); - } else { - filter_expr_ctx->reset_monitor_info(); - } return ret; } @@ -313,20 +425,16 @@ int ObJoinFilterOp::inner_rescan() int ObJoinFilterOp::inner_get_next_row() { int ret = OB_SUCCESS; - ObJoinFilterOpInput *filter_input_ = static_cast(input_); + ObJoinFilterOpInput *filter_input = static_cast(input_); while (OB_SUCC(ret)) { clear_evaluated_flag(); ret = child_->get_next_row(); if (OB_ITER_END == ret) { if (MY_SPEC.is_create_mode()) { - bool all_is_finished = false; - // sqc 下多个 worker 上的 filter 都完成了 child 迭代, - // 说明本 sqc 上的 filter 数据已经收集完毕,可以执行发送。 - // 对于local filter计划, 将filter写入manager - // 对于shuffle filter计划, 将filter信息写入exec_ctx,由recieve算子发送rpc. - if (OB_FAIL(filter_input_->check_finish(all_is_finished, MY_SPEC.is_shared_join_filter()))) { - LOG_WARN("fail to check all worker end", K(ret)); - } else if (all_is_finished && OB_FAIL(send_filter())) { + ret = OB_SUCCESS; + if (OB_FAIL(try_merge_join_filter())) { + LOG_WARN("fail to merge join filter", K(ret)); + } else if (OB_FAIL(try_send_join_filter())) { LOG_WARN("fail to send bloom filter to use filter", K(ret)); } if (OB_SUCC(ret)) { @@ -353,21 +461,10 @@ int ObJoinFilterOp::inner_get_next_row() } } if (MY_SPEC.is_use_mode() && ret == OB_ITER_END) { - ObExprJoinFilter::ObExprJoinFilterContext *filter_expr_ctx = NULL; - if (OB_ISNULL(filter_expr_ctx = static_cast( - ctx_.get_expr_op_ctx(MY_SPEC.filter_expr_id_)))) { - /*do nothing*/ - LOG_TRACE("join filter expr ctx is null"); + if (OB_FAIL(update_plan_monitor_info())) { + LOG_WARN("fail to update plan monitor info", K(ret)); } else { - // 记录相关信息到sql plan monitor中 - op_monitor_info_.otherstat_1_value_ = filter_expr_ctx->filter_count_; - op_monitor_info_.otherstat_2_value_ = filter_expr_ctx->total_count_; - op_monitor_info_.otherstat_3_value_ = filter_expr_ctx->check_count_; - op_monitor_info_.otherstat_4_value_ = filter_expr_ctx->ready_ts_; - op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::JOIN_FILTER_FILTERED_COUNT; - op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::JOIN_FILTER_TOTAL_COUNT; - op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::JOIN_FILTER_CHECK_COUNT; - op_monitor_info_.otherstat_4_id_ = ObSqlMonitorStatIds::JOIN_FILTER_READY_TIMESTAMP; + ret = OB_ITER_END; } } return ret; @@ -377,7 +474,7 @@ int ObJoinFilterOp::inner_get_next_batch(const int64_t max_row_cnt) { LOG_DEBUG("ObJoinFilterOp get_next_batch start"); int ret = OB_SUCCESS; - ObJoinFilterOpInput *filter_input_ = static_cast(input_); + ObJoinFilterOpInput *filter_input = static_cast(input_); // for batch int64_t batch_cnt = min(max_row_cnt, MY_SPEC.max_batch_size_); const ObBatchRows *child_brs = nullptr; @@ -387,9 +484,13 @@ int ObJoinFilterOp::inner_get_next_batch(const int64_t max_row_cnt) while (OB_SUCC(ret)) { clear_evaluated_flag(); if (OB_FAIL(child_->get_next_batch(batch_cnt, child_brs))) { - LOG_WARN("child_op failed to get next row", K(ret)); + if (OB_ITER_END != ret) { + LOG_WARN("failed to get next batch", K(ret)); + } + } else { + brs_.copy(child_brs); } - if (OB_SUCC(ret)) { + if (OB_SUCC(ret) && brs_.size_ > 0) { if (MY_SPEC.is_create_mode()) { if (OB_FAIL(insert_by_row_batch(child_brs))) { LOG_WARN("fail to insert bf", K(ret)); @@ -403,38 +504,25 @@ int ObJoinFilterOp::inner_get_next_batch(const int64_t max_row_cnt) ret = OB_ERR_UNEXPECTED; LOG_WARN("the mode of px bloom filter is unexpected", K(MY_SPEC.mode_), K(ret)); } + } else { + break; } } - if (OB_SUCC(ret)) { - ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx_); - if (OB_FAIL(brs_.copy(child_brs))) { - LOG_WARN("copy child_brs to brs_ failed", K(ret)); - } + if (OB_ITER_END == ret) { + brs_.size_ = 0; + brs_.end_ = true; + ret = OB_SUCCESS; } if (OB_SUCC(ret) && brs_.end_) { if (MY_SPEC.is_create_mode()) { - bool all_is_finished = false; - if (OB_FAIL(filter_input_->check_finish(all_is_finished, MY_SPEC.is_shared_join_filter()))) { - LOG_WARN("fail to check all worker end", K(ret)); - } else if (all_is_finished && OB_FAIL(send_filter())) { + if (OB_FAIL(try_merge_join_filter())) { + LOG_WARN("fail to merge join filter", K(ret)); + } else if (OB_FAIL(try_send_join_filter())) { LOG_WARN("fail to send bloom filter to use filter", K(ret)); } } else if (MY_SPEC.is_use_mode()) { - ObExprJoinFilter::ObExprJoinFilterContext *filter_expr_ctx = NULL; - if (OB_ISNULL(filter_expr_ctx = static_cast( - ctx_.get_expr_op_ctx(MY_SPEC.filter_expr_id_)))) { - // do nothing - LOG_TRACE("join filter expr ctx is null"); - } else { - // for sql plan monitor - op_monitor_info_.otherstat_1_value_ = filter_expr_ctx->filter_count_; - op_monitor_info_.otherstat_2_value_ = filter_expr_ctx->total_count_; - op_monitor_info_.otherstat_3_value_ = filter_expr_ctx->check_count_; - op_monitor_info_.otherstat_4_value_ = filter_expr_ctx->ready_ts_; - op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::JOIN_FILTER_FILTERED_COUNT; - op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::JOIN_FILTER_TOTAL_COUNT; - op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::JOIN_FILTER_CHECK_COUNT; - op_monitor_info_.otherstat_4_id_ = ObSqlMonitorStatIds::JOIN_FILTER_READY_TIMESTAMP; + if (OB_FAIL(update_plan_monitor_info())) { + LOG_WARN("fail to update plan monitor info", K(ret)); } } } @@ -442,184 +530,14 @@ int ObJoinFilterOp::inner_get_next_batch(const int64_t max_row_cnt) return ret; } -int ObJoinFilterOp::send_filter() -{ - int ret = OB_SUCCESS; - if (!MY_SPEC.is_shuffle() && OB_FAIL(send_local_filter())) { - LOG_WARN("fail to send local bloom filter", K(ret)); - } else if (MY_SPEC.is_shuffle() && - is_acceptable_filter() && - OB_FAIL(mark_rpc_filter())) { - LOG_WARN("fail to send rpc bllom filter", K(ret)); - } - return ret; -} - -bool ObJoinFilterOp::is_acceptable_filter() -{ - bool ret = true; - if (OB_NOT_NULL(filter_create_)) { - int64_t bits_cnt = 0; - int64_t total_cnt =0; - int64_t i = 0; - int64_t step = 1; - int64_t *bits_array_ptr = filter_create_->get_bits_array(); - int64_t len = filter_create_->get_bits_array_length(); - if (len > 1000L * 1000) { - //64MB bits. - step = round(((double)len) / 1000L * 1000); - } - if (OB_NOT_NULL(bits_array_ptr)) { - while (i < len) { - bits_cnt += ObBitVector::popcount64(bits_array_ptr[i]); - total_cnt += (sizeof(int64_t) * 8); - i += step; - } - double bits_rate = bits_cnt / (double)total_cnt; - if (bits_rate > ACCEPTABLE_FILTER_RATE && - bits_rate <= 1) { - ret = false; - } - LOG_TRACE("record join bloom filter bits rate", K(bits_rate), K(ret), K(bits_cnt), K(total_cnt), K(len)); - } - } - return ret; -} - -int ObJoinFilterOp::send_local_filter() -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(filter_create_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("filter create is unexpected", K(ret)); - } else { - filter_create_->px_bf_recieve_count_ = 1; - filter_create_->px_bf_recieve_size_ = 1; - } - if (OB_FAIL(ret)) { - /*do nothing*/ - } else if (OB_FAIL(ObPxBloomFilterManager::instance().set_px_bloom_filter(bf_key_, filter_create_))) { - LOG_WARN("fail to set px bloom filter in bloom filter manager", K(ret)); - } - return ret; -} - - -/* - * 发送rpc filter 有两种情况. - * - * HASH JOIN(root dfo) - * filter_c recieve(px) - * tsc transmit - * filter_use - * tsc - * 对于以上计划filter create是没有sqc proxy的, 将由px算子去link channel - * - * 第二种情况如下: - * - * p x - * HASH JOIN - * filter_c - * recieve recieve - * transmit transmit - *tsc filter_use - * tsc - * 对于这种计划需要join filter create算子获取channel, link channel并发送 - * */ - -int ObJoinFilterOp::mark_rpc_filter() -{ - int ret = OB_SUCCESS; - ObPxBloomFilterData *filter_data = NULL; - void *filter_ptr = NULL; - common::ObIArray &bf_ctx_array = ctx_.get_bloom_filter_ctx_array(); - if (OB_ISNULL(filter_ptr = ctx_.get_allocator().alloc(sizeof(ObPxBloomFilterData)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to init ObPxBloomFilterData", K(ret)); - } else if (OB_ISNULL(filter_data = new(filter_ptr) ObPxBloomFilterData())) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to init ObPxBloomFilterData", K(ret)); - } else if (OB_FAIL(filter_data->filter_.init(filter_create_))) { - LOG_WARN("fail to init filter data", K(ret)); - } else if (OB_FAIL(bf_ctx_array.push_back(ObJoinFilterDataCtx()))) { - LOG_WARN("failed to push back bloom filter context", K(ret)); - } else { - // don't need considerate concurrency control, cause px bloom filter will be ready one after other in execute plan - ObJoinFilterDataCtx &bf_ctx = bf_ctx_array.at(bf_ctx_array.count() - 1); - ObTenantConfigGuard tenant_config(TENANT_CONF(bf_key_.tenant_id_)); - if (tenant_config.is_valid() && true == tenant_config->_px_message_compression) { - bf_ctx.compressor_type_ = ObCompressorType::LZ4_COMPRESSOR; - } - ObJoinFilterOpInput *filter_input_ = static_cast(input_); - filter_data->tenant_id_ = bf_key_.tenant_id_; - filter_data->server_id_ = MY_SPEC.server_id_; - filter_data->filter_id_ = MY_SPEC.filter_id_; - filter_data->px_sequence_id_ = bf_key_.px_sequence_id_; - filter_data->bloom_filter_count_ = 0; - bf_ctx.filter_data_ = filter_data; - bf_ctx.filter_ready_ = true; - bf_ctx.ch_set_.reset(); - bf_ctx.ch_provider_ptr_ = filter_input_->share_info_.ch_provider_ptr_; - bf_ctx.bf_idx_at_sqc_proxy_ = filter_input_->get_bf_idx_at_sqc_proxy(); - LOG_DEBUG("join filter succ to mark rpc filter", K(bf_ctx_array.count()), K(bf_ctx_array), K(filter_data->filter_id_), K(ret)); - } - return ret; -} - -int ObJoinFilterOp::calc_hash_value(uint64_t &hash_value) -{ - int ret = OB_SUCCESS; - bool ignore = false; - if (OB_FAIL(calc_hash_value(hash_value, ignore))) { - LOG_WARN("fail to calc hash values", K(ret)); - } else if (ignore) { - ret = OB_NO_PARTITION_FOR_GIVEN_VALUE; - LOG_WARN("unexpected partition id", K(ret)); - } - return ret; -} - -int ObJoinFilterOp::calc_hash_value(uint64_t &hash_value, bool &ignore) -{ - int ret = OB_SUCCESS; - hash_value = ObExprJoinFilter::JOIN_FILTER_SEED; - ObDatum *datum = nullptr; - ignore = false; - if (MY_SPEC.is_partition_filter()) { - int64_t partition_id = 0; - if (OB_ISNULL(MY_SPEC.calc_tablet_id_expr_) || MY_SPEC.hash_funcs_.count() != 1) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected part id expr", K(ret)); - } else if (OB_FAIL(MY_SPEC.calc_tablet_id_expr_->eval(eval_ctx_, datum))) { - LOG_WARN("failed to eval datum", K(ret)); - } else if (ObExprCalcPartitionId::NONE_PARTITION_ID == (partition_id = datum->get_int())) { - ignore = true; - } else if (OB_FAIL(MY_SPEC.hash_funcs_.at(0).hash_func_(*datum, hash_value, hash_value))) { - LOG_WARN("failed to do hash", K(ret)); - } - } else { - for (int64_t idx = 0; OB_SUCC(ret) && idx < MY_SPEC.join_keys_.count() ; ++idx) { - if (OB_FAIL(MY_SPEC.join_keys_.at(idx)->eval(eval_ctx_, datum))) { - LOG_WARN("failed to eval datum", K(ret)); - } else if (OB_FAIL(MY_SPEC.hash_funcs_.at(idx).hash_func_(*datum, hash_value, hash_value))) { - LOG_WARN("failed to do hash", K(ret)); - } - } - } - return ret; -} - int ObJoinFilterOp::insert_by_row() { int ret = OB_SUCCESS; - uint64_t hash_value = 0; - bool ignore = false; - if (OB_FAIL(calc_hash_value(hash_value, ignore))) { - LOG_WARN("failed to calc hash value", K(ret)); - } else if (ignore) { - /*do nothing*/ - } else if (OB_FAIL(filter_create_->put(hash_value))) { - LOG_WARN("fail to put hash value to px bloom filter", K(ret)); + for (int i = 0; i < local_rf_msgs_.count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(local_rf_msgs_.at(i)->insert_by_row(MY_SPEC.join_keys_, + MY_SPEC.hash_funcs_, MY_SPEC.calc_tablet_id_expr_, eval_ctx_))) { + LOG_WARN("fail to insert rf by row", K(ret)); + } } return ret; } @@ -627,79 +545,12 @@ int ObJoinFilterOp::insert_by_row() int ObJoinFilterOp::insert_by_row_batch(const ObBatchRows *child_brs) { int ret = OB_SUCCESS; - if (child_brs->size_ > 0) { - uint64_t seed = ObExprJoinFilter::JOIN_FILTER_SEED; - if (MY_SPEC.is_partition_filter()) { - if (OB_ISNULL(MY_SPEC.calc_tablet_id_expr_) || MY_SPEC.hash_funcs_.count() != 1) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected part id expr", K(ret)); - } else if (OB_FAIL(MY_SPEC.calc_tablet_id_expr_->eval_batch(eval_ctx_, - *(child_brs->skip_), child_brs->size_))) { - LOG_WARN("failed to eval", K(ret)); - } else { - ObBatchDatumHashFunc hash_func_batch = MY_SPEC.hash_funcs_.at(0).batch_hash_func_; - hash_func_batch(batch_hash_values_, - MY_SPEC.calc_tablet_id_expr_->locate_batch_datums(eval_ctx_), - MY_SPEC.calc_tablet_id_expr_->is_batch_result(), - *child_brs->skip_, child_brs->size_, - &seed, - false); - } - } else { - for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.join_keys_.count(); ++i) { - ObExpr *expr = MY_SPEC.join_keys_.at(i); // expr ptr check in cg, not check here - if (OB_FAIL(expr->eval_batch(eval_ctx_, *(child_brs->skip_), child_brs->size_))) { - LOG_WARN("eval failed", K(ret)); - } else { - ObBatchDatumHashFunc hash_func_batch = MY_SPEC.hash_funcs_.at(i).batch_hash_func_; - const bool is_batch_seed = (i > 0); - hash_func_batch(batch_hash_values_, - expr->locate_batch_datums(eval_ctx_), expr->is_batch_result(), - *child_brs->skip_, child_brs->size_, - is_batch_seed ? batch_hash_values_ : &seed, - is_batch_seed); - } - } - } - for (int64_t i = 0; OB_SUCC(ret) && i < child_brs->size_; ++i) { - if (MY_SPEC.is_partition_filter()) { - ObDatum &datum = MY_SPEC.calc_tablet_id_expr_->locate_expr_datum(eval_ctx_, i); - if (ObExprCalcPartitionId::NONE_PARTITION_ID == datum.get_int()) { - continue; - } - } - if (OB_SUCC(ret)) { - if (child_brs->skip_->at(i)) { - continue; - } else if (OB_FAIL(filter_create_->put(batch_hash_values_[i]))) { - LOG_WARN("fail to put hash value to px bloom filter", K(ret)); - } - } - } - } - return ret; -} - -int ObJoinFilterOp::check_contain_row(bool &match) -{ - int ret = OB_SUCCESS; - if (OB_ISNULL(filter_use_)) { - if (OB_FAIL(ObPxBloomFilterManager::instance().get_px_bloom_filter(bf_key_, filter_use_))) { - ret = OB_SUCCESS; - } - } - if (OB_SUCC(ret) && OB_NOT_NULL(filter_use_)) { - if (!filter_use_->check_ready()) { - LOG_TRACE("px bloom filter is not ready, just by pass", - K(filter_use_->px_bf_recieve_count_), - K(filter_use_->px_bf_recieve_size_)); - } else { - uint64_t hash_value = 0; - if (OB_FAIL(calc_hash_value(hash_value))) { - LOG_WARN("failed to calc hash value", K(ret)); - } else if (OB_FAIL(filter_use_->might_contain(hash_value, match))) { - LOG_WARN("fail to check filter might contain value", K(ret)); - } + for (int i = 0; i < local_rf_msgs_.count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(local_rf_msgs_.at(i)->insert_by_row_batch(child_brs, + MY_SPEC.join_keys_, MY_SPEC.hash_funcs_, + MY_SPEC.calc_tablet_id_expr_, eval_ctx_, + batch_hash_values_))) { + LOG_WARN("fail to insert rf by row", K(ret)); } } return ret; @@ -708,14 +559,400 @@ int ObJoinFilterOp::check_contain_row(bool &match) int ObJoinFilterOp::inner_close() { int ret = OB_SUCCESS; - ObJoinFilterOpInput *filter_input_ = static_cast(input_); - if (MY_SPEC.is_create_mode()) { - // do nothing - } else if (filter_input_->check_release(MY_SPEC.is_shared_join_filter()) && - OB_FAIL(destroy_filter())) { - //当close引用计数为0时, 释放use端内存. - } else { - /*do nothing*/ + ObJoinFilterOpInput *filter_input = static_cast(input_); + if (MY_SPEC.is_create_mode() && OB_FAIL(close_join_filter_create())) { + LOG_WARN("fail to open join filter create op", K(ret)); + } else if ((MY_SPEC.is_use_mode() && OB_FAIL(close_join_filter_use()))) { + LOG_WARN("fail to open join filter use op", K(ret)); } return ret; } + +int ObJoinFilterOp::try_merge_join_filter() +{ + int ret = OB_SUCCESS; + ObJoinFilterOpInput *filter_input = static_cast(input_); + uint64_t *count_ptr = reinterpret_cast( + filter_input->share_info_.unfinished_count_ptr_); + ObPxSQCProxy *sqc_proxy = reinterpret_cast( + filter_input->share_info_.ch_provider_ptr_); + int64_t cur_cnt = 0; + + for (int i = 0; i < local_rf_msgs_.count() && OB_SUCC(ret); ++i) { + if (!MY_SPEC.is_shared_join_filter()) { + lucky_devil_champions_.at(i) = true; + } else if (local_rf_msgs_.at(i)->get_msg_type() == ObP2PDatahubMsgBase::BLOOM_FILTER_MSG) { + } else if (OB_FAIL(shared_rf_msgs_.at(i)->merge(*local_rf_msgs_.at(i)))) { + LOG_WARN("fail to do rf merge", K(ret)); + } + } + if (OB_SUCC(ret) && MY_SPEC.is_shared_join_filter()) { + cur_cnt = ATOMIC_AAF(count_ptr, -1); + for (int i = 0; i < local_rf_msgs_.count() && OB_SUCC(ret); ++i) { + if (local_rf_msgs_.at(i)->get_msg_type() == ObP2PDatahubMsgBase::BLOOM_FILTER_MSG) { + if (MY_SPEC.is_shuffle_) { + bool is_local_dh = false; + if (OB_FAIL(sqc_proxy->check_is_local_dh(local_rf_msgs_.at(i)->get_p2p_datahub_id(), + is_local_dh, + local_rf_msgs_.at(i)->get_msg_receive_expect_cnt()))) { + LOG_WARN("fail to check local dh", K(ret)); + } else if (is_local_dh) { + lucky_devil_champions_.at(i) = (0 == cur_cnt); + } else { + lucky_devil_champions_.at(i) = true; + } + } else { + lucky_devil_champions_.at(i) = (0 == cur_cnt); + } + if (OB_SUCC(ret)) { + if (0 == cur_cnt) { + bool *create_finish = static_cast(shared_rf_msgs_.at(i))->create_finish_; + if (OB_NOT_NULL(create_finish)) { + *create_finish = true; + } + } + } + } else if (0 == cur_cnt) { + lucky_devil_champions_.at(i) = true; + } + } + } + return ret; +} + +int ObJoinFilterOp::try_send_join_filter() +{ + int ret = OB_SUCCESS; + ObJoinFilterOpInput *filter_input = static_cast(input_); + ObPxSQCProxy *sqc_proxy = reinterpret_cast( + filter_input->share_info_.ch_provider_ptr_); + CK(OB_NOT_NULL(sqc_proxy)); + + for (int i = 0; i < local_rf_msgs_.count() && OB_SUCC(ret); ++i) { + if (!lucky_devil_champions_.at(i)) { + } else if (!MY_SPEC.is_shared_join_filter()) { + if (OB_FAIL(PX_P2P_DH.send_local_p2p_msg(*local_rf_msgs_.at(i)))) { + LOG_WARN("fail to send local p2p msg", K(ret)); + } + } else if (!MY_SPEC.is_shuffle_) { + if (OB_FAIL(PX_P2P_DH.send_local_p2p_msg(*shared_rf_msgs_.at(i)))) { + LOG_WARN("fail to send local p2p msg", K(ret)); + } + } else if (MY_SPEC.is_shared_join_filter()) { + if (OB_FAIL(PX_P2P_DH.send_p2p_msg(*shared_rf_msgs_.at(i), *sqc_proxy))) { + LOG_WARN("fail to send p2p msg", K(ret)); + } + } + } + return ret; +} + +int ObJoinFilterOp::prepre_bloom_filter_ctx(ObBloomFilterSendCtx *bf_ctx) +{ + int ret = OB_SUCCESS; + int64_t &each_group_size = bf_ctx->get_each_group_size(); + if (OB_FAIL(calc_each_bf_group_size(each_group_size))) { + LOG_WARN("fail to calc each bf group size", K(ret)); + } else { + ObRFBloomFilterMsg *bf_msg = filter_create_msg_->bf_msg_; + ObPxBloomFilter &bf = bf_msg->bloom_filter_; + int64_t sqc_count = 0; + int64_t peer_sqc_count = 0; + int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); + // 125 = 1000/8, send size means how many byte of partial bloom filter will be send at once + int64_t send_size = GCONF._send_bloom_filter_size * 125; + // how many piece of partial bloom filter will be send by all threads(sqc level) + int64_t send_count = ceil(bf.get_bits_array_length() / (double)send_size); + int64_t bloom_filter_count = send_count * send_size; + ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); + if (tenant_config.is_valid() && true == tenant_config->_px_message_compression) { + bf_ctx->set_bf_compress_type(ObCompressorType::LZ4_COMPRESSOR); + } + if (OB_FAIL(bf_ctx->generate_filter_indexes(each_group_size, peer_sqc_count))) { + LOG_WARN("failed to generate filter indexs", K(ret)); + } else { + bf_ctx->set_per_addr_bf_count(send_count); + bf_ctx->set_bloom_filter_ready(true); + } + } + return ret; +} + +int ObJoinFilterOp::calc_each_bf_group_size(int64_t &each_group_size) +{ + int ret = OB_SUCCESS; + if (0 == each_group_size) { // only need calc once + int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); + int64_t peer_target_cnt = 0; + 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(peer_target_cnt); // 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); + } + return ret; +} + +int ObJoinFilterOp::update_plan_monitor_info() +{ + int ret = OB_SUCCESS; + op_monitor_info_.otherstat_1_value_ = 0; + op_monitor_info_.otherstat_2_value_ = 0; + op_monitor_info_.otherstat_3_value_ = 0; + op_monitor_info_.otherstat_4_value_ = 0; + op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::JOIN_FILTER_FILTERED_COUNT; + op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::JOIN_FILTER_TOTAL_COUNT; + op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::JOIN_FILTER_CHECK_COUNT; + op_monitor_info_.otherstat_4_id_ = ObSqlMonitorStatIds::JOIN_FILTER_READY_TIMESTAMP; + op_monitor_info_.otherstat_5_id_ = ObSqlMonitorStatIds::JOIN_FILTER_ID; + op_monitor_info_.otherstat_5_value_ = MY_SPEC.filter_id_; + op_monitor_info_.otherstat_6_id_ = ObSqlMonitorStatIds::JOIN_FILTER_LENGTH; + op_monitor_info_.otherstat_6_value_ = MY_SPEC.filter_len_; + for (int i = 0; i < MY_SPEC.rf_infos_.count() && OB_SUCC(ret); ++i) { + if (OB_INVALID_ID != MY_SPEC.rf_infos_.at(i).filter_expr_id_) { + ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = NULL; + if (OB_ISNULL(join_filter_ctx = static_cast( + ctx_.get_expr_op_ctx(MY_SPEC.rf_infos_.at(i).filter_expr_id_)))) { + LOG_TRACE("join filter expr ctx is null"); + } else { + op_monitor_info_.otherstat_1_value_ += join_filter_ctx->filter_count_; + op_monitor_info_.otherstat_2_value_ += join_filter_ctx->total_count_; + op_monitor_info_.otherstat_3_value_ += join_filter_ctx->check_count_; + op_monitor_info_.otherstat_4_value_ = max(join_filter_ctx->ready_ts_, op_monitor_info_.otherstat_4_value_); + } + } + } + return ret; +} + +int ObJoinFilterOp::open_join_filter_create() +{ + int ret = OB_SUCCESS; + int64_t filter_len = MY_SPEC.filter_len_; + common::ObIAllocator &allocator = ctx_.get_allocator(); + int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); + int64_t timeout_ts = GET_PHY_PLAN_CTX(ctx_)->get_timeout_timestamp(); + ObJoinFilterOpInput *filter_input = static_cast(input_); + ObPxSQCProxy *sqc_proxy = reinterpret_cast( + filter_input->share_info_.ch_provider_ptr_); + if (!MY_SPEC.is_shared_join_filter() && OB_FAIL(ObPxEstimateSizeUtil::get_px_size( + &ctx_, MY_SPEC.px_est_size_factor_, filter_len, filter_len))) { + LOG_WARN("failed to get px size", K(ret)); + } else if (!MY_SPEC.is_shared_join_filter()) { + ObP2PDatahubMsgBase *msg_ptr = nullptr; + for (int i = 0; i < MY_SPEC.rf_infos_.count() && OB_SUCC(ret); ++i) { + msg_ptr = nullptr; + if (OB_FAIL(PX_P2P_DH.alloc_msg(allocator, MY_SPEC.rf_infos_.at(i).dh_msg_type_, msg_ptr))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else if (OB_FAIL(local_rf_msgs_.push_back(msg_ptr))) { + LOG_WARN("fail to push back msg ptr", K(ret)); + } else if (OB_FAIL(msg_ptr->init(MY_SPEC.rf_infos_.at(i).p2p_datahub_id_, + filter_input->px_sequence_id_, filter_input->task_id_, tenant_id, timeout_ts, filter_input->register_dm_info_))) { + LOG_WARN("fail to init msg", K(ret)); + } else if (OB_FAIL(ObJoinFilterOpInput::construct_msg_details(MY_SPEC, sqc_proxy, + filter_input->config_, *msg_ptr, 1))) { + LOG_WARN("fail to construct msg details", K(ret)); + } else if (OB_FAIL(lucky_devil_champions_.push_back(false))) { + LOG_WARN("fail to push back flag", K(ret)); + } + } + } else if (OB_FAIL(init_shared_msgs_from_input())) { + LOG_WARN("fail to init shared msgs from input", K(ret)); + } + if (OB_SUCC(ret) && MY_SPEC.max_batch_size_ > 0) { + if (OB_ISNULL(batch_hash_values_ = + (uint64_t *)ctx_.get_allocator().alloc(sizeof(uint64_t) * MY_SPEC.max_batch_size_))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc batch_hash_values_", K(ret), K(MY_SPEC.max_batch_size_)); + } + } + return ret; +} + +int ObJoinFilterOp::open_join_filter_use() +{ + int ret = OB_SUCCESS; + ObJoinFilterOpInput *filter_input = static_cast(input_); + int64_t task_id = MY_SPEC.is_shared_join_filter() ? 0 : filter_input->task_id_; + int64_t px_seq_id = filter_input->px_sequence_id_; + for (int i = 0; i < MY_SPEC.rf_infos_.count() && OB_SUCC(ret); ++i) { + if (OB_INVALID_ID != MY_SPEC.rf_infos_.at(i).filter_expr_id_) { + ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = NULL; + if (OB_ISNULL(join_filter_ctx = static_cast( + ctx_.get_expr_op_ctx(MY_SPEC.rf_infos_.at(i).filter_expr_id_)))) { + if (OB_FAIL(ctx_.create_expr_op_ctx(MY_SPEC.rf_infos_.at(i).filter_expr_id_,join_filter_ctx))) { + LOG_WARN("failed to create operator ctx", K(ret), K(MY_SPEC.rf_infos_.at(i).filter_expr_id_)); + } else { + ObP2PDhKey dh_key(MY_SPEC.rf_infos_.at(i).p2p_datahub_id_, px_seq_id, task_id); + join_filter_ctx->rf_key_ = dh_key; + int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); + join_filter_ctx->window_size_ = ADAPTIVE_BF_WINDOW_ORG_SIZE; + join_filter_ctx->max_wait_time_ms_ = filter_input->config_.runtime_filter_wait_time_ms_; + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("join filter ctx is unexpected", K(ret)); + } + } + } + return ret; +} + +int ObJoinFilterOp::init_shared_msgs_from_input() +{ + int ret = OB_SUCCESS; + ObJoinFilterOpInput *op_input = static_cast(input_); + ObArray *array_ptr = + reinterpret_cast *>(op_input->share_info_.shared_msgs_); + if (OB_ISNULL(array_ptr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("array ptr is nullptr", K(ret)); + } else if (OB_FAIL(shared_rf_msgs_.assign(*array_ptr))) { + LOG_WARN("fail to assign array ptr", K(ret)); + } else { + for (int i = 0; i < shared_rf_msgs_.count() && OB_SUCC(ret); ++i) { + if (OB_ISNULL(shared_rf_msgs_.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected shared rf msgs", K(ret)); + } else if (OB_FAIL(init_local_msg_from_shared_msg(*shared_rf_msgs_.at(i)))) { + LOG_WARN("fail to copy local msgs", K(ret)); + } else if (OB_FAIL(lucky_devil_champions_.push_back(false))) { + LOG_WARN("fail to push back flag", K(ret)); + } + } + } + return ret; +} + +int ObJoinFilterOp::init_local_msg_from_shared_msg(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + ObJoinFilterOpInput *filter_input = static_cast(input_); + ObPxSQCProxy *sqc_proxy = reinterpret_cast( + filter_input->share_info_.ch_provider_ptr_); + switch(msg.get_msg_type()) { + case ObP2PDatahubMsgBase::BLOOM_FILTER_MSG: { + ret = local_rf_msgs_.push_back(&msg); + break; + } + case ObP2PDatahubMsgBase::RANGE_FILTER_MSG: { + ObP2PDatahubMsgBase *range_ptr = nullptr; + if (OB_FAIL(PX_P2P_DH.alloc_msg(ctx_.get_allocator(), + ObP2PDatahubMsgBase::RANGE_FILTER_MSG, range_ptr))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else if (OB_FAIL(range_ptr->init(msg.get_p2p_datahub_id(), + msg.get_px_seq_id(), 0/*task_id*/, msg.get_tenant_id(), + msg.get_timeout_ts(), filter_input->register_dm_info_))) { + LOG_WARN("fail to init msg", K(ret)); + } else if (OB_FAIL(ObJoinFilterOpInput::construct_msg_details(MY_SPEC, + sqc_proxy, filter_input->config_, *range_ptr, msg.get_msg_receive_expect_cnt()))) { + LOG_WARN("fail to construct msg details", K(ret)); + } else if (OB_FAIL(local_rf_msgs_.push_back(range_ptr))) { + LOG_WARN("fail to push back local rf msgs", K(ret)); + } + break; + } + case ObP2PDatahubMsgBase::IN_FILTER_MSG: { + ObP2PDatahubMsgBase *in_ptr = nullptr; + if (OB_FAIL(PX_P2P_DH.alloc_msg(ctx_.get_allocator(), + ObP2PDatahubMsgBase::IN_FILTER_MSG, in_ptr))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else { + ObRFInFilterMsg &in_msg = static_cast(*in_ptr); + if (OB_FAIL(in_msg.init(msg.get_p2p_datahub_id(), + msg.get_px_seq_id(), 0/*task_id*/, msg.get_tenant_id(), + msg.get_timeout_ts(), filter_input->register_dm_info_))) { + LOG_WARN("fail to init msg", K(ret)); + } else if (OB_FAIL(ObJoinFilterOpInput::construct_msg_details(MY_SPEC, + sqc_proxy, filter_input->config_, *in_ptr, msg.get_msg_receive_expect_cnt()))) { + LOG_WARN("fail to construct msg details", K(ret)); + } else if (OB_FAIL(local_rf_msgs_.push_back(in_ptr))) { + LOG_WARN("fail to push back local rf msgs", K(ret)); + } + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected msg type", K(ret), K(msg.get_msg_type())); + } + } + return ret; +} + +int ObJoinFilterOp::close_join_filter_create() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(release_local_msg())) { + LOG_WARN("failed release local msg", K(ret)); + } else if (OB_FAIL(release_shared_msg())) { + LOG_WARN("failed release local msg", K(ret)); + } + return ret; +} + +int ObJoinFilterOp::release_local_msg() +{ + int ret = OB_SUCCESS; + ObP2PDatahubMsgBase *msg = nullptr; + for (int i = 0; i < local_rf_msgs_.count(); ++i) { + if (OB_NOT_NULL(local_rf_msgs_.at(i))) { + msg = nullptr; + ObP2PDhKey key; + key.p2p_datahub_id_ = + local_rf_msgs_.at(i)->get_p2p_datahub_id(); + key.task_id_ = local_rf_msgs_.at(i)->get_task_id(); + key.px_sequence_id_ = local_rf_msgs_.at(i)->get_px_seq_id(); + if (!MY_SPEC.is_shared_join_filter()) { + PX_P2P_DH.erase_msg(key, msg); + local_rf_msgs_.at(i)->destroy(); + } else if (local_rf_msgs_.at(i)->get_msg_type() != ObP2PDatahubMsgBase::BLOOM_FILTER_MSG) { + local_rf_msgs_.at(i)->destroy(); + } + } + } + return ret; +} + +int ObJoinFilterOp::release_shared_msg() +{ + int ret = OB_SUCCESS; + ObP2PDatahubMsgBase *msg = nullptr; + ObJoinFilterOpInput *filter_input = static_cast(input_); + if (MY_SPEC.is_shared_join_filter() && !MY_SPEC.is_shuffle_) { + bool need_release = filter_input->check_release(); + for (int i = 0; need_release && i < shared_rf_msgs_.count(); ++i) { + if (OB_NOT_NULL(shared_rf_msgs_.at(i))) { + msg = nullptr; + ObP2PDhKey key; + key.p2p_datahub_id_ = + shared_rf_msgs_.at(i)->get_p2p_datahub_id(); + key.task_id_ = shared_rf_msgs_.at(i)->get_task_id(); + key.px_sequence_id_ = shared_rf_msgs_.at(i)->get_px_seq_id(); + PX_P2P_DH.erase_msg(key, msg); + shared_rf_msgs_.at(i)->destroy(); + } + } + } + return ret; +} + +int ObJoinFilterOp::close_join_filter_use() +{ + int ret = OB_SUCCESS; + /*do nothing*/ + return ret; +} diff --git a/src/sql/engine/join/ob_join_filter_op.h b/src/sql/engine/join/ob_join_filter_op.h index 22a1438f8..5aa895606 100644 --- a/src/sql/engine/join/ob_join_filter_op.h +++ b/src/sql/engine/join/ob_join_filter_op.h @@ -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(&sqc_proxy); } - bool is_local_create() { return is_local_create_; } ObJoinFilterOp *get_filter() { return reinterpret_cast(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 &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 rf_infos_; + common::ObFixedArray 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 &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 shared_rf_msgs_; // sqc level share + ObArray local_rf_msgs_; uint64_t *batch_hash_values_; + ObArray lucky_devil_champions_; }; } diff --git a/src/sql/engine/ob_physical_plan.cpp b/src/sql/engine/ob_physical_plan.cpp index 93b25080a..3eeb021f1 100644 --- a/src/sql/engine/ob_physical_plan.cpp +++ b/src/sql/engine/ob_physical_plan.cpp @@ -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) diff --git a/src/sql/engine/ob_physical_plan.h b/src/sql/engine/ob_physical_plan.h index 36fcce440..7118c891f 100644 --- a/src/sql/engine/ob_physical_plan.h +++ b/src/sql/engine/ob_physical_plan.h @@ -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) diff --git a/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp b/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp index 5a31fdb0c..9a1277be6 100644 --- a/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp @@ -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. diff --git a/src/sql/engine/px/exchange/ob_px_ms_receive_op.cpp b/src/sql/engine/px/exchange/ob_px_ms_receive_op.cpp index b7243062a..873049d04 100644 --- a/src/sql/engine/px/exchange/ob_px_ms_receive_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_ms_receive_op.cpp @@ -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)); } diff --git a/src/sql/engine/px/exchange/ob_px_receive_op.cpp b/src/sql/engine/px/exchange/ob_px_receive_op.cpp index 328faf13f..8881b1961 100644 --- a/src/sql/engine/px/exchange/ob_px_receive_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_receive_op.cpp @@ -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(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(input_); + /*ObPxReceiveOpInput *recv_input = reinterpret_cast(input_); ObPxSQCProxy *sqc_proxy = reinterpret_cast(recv_input->get_ch_provider()); ObPhysicalPlanCtx *phy_plan_ctx = GET_PHY_PLAN_CTX(ctx_); common::ObIArray &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; } diff --git a/src/sql/engine/px/exchange/ob_px_transmit_op.cpp b/src/sql/engine/px/exchange/ob_px_transmit_op.cpp index 891d0a822..e1c10b6ad 100644 --- a/src/sql/engine/px/exchange/ob_px_transmit_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_transmit_op.cpp @@ -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()); diff --git a/src/sql/engine/px/ob_dfo.cpp b/src/sql/engine/px/ob_dfo.cpp index b519db7eb..1368749d4 100644 --- a/src/sql/engine/px/ob_dfo.cpp +++ b/src/sql/engine/px/ob_dfo.cpp @@ -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 &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; -} \ No newline at end of file +} diff --git a/src/sql/engine/px/ob_dfo.h b/src/sql/engine/px/ob_dfo.h index 73a653c44..a41fb9ae2 100644 --- a/src/sql/engine/px/ob_dfo.h +++ b/src/sql/engine/px/ob_dfo.h @@ -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 p2p_sequence_ids_; + common::ObSArray> 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 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 &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 &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 &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 &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 external_table_files_; + // for dm + ObPxDetectableIds px_detectable_ids_; + common::ObIDetectCallback *detect_cb_; + uint64_t node_sequence_id_; + // --------------- + // for p2p dh mgr + common::ObArrayp2p_dh_ids_; //for dh create + common::ObArrayp2p_dh_addrs_; //for dh use + ObDASTableLoc *p2p_dh_loc_; + bool need_p2p_info_; + ObP2PDhMapInfo p2p_dh_map_info_; + // --------------- + ObPxCoordInfo *coord_info_ptr_; }; diff --git a/src/sql/engine/px/ob_dfo_mgr.cpp b/src/sql/engine/px/ob_dfo_mgr.cpp index 7ddba4d2f..92cace5e5 100644 --- a/src/sql/engine/px/ob_dfo_mgr.cpp +++ b/src/sql/engine/px/ob_dfo_mgr.cpp @@ -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(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(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(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(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(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(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(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(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(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()); + } } } } diff --git a/src/sql/engine/px/ob_dfo_scheduler.cpp b/src/sql/engine/px/ob_dfo_scheduler.cpp index b1195912f..1eb2823e5 100644 --- a/src/sql/engine/px/ob_dfo_scheduler.cpp +++ b/src/sql/engine/px/ob_dfo_scheduler.cpp @@ -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& 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 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 peer_states; + ObArray 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; } diff --git a/src/sql/engine/px/ob_dfo_scheduler.h b/src/sql/engine/px/ob_dfo_scheduler.h index d3bf742de..4abbecc4a 100644 --- a/src/sql/engine/px/ob_dfo_scheduler.h +++ b/src/sql/engine/px/ob_dfo_scheduler.h @@ -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: diff --git a/src/sql/engine/px/ob_granule_iterator_op.cpp b/src/sql/engine/px/ob_granule_iterator_op.cpp index d8eeb7764..5c25125c5 100644 --- a/src/sql/engine/px/ob_granule_iterator_op.cpp +++ b/src/sql/engine/px/ob_granule_iterator_op.cpp @@ -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(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(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; diff --git a/src/sql/engine/px/ob_granule_iterator_op.h b/src/sql/engine/px/ob_granule_iterator_op.h index bf5550657..74ebc1691 100644 --- a/src/sql/engine/px/ob_granule_iterator_op.h +++ b/src/sql/engine/px/ob_granule_iterator_op.h @@ -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_; diff --git a/src/sql/engine/px/ob_granule_pump.h b/src/sql/engine/px/ob_granule_pump.h index 8ffca98f0..f319905b4 100644 --- a/src/sql/engine/px/ob_granule_pump.h +++ b/src/sql/engine/px/ob_granule_pump.h @@ -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 { diff --git a/src/sql/engine/px/ob_px_basic_info.h b/src/sql/engine/px/ob_px_basic_info.h index cc67f53dd..7a5512c2f 100644 --- a/src/sql/engine/px/ob_px_basic_info.h +++ b/src/sql/engine/px/ob_px_basic_info.h @@ -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 diff --git a/src/sql/engine/px/ob_px_bloom_filter.cpp b/src/sql/engine/px/ob_px_bloom_filter.cpp index 7b4de763d..6279d95d9 100644 --- a/src/sql/engine/px/ob_px_bloom_filter.cpp +++ b/src/sql/engine/px/ob_px_bloom_filter.cpp @@ -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(bits_array_buf) + + CACHE_LINE_SIZE - 1) >> LOG_CACHE_LINE_SIZE) << LOG_CACHE_LINE_SIZE; + bits_array_ = reinterpret_cast(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_, diff --git a/src/sql/engine/px/ob_px_bloom_filter.h b/src/sql/engine/px/ob_px_bloom_filter.h index 6a970ebd8..0df0789eb 100644 --- a/src/sql/engine/px/ob_px_bloom_filter.h +++ b/src/sql/engine/px/ob_px_bloom_filter.h @@ -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 diff --git a/src/sql/engine/px/ob_px_coord_op.cpp b/src/sql/engine/px/ob_px_coord_op.cpp index 14d3c1648..14fe37121 100644 --- a/src/sql/engine/px/ob_px_coord_op.cpp +++ b/src/sql/engine/px/ob_px_coord_op.cpp @@ -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 *, 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 *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)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret)); + } else { + p2p_ids = new(ptr) ObSArray(); + 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 &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 &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(get_spec()).batch_op_info_.is_inited()) { ObDTLIntermResultKey key; ObDtlChannelInfo ci; diff --git a/src/sql/engine/px/ob_px_coord_op.h b/src/sql/engine/px/ob_px_coord_op.h index 2ed79f190..d2ae991a6 100644 --- a/src/sql/engine/px/ob_px_coord_op.h +++ b/src/sql/engine/px/ob_px_coord_op.h @@ -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_; diff --git a/src/sql/engine/px/ob_px_rpc_processor.cpp b/src/sql/engine/px/ob_px_rpc_processor.cpp index 5edd15965..738e415ec 100644 --- a/src/sql/engine/px/ob_px_rpc_processor.cpp +++ b/src/sql/engine/px/ob_px_rpc_processor.cpp @@ -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); diff --git a/src/sql/engine/px/ob_px_scheduler.cpp b/src/sql/engine/px/ob_px_scheduler.cpp index eb7c40485..3c8d72e64 100644 --- a/src/sql/engine/px/ob_px_scheduler.cpp +++ b/src/sql/engine/px/ob_px_scheduler.cpp @@ -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(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 &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 diff --git a/src/sql/engine/px/ob_px_scheduler.h b/src/sql/engine/px/ob_px_scheduler.h index 4535ea338..89018ac12 100644 --- a/src/sql/engine/px/ob_px_scheduler.h +++ b/src/sql/engine/px/ob_px_scheduler.h @@ -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::ObSArrayaddrs_; + 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 temp_access_ops_; + ObSEArray 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 *pruning_table_location_; TableAccessType table_access_type_; + ObDetectableId qc_detectable_id_; + // key = p2p_dh_id value = dfo_id + target_addrs + hash::ObHashMap p2p_dfo_map_; + ObTempTableP2PInfo p2p_temp_table_info_; }; class ObDfoSchedulerBasic; diff --git a/src/sql/engine/px/ob_px_sqc_handler.cpp b/src/sql/engine/px/ob_px_sqc_handler.cpp index aef698311..e19ffd714 100644 --- a/src/sql/engine/px/ob_px_sqc_handler.cpp +++ b/src/sql/engine/px/ob_px_sqc_handler.cpp @@ -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; } diff --git a/src/sql/engine/px/ob_px_sqc_handler.h b/src/sql/engine/px/ob_px_sqc_handler.h index 888219b63..bb08aec4e 100644 --- a/src/sql/engine/px/ob_px_sqc_handler.h +++ b/src/sql/engine/px/ob_px_sqc_handler.h @@ -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_; }; } diff --git a/src/sql/engine/px/ob_px_sqc_proxy.cpp b/src/sql/engine/px/ob_px_sqc_proxy.cpp index a981a24d4..eada3a685 100644 --- a/src/sql/engine/px/ob_px_sqc_proxy.cpp +++ b/src/sql/engine/px/ob_px_sqc_proxy.cpp @@ -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 *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; +} diff --git a/src/sql/engine/px/ob_px_sqc_proxy.h b/src/sql/engine/px/ob_px_sqc_proxy.h index dc50cc2d3..9382e0c80 100644 --- a/src/sql/engine/px/ob_px_sqc_proxy.h +++ b/src/sql/engine/px/ob_px_sqc_proxy.h @@ -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 &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 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 *, + 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 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); }; diff --git a/src/sql/engine/px/ob_px_sub_coord.cpp b/src/sql/engine/px/ob_px_sub_coord.cpp index 650e9f59a..9be9592ef 100644 --- a/src/sql/engine/px/ob_px_sub_coord.cpp +++ b/src/sql/engine/px/ob_px_sub_coord.cpp @@ -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 &tsc_location_keys) { int ret = OB_SUCCESS; - if (IS_PX_BLOOM_FILTER(root.get_type())) { - ObJoinFilterSpec &filter_op = reinterpret_cast(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(&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(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(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 *array_ptr = + reinterpret_cast *>(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; diff --git a/src/sql/engine/px/ob_px_sub_coord.h b/src/sql/engine/px/ob_px_sub_coord.h index c424c156c..a90859099 100644 --- a/src/sql/engine/px/ob_px_sub_coord.h +++ b/src/sql/engine/px/ob_px_sub_coord.h @@ -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> &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 rf_msgs_; // for clear DISALLOW_COPY_AND_ASSIGN(ObPxSubCoord); }; } diff --git a/src/sql/engine/px/ob_px_task_process.cpp b/src/sql/engine/px/ob_px_task_process.cpp index d56b37a45..87a7a503c 100644 --- a/src/sql/engine/px/ob_px_task_process.cpp +++ b/src/sql/engine/px/ob_px_task_process.cpp @@ -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(&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(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_); } diff --git a/src/sql/engine/px/ob_px_util.cpp b/src/sql/engine/px/ob_px_util.cpp index 9521d4611..47616bf8b 100644 --- a/src/sql/engine/px/ob_px_util.cpp +++ b/src/sql/engine/px/ob_px_util.cpp @@ -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 -static int get_location_addrs(const T &locations, - ObIArray &addrs) + +int ObPXServerAddrUtil::generate_dh_map_info(ObDfo &dfo) { int ret = OB_SUCCESS; - hash::ObHashSet 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 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 scan_ops; @@ -544,6 +576,8 @@ int ObPXServerAddrUtil::alloc_by_temp_child_distribution_inner(ObExecContext &ex ObArray 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; diff --git a/src/sql/engine/px/ob_px_util.h b/src/sql/engine/px/ob_px_util.h index f3ec5fa08..f78c10f06 100644 --- a/src/sql/engine/px/ob_px_util.h +++ b/src/sql/engine/px/ob_px_util.h @@ -247,6 +247,8 @@ private: static int assign_external_files_to_sqc(const common::ObIArray &files, bool is_file_on_disk, common::ObIArray &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 +static int get_location_addrs(const T &locations, + ObIArray &addrs) +{ + int ret = OB_SUCCESS; + hash::ObHashSet 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; +} + } } diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp new file mode 100644 index 000000000..663db0595 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp @@ -0,0 +1,376 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h" +#include "lib/rc/context.h" +#include "sql/engine/px/ob_px_sqc_proxy.h" +#include "share/ob_rpc_share.h" +#include "share/detect/ob_detect_manager_utils.h" + +using namespace oceanbase; +using namespace oceanbase::common; +using namespace oceanbase::share; +using namespace oceanbase::sql; +using namespace oceanbase::obrpc; + +ObP2PDatahubManager &ObP2PDatahubManager::instance() +{ + static ObP2PDatahubManager the_p2p_dh_mgr; + return the_p2p_dh_mgr; +} + +int ObP2PDatahubManager::init() +{ + int ret = OB_SUCCESS; + if (IS_INIT) { + ret = OB_INIT_TWICE; + LOG_WARN("no need to init twice datahub manager", K(ret)); + } else if (OB_FAIL(map_.create(BUCKET_NUM, + "PxP2PDhMgrKey", + "PxP2PDhMgrNode"))) { + LOG_WARN("create hash table failed", K(ret)); + } else if (OB_FAIL(share::init_obrpc_proxy(p2p_dh_proxy_))) { + LOG_WARN("fail to init obrpc proxy", K(ret)); + } else { + is_inited_ = true; + } + return ret; +} + +void ObP2PDatahubManager::destroy() +{ + if (IS_INIT) { + map_.destroy(); + } +} + +int ObP2PDatahubManager::process_msg(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + ObP2PDatahubMsgBase *new_msg = nullptr; + bool need_free = false; + if (!msg.is_valid_type()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid type", K(ret), K(msg.get_msg_type())); + } else if (OB_FAIL(deep_copy_msg(msg, new_msg))) { + need_free = true; + LOG_WARN("fail to copy msg", K(ret)); + } else if (OB_ISNULL(new_msg)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected new msg", K(ret)); + } else if (OB_FAIL(new_msg->process_msg_internal(need_free))) { + need_free = true; + LOG_WARN("fail to process msg", K(ret)); + } + if (need_free && OB_NOT_NULL(new_msg)) { + new_msg->destroy(); + ob_free(new_msg); + new_msg = nullptr; + } + return ret; +} + +template +int ObP2PDatahubManager::alloc_msg(int64_t tenant_id, T *&msg_ptr) +{ + int ret = OB_SUCCESS; + void *ptr = nullptr; + ObMemAttr attr(tenant_id, "PxP2PDhMsg", common::ObCtxIds::DEFAULT_CTX_ID); + if (OB_ISNULL(ptr = (ob_malloc(sizeof(T), attr)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc memory for p2p dh msg", K(ret)); + } else { + msg_ptr = new(ptr) T(); + } + return ret; +} + +template +int ObP2PDatahubManager::alloc_msg( + common::ObIAllocator &allocator, + T *&msg_ptr) +{ + int ret = OB_SUCCESS; + void *ptr = nullptr; + if (OB_ISNULL(ptr = (allocator.alloc(sizeof(T))))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc memory for p2p dh msg", K(ret)); + } else { + msg_ptr = new(ptr) T(); + } + return ret; +} + +int ObP2PDatahubManager::alloc_msg( + common::ObIAllocator &allocator, + ObP2PDatahubMsgBase::ObP2PDatahubMsgType type, + ObP2PDatahubMsgBase *&msg_ptr) +{ + int ret = OB_SUCCESS; + switch(type) { + case ObP2PDatahubMsgBase::BLOOM_FILTER_MSG: { + ObRFBloomFilterMsg *bf_ptr = nullptr; + if (OB_FAIL(alloc_msg(allocator, bf_ptr))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else { + msg_ptr = bf_ptr; + } + break; + } + case ObP2PDatahubMsgBase::RANGE_FILTER_MSG: { + ObRFRangeFilterMsg *range_ptr = nullptr; + if (OB_FAIL(alloc_msg(allocator, range_ptr))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else { + msg_ptr = range_ptr; + } + break; + } + case ObP2PDatahubMsgBase::IN_FILTER_MSG: { + ObRFInFilterMsg *in_ptr = nullptr; + if (OB_FAIL(alloc_msg(allocator, in_ptr))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else { + msg_ptr = in_ptr; + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected type", K(type), K(ret)); + } + } + if (OB_SUCC(ret) && OB_NOT_NULL(msg_ptr)) { + msg_ptr->set_msg_type(type); + } + return ret; +} + +void ObP2PDatahubManager::free_msg(ObP2PDatahubMsgBase *&msg) +{ + if (OB_NOT_NULL(msg)) { + msg->destroy(); + ob_free(msg); + msg = nullptr; + } +} + +int ObP2PDatahubManager::deep_copy_msg(ObP2PDatahubMsgBase &msg, ObP2PDatahubMsgBase *&new_msg) +{ + return msg.deep_copy_msg(new_msg); +} + +int ObP2PDatahubManager::P2PMsgMergeCall::operator() (common::hash::HashMapPair &entry) +{ + int ret = OB_SUCCESS; + if (!dh_msg_.is_active()) { + entry.second->set_is_active(false); + } else if (!dh_msg_.is_empty() && (OB_FAIL(entry.second->merge(dh_msg_)))) { + LOG_WARN("fail to merge dh msg", K(ret_), K(entry.first)); + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(entry.second->process_receive_count(dh_msg_))) { + LOG_WARN("fail to process receive count", K(ret)); + } + need_free_ = true; + return ret; +} + +int ObP2PDatahubManager::send_local_msg(ObP2PDatahubMsgBase *msg) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(msg)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("msg is null", K(ret)); + } else { + ObP2PDhKey dh_key(msg->get_p2p_datahub_id(), + msg->get_px_seq_id(), + msg->get_task_id(), + ObTimeUtility::current_time(), msg->get_timeout_ts()); + if (OB_FAIL(map_.set_refactored(dh_key, msg))) { + LOG_TRACE("fail to insert p2p dh msg", K(ret)); + } else { + msg->set_is_ready(true); + } + } + return ret; +} + +int ObP2PDatahubManager::atomic_get_msg(ObP2PDhKey &dh_key, ObP2PDatahubMsgBase *&msg) +{ + int ret = OB_SUCCESS; + P2PMsgGetCall call(msg); + if (OB_FAIL(map_.read_atomic(dh_key, call))) { + LOG_TRACE("fail to get p2p msg in PX_P2P_DH", K(ret)); + } else if (OB_SUCCESS != call.ret_) { + ret = call.ret_; + LOG_TRACE("fail to get p2p msg in PX_P2P_DH", K(ret)); + } + return ret; +} + +int ObP2PDatahubManager::set_msg(ObP2PDhKey &dh_key, ObP2PDatahubMsgBase *&msg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(map_.set_refactored(dh_key, msg))) { + LOG_WARN("fail to insert p2p dh msg", K(ret)); + } + return ret; +} + +int ObP2PDatahubManager::erase_msg(ObP2PDhKey &dh_key, + ObP2PDatahubMsgBase *&msg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(map_.erase_refactored(dh_key, &msg))) { + LOG_TRACE("fail to erase from map", K(ret)); + } + return ret; +} + +int ObP2PDatahubManager::erase_msg_if(ObP2PDhKey &dh_key, + ObP2PDatahubMsgBase *&msg, bool& is_erased, bool need_unreg_dm) +{ + int ret = OB_SUCCESS; + P2PMsgEraseIfCall erase_if_call; + if (OB_FAIL(map_.erase_if(dh_key, erase_if_call, is_erased, &msg))) { + LOG_TRACE("fail to erase if from map", K(ret)); + } else if (is_erased && OB_NOT_NULL(msg)) { + if (need_unreg_dm) { + ObDetectManagerUtils::p2p_datahub_unregister_check_item_from_dm( + msg->get_register_dm_info().detectable_id_, msg->get_dm_cb_node_seq_id()); + } + PX_P2P_DH.free_msg(msg); + } + return ret; +} + +int ObP2PDatahubManager::generate_p2p_dh_id(int64_t &p2p_dh_id) +{ + int ret = OB_SUCCESS; + // generate p2p dh id + // | <16> | <28> | 20 + // server_id timestamp sequence + if (0 >= GCTX.server_id_) { + ret = OB_SERVER_IS_INIT; + LOG_WARN("server id is unexpected", K(ret)); + } else { + const uint64_t svr_id = GCTX.server_id_; + int64_t ts = (common::ObTimeUtility::current_time() / 1000000) << 20; + int64_t seq_id = ATOMIC_AAF(&p2p_dh_id_, 1); + p2p_dh_id = (ts & 0x0000FFFFFFFFFFFF) | (svr_id << 48) | seq_id; + } + return ret; +} + +int ObP2PDatahubManager::send_p2p_msg( + ObP2PDatahubMsgBase &msg, + ObPxSQCProxy &sqc_proxy) +{ + int ret = OB_SUCCESS; + int64_t p2p_dh_id = msg.get_p2p_datahub_id(); + ObPxSQCProxy::SQCP2PDhMap &dh_map = sqc_proxy.get_p2p_dh_map(); + ObSArray *target_addrs = nullptr; + if (OB_FAIL(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.get_msg_receive_expect_cnt()) { + ObP2PDatahubMsgBase *new_msg = nullptr; + if (OB_FAIL(deep_copy_msg(msg, new_msg))) { + LOG_WARN("fail to copy msg", K(ret)); + } else if (OB_ISNULL(new_msg)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpeceted new msg", K(ret)); + } + if (OB_SUCC(ret)) { + ObP2PDatahubMsgGuard guard(new_msg); + if (OB_FAIL(send_local_msg(new_msg))) { + // set failed, which means final_msg is not exists in dh map, let it go + guard.release(); + if (ret == OB_HASH_EXIST) { + // it's ok. + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to do local msg send", K(ret)); + } + if (OB_NOT_NULL(new_msg)) { + new_msg->destroy(); + ob_free(new_msg); + } + } else { + ObP2PDhKey dh_key(new_msg->get_p2p_datahub_id(), + new_msg->get_px_seq_id(), + new_msg->get_task_id(), + ObTimeUtility::current_time(), new_msg->get_timeout_ts()); + int reg_ret = ObDetectManagerUtils::p2p_datahub_register_check_item_into_dm( + new_msg->get_register_dm_info(), dh_key, new_msg->get_dm_cb_node_seq_id()); + if (OB_SUCCESS != reg_ret) { + LOG_WARN("[DM] failed to register check item to dm", K(reg_ret)); + } + LOG_TRACE("[DM] p2p dh register check item to dm", K(reg_ret), K(new_msg->get_register_dm_info()), + K(dh_key), K(new_msg->get_dm_cb_node_seq_id()), K(new_msg)); + } + } + } else if (OB_FAIL(msg.broadcast(*target_addrs, p2p_dh_proxy_))) { + LOG_WARN("fail to do broadcast"); + } + return ret; +} + +int ObP2PDatahubManager::send_local_p2p_msg(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(send_local_msg(&msg))) { + if (OB_HASH_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to send local msg", K(ret)); + } + } + return ret; +} + +void ObP2PDatahubManager::P2PMsgGetCall::operator() (common::hash::HashMapPair &entry) +{ + dh_msg_ = entry.second; + if (OB_NOT_NULL(dh_msg_)) { + dh_msg_->inc_ref_count(); + } else { + int ret = OB_UNEXPECT_INTERNAL_ERROR; + LOG_WARN("dh_msg_ is null", K(ret)); + } +} + +bool ObP2PDatahubManager::P2PMsgEraseIfCall::operator() (common::hash::HashMapPair &entry) +{ + bool need_erase = false; + if (OB_NOT_NULL(entry.second)) { + // only if the ref count is 1, we can decrease ref count to 0 and erase it from map + if (1 == entry.second->cas_ref_count(1, 0)) { + need_erase = true; + } + } else { + int ret = OB_UNEXPECT_INTERNAL_ERROR; + LOG_WARN("dh_msg_ is null", K(ret)); + } + return need_erase; +} diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h new file mode 100644 index 000000000..7db846567 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h @@ -0,0 +1,107 @@ +/** + * 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 __SQL_ENG_P2P_DH_MGR_H__ +#define __SQL_ENG_P2P_DH_MGR_H__ +#include "lib/ob_define.h" +#include "lib/hash/ob_hashmap.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObPxSQCProxy; + +class ObP2PDatahubManager +{ +public: + struct P2PMsgMergeCall + { + P2PMsgMergeCall(ObP2PDatahubMsgBase &db_msg) : dh_msg_(db_msg), need_free_(false) {}; + ~P2PMsgMergeCall() = default; + int operator() (common::hash::HashMapPair &entry); + int ret_; + ObP2PDatahubMsgBase &dh_msg_; + bool need_free_; + }; + struct P2PMsgGetCall + { + P2PMsgGetCall(ObP2PDatahubMsgBase *&db_msg) : dh_msg_(db_msg), ret_(OB_SUCCESS) {}; + ~P2PMsgGetCall() = default; + void operator() (common::hash::HashMapPair &entry); + ObP2PDatahubMsgBase *&dh_msg_; + int ret_; + }; + + struct P2PMsgEraseIfCall + { + P2PMsgEraseIfCall() : ret_(OB_SUCCESS) {}; + ~P2PMsgEraseIfCall() = default; + bool operator() (common::hash::HashMapPair &entry); + int ret_; + }; + +public: + ObP2PDatahubManager() : map_(), is_inited_(false), + p2p_dh_proxy_(), p2p_dh_id_(0) + {} + ~ObP2PDatahubManager() { destroy(); } + static ObP2PDatahubManager &instance(); + typedef common::hash::ObHashMap MsgMap; + int init(); + void destroy(); + int process_msg(ObP2PDatahubMsgBase &msg); + int send_p2p_msg( + ObP2PDatahubMsgBase &msg, + ObPxSQCProxy &sqc_proxy); + int send_local_p2p_msg(ObP2PDatahubMsgBase &msg); + template + int alloc_msg(int64_t tenant_id, T *&msg_ptr); + + int alloc_msg(common::ObIAllocator &allocator, + ObP2PDatahubMsgBase::ObP2PDatahubMsgType type, + ObP2PDatahubMsgBase *&msg_ptr); + int send_local_msg(ObP2PDatahubMsgBase *msg); + int atomic_get_msg(ObP2PDhKey &dh_key, ObP2PDatahubMsgBase *&msg); + int set_msg(ObP2PDhKey &dh_key, ObP2PDatahubMsgBase *&msg); + int erase_msg(ObP2PDhKey &dh_key, ObP2PDatahubMsgBase *&msg); + int erase_msg_if(ObP2PDhKey &dh_key, ObP2PDatahubMsgBase *&msg, bool& is_erased, bool need_unreg_dm=true); + MsgMap &get_map() { return map_; } + int deep_copy_msg(ObP2PDatahubMsgBase &msg, ObP2PDatahubMsgBase *&new_msg); + void free_msg(ObP2PDatahubMsgBase *&msg); + obrpc::ObP2PDhRpcProxy &get_proxy() { return p2p_dh_proxy_; } + int generate_p2p_dh_id(int64_t &p2p_dh_id); +private: + template + int alloc_msg(common::ObIAllocator &allocator, + T *&msg_ptr); +private: + static const int64_t BUCKET_NUM = 131072; //2^17 +private: + MsgMap map_; + bool is_inited_; + obrpc::ObP2PDhRpcProxy p2p_dh_proxy_; + int64_t p2p_dh_id_; +private: + DISALLOW_COPY_AND_ASSIGN(ObP2PDatahubManager); +}; + +#define PX_P2P_DH (::oceanbase::sql::ObP2PDatahubManager::instance()) + +} //end sql; +} //end oceanbase + + +#endif diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.cpp b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.cpp new file mode 100644 index 000000000..9d4d67686 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.cpp @@ -0,0 +1,170 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "ob_p2p_dh_msg.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "share/detect/ob_detect_manager_utils.h" +using namespace oceanbase; +using namespace common; +using namespace sql; + +OB_SERIALIZE_MEMBER(ObP2PDatahubMsgBase, + trace_id_, p2p_datahub_id_, px_sequence_id_, + task_id_, tenant_id_, timeout_ts_, msg_type_, + msg_receive_cur_cnt_, msg_receive_expect_cnt_, + is_active_, is_empty_, register_dm_info_); + +int ObP2PDatahubMsgBase::broadcast( + ObIArray &target_addrs, + obrpc::ObP2PDhRpcProxy &p2p_dh_proxy) +{ + int ret = OB_SUCCESS; + ObPxP2PDatahubArg arg; + arg.msg_ = this; + for (int i = 0; i < target_addrs.count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(p2p_dh_proxy. + to(target_addrs.at(i)). + by(tenant_id_). + timeout(timeout_ts_). + send_p2p_dh_message(arg, nullptr))) { + LOG_WARN("fail to send p2p2 dh msg", K(ret)); + } + } + return ret; +} + +int ObP2PDatahubMsgBase::init(int64_t p2p_dh_id, + int64_t px_sequence_id, int64_t task_id, + int64_t tenant_id, int64_t timeout_ts, + const ObRegisterDmInfo ®ister_dm_info) +{ + int ret = OB_SUCCESS; + trace_id_ = *ObCurTraceId::get_trace_id(); + p2p_datahub_id_ = p2p_dh_id; + px_sequence_id_ = px_sequence_id; + task_id_ = task_id; + tenant_id_ = tenant_id; + timeout_ts_ = timeout_ts; + is_active_ = true; + is_ready_ = false; + is_empty_ = true; + allocator_.set_tenant_id(tenant_id); + allocator_.set_label("ObP2PDHMsg"); + register_dm_info_ = register_dm_info; + return ret; +} + +int ObP2PDatahubMsgBase::assign(const ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + trace_id_ = msg.get_trace_id(); + p2p_datahub_id_ = msg.get_p2p_datahub_id(); + px_sequence_id_ = msg.get_px_seq_id(); + task_id_ = msg.get_task_id(); + tenant_id_ = msg.get_tenant_id(); + timeout_ts_ = msg.get_timeout_ts(); + msg_type_ = msg.get_msg_type(); + is_active_ = msg.is_active(); + is_ready_ = msg.check_ready(); + is_empty_ = msg.is_empty(); + msg_receive_cur_cnt_ = msg.get_msg_receive_cur_cnt(); + msg_receive_expect_cnt_ = msg.get_msg_receive_expect_cnt(); + allocator_.set_tenant_id(tenant_id_); + allocator_.set_label("ObP2PDHMsg"); + register_dm_info_ = msg.register_dm_info_; + return ret; +} + +int ObP2PDatahubMsgBase::process_receive_count(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + CK(msg.get_msg_receive_expect_cnt() > 0 && msg_receive_expect_cnt_ > 0); + if (OB_SUCC(ret)) { + ATOMIC_AAF(&msg_receive_cur_cnt_, msg.get_msg_receive_cur_cnt()); + if (msg_receive_cur_cnt_ > msg_receive_expect_cnt_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected receive count", K(ret)); + } + check_finish_receive(); + } + return ret; +} + +void ObP2PDatahubMsgBase::check_finish_receive() +{ + if (is_active_) { + if (msg_receive_cur_cnt_ == msg_receive_expect_cnt_) { + is_ready_ = true; + } + } +} + +int ObP2PDatahubMsgBase::process_msg_internal(bool &need_free) +{ + int ret = OB_SUCCESS; + ObP2PDhKey dh_key(p2p_datahub_id_, px_sequence_id_, task_id_); + ObP2PDatahubManager::P2PMsgMergeCall call(*this); + ObP2PDatahubManager::MsgMap &map = PX_P2P_DH.get_map(); + start_time_ = ObTimeUtility::current_time(); + ObP2PDatahubMsgGuard guard(this); + do { + if (OB_HASH_EXIST == (ret = map.set_refactored(dh_key, this))) { + if (OB_FAIL(map.atomic_refactored(dh_key, call))) { + if (OB_HASH_NOT_EXIST != ret) { + LOG_WARN("fail to merge p2p dh msg", K(ret)); + } + } + } else if (OB_SUCCESS == ret) { + // set_refactored success, means this msg is in map, so register check item into dm + int reg_ret = ObDetectManagerUtils::p2p_datahub_register_check_item_into_dm(register_dm_info_, + dh_key, dm_cb_node_seq_id_); + if (OB_SUCCESS != reg_ret) { + LOG_WARN("[DM] failed to register check item to dm", K(reg_ret)); + } + LOG_TRACE("[DM] p2p msg register check item to dm", K(reg_ret), K(register_dm_info_), + K(dh_key), K(dm_cb_node_seq_id_), K(this)); + } + } while (ret == OB_HASH_NOT_EXIST); + if (call.need_free_) { + need_free = true; + // msg not in map, dec ref count + guard.dec_msg_ref_count(); + } + return ret; +} + +ObP2PDatahubMsgGuard::ObP2PDatahubMsgGuard(ObP2PDatahubMsgBase *msg) : msg_(msg) +{ + // one for dh map hold msg and one for we use msg to reg dm + msg->inc_ref_count(2); +} + +ObP2PDatahubMsgGuard::~ObP2PDatahubMsgGuard() +{ + dec_msg_ref_count(); +} + +void ObP2PDatahubMsgGuard::release() +{ + msg_ = nullptr; +} + +void ObP2PDatahubMsgGuard::dec_msg_ref_count() +{ + if (OB_NOT_NULL(msg_)) { + msg_->dec_ref_count(); + } +} diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h new file mode 100644 index 000000000..007e2d4b9 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h @@ -0,0 +1,162 @@ +/** + * 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 __SQL_ENG_P2P_DH_MSG_H__ +#define __SQL_ENG_P2P_DH_MSG_H__ +#include "lib/ob_define.h" +#include "lib/hash/ob_hashmap.h" +#include "lib/container/ob_iarray.h" +#include "lib/allocator/page_arena.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h" +#include "sql/engine/expr/ob_expr_join_filter.h" +#include "share/detect/ob_detectable_id.h" + + +namespace oceanbase +{ +namespace sql +{ + +class ObBatchRows; +class ObP2PDatahubMsgBase +{ + OB_UNIS_VERSION_V(1); +public: + enum ObP2PDatahubMsgType + { + NOT_INIT = 0, + BLOOM_FILTER_MSG = 1, + RANGE_FILTER_MSG = 2, + IN_FILTER_MSG = 3, + MAX_TYPE = 4, + }; +public: + ObP2PDatahubMsgBase() : trace_id_(), p2p_datahub_id_(OB_INVALID_ID), + px_sequence_id_(OB_INVALID_ID), task_id_(OB_INVALID_ID), + tenant_id_(OB_INVALID_ID), timeout_ts_(0), + start_time_(0), msg_type_(NOT_INIT), + lock_(), allocator_(), msg_receive_cur_cnt_(0), + msg_receive_expect_cnt_(0), is_active_(true), + is_ready_(false), is_empty_(true), ref_count_(0), + register_dm_info_(), dm_cb_node_seq_id_(0) {} + virtual ~ObP2PDatahubMsgBase() {} + virtual int assign(const ObP2PDatahubMsgBase &); + virtual int merge(ObP2PDatahubMsgBase &) = 0; + virtual int deep_copy_msg(ObP2PDatahubMsgBase *&new_msg_ptr) = 0; + virtual int broadcast( + ObIArray &target_addrs, + obrpc::ObP2PDhRpcProxy &p2p_dh_proxy); + virtual int might_contain(const ObExpr &expr, + ObEvalCtx &ctx, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx, + ObDatum &res) + { return OB_SUCCESS; } + virtual int might_contain_batch( + const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const int64_t batch_size, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx) + { return OB_SUCCESS; } + virtual int insert_by_row( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs_, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx) + { return OB_SUCCESS; } + virtual int insert_by_row_batch( + const ObBatchRows *child_brs, + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t *batch_hash_values + ) { return OB_SUCCESS; } + virtual int insert_by_batch() { return OB_SUCCESS; } + virtual int destroy() = 0; + virtual int process_receive_count(ObP2PDatahubMsgBase &); + virtual int process_msg_internal(bool &need_free); + virtual int reuse() { return OB_SUCCESS; } + void check_finish_receive(); + bool check_ready() const { return is_ready_; } + ObP2PDatahubMsgType get_msg_type() const { return msg_type_; } + void set_msg_type(ObP2PDatahubMsgType type) { msg_type_ = type; } + int64_t get_p2p_datahub_id() const { return p2p_datahub_id_; } + int64_t get_tenant_id() const { return tenant_id_; } + int64_t get_timeout_ts() const { return timeout_ts_; } + void reset_status() { + is_active_ = true; + is_ready_ = false; + is_empty_ = true; + } + bool is_active() const { return is_active_; } + void set_is_active(bool flag) { is_active_ = flag; } + bool is_empty() const { return is_empty_; } + int init(int64_t p2p_dh_id, int64_t px_seq_id, int64_t task_id, + int64_t tenant_id, int64_t timeout_ts, const ObRegisterDmInfo ®ister_dm_info); + common::ObIAllocator &get_allocator() { return allocator_; } + int64_t get_task_id() const { return task_id_; } + void set_is_ready(bool flag) { is_ready_ = flag; } + int64_t get_msg_receive_expect_cnt() const { return msg_receive_expect_cnt_;}; + int64_t get_msg_receive_cur_cnt() const { return msg_receive_cur_cnt_; } + void set_msg_cur_cnt(int64_t cnt) { msg_receive_cur_cnt_ = cnt; } + void set_msg_expect_cnt(int64_t cnt) { msg_receive_expect_cnt_ = cnt; } + bool is_valid_type() { return NOT_INIT < msg_type_ < MAX_TYPE; } + common::ObCurTraceId::TraceId get_trace_id() const { return trace_id_; } + int64_t get_start_time() const { return start_time_; } + int64_t get_px_seq_id() const { return px_sequence_id_; } + // in the following two scenes that ref_count_ should been increased + // 1. if it's a shared msg, and insert into PX_P2P_DH by rpc thread (or only one server int the p2pmap, insert by local thread) + // 2. someone use PX_P2P_DH.atomic_get_msg to get msg ptr + void inc_ref_count(int64_t count=1) { ATOMIC_AAF(&ref_count_, count); } + // appear in pairs with increase + int64_t dec_ref_count() { return ATOMIC_SAF(&ref_count_, 1); } + int64_t cas_ref_count(int64_t expect, int64_t new_val) { return ATOMIC_CAS(&ref_count_, expect, new_val); } + const ObRegisterDmInfo &get_register_dm_info() { return register_dm_info_; } + uint64_t &get_dm_cb_node_seq_id() { return dm_cb_node_seq_id_; } + TO_STRING_KV(K(p2p_datahub_id_), K_(px_sequence_id), K(tenant_id_), K(timeout_ts_), K(is_active_), K(msg_type_)); +protected: + common::ObCurTraceId::TraceId trace_id_; + int64_t p2p_datahub_id_; + int64_t px_sequence_id_; + int64_t task_id_; + int64_t tenant_id_; + int64_t timeout_ts_; + int64_t start_time_; + ObP2PDatahubMsgType msg_type_; + mutable common::ObSpinLock lock_; + common::ObArenaAllocator allocator_; + int64_t msg_receive_cur_cnt_; + int64_t msg_receive_expect_cnt_; + bool is_active_; + bool is_ready_; + bool is_empty_; + int64_t ref_count_; + ObRegisterDmInfo register_dm_info_; + uint64_t dm_cb_node_seq_id_; + DISALLOW_COPY_AND_ASSIGN(ObP2PDatahubMsgBase); +}; + +// guard for set msg into PX_P2P_DH map and register into dm +struct ObP2PDatahubMsgGuard +{ + ObP2PDatahubMsgGuard(ObP2PDatahubMsgBase *msg); + ~ObP2PDatahubMsgGuard(); + void dec_msg_ref_count(); + void release(); + ObP2PDatahubMsgBase *msg_; +}; + +} +} + + +#endif diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.cpp b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.cpp new file mode 100644 index 000000000..93f9fbaa6 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.cpp @@ -0,0 +1,60 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" + +using namespace oceanbase; +using namespace common; +using namespace sql; +using namespace obrpc; + + +int ObPxP2pDhMsgP::process() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(arg_.msg_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to process px p2p datahub msg", K(ret), K(arg_)); + } else if (OB_FAIL(PX_P2P_DH.process_msg(*arg_.msg_))) { + LOG_WARN("failed to process px p2p datahub msg on dh mgr", K(ret)); + } + return ret; +} + +void ObPxP2pDhMsgCB::on_timeout() +{ + int ret = OB_SUCCESS; + LOG_WARN("failed to send p2p datahub message,", K(get_error()), K(start_time_), + K(timeout_ts_), K(addr_), K(trace_id_)); +} + + +int ObPxP2pDhClearMsgP::process() +{ + int ret = OB_SUCCESS; + ObIArray &array = arg_.p2p_dh_ids_; + ObP2PDhKey key; + ObP2PDatahubMsgBase *msg = nullptr; + for (int i = 0; i < array.count(); ++i) { + ret = OB_SUCCESS; + key.p2p_datahub_id_ = array.at(i); + key.task_id_ = 0; + key.px_sequence_id_ = arg_.px_seq_id_; + bool is_erased = false; + if (OB_FAIL(PX_P2P_DH.erase_msg_if(key, msg, is_erased, true/* need unregister dm */)) || !is_erased) { + LOG_TRACE("fail to erase msg", K(ret)); + } + } + return ret; +} diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.h b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.h new file mode 100644 index 000000000..75ab16ef9 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.h @@ -0,0 +1,92 @@ +/** + * 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 __SQL_ENG_P2P_DH_RPC_PROCESS_H__ +#define __SQL_ENG_P2P_DH_RPC_PROCESS_H__ +#include "rpc/obrpc/ob_rpc_processor.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h" +#include "observer/ob_server_struct.h" +#include "lib/ob_define.h" +namespace oceanbase { +namespace sql { + +class ObPxP2pDhMsgP + : public obrpc::ObRpcProcessor> +{ +public: + ObPxP2pDhMsgP(const observer::ObGlobalContext &gctx) { UNUSED(gctx);} + virtual ~ObPxP2pDhMsgP() = default; + //virtual int init() final; + //virtual void destroy() final; + virtual int process() final; + DISALLOW_COPY_AND_ASSIGN(ObPxP2pDhMsgP); +}; + +class ObPxP2pDhMsgCB + : public obrpc::ObP2PDhRpcProxy::AsyncCB +{ +public: + ObPxP2pDhMsgCB(const common::ObAddr &server, + const common::ObCurTraceId::TraceId &trace_id, + int64_t start_time, + int64_t timeout_ts, + int64_t p2p_datahub_id) + : addr_(server), + start_time_(start_time), + timeout_ts_(timeout_ts), + p2p_datahub_id_(p2p_datahub_id) + { + trace_id_.set(trace_id); + } + virtual ~ObPxP2pDhMsgCB() {} +public: + virtual int process() { return OB_SUCCESS; } + virtual void on_invalid() {} + virtual void on_timeout(); + 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) ObPxP2pDhMsgCB(addr_, trace_id_, + start_time_, timeout_ts_, p2p_datahub_id_); + } + return newcb; + } + virtual void set_args(const Request &arg) { UNUSED(arg); } +private: + common::ObCurTraceId::TraceId trace_id_; + common::ObAddr addr_; + int64_t start_time_; + int64_t timeout_ts_; + int64_t p2p_datahub_id_; + DISALLOW_COPY_AND_ASSIGN(ObPxP2pDhMsgCB); +}; + + +class ObPxP2pDhClearMsgP + : public obrpc::ObRpcProcessor> +{ +public: + ObPxP2pDhClearMsgP(const observer::ObGlobalContext &gctx) { UNUSED(gctx);} + virtual ~ObPxP2pDhClearMsgP() = default; + //virtual int init() final; + //virtual void destroy() final; + virtual int process() final; + DISALLOW_COPY_AND_ASSIGN(ObPxP2pDhClearMsgP); +}; + + +} +} + +#endif diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.cpp b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.cpp new file mode 100644 index 000000000..d372a19b5 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.cpp @@ -0,0 +1,54 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#define USING_LOG_PREFIX SQL_ENG +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" + +using namespace oceanbase::common; +using namespace oceanbase::sql; + +OB_DEF_SERIALIZE(ObPxP2PDatahubArg) +{ + int ret = OB_SUCCESS; + CK(OB_NOT_NULL(msg_)); + OB_UNIS_ENCODE(msg_->get_msg_type()); + OB_UNIS_ENCODE(*msg_); + return ret; +} + +OB_DEF_DESERIALIZE(ObPxP2PDatahubArg) +{ + int ret = OB_SUCCESS; + CK(OB_NOT_NULL(CURRENT_CONTEXT)); + ObP2PDatahubMsgBase::ObP2PDatahubMsgType msg_type = ObP2PDatahubMsgBase::NOT_INIT; + OB_UNIS_DECODE(msg_type); + ObIAllocator &allocator = CURRENT_CONTEXT->get_arena_allocator(); + if (OB_FAIL(PX_P2P_DH.alloc_msg(allocator, msg_type, msg_))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else { + OB_UNIS_DECODE(*msg_); + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObPxP2PDatahubArg) +{ + int64_t len = 0; + + LST_DO_CODE(OB_UNIS_ADD_LEN, msg_->get_msg_type(), *msg_); + return len; +} + +OB_SERIALIZE_MEMBER(ObPxP2PDatahubMsgResponse, rc_); + +OB_SERIALIZE_MEMBER(ObPxP2PClearMsgArg, p2p_dh_ids_, px_seq_id_); diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h new file mode 100644 index 000000000..8640fe9d6 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h @@ -0,0 +1,61 @@ +/** + * 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 OB_P2P_DH_RPC_PROXY_H +#define OB_P2P_DH_RPC_PROXY_H + +#include "rpc/obrpc/ob_rpc_proxy.h" +#include "observer/ob_server_struct.h" +namespace oceanbase { +namespace sql { + +class ObP2PDatahubMsgBase; +struct ObPxP2PDatahubArg +{ + OB_UNIS_VERSION(1); +public: + ObP2PDatahubMsgBase *msg_; + TO_STRING_KV(KP(msg_)); +}; + +struct ObPxP2PDatahubMsgResponse +{ + OB_UNIS_VERSION(1); +public: + int rc_; + TO_STRING_KV(K_(rc)); +}; + +struct ObPxP2PClearMsgArg +{ + OB_UNIS_VERSION(1); +public: + ObSArray p2p_dh_ids_; + int64_t px_seq_id_; + TO_STRING_KV(K(p2p_dh_ids_)); +}; + +} +namespace obrpc { + +class ObP2PDhRpcProxy + : public ObRpcProxy +{ +public: + DEFINE_TO(ObP2PDhRpcProxy); + RPC_AP(PR5 send_p2p_dh_message, OB_PX_P2P_DH_MSG, (sql::ObPxP2PDatahubArg), sql::ObPxP2PDatahubMsgResponse); + RPC_AP(PR5 clear_dh_msg, OB_PX_CLAER_DH_MSG, (sql::ObPxP2PClearMsgArg), sql::ObPxP2PDatahubMsgResponse); +}; + +} +} + +#endif diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h new file mode 100644 index 000000000..ae3e7cf70 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h @@ -0,0 +1,76 @@ +/** + * 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 __SQL_ENG_P2P_DH_SHARE_INFO_H__ +#define __SQL_ENG_P2P_DH_SHARE_INFO_H__ + +#include "lib/ob_define.h" +#include "lib/hash/ob_hashmap.h" +namespace oceanbase +{ +namespace sql +{ + +class ObP2PDatahubManagerHashWrapper +{ + OB_UNIS_VERSION(1); +public: + ObP2PDatahubManagerHashWrapper() : + p2p_datahub_id_(OB_INVALID_ID), px_sequence_id_(OB_INVALID_ID), task_id_(0), + start_time_(0), timeout_ts_(0) {} + ObP2PDatahubManagerHashWrapper(int64_t p2p_datahub_id, int64_t px_sequence_id, int64_t task_id) : + p2p_datahub_id_(p2p_datahub_id), + px_sequence_id_(px_sequence_id), + task_id_(task_id), + start_time_(0), timeout_ts_(0) {} + ObP2PDatahubManagerHashWrapper(int64_t p2p_datahub_id, int64_t px_sequence_id, + int64_t task_id, int64_t start_time, int64_t timeout_ts) : + p2p_datahub_id_(p2p_datahub_id), + px_sequence_id_(px_sequence_id), + task_id_(task_id), + start_time_(start_time), timeout_ts_(timeout_ts) {} + ~ObP2PDatahubManagerHashWrapper() {} + inline bool operator==(const ObP2PDatahubManagerHashWrapper &other) const + { + return p2p_datahub_id_ == other.p2p_datahub_id_ && + px_sequence_id_ == other.px_sequence_id_ && + task_id_ == other.task_id_; + } + inline uint64_t hash() const; + int hash(uint64_t &hash_val) const { hash_val = hash(); return OB_SUCCESS; } + + // generated by the optimizer, + // a unique identifier represents data that needs to be accessed via P2P Datahub. + int64_t p2p_datahub_id_; + // The use of PX ID as a unique dynamic identifier during execution, + // in combination with P2P Datahub ID, ensures uniqueness. + int64_t px_sequence_id_; + // task id is used for partition wise + int64_t task_id_; + int64_t start_time_; + int64_t timeout_ts_; + TO_STRING_KV(K_(p2p_datahub_id), K_(px_sequence_id), K_(task_id), K_(start_time), K_(timeout_ts)); +}; + +typedef ObP2PDatahubManagerHashWrapper ObP2PDhKey; +inline uint64_t ObP2PDatahubManagerHashWrapper::hash() const +{ + uint64_t hash_ret = 0; + hash_ret = common::murmurhash(&p2p_datahub_id_, sizeof(uint64_t), 0); + hash_ret = common::murmurhash(&px_sequence_id_, sizeof(uint64_t), hash_ret); + hash_ret = common::murmurhash(&task_id_, sizeof(uint64_t), hash_ret); + return hash_ret; +} + +} +} + +#endif diff --git a/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.cpp b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.cpp new file mode 100644 index 000000000..99476e3ab --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.cpp @@ -0,0 +1,1482 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG +#include "sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_process.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_rpc_proxy.h" +#include "sql/engine/expr/ob_expr_join_filter.h" +#include "sql/engine/expr/ob_expr_calc_partition_id.h" +#include "sql/engine/ob_operator.h" +#include "share/detect/ob_detect_manager_utils.h" + +using namespace oceanbase::common; +using namespace oceanbase::sql; +using namespace oceanbase::share; + +OB_SERIALIZE_MEMBER((ObRFBloomFilterMsg, ObP2PDatahubMsgBase), + phase_, bloom_filter_, next_peer_addrs_, + expect_first_phase_count_, piece_size_); +OB_SERIALIZE_MEMBER(ObRFRangeFilterMsg::MinMaxCellSize, min_datum_buf_size_, max_datum_buf_size_); + +OB_DEF_SERIALIZE(ObRFRangeFilterMsg) +{ + int ret = OB_SUCCESS; + BASE_SER((ObRFRangeFilterMsg, ObP2PDatahubMsgBase)); + LST_DO_CODE(OB_UNIS_ENCODE, + lower_bounds_, + upper_bounds_, + need_null_cmp_flags_, + cells_size_, + null_first_cmp_funcs_, + null_last_cmp_funcs_); + return ret; +} + +OB_DEF_DESERIALIZE(ObRFRangeFilterMsg) +{ + int ret = OB_SUCCESS; + BASE_DESER((ObRFRangeFilterMsg, ObP2PDatahubMsgBase)); + LST_DO_CODE(OB_UNIS_DECODE, + lower_bounds_, + upper_bounds_, + need_null_cmp_flags_, + cells_size_, + null_first_cmp_funcs_, + null_last_cmp_funcs_); + if (OB_FAIL(adjust_cell_size())) { + LOG_WARN("fail do adjust cell size", K(ret)); + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObRFRangeFilterMsg) +{ + int64_t len = 0; + BASE_ADD_LEN((ObRFInFilterMsg, ObP2PDatahubMsgBase)); + LST_DO_CODE(OB_UNIS_ADD_LEN, + lower_bounds_, + upper_bounds_, + need_null_cmp_flags_, + cells_size_, + null_first_cmp_funcs_, + null_last_cmp_funcs_); + return len; +} + +OB_DEF_SERIALIZE(ObRFInFilterMsg) +{ + int ret = OB_SUCCESS; + BASE_SER((ObRFInFilterMsg, ObP2PDatahubMsgBase)); + int cnt = is_active_? serial_rows_.count() : 0; + OB_UNIS_ENCODE(cnt); + OB_UNIS_ENCODE(cmp_funcs_); + OB_UNIS_ENCODE(hash_funcs_); + OB_UNIS_ENCODE(col_cnt_); + OB_UNIS_ENCODE(max_in_num_); + OB_UNIS_ENCODE(need_null_cmp_flags_); + if (is_active_) { + for (int i = 0; OB_SUCC(ret) && i < serial_rows_.count(); ++i) { + if (OB_FAIL(serial_rows_.at(i)->serialize(buf, buf_len, pos))) { + LOG_WARN("fail to serialize rows", K(ret)); + } + } + } + return ret; +} + +OB_DEF_DESERIALIZE(ObRFInFilterMsg) +{ + int ret = OB_SUCCESS; + int64_t row_cnt = 0; + BASE_DESER((ObRFInFilterMsg, ObP2PDatahubMsgBase)); + OB_UNIS_DECODE(row_cnt); + OB_UNIS_DECODE(cmp_funcs_); + OB_UNIS_DECODE(hash_funcs_); + OB_UNIS_DECODE(col_cnt_); + OB_UNIS_DECODE(max_in_num_); + OB_UNIS_DECODE(need_null_cmp_flags_); + if (OB_SUCC(ret) && is_active_) { + ObFixedArray *new_row = nullptr; + void *array_ptr = nullptr; + int64_t buckets_cnt = max(row_cnt, 1); + if (OB_FAIL(serial_rows_.reserve(row_cnt))) { + LOG_WARN("fail to init row cnt", K(ret)); + } else if (OB_FAIL(rows_set_.create(buckets_cnt * 2, + "RFDEInFilter", + "RFDEInFilter"))) { + LOG_WARN("fail to init in hash set", K(ret)); + } else if (OB_FAIL(cur_row_.prepare_allocate(col_cnt_))) { + LOG_WARN("fail to prepare allocate col cnt datum", K(ret)); + } + for (int i = 0; OB_SUCC(ret) && i < row_cnt; ++i) { + new_row = nullptr; + array_ptr = nullptr; + if (OB_ISNULL(array_ptr = allocator_.alloc(sizeof(ObFixedArray)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret)); + } else { + new_row = new(array_ptr) ObFixedArray(allocator_); + if (OB_FAIL(new_row->deserialize(buf, data_len, pos))) { + LOG_WARN("fail to serialize rows", K(ret)); + } else if (OB_FAIL(serial_rows_.push_back(new_row))) { + LOG_WARN("fail to push back new row", K(ret)); + } else { + ObRFInFilterNode node(&cmp_funcs_, &hash_funcs_, new_row); + if (OB_FAIL(rows_set_.set_refactored(node))) { + LOG_WARN("fail to insert in filter node", K(ret)); + } + } + } + } + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObRFInFilterMsg) +{ + int64_t len = 0; + BASE_ADD_LEN((ObRFInFilterMsg, ObP2PDatahubMsgBase)); + int cnt = is_active_? serial_rows_.count() : 0; + OB_UNIS_ADD_LEN(cnt); + OB_UNIS_ADD_LEN(cmp_funcs_); + OB_UNIS_ADD_LEN(hash_funcs_); + OB_UNIS_ADD_LEN(col_cnt_); + OB_UNIS_ADD_LEN(max_in_num_); + OB_UNIS_ADD_LEN(need_null_cmp_flags_); + if (is_active_) { + for (int i = 0; i < serial_rows_.count(); ++i) { + len += serial_rows_.at(i)->get_serialize_size(); + } + } + return len; +} + + +//ObRFBloomFilterMsg +int ObRFBloomFilterMsg::process_msg_internal(bool &need_free) +{ + int ret = OB_SUCCESS; + ObP2PDhKey dh_key(p2p_datahub_id_, px_sequence_id_, task_id_); + ObP2PDatahubManager::P2PMsgMergeCall call(*this); + ObP2PDatahubManager::MsgMap &map = PX_P2P_DH.get_map(); + start_time_ = ObTimeUtility::current_time(); + if (OB_FAIL(generate_receive_count_array(piece_size_))) { + LOG_WARN("fail to generate receive count array", K(ret)); + } + ObP2PDatahubMsgGuard guard(this); + do { + if (OB_HASH_EXIST == (ret = map.set_refactored(dh_key, this))) { + if (OB_FAIL(map.read_atomic(dh_key, call))) { + if (OB_HASH_NOT_EXIST != ret) { + LOG_WARN("fail to merge p2p dh msg", K(ret)); + } + } + } else if (OB_SUCCESS == ret) { + // set_refactored success, means this msg is in map, so register check item into dm + int reg_ret = ObDetectManagerUtils::p2p_datahub_register_check_item_into_dm(register_dm_info_, + dh_key, dm_cb_node_seq_id_); + if (OB_SUCCESS != reg_ret) { + LOG_WARN("[DM] failed to register check item to dm", K(reg_ret)); + } + LOG_TRACE("[DM] rf register check item to dm", K(reg_ret), K(register_dm_info_), + K(dh_key), K(dm_cb_node_seq_id_), K(this)); + } + } while (ret == OB_HASH_NOT_EXIST); + if (call.need_free_) { + need_free = true; + // msg not in map, dec ref count + guard.dec_msg_ref_count(); + } + return ret; +} + +int ObRFBloomFilterMsg::generate_receive_count_array(int64_t piece_size) +{ + int ret = OB_SUCCESS; + int64_t bits_array_length = bloom_filter_.get_bits_array_length(); + 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 * piece_size; + if (begin_idx >= bits_array_length) { + begin_idx = bits_array_length - 1; + } + OZ(receive_count_array_.push_back(BloomFilterReceiveCount(begin_idx, 0))); + } + return ret; +} + +int ObRFBloomFilterMsg::reuse() +{ + int ret = OB_SUCCESS; + is_empty_ = true; + bloom_filter_.reset_filter(); + return ret; +} + +int ObRFBloomFilterMsg::process_first_phase_recieve_count( + ObRFBloomFilterMsg &msg, bool &first_phase_end) +{ + int ret = OB_SUCCESS; + CK(msg.get_msg_receive_expect_cnt() > 0 && msg_receive_expect_cnt_ > 0); + int64_t begin_idx = msg.bloom_filter_.get_begin_idx(); + ATOMIC_INC(&msg_receive_cur_cnt_); + if (msg_receive_cur_cnt_ > msg_receive_expect_cnt_) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("fail to process recieve count", K(ret), K(msg_receive_cur_cnt_), + K(msg_receive_expect_cnt_)); + } else if (receive_count_array_.empty()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("emptry receive count array", K(ret)); + } else { + bool find = false; + for (int i = 0; OB_SUCC(ret) && i < receive_count_array_.count(); ++i) { + if (begin_idx == receive_count_array_.at(i).begin_idx_) { + int64_t cur_count = ATOMIC_AAF(&receive_count_array_.at(i).reciv_count_, 1); + first_phase_end = (cur_count == expect_first_phase_count_); + find = true; + break; + } + } + if (!find) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected process first phase", K(ret), K(receive_count_array_.count())); + } + } + return ret; +} + +int ObRFBloomFilterMsg::process_receive_count(ObP2PDatahubMsgBase &rf_msg) +{ + int ret = OB_SUCCESS; + bool first_phase_end = false; + ObRFBloomFilterMsg &bf_msg = static_cast(rf_msg); + auto process_second_phase = [&](ObRFBloomFilterMsg &bf_msg) { + if (OB_FAIL(ObP2PDatahubMsgBase::process_receive_count(bf_msg))) { + LOG_WARN("fail to process receive count", K(ret)); + } + return ret; + }; + + auto process_first_phase = [&](ObRFBloomFilterMsg &bf_msg) { + if (OB_FAIL(process_first_phase_recieve_count( + bf_msg, first_phase_end))) { + LOG_WARN("fail to process receive count", K(ret)); + } + return ret; + }; + if (is_first_phase()) { + if (OB_FAIL(process_first_phase(bf_msg))) { + LOG_WARN("fail to process first phase", K(ret)); + } else if (first_phase_end && !bf_msg.get_next_phase_addrs().empty()) { + obrpc::ObP2PDhRpcProxy &rpc_proxy = PX_P2P_DH.get_proxy(); + ObPxP2PDatahubArg arg; + ObRFBloomFilterMsg second_phase_msg; + arg.msg_ = &second_phase_msg; + if (OB_FAIL(second_phase_msg.shadow_copy(*this))) { + LOG_WARN("fail to shadow copy second phase msg", K(ret)); + } else { + second_phase_msg.set_msg_cur_cnt(expect_first_phase_count_); + } + for (int i = 0; OB_SUCC(ret) && i < bf_msg.get_next_phase_addrs().count(); ++i) { + if (bf_msg.get_next_phase_addrs().at(i) != GCTX.self_addr()) { + if (OB_FAIL(rpc_proxy.to(bf_msg.get_next_phase_addrs().at(i)) + .by(bf_msg.get_tenant_id()) + .timeout(bf_msg.get_timeout_ts()) + .compressed(ObCompressorType::LZ4_COMPRESSOR) + .send_p2p_dh_message(arg, NULL))) { + LOG_WARN("fail to send bloom filter", K(ret)); + } + } + } + } else if (bf_msg.get_next_phase_addrs().empty()) { + (void)check_finish_receive(); + } + } else if (OB_FAIL(process_second_phase(bf_msg))) { + LOG_WARN("fail to process second phase", K(ret)); + } + return ret; +} + +int ObRFBloomFilterMsg::deep_copy_msg(ObP2PDatahubMsgBase *&new_msg_ptr) +{ + int ret = OB_SUCCESS; + ObRFBloomFilterMsg *bf_msg = nullptr; + if (OB_FAIL(PX_P2P_DH.alloc_msg(tenant_id_, bf_msg))) { + LOG_WARN("fail to alloc rf msg", K(ret)); + } else if (OB_FAIL(bf_msg->assign(*this))) { + LOG_WARN("fail to assign rf msg", K(ret)); + } else { + new_msg_ptr = bf_msg; + } + return ret; +} + +int ObRFBloomFilterMsg::assign(const ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + const ObRFBloomFilterMsg &other_msg = static_cast(msg); + phase_ = other_msg.phase_; + expect_first_phase_count_ = other_msg.expect_first_phase_count_; + piece_size_ = other_msg.piece_size_; + if (OB_FAIL(ObP2PDatahubMsgBase::assign(msg))) { + LOG_WARN("failed to assign base data", K(ret)); + } else if (OB_FAIL(next_peer_addrs_.assign(other_msg.next_peer_addrs_))) { + LOG_WARN("fail to assign bf msg", K(ret)); + } else if (OB_FAIL(bloom_filter_.assign(other_msg.bloom_filter_))) { + LOG_WARN("fail to assign bf msg", K(ret)); + } else if (OB_FAIL(filter_indexes_.assign(other_msg.filter_indexes_))) { + LOG_WARN("failed to assign filter indexes", K(ret)); + } + return ret; +} + +int ObRFBloomFilterMsg::shadow_copy(const ObRFBloomFilterMsg &other_msg) +{ + int ret = OB_SUCCESS; + phase_ = other_msg.phase_; + expect_first_phase_count_ = other_msg.expect_first_phase_count_; + piece_size_ = other_msg.piece_size_; + if (OB_FAIL(ObP2PDatahubMsgBase::assign(other_msg))) { + LOG_WARN("failed to assign base data", K(ret)); + } else if (OB_FAIL(bloom_filter_.init(&other_msg.bloom_filter_))) { + LOG_WARN("fail to assign bf msg", K(ret)); + } + return ret; +} + +int ObRFBloomFilterMsg::merge(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + ObRFBloomFilterMsg &bf_msg = static_cast(msg); + if (bf_msg.is_empty_) { + } else if (OB_FAIL(bloom_filter_.merge_filter(&bf_msg.bloom_filter_))) { + LOG_WARN("fail to merge bloom filter msg", K(ret)); + } else { + is_empty_ = false; + } + return ret; +} + +int ObRFBloomFilterMsg::destroy() +{ + int ret = OB_SUCCESS; + next_peer_addrs_.reset(); + bloom_filter_.reset(); + filter_indexes_.reset(); + receive_count_array_.reset(); + allocator_.reset(); + return ret; +} + +int ObRFBloomFilterMsg::might_contain(const ObExpr &expr, + ObEvalCtx &ctx, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx, + ObDatum &res) +{ + int ret = OB_SUCCESS; + uint64_t hash_val = ObExprJoinFilter::JOIN_FILTER_SEED; + ObDatum *datum = nullptr; + ObHashFunc hash_func; + if (OB_UNLIKELY(is_empty_)) { + res.set_int(0); + filter_ctx.filter_count_++; + filter_ctx.check_count_++; + } else { + 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( + expr.inner_functions_[GET_FUNC(i, ObExprJoinFilter::HASH_ROW)]); + if (OB_FAIL(hash_func.hash_func_(*datum, hash_val, hash_val))) { + LOG_WARN("fail to calc hash val", K(ret)); + } + } + } + } + if (OB_SUCC(ret)) { + bool is_match = true; + if (OB_FAIL(bloom_filter_.might_contain(hash_val, is_match))) { + LOG_WARN("fail to check filter might contain value", K(ret), K(hash_val)); + } else { + if (!is_match) { + filter_ctx.filter_count_++; + } + filter_ctx.check_count_++; + res.set_int(is_match ? 1 : 0); + ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + } + } + } + return ret; +} + +int ObRFBloomFilterMsg::might_contain_batch( + const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const int64_t batch_size, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx) +{ + int ret = OB_SUCCESS; + bool is_match = true; + uint64_t seed = ObExprJoinFilter::JOIN_FILTER_SEED; + ObDatum *results = expr.locate_batch_datums(ctx); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + uint64_t *hash_values = reinterpret_cast( + ctx.frames_[expr.frame_idx_] + expr.res_buf_off_); + if (OB_UNLIKELY(is_empty_)) { + if (OB_FAIL(ObBitVector::flip_foreach(skip, batch_size, + [&](int64_t idx) __attribute__((always_inline)) { + int ret = OB_SUCCESS; + eval_flags.set(idx); + results[idx].set_int(0); + ++filter_ctx.filter_count_; + ++filter_ctx.check_count_; + ++filter_ctx.total_count_; + return ret; + }))) { + LOG_WARN("fail to do for each operation", K(ret)); + } + } else { + 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( + expr.inner_functions_[GET_FUNC(i, ObExprJoinFilter::HASH_BATCH)]); + 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_.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_.might_contain(hash_values[idx], is_match); + if (OB_SUCC(ret)) { + filter_ctx.filter_count_ += !is_match; + eval_flags.set(idx); + results[idx].set_int(is_match); + ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + ++filter_ctx.check_count_; + ++filter_ctx.total_count_; + } + return ret; + }))) { + LOG_WARN("failed to process prefetch block", K(ret)); + } + } + return ret; +} + +int ObRFBloomFilterMsg::insert_by_row_batch( + const ObBatchRows *child_brs, + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t *batch_hash_values) +{ + int ret = OB_SUCCESS; + if (child_brs->size_ > 0) { + uint64_t seed = ObExprJoinFilter::JOIN_FILTER_SEED; + if (OB_NOT_NULL(calc_tablet_id_expr)) { + if (OB_ISNULL(calc_tablet_id_expr) || hash_funcs.count() != 1) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected part id expr", K(ret)); + } else if (OB_FAIL(calc_tablet_id_expr->eval_batch(eval_ctx, + *(child_brs->skip_), child_brs->size_))) { + LOG_WARN("failed to eval", K(ret)); + } else { + ObBatchDatumHashFunc hash_func_batch = hash_funcs.at(0).batch_hash_func_; + hash_func_batch(batch_hash_values, + calc_tablet_id_expr->locate_batch_datums(eval_ctx), + calc_tablet_id_expr->is_batch_result(), + *child_brs->skip_, child_brs->size_, + &seed, + false); + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < expr_array.count(); ++i) { + ObExpr *expr = expr_array.at(i); // expr ptr check in cg, not check here + if (OB_FAIL(expr->eval_batch(eval_ctx, *(child_brs->skip_), child_brs->size_))) { + LOG_WARN("eval failed", K(ret)); + } else { + ObBatchDatumHashFunc hash_func_batch = hash_funcs.at(i).batch_hash_func_; + const bool is_batch_seed = (i > 0); + hash_func_batch(batch_hash_values, + expr->locate_batch_datums(eval_ctx), expr->is_batch_result(), + *child_brs->skip_, child_brs->size_, + is_batch_seed ? batch_hash_values : &seed, + is_batch_seed); + } + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < child_brs->size_; ++i) { + if (OB_NOT_NULL(calc_tablet_id_expr)) { + ObDatum &datum = calc_tablet_id_expr->locate_expr_datum(eval_ctx, i); + if (ObExprCalcPartitionId::NONE_PARTITION_ID == datum.get_int()) { + continue; + } + } + if (OB_SUCC(ret)) { + if (child_brs->skip_->at(i)) { + continue; + } else if (OB_FAIL(bloom_filter_.put(batch_hash_values[i]))) { + LOG_WARN("fail to put hash value to px bloom filter", K(ret)); + } else if (is_empty_) { + is_empty_ = false; + } + } + } + } + return ret; +} +int ObRFBloomFilterMsg::insert_by_row( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx) +{ + int ret = OB_SUCCESS; + uint64_t hash_value = 0; + bool ignore = false; + if (OB_FAIL(calc_hash_value(expr_array, + hash_funcs, calc_tablet_id_expr, + eval_ctx, hash_value, ignore))) { + LOG_WARN("failed to calc hash value", K(ret)); + } else if (ignore) { + /*do nothing*/ + } else if (OB_FAIL(bloom_filter_.put(hash_value))) { + LOG_WARN("fail to put hash value to px bloom filter", K(ret)); + } else if (is_empty_) { + is_empty_ = false; + } + return ret; +} + +int ObRFBloomFilterMsg::calc_hash_value( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t &hash_value, bool &ignore) +{ + int ret = OB_SUCCESS; + hash_value = ObExprJoinFilter::JOIN_FILTER_SEED; + ignore = false; + ObDatum *datum = nullptr; + if (OB_NOT_NULL(calc_tablet_id_expr)) { + int64_t partition_id = 0; + if (hash_funcs.count() != 1) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected part id expr", K(ret)); + } else if (OB_FAIL(calc_tablet_id_expr->eval(eval_ctx, datum))) { + LOG_WARN("failed to eval datum", K(ret)); + } else if (ObExprCalcPartitionId::NONE_PARTITION_ID == (partition_id = datum->get_int())) { + ignore = true; + } else if (OB_FAIL(hash_funcs.at(0).hash_func_(*datum, hash_value, hash_value))) { + LOG_WARN("failed to do hash funcs", K(ret)); + } + } else { + for (int64_t idx = 0; OB_SUCC(ret) && idx < expr_array.count() ; ++idx) { + if (OB_FAIL(expr_array.at(idx)->eval(eval_ctx, datum))) { + LOG_WARN("failed to eval datum", K(ret)); + } else if (OB_FAIL(hash_funcs.at(idx).hash_func_(*datum, hash_value, hash_value))) { + LOG_WARN("failed to do hash funcs", K(ret)); + } + } + } + return ret; +} + +int ObRFBloomFilterMsg::broadcast(ObIArray &target_addrs, + obrpc::ObP2PDhRpcProxy &p2p_dh_proxy) +{ + int ret = OB_SUCCESS; + int64_t start_time = ObTimeUtility::current_time(); + CK(OB_NOT_NULL(filter_idx_) && OB_NOT_NULL(create_finish_)); + int64_t cur_idx = 0; + ObRFBloomFilterMsg msg; + ObPxP2pDhMsgCB msg_cb(GCTX.self_addr(), + *ObCurTraceId::get_trace_id(), + ObTimeUtility::current_time(), + timeout_ts_, + p2p_datahub_id_); + ObPxP2PDatahubArg arg; + if (OB_FAIL(msg.shadow_copy(*this))) { + LOG_WARN("fail to shadow copy second phase msg", K(ret)); + } else if (OB_ISNULL(create_finish_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected create finish ptr", K(ret)); + } + arg.msg_ = &msg; + while (!*create_finish_ && OB_SUCC(ret)) { + if (OB_FAIL(THIS_WORKER.check_status())) { + LOG_WARN("fail to check status", K(ret)); + } + ob_usleep(10); + } + while (*filter_idx_ < filter_indexes_.count() && OB_SUCC(ret)) { + cur_idx = ATOMIC_FAA(filter_idx_, 1); + if (cur_idx < filter_indexes_.count()) { + msg.next_peer_addrs_.reuse(); + auto addr_filter_idx = filter_indexes_.at(cur_idx); + msg.bloom_filter_.set_begin_idx(addr_filter_idx.begin_idx_); + msg.bloom_filter_.set_end_idx(addr_filter_idx.end_idx_); + for (int i = 0; OB_SUCC(ret) && i < addr_filter_idx.channel_ids_.count(); ++i) { + if (OB_FAIL(msg.next_peer_addrs_.push_back( + target_addrs.at(addr_filter_idx.channel_ids_.at(i))))) { + LOG_WARN("failed push back peer addr", K(i), K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (addr_filter_idx.channel_id_ >= target_addrs.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected channel id", K(addr_filter_idx.channel_id_), K(target_addrs.count())); + } else if (OB_FAIL(p2p_dh_proxy.to(target_addrs.at(addr_filter_idx.channel_id_)) + .by(tenant_id_) + .timeout(timeout_ts_) + .compressed(ObCompressorType::LZ4_COMPRESSOR) + .send_p2p_dh_message(arg, &msg_cb))) { + LOG_WARN("fail to send bloom filter", K(ret)); + } + } + } + return ret; +} + +int ObRFBloomFilterMsg::generate_filter_indexes( + int64_t each_group_size, + int64_t addr_cnt, + int64_t piece_size) +{ + int ret = OB_SUCCESS; + int64_t filter_len = bloom_filter_.get_bits_array_length(); + int64_t count = ceil(filter_len / (double)piece_size); + int64_t start_idx = 0, end_idx = 0; + int64_t group_addr_cnt = each_group_size > addr_cnt ? + addr_cnt : each_group_size; + BloomFilterIndex filter_index; + for (int i = 0; OB_SUCC(ret) && i < count; ++i) { + start_idx = i * piece_size; + end_idx = (i + 1) * piece_size; + if (start_idx >= filter_len) { + start_idx = filter_len - 1; + } + if (end_idx >= filter_len) { + end_idx = filter_len - 1; + } + filter_index.begin_idx_ = start_idx; + filter_index.end_idx_ = end_idx; + int64_t group_count = ceil((double)addr_cnt / group_addr_cnt); + int64_t start_channel = ObRandom::rand(0, group_count - 1); + start_channel *= group_addr_cnt; + int pos = 0; + for (int j = start_channel; OB_SUCC(ret) && + j < start_channel + addr_cnt; + j += group_addr_cnt) { + pos = (j >= addr_cnt ? j - addr_cnt : j); + pos = (pos / group_addr_cnt) * group_addr_cnt; + filter_index.channel_ids_.reset(); + if (pos + group_addr_cnt > addr_cnt) { + filter_index.channel_id_ = (i % (addr_cnt - pos)) + pos; + } else { + filter_index.channel_id_ = (i % group_addr_cnt) + pos; + } + for (int k = pos; OB_SUCC(ret) && k < addr_cnt && k < pos + group_addr_cnt; ++k) { + OZ(filter_index.channel_ids_.push_back(k)); + } + OZ(filter_indexes_.push_back(filter_index)); + } + } + return ret; +} +//end ObRFBloomFilterMsg + +//ObRFRangeFilterMsg +ObRFRangeFilterMsg::ObRFRangeFilterMsg() +: ObP2PDatahubMsgBase(), lower_bounds_(allocator_), upper_bounds_(allocator_), + need_null_cmp_flags_(allocator_), cells_size_(allocator_), + null_first_cmp_funcs_(allocator_), null_last_cmp_funcs_(allocator_) +{ +} + +int ObRFRangeFilterMsg::reuse() +{ + int ret = OB_SUCCESS; + is_empty_ = true; + lower_bounds_.reset(); + upper_bounds_.reset(); + cells_size_.reset(); + if (OB_FAIL(lower_bounds_.prepare_allocate(null_first_cmp_funcs_.count()))) { + LOG_WARN("fail to prepare allocate col cnt", K(ret)); + } else if (OB_FAIL(upper_bounds_.prepare_allocate(null_first_cmp_funcs_.count()))) { + LOG_WARN("fail to prepare allocate col cnt", K(ret)); + } else if (OB_FAIL(cells_size_.prepare_allocate(null_first_cmp_funcs_.count()))) { + LOG_WARN("fail to prepare allocate col cnt", K(ret)); + } + return ret; +} + +int ObRFRangeFilterMsg::assign(const ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + const ObRFRangeFilterMsg &other_msg = static_cast(msg); + if (OB_FAIL(ObP2PDatahubMsgBase::assign(msg))) { + LOG_WARN("failed to assign base data", K(ret)); + } else if (OB_FAIL(lower_bounds_.assign(other_msg.lower_bounds_))) { + LOG_WARN("fail to assign lower bounds", K(ret)); + } else if (OB_FAIL(upper_bounds_.assign(other_msg.upper_bounds_))) { + LOG_WARN("fail to assign upper bounds", K(ret)); + } else if (OB_FAIL(null_first_cmp_funcs_.assign(other_msg.null_first_cmp_funcs_))) { + LOG_WARN("failed to assign cmp funcs", K(ret)); + } else if (OB_FAIL(null_last_cmp_funcs_.assign(other_msg.null_last_cmp_funcs_))) { + LOG_WARN("failed to assign cmp funcs", K(ret)); + } else if (OB_FAIL(need_null_cmp_flags_.assign(other_msg.need_null_cmp_flags_))) { + LOG_WARN("failed to assign cmp flags", K(ret)); + } else if (OB_FAIL(cells_size_.assign(other_msg.cells_size_))) { + LOG_WARN("failed to assign cell size", K(ret)); + } else if (OB_FAIL(adjust_cell_size())) { + LOG_WARN("fail to adjust cell size", K(ret)); + } + return ret; +} + +int ObRFRangeFilterMsg::deep_copy_msg(ObP2PDatahubMsgBase *&new_msg_ptr) +{ + int ret = OB_SUCCESS; + ObRFRangeFilterMsg *rf_msg = nullptr; + if (OB_FAIL(PX_P2P_DH.alloc_msg(tenant_id_, rf_msg))) { + LOG_WARN("fail to alloc rf msg", K(ret)); + } else if (OB_FAIL(rf_msg->assign(*this))) { + LOG_WARN("fail to assign rf msg", K(ret)); + } else { + for (int i = 0; i < rf_msg->lower_bounds_.count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(rf_msg->lower_bounds_.at(i).deep_copy(lower_bounds_.at(i), + rf_msg->get_allocator()))) { + LOG_WARN("fail to deep copy rf msg", K(ret)); + } else if (OB_FAIL(rf_msg->upper_bounds_.at(i).deep_copy(upper_bounds_.at(i), + rf_msg->get_allocator()))) { + LOG_WARN("fail to deep copy rf msg", K(ret)); + } + } + if (OB_SUCC(ret)) { + new_msg_ptr = rf_msg; + } + } + return ret; +} + +int ObRFRangeFilterMsg::merge(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + ObRFRangeFilterMsg &range_msg = static_cast(msg); + CK(range_msg.lower_bounds_.count() == lower_bounds_.count() && + range_msg.upper_bounds_.count() == upper_bounds_.count()); + if (OB_FAIL(ret)) { + LOG_WARN("unexpected bounds count", K(lower_bounds_.count()), K(range_msg.lower_bounds_.count())); + } else if (range_msg.is_empty_) { + /*do nothing*/ + } else { + ObSpinLockGuard guard(lock_); + if (OB_FAIL(get_min(range_msg.lower_bounds_))) { + LOG_WARN("fail to get min lower bounds", K(ret)); + } else if (OB_FAIL(get_max(range_msg.upper_bounds_))) { + LOG_WARN("fail to get max lower bounds", K(ret)); + } else if (is_empty_) { + is_empty_ = false; + } + } + return ret; +} + +int ObRFRangeFilterMsg::get_min(ObIArray &vals) +{ + int ret = OB_SUCCESS; + for (int i = 0; i < vals.count() && OB_SUCC(ret); ++i) { + // null value is also suitable + if (OB_FAIL(get_min(null_first_cmp_funcs_.at(i), lower_bounds_.at(i), + vals.at(i), cells_size_.at(i).min_datum_buf_size_))) { + LOG_WARN("fail to compare value", K(ret)); + } + } + return ret; +} + +int ObRFRangeFilterMsg::get_max(ObIArray &vals) +{ + int ret = OB_SUCCESS; + for (int i = 0; i < vals.count() && OB_SUCC(ret); ++i) { + // null value is also suitable + if (OB_FAIL(get_max(null_last_cmp_funcs_.at(i), upper_bounds_.at(i), + vals.at(i), cells_size_.at(i).max_datum_buf_size_))) { + LOG_WARN("fail to compare value", K(ret)); + } + } + return ret; +} + +int ObRFRangeFilterMsg::get_min(ObCmpFunc &func, ObDatum &l, ObDatum &r, int64_t &cell_size) +{ + int ret = OB_SUCCESS; + int cmp = 0; + if (is_empty_ || (OB_ISNULL(l.ptr_))) { + if (OB_FAIL(dynamic_copy_cell(r, l, cell_size))) { + LOG_WARN("fail to deep copy datum"); + } + } else if (OB_FAIL(func.cmp_func_(l, r, cmp))) { + LOG_WARN("fail to cmp", K(ret)); + } else if (cmp > 0) { + if (OB_FAIL(dynamic_copy_cell(r, l, cell_size))) { + LOG_WARN("fail to deep copy datum"); + } + } + return ret; +} + +int ObRFRangeFilterMsg::adjust_cell_size() +{ + int ret = OB_SUCCESS; + CK(cells_size_.count() == lower_bounds_.count() && + lower_bounds_.count() == upper_bounds_.count()); + for (int i = 0; OB_SUCC(ret) && i < cells_size_.count(); ++i) { + cells_size_.at(i).min_datum_buf_size_ = + std::min(cells_size_.at(i).min_datum_buf_size_, (int64_t)lower_bounds_.at(i).len_); + cells_size_.at(i).max_datum_buf_size_ = + std::min(cells_size_.at(i).max_datum_buf_size_, (int64_t)upper_bounds_.at(i).len_); + } + return ret; +} + +int ObRFRangeFilterMsg::dynamic_copy_cell(const ObDatum &src, ObDatum &target, int64_t &cell_size) +{ + int ret = OB_SUCCESS; + int64_t need_size = src.len_; + if (src.is_null()) { + target.null_ = 1; + } else if (OB_UNLIKELY(0 == src.len_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null src datum", K(ret)); + } else { + if (need_size > cell_size) { + need_size = need_size * 2; + char *buff_ptr = NULL; + if (OB_ISNULL(buff_ptr = static_cast(allocator_.alloc(need_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(WARN, "fall to alloc buff", K(need_size), K(ret)); + } else { + memcpy(buff_ptr, src.ptr_, src.len_); + target.pack_ = src.pack_; + target.ptr_ = buff_ptr; + cell_size = need_size; + } + } else { + memcpy(const_cast(target.ptr_), src.ptr_, src.len_); + target.pack_ = src.pack_; + } + } + return ret; +} + +int ObRFRangeFilterMsg::get_max(ObCmpFunc &func, ObDatum &l, ObDatum &r, int64_t &cell_size) +{ + int ret = OB_SUCCESS; + int cmp = 0; + if (is_empty_ || OB_ISNULL(l.ptr_)) { + if (OB_FAIL(dynamic_copy_cell(r, l, cell_size))) { + LOG_WARN("fail to deep copy datum"); + } + } else if (OB_FAIL(func.cmp_func_(l, r, cmp))) { + LOG_WARN("fail to cmp value", K(ret)); + } else if (cmp < 0) { + if (OB_FAIL(dynamic_copy_cell(r, l, cell_size))) { + LOG_WARN("fail to deep copy datum"); + } + } + return ret; +} + +int ObRFRangeFilterMsg::might_contain(ObIArray &vals, bool &is_match) +{ + int ret = OB_SUCCESS; + is_match = true; + int cmp_min = 0; + int cmp_max = 0; + for (int i = 0; i < vals.count() && OB_SUCC(ret); ++i) { + cmp_min = 0; + cmp_max = 0; + if (OB_FAIL(null_first_cmp_funcs_.at(i).cmp_func_(lower_bounds_.at(i), vals.at(i), cmp_min))) { + LOG_WARN("fail to compare value", K(ret)); + } else if (cmp_min > 0) { + is_match = false; + break; + } else if (OB_FAIL(null_last_cmp_funcs_.at(i).cmp_func_(upper_bounds_.at(i), vals.at(i), cmp_max))) { + LOG_WARN("fail to compare value", K(ret)); + } else if (cmp_max < 0) { + is_match = false; + break; + } + } + return ret; +} + +int ObRFRangeFilterMsg::insert_by_row( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx) +{ + int ret = OB_SUCCESS; + UNUSED(hash_funcs); + ObDatum *datum = nullptr; + if (is_empty_) { + bool ignore_null = false; + for (int64_t i = 0; OB_SUCC(ret) && i < expr_array.count(); ++i) { + ObExpr *expr = expr_array.at(i); + if (OB_FAIL(expr->eval(eval_ctx, datum))) { + LOG_WARN("fail to eval expr", K(ret)); + } else if (datum->is_null() && !need_null_cmp_flags_.at(i)) { + ignore_null = true; + break; + } else if (OB_FAIL(dynamic_copy_cell(*datum, lower_bounds_.at(i), cells_size_.at(i).min_datum_buf_size_))) { + LOG_WARN("fail to deep copy datum", K(ret)); + } else if (OB_FAIL(dynamic_copy_cell(*datum, upper_bounds_.at(i), cells_size_.at(i).max_datum_buf_size_))) { + LOG_WARN("fail to deep copy datum", K(ret)); + } + } + if (!ignore_null) { + is_empty_ = false; + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < expr_array.count(); ++i) { + ObExpr *expr = expr_array.at(i); + if (OB_FAIL(expr->eval(eval_ctx, datum))) { + LOG_WARN("fail to eval expr", K(ret)); + } else if (datum->is_null() && !need_null_cmp_flags_.at(i)) { + /*do nothing*/ + break; + } else if (OB_FAIL(get_min(null_last_cmp_funcs_.at(i), lower_bounds_.at(i), *datum, cells_size_.at(i).min_datum_buf_size_))) { + LOG_WARN("failed to compare value", K(ret)); + } else if (OB_FAIL(get_max(null_first_cmp_funcs_.at(i), upper_bounds_.at(i), *datum, cells_size_.at(i).max_datum_buf_size_))) { + LOG_WARN("failed to compare value", K(ret)); + } + } + } + return ret; +} + +int ObRFRangeFilterMsg::insert_by_row_batch( + const ObBatchRows *child_brs, + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t *batch_hash_values) +{ + int ret = OB_SUCCESS; + UNUSED(batch_hash_values); + UNUSED(calc_tablet_id_expr); + if (child_brs->size_ > 0) { + ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx); + batch_info_guard.set_batch_size(child_brs->size_); + for (int64_t idx = 0; OB_SUCC(ret) && idx < child_brs->size_; ++idx) { + if (child_brs->skip_->at(idx)) { + continue; + } else { + batch_info_guard.set_batch_idx(idx); + if (OB_FAIL(insert_by_row(expr_array, hash_funcs, + calc_tablet_id_expr, eval_ctx))) { + LOG_WARN("fail to insert by row", K(ret)); + } + } + } + } + return ret; +} + +int ObRFRangeFilterMsg::might_contain(const ObExpr &expr, + ObEvalCtx &ctx, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx, + ObDatum &res) +{ + int ret = OB_SUCCESS; + ObDatum *datum = nullptr; + ObCmpFunc cmp_func_null_last; + ObCmpFunc cmp_func_null_first; + int cmp_min = 0; + int cmp_max = 0; + bool is_match = true; + if (OB_UNLIKELY(is_empty_)) { + res.set_int(0); + filter_ctx.filter_count_++; + filter_ctx.check_count_++; + } else { + 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 { + cmp_min = 0; + cmp_max = 0; + cmp_func_null_first.cmp_func_ = reinterpret_cast( + expr.inner_functions_[GET_FUNC(i, ObExprJoinFilter::NULL_FIRST_COMPARE)]); + cmp_func_null_last.cmp_func_ = reinterpret_cast( + expr.inner_functions_[GET_FUNC(i, ObExprJoinFilter::NULL_LAST_COMPARE)]); + if (OB_FAIL(cmp_func_null_first.cmp_func_(*datum, lower_bounds_.at(i), cmp_min))) { + LOG_WARN("fail to compare value", K(ret)); + } else if (cmp_min < 0) { + is_match = false; + break; + } else if (OB_FAIL(cmp_func_null_last.cmp_func_(*datum, upper_bounds_.at(i), cmp_max))) { + LOG_WARN("fail to compare value", K(ret)); + } else if (cmp_max > 0) { + is_match = false; + break; + } + } + } + } + if (OB_SUCC(ret)) { + if (!is_match) { + filter_ctx.filter_count_++; + } + filter_ctx.check_count_++; + res.set_int(is_match ? 1 : 0); + ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + } + } + return ret; +} + +int ObRFRangeFilterMsg::might_contain_batch( + const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const int64_t batch_size, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx) +{ + int ret = OB_SUCCESS; + ObDatum *results = expr.locate_batch_datums(ctx); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + ObEvalCtx::BatchInfoScopeGuard batch_info_guard(ctx); + batch_info_guard.set_batch_size(batch_size); + for (int64_t i = 0; OB_SUCC(ret) && i < batch_size; ++i) { + if (skip.at(i)) { + continue; + } else if (OB_UNLIKELY(is_empty_)) { + results[i].set_int(0); + eval_flags.set(i); + ++filter_ctx.filter_count_; + ++filter_ctx.check_count_; + ++filter_ctx.total_count_; + } else { + batch_info_guard.set_batch_idx(i); + ObDatum &result = results[i]; + if (OB_FAIL(might_contain(expr, ctx, filter_ctx, result))) { + LOG_WARN("fail to check expr value", K(ret)); + } else { + ++filter_ctx.total_count_; + eval_flags.set(i); + } + } + } + return ret; +} +// end ObRFRangeFilterMsg + +// ObRFInFilterMsg + +int ObRFInFilterMsg::assign(const ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + const ObRFInFilterMsg &other_msg = static_cast(msg); + if (OB_FAIL(ObP2PDatahubMsgBase::assign(msg))) { + LOG_WARN("failed to assign base data", K(ret)); + } else if (OB_FAIL(cmp_funcs_.assign(other_msg.cmp_funcs_))) { + LOG_WARN("fail to assign bf msg", K(ret)); + } else if (OB_FAIL(hash_funcs_.assign(other_msg.hash_funcs_))) { + LOG_WARN("fail to assign bf msg", K(ret)); + } else if (OB_FAIL(cur_row_.assign(other_msg.cur_row_))) { + LOG_WARN("failed to assign filter indexes", K(ret)); + } else if (OB_FAIL(need_null_cmp_flags_.assign(other_msg.need_null_cmp_flags_))) { + LOG_WARN("failed to assign filter indexes", K(ret)); + } else { + col_cnt_ = other_msg.col_cnt_; + max_in_num_ = other_msg.max_in_num_; + } + return ret; +} + +int ObRFInFilterMsg::deep_copy_msg(ObP2PDatahubMsgBase *&new_msg_ptr) +{ + int ret = OB_SUCCESS; + ObRFInFilterMsg *in_msg = nullptr; + int64_t row_cnt = max(serial_rows_.count(), 1); + if (OB_FAIL(PX_P2P_DH.alloc_msg(tenant_id_, in_msg))) { + LOG_WARN("fail to alloc rf msg", K(ret)); + } else if (OB_FAIL(in_msg->assign(*this))) { + LOG_WARN("fail to assign rf msg", K(ret)); + } else if (OB_FAIL(in_msg->rows_set_.create(row_cnt * 2, + "RFCPInFilter", + "RFCPInFilter"))) { + LOG_WARN("fail to init in hash set", K(ret)); + } else { + int64_t row_cnt = serial_rows_.count(); + if (0 == row_cnt) { + } else { + for (int i = 0; i < row_cnt && OB_SUCC(ret); ++i) { + for (int j = 0; j < col_cnt_ && OB_SUCC(ret); ++j) { + in_msg->cur_row_.at(j) = serial_rows_.at(i)->at(j); + } + if (OB_SUCC(ret)) { + if (OB_FAIL(in_msg->append_row())) { + LOG_WARN("fail to append row", K(ret)); + } + } + } + } + if (OB_SUCC(ret)) { + new_msg_ptr = in_msg; + } + } + return ret; +} + +int ObRFInFilterMsg::insert_by_row_batch( + const ObBatchRows *child_brs, + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t *batch_hash_values) +{ + int ret = OB_SUCCESS; + UNUSED(batch_hash_values); + UNUSED(calc_tablet_id_expr); + if (child_brs->size_ > 0 && is_active_) { + ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx); + batch_info_guard.set_batch_size(child_brs->size_); + for (int64_t idx = 0; OB_SUCC(ret) && idx < child_brs->size_; ++idx) { + if (child_brs->skip_->at(idx)) { + continue; + } else { + batch_info_guard.set_batch_idx(idx); + ObDatum *datum = nullptr; + bool ignore_null_row = false; + for (int64_t i = 0; OB_SUCC(ret) && i < expr_array.count(); ++i) { + ObExpr *expr = expr_array.at(i); + if (OB_FAIL(expr->eval(eval_ctx, datum))) { + LOG_WARN("fail to eval expr", K(ret)); + } else if (datum->is_null() && !need_null_cmp_flags_.at(i)) { + ignore_null_row = true; + break; + } else { + cur_row_.at(i) = (*datum); + } + } + if (OB_SUCC(ret) && !ignore_null_row) { + if (OB_FAIL(insert_node())) { + LOG_WARN("fail to insert node", K(ret)); + } + } + } + } + } + return ret; +} + +int ObRFInFilterMsg::insert_node() +{ + int ret = OB_SUCCESS; + ObRFInFilterNode node(&cmp_funcs_, &hash_funcs_, &cur_row_); + if (OB_FAIL(rows_set_.exist_refactored(node))) { + if (OB_HASH_NOT_EXIST == ret) { + ret = OB_SUCCESS; + if (serial_rows_.count() > max_in_num_) { + is_active_ = false; + } else if (OB_FAIL(append_row())) { + LOG_WARN("fail to append row", K(ret)); + } else if (is_empty_) { + is_empty_ = false; + } + } else if (OB_HASH_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to check node", K(ret)); + } + } + return ret; +} + +int ObRFInFilterMsg::insert_by_row( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx) +{ + int ret = OB_SUCCESS; + ObDatum *datum = nullptr; + if (is_active_) { + bool ignore_null_row = false; + for (int64_t idx = 0; OB_SUCC(ret) && idx < expr_array.count() ; ++idx) { + datum = nullptr; + if (OB_FAIL(expr_array.at(idx)->eval(eval_ctx, datum))) { + LOG_WARN("failed to eval datum", K(ret)); + } else if (datum->is_null() && !need_null_cmp_flags_.at(idx)) { + ignore_null_row = true; + break; + } else { + cur_row_.at(idx) = (*datum); + } + } + if (OB_SUCC(ret) && !ignore_null_row) { + if (OB_FAIL(insert_node())) { + LOG_WARN("fail to insert node", K(ret)); + } + } + } + + return ret; +} + +int ObRFInFilterMsg::append_row() +{ + int ret = OB_SUCCESS; + ObFixedArray *new_row = nullptr; + void *array_ptr = nullptr; + if (OB_ISNULL(array_ptr = allocator_.alloc(sizeof(ObFixedArray)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc memory", K(ret)); + } else { + new_row = new(array_ptr) ObFixedArray(allocator_); + if (OB_FAIL(new_row->init(cur_row_.count()))) { + LOG_WARN("fail to init cur row", K(ret)); + } else { + ObDatum datum; + for (int i = 0; i < cur_row_.count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(datum.deep_copy(cur_row_.at(i), allocator_))) { + LOG_WARN("fail to deep copy datum", K(ret)); + } else if (OB_FAIL(new_row->push_back(datum))) { + LOG_WARN("fail to push back new row", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(serial_rows_.push_back(new_row))) { + LOG_WARN("fail to push back serial rows", K(ret)); + } else { + ObRFInFilterNode node(&cmp_funcs_, &hash_funcs_, new_row); + if (OB_FAIL(rows_set_.set_refactored(node))) { + LOG_WARN("fail to insert in filter node", K(ret)); + } + } + } + } + } + return ret; +} + +int ObRFInFilterMsg::ObRFInFilterNode::hash(uint64_t &hash_ret) const +{ + int ret = OB_SUCCESS; + hash_ret = 0; + for (int i = 0; i < row_->count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(hash_funcs_->at(i).hash_func_(row_->at(i), hash_ret, hash_ret))) { + LOG_WARN("fail to calc hash value", K(ret), K(hash_ret)); + } + } + return ret; +} + +bool ObRFInFilterMsg::ObRFInFilterNode::operator==(const ObRFInFilterNode &other) const +{ + int cmp_ret = 0; + bool ret = true; + for (int i = 0; i < other.row_->count(); ++i) { + if (row_->at(i).is_null() && other.row_->at(i).is_null()) { + continue; + } else { + int tmp_ret = cmp_funcs_->at(i).cmp_func_(row_->at(i), other.row_->at(i), cmp_ret); + if (cmp_ret != 0) { + ret = false; + break; + } + } + } + return ret; +} + +int ObRFInFilterMsg::merge(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + ObRFInFilterMsg &in_msg = static_cast(msg); + if (!msg.is_active()) { + is_active_ = false; + } else if (!msg.is_empty() && is_active_) { + ObSpinLockGuard guard(lock_); + for (int i = 0; i < in_msg.serial_rows_.count() && OB_SUCC(ret); ++i) { + for (int j = 0; j < in_msg.serial_rows_.at(i)->count(); ++j) { + cur_row_.at(j) = in_msg.serial_rows_.at(i)->at(j); + } + if (OB_FAIL(insert_node())) { + LOG_WARN("fail to insert node", K(ret)); + } + } + } + return ret; +} + +int ObRFInFilterMsg::might_contain(const ObExpr &expr, + ObEvalCtx &ctx, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx, + ObDatum &res) +{ + int ret = OB_SUCCESS; + ObDatum *datum = nullptr; + bool is_match = true; + ObSEArray cur_row; + if (OB_UNLIKELY(!is_active_)) { + res.set_int(1); + } else if (OB_UNLIKELY(is_empty_)) { + res.set_int(0); + filter_ctx.filter_count_++; + filter_ctx.check_count_++; + } else { + 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 if (OB_FAIL(cur_row.push_back(*datum))) { + LOG_WARN("failed to push back datum", K(ret)); + } + } + } + if (OB_SUCC(ret)) { + ObRFInFilterNode node(&cmp_funcs_, &hash_funcs_, &cur_row); + if (OB_FAIL(rows_set_.exist_refactored(node))) { + if (OB_HASH_NOT_EXIST == ret) { + is_match = false; + ret = OB_SUCCESS; + } else if (OB_HASH_EXIST == ret) { + is_match = true; + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to check node", K(ret)); + } + } + } + if (OB_SUCC(ret)) { + if (!is_match) { + filter_ctx.filter_count_++; + } + filter_ctx.check_count_++; + res.set_int(is_match ? 1 : 0); + ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + } + } + return ret; +} + +int ObRFInFilterMsg::reuse() +{ + int ret = OB_SUCCESS; + is_empty_ = true; + serial_rows_.reset(); + rows_set_.reuse(); + return ret; +} + +int ObRFInFilterMsg::might_contain_batch( + const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const int64_t batch_size, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx) +{ + int ret = OB_SUCCESS; + ObDatum *results = expr.locate_batch_datums(ctx); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + ObEvalCtx::BatchInfoScopeGuard batch_info_guard(ctx); + batch_info_guard.set_batch_size(batch_size); + for (int64_t i = 0; OB_SUCC(ret) && i < batch_size; ++i) { + if (skip.at(i)) { + continue; + } else if (OB_UNLIKELY(!is_active_)) { + eval_flags.set(i); + results[i].set_int(1); + } else if (OB_UNLIKELY(is_empty_)) { + eval_flags.set(i); + results[i].set_int(0); + ++filter_ctx.filter_count_; + ++filter_ctx.check_count_; + ++filter_ctx.total_count_; + } else { + batch_info_guard.set_batch_idx(i); + ObDatum *result = &results[i]; + if (OB_FAIL(might_contain(expr, ctx, filter_ctx, *result))) { + LOG_WARN("fail to check expr value", K(ret)); + } else { + ++filter_ctx.total_count_; + eval_flags.set(i); + } + } + } + return ret; +} + +int ObRFInFilterMsg::destroy() +{ + int ret = OB_SUCCESS; + rows_set_.destroy(); + hash_funcs_.reset(); + cmp_funcs_.reset(); + need_null_cmp_flags_.reset(); + cur_row_.reset(); + for (int i = 0; i < serial_rows_.count(); ++i) { + if (OB_NOT_NULL(serial_rows_.at(i))) { + serial_rows_.at(i)->reset(); + } + } + serial_rows_.reset(); + allocator_.reset(); + return ret; +} +//end ObRFInFilterMsg diff --git a/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h new file mode 100644 index 000000000..3845a3e76 --- /dev/null +++ b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h @@ -0,0 +1,238 @@ +/** + * 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 __SQL_ENG_P2P_RUNTIME_FILTER_DH_MSG_H__ +#define __SQL_ENG_P2P_RUNTIME_FILTER_DH_MSG_H__ +#include "lib/ob_define.h" +#include "lib/hash/ob_hashmap.h" +#include "lib/container/ob_array.h" +#include "share/datum/ob_datum.h" +#include "sql/engine/px/ob_px_bloom_filter.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h" + + + +namespace oceanbase +{ +namespace sql +{ + +class ObP2PDatahubMsgBase; + +class ObRFBloomFilterMsg final : public ObP2PDatahubMsgBase +{ + OB_UNIS_VERSION_V(1); +public: + enum ObSendBFPhase + { + FIRST_LEVEL, + SECOND_LEVEL + }; + ObRFBloomFilterMsg() : phase_(), bloom_filter_(), + next_peer_addrs_(), expect_first_phase_count_(0), + piece_size_(0), filter_indexes_(), receive_count_array_(), + filter_idx_(nullptr), create_finish_(nullptr) {} + ~ObRFBloomFilterMsg() { destroy(); } + virtual int assign(const ObP2PDatahubMsgBase &) final; + virtual int merge(ObP2PDatahubMsgBase &) final; + virtual int broadcast(ObIArray &target_addrs, + obrpc::ObP2PDhRpcProxy &p2p_dh_proxy) final; + bool is_first_phase() { return FIRST_LEVEL == phase_; } + virtual int might_contain(const ObExpr &expr, + ObEvalCtx &ctx, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx, + ObDatum &res) override; + virtual int might_contain_batch( + const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const int64_t batch_size, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx) override; + virtual int insert_by_row( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx) override; + virtual int insert_by_row_batch( + const ObBatchRows *child_brs, + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t *batch_hash_values) override; + virtual int reuse() override; + virtual int process_receive_count(ObP2PDatahubMsgBase &) override; + common::ObIArray& get_next_phase_addrs() { return next_peer_addrs_; } + virtual int deep_copy_msg(ObP2PDatahubMsgBase *&new_msg_ptr); + virtual int destroy(); + int generate_filter_indexes(int64_t each_group_size, + int64_t addr_cnt, int64_t piece_size); + int process_first_phase_recieve_count( + ObRFBloomFilterMsg &msg, bool &first_phase_end); + virtual int process_msg_internal(bool &need_free); +private: +int calc_hash_value( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t &hash_value, bool &ignore); +int shadow_copy(const ObRFBloomFilterMsg &msg); +int generate_receive_count_array(int64_t piece_size); +public: + ObSendBFPhase phase_; + ObPxBloomFilter bloom_filter_; + common::ObSArray next_peer_addrs_; + int64_t expect_first_phase_count_; + int64_t piece_size_; + common::ObArray filter_indexes_; + common::ObArray receive_count_array_; + int64_t *filter_idx_; //for shared msg + bool *create_finish_; //for shared msg +}; + +class ObRFRangeFilterMsg : public ObP2PDatahubMsgBase +{ + OB_UNIS_VERSION_V(1); +public: + struct MinMaxCellSize + { + OB_UNIS_VERSION_V(1); + public: + MinMaxCellSize() : min_datum_buf_size_(0), max_datum_buf_size_(0) {} + virtual ~MinMaxCellSize() = default; + // record the real datum buf for lower bound + int64_t min_datum_buf_size_; + // record the real datum buf for upper bound + int64_t max_datum_buf_size_; + TO_STRING_KV(K_(min_datum_buf_size), K_(max_datum_buf_size)); + }; +public: + ObRFRangeFilterMsg(); + virtual int assign(const ObP2PDatahubMsgBase &) final; + virtual int merge(ObP2PDatahubMsgBase &) final; + virtual int deep_copy_msg(ObP2PDatahubMsgBase *&new_msg_ptr); + virtual int destroy() { + lower_bounds_.reset(); + upper_bounds_.reset(); + null_first_cmp_funcs_.reset(); + null_last_cmp_funcs_.reset(); + need_null_cmp_flags_.reset(); + cells_size_.reset(); + allocator_.reset(); + return OB_SUCCESS; + } + virtual int might_contain(const ObExpr &expr, + ObEvalCtx &ctx, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx, + ObDatum &res) override; + virtual int might_contain_batch( + const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const int64_t batch_size, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx) override; + virtual int insert_by_row( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx) override; + virtual int insert_by_row_batch( + const ObBatchRows *child_brs, + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t *batch_hash_values) override; + virtual int reuse() override; + int might_contain(ObIArray &vals, bool &is_match); + int adjust_cell_size(); +private: + int get_min(ObIArray &vals); + int get_max(ObIArray &vals); + int get_min(ObCmpFunc &func, ObDatum &l, ObDatum &r, int64_t &cell_size); + int get_max(ObCmpFunc &func, ObDatum &l, ObDatum &r, int64_t &cell_size); + int dynamic_copy_cell(const ObDatum &src, ObDatum &target, int64_t &cell_size); + +public: + ObFixedArray lower_bounds_; + ObFixedArray upper_bounds_; + ObFixedArray need_null_cmp_flags_; + ObFixedArray cells_size_; + ObCmpFuncs null_first_cmp_funcs_; + ObCmpFuncs null_last_cmp_funcs_; +}; + +class ObRFInFilterMsg : public ObP2PDatahubMsgBase +{ + OB_UNIS_VERSION_V(1); +public: + struct ObRFInFilterNode { + ObRFInFilterNode() = default; + ObRFInFilterNode(ObCmpFuncs *cmp_funcs, ObHashFuncs *hash_funcs, ObIArray *row) + : cmp_funcs_(cmp_funcs), hash_funcs_(hash_funcs), row_(row) {} + int hash(uint64_t &hash_ret) const; + inline bool operator==(const ObRFInFilterNode &other) const; + ObCmpFuncs *cmp_funcs_; + ObHashFuncs *hash_funcs_; + ObIArray *row_; + }; +public: + ObRFInFilterMsg() : ObP2PDatahubMsgBase(), rows_set_(), + cmp_funcs_(allocator_), hash_funcs_(allocator_), + serial_rows_(), need_null_cmp_flags_(allocator_), + cur_row_(allocator_), col_cnt_(0), + max_in_num_(0) {} + virtual int assign(const ObP2PDatahubMsgBase &); + virtual int merge(ObP2PDatahubMsgBase &) final; + virtual int deep_copy_msg(ObP2PDatahubMsgBase *&new_msg_ptr); + virtual int destroy(); + virtual int might_contain(const ObExpr &expr, + ObEvalCtx &ctx, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx, + ObDatum &res) override; + virtual int might_contain_batch( + const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const int64_t batch_size, + ObExprJoinFilter::ObExprJoinFilterContext &filter_ctx) override; + virtual int insert_by_row( + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx) override; + virtual int insert_by_row_batch( + const ObBatchRows *child_brs, + const common::ObIArray &expr_array, + const common::ObHashFuncs &hash_funcs, + const ObExpr *calc_tablet_id_expr, + ObEvalCtx &eval_ctx, + uint64_t *batch_hash_values) override; + virtual int reuse() override; +private: + int append_row(); + int insert_node(); +public: + hash::ObHashSet rows_set_; + ObCmpFuncs cmp_funcs_; + ObHashFuncs hash_funcs_; + ObSArray *> serial_rows_; + ObFixedArray need_null_cmp_flags_; + ObFixedArray cur_row_; + int64_t col_cnt_; + int64_t max_in_num_; +}; + +} +} + +#endif diff --git a/src/sql/optimizer/ob_join_order.cpp b/src/sql/optimizer/ob_join_order.cpp index d10b0c237..7220d75f8 100644 --- a/src/sql/optimizer/ob_join_order.cpp +++ b/src/sql/optimizer/ob_join_order.cpp @@ -9251,7 +9251,7 @@ int ObJoinOrder::find_possible_join_filter_tables(const Path &left_path, right_path, left_path.parent_->get_tables(), right_tables, - !get_plan()->get_optimizer_context().enable_bloom_filter(), + !get_plan()->get_optimizer_context().enable_runtime_filter(), !right_need_exchange, is_fully_partition_wise, left_exprs, @@ -9313,7 +9313,7 @@ int ObJoinOrder::find_possible_join_filter_tables(const ObLogPlanHint &log_plan_ can_join_filter, force_hint))) { LOG_WARN("failed to check use join filter", K(ret)); - } else if (is_current_dfo && !is_fully_partition_wise && + } else if (!is_fully_partition_wise && OB_FAIL(log_plan_hint.check_use_join_filter(access.table_id_, left_tables, true, @@ -9379,13 +9379,9 @@ int ObJoinOrder::find_possible_join_filter_tables(const ObLogPlanHint &log_plan_ LOG_WARN("failed to push back info", K(ret)); } } - } else if (((GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_1_0_0) || is_current_dfo) && right_path.is_join_path()) { + } else if (right_path.is_join_path()) { const JoinPath &join_path = static_cast(right_path); - if (!is_current_dfo && join_path.is_left_need_exchange()) { - // Stop recursive join_path.left_path_ if GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_1_0_0 and - // is_current_dfo is false and join_path.is_left_need_exchange(). - // Do not allow bloom filters to be sent between non-adjacent dfos. - } else if (OB_ISNULL(join_path.left_path_) || + if (OB_ISNULL(join_path.left_path_) || OB_ISNULL(join_path.left_path_->parent_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); @@ -9403,10 +9399,6 @@ int ObJoinOrder::find_possible_join_filter_tables(const ObLogPlanHint &log_plan_ LOG_WARN("failed to find shuffle table scan", K(ret)); } if (OB_FAIL(ret)) { - } else if (!is_current_dfo && join_path.is_right_need_exchange()) { - // stop recursive join_path.right_path_ if GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_1_0_0 and - // is_current_dfo is false and join_path.is_left_need_exchange(). - // Do not allow bloom filters to be sent between non-adjacent dfos. } else if (OB_ISNULL(join_path.right_path_) || OB_ISNULL(join_path.right_path_->parent_)) { ret = OB_ERR_UNEXPECTED; @@ -9514,20 +9506,11 @@ int ObJoinOrder::check_normal_join_filter_valid(const Path& left_path, OB_ISNULL(stmt = plan->get_stmt())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected null plan", K(ret)); - } else if (OB_FAIL(find_shuffle_join_filter(left_path, left_find))) { - LOG_WARN("failed to find shuffle join filter", K(ret)); - } else if (OB_FAIL(find_shuffle_join_filter(right_path, right_find))) { - LOG_WARN("failed to find shuffle join filter", K(ret)); - } else { - cur_dfo_has_shuffle_bf = left_find || right_find; } for (int i = 0; OB_SUCC(ret) && i < join_filter_infos.count(); ++i) { JoinFilterInfo &info = join_filter_infos.at(i); double join_filter_sel = 1.0; - if ((GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_1_0_0) && cur_dfo_has_shuffle_bf && !info.in_current_dfo_) { - info.can_use_join_filter_ = false; - OPT_TRACE("current dfo has shuffle join filter, this hash join will not use join filter"); - } else if (!info.can_use_join_filter_ || info.lexprs_.empty()) { + if (!info.can_use_join_filter_ || info.lexprs_.empty()) { info.can_use_join_filter_ = false; } else if (OB_FAIL(calc_join_filter_selectivity(left_path, info, diff --git a/src/sql/optimizer/ob_log_granule_iterator.cpp b/src/sql/optimizer/ob_log_granule_iterator.cpp index bf697b75d..7ad9b8a5a 100644 --- a/src/sql/optimizer/ob_log_granule_iterator.cpp +++ b/src/sql/optimizer/ob_log_granule_iterator.cpp @@ -104,7 +104,7 @@ int ObLogGranuleIterator::get_plan_item_info(PlanText &plan_text, get_join_filter_info().is_inited_ && OB_INVALID_ID != get_join_filter_info().filter_id_) { BEGIN_BUF_PRINT; - if (OB_FAIL(BUF_PRINTF(":BF%04ld", get_join_filter_info().filter_id_))) { + if (OB_FAIL(BUF_PRINTF(":RF%04ld", get_join_filter_info().filter_id_))) { LOG_WARN("failed to print str", K(ret)); } END_BUF_PRINT(plan_item.object_alias_, diff --git a/src/sql/optimizer/ob_log_join_filter.cpp b/src/sql/optimizer/ob_log_join_filter.cpp index 22470f448..b0fdf74fa 100644 --- a/src/sql/optimizer/ob_log_join_filter.cpp +++ b/src/sql/optimizer/ob_log_join_filter.cpp @@ -36,7 +36,7 @@ int ObLogJoinFilter::est_cost() ObLogicalOperator *first_child = get_child(ObLogicalOperator::first_child); if (OB_ISNULL(first_child)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("first_child is null", K(ret)); + LOG_WARN("first_child is null", K(ret)); } else { // refine this set_op_cost(0.0); @@ -81,15 +81,70 @@ int ObLogJoinFilter::get_plan_item_info(PlanText &plan_text, ObSqlPlanItem &plan_item) { int ret = OB_SUCCESS; + bool is_first = false; + static const char *join_filter_type_name[] = + {"bloom", "range", "in"}; + int64_t arr_len = sizeof(join_filter_type_name) / sizeof(const char *); if (OB_FAIL(ObLogicalOperator::get_plan_item_info(plan_text, plan_item))) { LOG_WARN("failed to get plan item info", K(ret)); } else if (OB_INVALID_ID != get_filter_id()) { BEGIN_BUF_PRINT; - if (OB_FAIL(BUF_PRINTF(":BF%04ld", get_filter_id()))) { + if (OB_FAIL(BUF_PRINTF(":RF%04ld", get_filter_id()))) { LOG_WARN("failed to print str", K(ret)); } END_BUF_PRINT(plan_item.object_alias_, plan_item.object_alias_len_); } + if (OB_SUCC(ret) && is_create_) { + BEGIN_BUF_PRINT; + int64_t idx = 0; + const char *type_name = nullptr; + if (OB_FAIL(BUF_PRINTF("RF_TYPE("))) { + LOG_WARN("fail to print rf", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < join_filter_types_.count(); ++i) { + idx = (int64_t)(join_filter_types_.at(i)) - 1; + if (idx >= arr_len || idx < 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected type", K(ret)); + } else if (FALSE_IT(type_name = join_filter_type_name[idx])) { + } else if (is_first && OB_FAIL(BUF_PRINTF(", "))) { + LOG_WARN("BUF_PRINTF fails", K(ret)); + } else if (OB_FAIL(BUF_PRINTF("%.*s", + (int)strlen(type_name), + type_name))) { + LOG_WARN("BUF_PRINTF fails", K(ret)); + } else { + is_first = true; + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(BUF_PRINTF(")"))) { + LOG_WARN("fail to print rf", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (!join_exprs_.empty()) { + if (OB_FAIL(BUF_PRINTF(", RF_EXPR["))) { + LOG_WARN("fail to print rf", K(ret)); + } else { + int cnt = join_exprs_.count(); + for (int i = 0; i < cnt && OB_SUCC(ret); ++i) { + if (OB_FAIL(join_exprs_.at(i)->get_name(buf, buf_len, pos, type))) { + LOG_WARN("fail to get name", K(ret)); + } else if (i != cnt - 1 && OB_FAIL(BUF_PRINTF(", "))) { + LOG_WARN("fail to print buf", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(BUF_PRINTF("]"))) { + LOG_WARN("fail to print buf", K(ret)); + } + } + } + } + END_BUF_PRINT(plan_item.special_predicates_, + plan_item.special_predicates_len_); + } return ret; } diff --git a/src/sql/optimizer/ob_log_join_filter.h b/src/sql/optimizer/ob_log_join_filter.h index d7536e2ba..0aa460aae 100644 --- a/src/sql/optimizer/ob_log_join_filter.h +++ b/src/sql/optimizer/ob_log_join_filter.h @@ -17,6 +17,8 @@ #include "sql/optimizer/ob_log_join.h" #include "sql/resolver/dml/ob_sql_hint.h" #include "sql/engine/px/ob_px_basic_info.h" +#include "sql/engine/join/ob_join_filter_op.h" +#include "sql/engine/expr/ob_expr_join_filter.h" namespace oceanbase { namespace sql @@ -26,11 +28,15 @@ class ObLogJoinFilter : public ObLogicalOperator { public: ObLogJoinFilter(ObLogPlan &plan) : - ObLogicalOperator(plan), is_create_(false), filter_id_(common::OB_INVALID_ID), + ObLogicalOperator(plan), is_create_(false), + filter_id_(common::OB_INVALID_ID), filter_len_(0), join_exprs_(), is_use_filter_shuffle_(false), - join_filter_expr_(NULL), - filter_type_(JoinFilterType::INVALID_TYPE), + join_filter_exprs_(), + join_filter_types_(), + p2p_sequence_ids_(), + is_null_safe_cmps_(), + filter_type_(JoinFilterSharedType::INVALID_TYPE), calc_tablet_id_expr_(NULL), skip_subpart_(false) { } @@ -57,23 +63,41 @@ public: inline void set_is_use_filter_shuffle(bool flag) { is_use_filter_shuffle_ = flag; } inline bool is_use_filter_shuffle() { return is_use_filter_shuffle_; } inline bool is_partition_filter() const - { return filter_type_ == JoinFilterType::NONSHARED_PARTITION_JOIN_FILTER || - filter_type_ == JoinFilterType::SHARED_PARTITION_JOIN_FILTER; }; + { return filter_type_ == JoinFilterSharedType::NONSHARED_PARTITION_JOIN_FILTER || + filter_type_ == JoinFilterSharedType::SHARED_PARTITION_JOIN_FILTER; }; common::ObIArray &get_join_exprs() { return join_exprs_; } - inline void set_join_filter_expr(ObRawExpr *filter_expr) { join_filter_expr_ = filter_expr; } - const ObRawExpr *get_join_filter_expr() { return join_filter_expr_; } + int add_join_filter_expr(ObRawExpr *filter_expr) + { return join_filter_exprs_.push_back(filter_expr); } + const common::ObIArray &get_join_filter_exprs() + { return join_filter_exprs_; } +common::ObIArray &get_join_filter_exprs_for_update() + { return join_filter_exprs_; } + common::ObIArray &get_is_null_safe_cmps() + { return is_null_safe_cmps_; } + const common::ObIArray &get_join_filter_types() + { return join_filter_types_; } + int add_join_filter_type(RuntimeFilterType type) + { return join_filter_types_.push_back(type); } + const common::ObIArray &get_p2p_sequence_ids() + { return p2p_sequence_ids_; } + int add_p2p_sequence_id(int64_t id) + { return p2p_sequence_ids_.push_back(id); } inline void set_tablet_id_expr(ObRawExpr *tablet_id_expr) { calc_tablet_id_expr_ = tablet_id_expr; } const ObRawExpr *get_tablet_id_expr() { return calc_tablet_id_expr_; } inline void set_is_shared_join_filter() - { filter_type_ = JoinFilterType::SHARED_JOIN_FILTER; } + { filter_type_ = JoinFilterSharedType::SHARED_JOIN_FILTER; } inline void set_is_non_shared_join_filter() - { filter_type_ = JoinFilterType::NONSHARED_JOIN_FILTER; } + { filter_type_ = JoinFilterSharedType::NONSHARED_JOIN_FILTER; } + inline bool is_shared_join_filter() { + return filter_type_ == JoinFilterSharedType::SHARED_JOIN_FILTER || + filter_type_ == JoinFilterSharedType::SHARED_PARTITION_JOIN_FILTER; + } inline void set_is_shared_partition_join_filter() - { filter_type_ = JoinFilterType::SHARED_PARTITION_JOIN_FILTER; } + { filter_type_ = JoinFilterSharedType::SHARED_PARTITION_JOIN_FILTER; } inline void set_is_no_shared_partition_join_filter() - { filter_type_ = JoinFilterType::NONSHARED_PARTITION_JOIN_FILTER; } - JoinFilterType get_filter_type() { return filter_type_; } + { filter_type_ = JoinFilterSharedType::NONSHARED_PARTITION_JOIN_FILTER; } + JoinFilterSharedType get_filter_type() { return filter_type_; } virtual int inner_replace_op_exprs( const common::ObIArray> &to_replace_exprs) override; virtual int get_plan_item_info(PlanText &plan_text, @@ -85,8 +109,11 @@ private: //equal join condition expr common::ObSEArray join_exprs_; bool is_use_filter_shuffle_; // 标记use端filter是否有shuffle - ObRawExpr *join_filter_expr_; - JoinFilterType filter_type_; + common::ObSEArray join_filter_exprs_; + common::ObSEArray join_filter_types_; + common::ObSEArray p2p_sequence_ids_; + common::ObSEArray is_null_safe_cmps_; + JoinFilterSharedType filter_type_; ObRawExpr *calc_tablet_id_expr_; // 计算tablet_id的expr bool skip_subpart_; // Ignore 2-level subpart_id when calculating partition id DISALLOW_COPY_AND_ASSIGN(ObLogJoinFilter); diff --git a/src/sql/optimizer/ob_log_plan.cpp b/src/sql/optimizer/ob_log_plan.cpp index 941125451..8c83cacad 100644 --- a/src/sql/optimizer/ob_log_plan.cpp +++ b/src/sql/optimizer/ob_log_plan.cpp @@ -9683,7 +9683,7 @@ int ObLogPlan::plan_tree_traverse(const TraverseOp &operation, void *ctx) ctx = &md_ctx; break; } - case BLOOM_FILTER: { + case RUNTIME_FILTER: { ctx = &bf_ctx; break; } @@ -11121,7 +11121,7 @@ int ObLogPlan::generate_plan() /*do nothing*/ } else if (OB_FAIL(do_post_plan_processing())) { LOG_WARN("failed to post plan processing", K(ret)); - } else if (OB_FAIL(plan_traverse_loop(BLOOM_FILTER, + } else if (OB_FAIL(plan_traverse_loop(RUNTIME_FILTER, ALLOC_GI, PX_PIPE_BLOCKING, ALLOC_MONITORING_DUMP, @@ -13287,6 +13287,8 @@ int ObLogPlan::find_possible_join_filter_tables(ObLogicalOperator *op, ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (op->get_type() == log_op_def::LOG_SET) { + ObLogSet *log_set = static_cast(op); + bool is_ext_pw = (log_set->get_distributed_algo() == DistAlgo::DIST_SET_PARTITION_WISE); for (int64_t i = 0; OB_SUCC(ret) && i < op->get_num_of_child(); ++i) { ObLogicalOperator* child_op; ObLogPlan* child_plan; @@ -13299,7 +13301,7 @@ int ObLogPlan::find_possible_join_filter_tables(ObLogicalOperator *op, ObTabletID::INVALID_TABLET_ID, hint_info, is_current_dfo, - op->is_fully_paratition_wise(), + op->is_fully_paratition_wise() || is_ext_pw, left_join_conditions, right_join_conditions, join_filter_infos))) { @@ -13323,7 +13325,7 @@ int ObLogPlan::find_possible_join_filter_tables(ObLogicalOperator *op, can_join_filter, force_hint))) { LOG_WARN("failed to check use join filter", K(ret)); - } else if (is_current_dfo && !is_fully_partition_wise && + } else if (!is_fully_partition_wise && OB_FAIL(hint_info.check_use_join_filter(*stmt, stmt->get_query_ctx()->get_query_hint(), scan->get_table_id(), @@ -13344,7 +13346,7 @@ int ObLogPlan::find_possible_join_filter_tables(ObLogicalOperator *op, info.can_use_join_filter_ = can_join_filter; info.force_filter_ = force_hint; info.need_partition_join_filter_ = can_part_join_filter; - info.force_part_filter_ = is_current_dfo ? force_part_hint : NULL; + info.force_part_filter_ = force_part_hint; info.in_current_dfo_ = is_current_dfo; if (info.can_use_join_filter_ || info.need_partition_join_filter_) { if (OB_FAIL(get_join_filter_exprs(left_join_conditions, @@ -13421,8 +13423,7 @@ int ObLogPlan::find_possible_join_filter_tables(ObLogicalOperator *op, join_filter_infos))) { LOG_WARN("failed to find pushdown join filter table", K(ret)); } - } else if (((GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_1_0_0) || is_current_dfo) && - log_op_def::LOG_JOIN == op->get_type()) { + } else if (log_op_def::LOG_JOIN == op->get_type()) { ObLogJoin* join_op = static_cast(op); ObLogicalOperator* left_op; ObLogicalOperator* right_op; @@ -13450,15 +13451,14 @@ int ObLogPlan::find_possible_join_filter_tables(ObLogicalOperator *op, join_filter_infos)))) { LOG_WARN("failed to find shuffle table scan", K(ret)); } - } else if (!is_current_dfo && log_op_def::LOG_JOIN == op->get_type()) { - /* do nothing */ - } else if (!is_current_dfo && (log_op_def::LOG_EXCHANGE == op->get_type() && - static_cast(op)->is_consumer())) { - /* do nothing */ } else if (log_op_def::LOG_EXCHANGE == op->get_type() && static_cast(op)->is_consumer() && OB_FALSE_IT(is_current_dfo = false)) { /* do nothing */ + } else if (log_op_def::LOG_EXCHANGE == op->get_type() && + static_cast(op)->is_consumer() && + static_cast(op)->is_local()) { + /* do nothing */ } else if (log_op_def::LOG_SUBPLAN_FILTER == op->get_type()) { if (OB_FAIL(SMART_CALL(find_possible_join_filter_tables(op->get_child(ObLogicalOperator::first_child), hint_info, diff --git a/src/sql/optimizer/ob_logical_operator.cpp b/src/sql/optimizer/ob_logical_operator.cpp index 194c7f8ca..87e20c4ef 100644 --- a/src/sql/optimizer/ob_logical_operator.cpp +++ b/src/sql/optimizer/ob_logical_operator.cpp @@ -57,6 +57,9 @@ #include "sql/optimizer/ob_join_order.h" #include "sql/optimizer/ob_opt_selectivity.h" #include "sql/optimizer/ob_log_merge.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "sql/engine/expr/ob_expr_join_filter.h" + using namespace oceanbase::sql; using namespace oceanbase::share; @@ -1404,7 +1407,7 @@ int ObLogicalOperator::do_pre_traverse_operation(const TraverseOp &op, void *ctx } break; } - case BLOOM_FILTER: { + case RUNTIME_FILTER: { break; } case PROJECT_PRUNING: { @@ -1515,10 +1518,10 @@ int ObLogicalOperator::do_post_traverse_operation(const TraverseOp &op, void *ct } else { /* Do nothing */ } break; } - case BLOOM_FILTER: { + case RUNTIME_FILTER: { AllocBloomFilterContext *alloc_bf_ctx = static_cast(ctx); CK( OB_NOT_NULL(alloc_bf_ctx)); - OC( (allocate_bf_node_for_hash_join)(*alloc_bf_ctx)); + OC( (allocate_runtime_filter_for_hash_join)(*alloc_bf_ctx)); break; } case ALLOC_MONITORING_DUMP: { @@ -4416,36 +4419,140 @@ int ObLogicalOperator::allocate_monitoring_dump_node_above(uint64_t flags, uint6 return ret; } -int ObLogicalOperator::push_down_bloom_filter_expr(ObLogicalOperator *op, - ObLogicalOperator *join_filter_op, double join_filter_rate) +int ObLogicalOperator::add_join_filter_info( + ObLogicalOperator *join_filter_create_op, + ObLogicalOperator *join_filter_use_op, + ObRawExpr *join_filter_expr, + RuntimeFilterType type) { int ret = OB_SUCCESS; - CK(OB_NOT_NULL(op) && OB_NOT_NULL(join_filter_op)); - if (OB_SUCC(ret)) { - ObLogJoinFilter *join_filter_use = static_cast(join_filter_op); - common::ObIArray &exprs = op->get_filter_exprs(); - ObRawExprFactory &expr_factory = get_plan()->get_optimizer_context().get_expr_factory(); - common::ObIArray &join_exprs = join_filter_use->get_join_exprs(); - ObOpRawExpr *join_filter_expr = NULL; - ObSQLSessionInfo *session_info = get_plan()->get_optimizer_context().get_session_info(); - if (OB_FAIL(expr_factory.create_raw_expr(T_OP_JOIN_BLOOM_FILTER, join_filter_expr))) { - LOG_WARN("fail to create raw expr", K(ret)); + ObLogJoinFilter *join_filter_create = static_cast(join_filter_create_op); + ObLogJoinFilter *join_filter_use = static_cast(join_filter_use_op); + int64_t p2p_sequence_id = OB_INVALID_ID; + if (OB_ISNULL(join_filter_create) || OB_ISNULL(join_filter_use) || + OB_ISNULL(join_filter_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("null operator", K(join_filter_use), K(join_filter_create_op), K(ret)); + } else if (OB_FAIL(join_filter_use->add_join_filter_expr(join_filter_expr))) { + LOG_WARN("fail to add join filter use", K(ret)); + } else if (OB_FAIL(join_filter_use->add_join_filter_type(type))) { + LOG_WARN("fail to add join filter use", K(ret)); + } else if (OB_FAIL(join_filter_create->add_join_filter_type(type))) { + LOG_WARN("fail to add join filter use", K(ret)); + } else if (OB_FAIL(PX_P2P_DH.generate_p2p_dh_id(p2p_sequence_id))) { + LOG_WARN("fail to generate p2p dh id", K(ret)); + } else if (OB_FAIL(join_filter_create->add_p2p_sequence_id(p2p_sequence_id))) { + LOG_WARN("fail to add join filter use", K(ret)); + } else if (OB_FAIL(join_filter_use->add_p2p_sequence_id(p2p_sequence_id))) { + LOG_WARN("fail to add join filter use", K(ret)); + } + return ret; +} + +int ObLogicalOperator::add_partition_join_filter_info( + ObLogicalOperator *join_filter_create_op, + RuntimeFilterType type) +{ + int ret = OB_SUCCESS; + int64_t p2p_sequence_id = OB_INVALID_ID; + ObLogJoinFilter *join_filter_create = static_cast(join_filter_create_op); + if (OB_ISNULL(join_filter_create)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("null operator", K(join_filter_create_op), K(ret)); + } else if (OB_FAIL(join_filter_create->add_join_filter_type(type))) { + LOG_WARN("fail to add join filter use", K(ret)); + } else if (OB_FAIL(PX_P2P_DH.generate_p2p_dh_id(p2p_sequence_id))) { + LOG_WARN("fail to generate p2p dh id", K(ret)); + } else if (OB_FAIL(join_filter_create->add_p2p_sequence_id(p2p_sequence_id))) { + LOG_WARN("fail to add join filter use", K(ret)); + } else { + ObLogJoin *join_op = static_cast(this); + ObLogJoinFilter *join_filter_create = static_cast(join_filter_create_op); + if (LOG_JOIN != get_type()) { + //do nothing } else { - for (int i = 0; i < join_exprs.count() && OB_SUCC(ret); ++i) { - if (OB_FAIL(join_filter_expr->add_param_expr(join_exprs.at(i)))) { - LOG_WARN("fail to add param expr", K(ret)); + for (int i = 0; i < join_op->get_join_conditions().count(); ++i) { + if (OB_FAIL(join_filter_create->get_is_null_safe_cmps().push_back(false))) { + LOG_WARN("fail to push back is null safe flag", K(ret)); } } - if (OB_SUCC(ret)) { - if (OB_FAIL(join_filter_expr->formalize(session_info))) { - LOG_WARN("fail to formalize expr", K(ret)); - } else if (OB_FAIL(exprs.push_back(join_filter_expr))) { - LOG_WARN("fail to to push back expr", K(ret)); - } else if (OB_FAIL(add_var_to_array_no_dup(get_plan()->get_predicate_selectivities(), - ObExprSelPair(join_filter_expr, join_filter_rate)))) { - LOG_WARN("fail to add join filter expr", K(ret)); - } else { - join_filter_use->set_join_filter_expr(join_filter_expr); + } + } + return ret; +} + +int ObLogicalOperator::generate_runtime_filter_expr( + ObLogicalOperator *op, + ObLogicalOperator *join_filter_create_op, + ObLogicalOperator *join_filter_use_op, + double join_filter_rate, + RuntimeFilterType type) +{ + int ret = OB_SUCCESS; + ObLogJoinFilter *join_filter_use = static_cast(join_filter_use_op); + ObLogJoinFilter *join_filter_create = static_cast(join_filter_create_op); + common::ObIArray &exprs = op->get_filter_exprs(); + ObRawExprFactory &expr_factory = get_plan()->get_optimizer_context().get_expr_factory(); + common::ObIArray &join_exprs = join_filter_use->get_join_exprs(); + ObOpRawExpr *join_filter_expr = NULL; + ObSQLSessionInfo *session_info = get_plan()->get_optimizer_context().get_session_info(); + if (OB_FAIL(expr_factory.create_raw_expr(T_OP_RUNTIME_FILTER, join_filter_expr))) { + LOG_WARN("fail to create raw expr", K(ret)); + } else { + join_filter_expr->set_runtime_filter_type(type); + for (int i = 0; i < join_exprs.count() && OB_SUCC(ret); ++i) { + if (OB_FAIL(join_filter_expr->add_param_expr(join_exprs.at(i)))) { + LOG_WARN("fail to add param expr", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(join_filter_expr->formalize(session_info))) { + LOG_WARN("fail to formalize expr", K(ret)); + } else if (OB_FAIL(exprs.push_back(join_filter_expr))) { + LOG_WARN("fail to to push back expr", K(ret)); + } else if (OB_FAIL(add_var_to_array_no_dup(get_plan()->get_predicate_selectivities(), + ObExprSelPair(join_filter_expr, join_filter_rate)))) { + LOG_WARN("fail to add join filter expr", K(ret)); + } else if (OB_FAIL(add_join_filter_info(join_filter_create_op, + join_filter_use_op, join_filter_expr, type))) { + LOG_WARN("fail to add join filter info", K(ret)); + } + } + } + return ret; +} + +int ObLogicalOperator::create_runtime_filter_info(ObLogicalOperator *op, + ObLogicalOperator *join_filter_create_op, + ObLogicalOperator *join_filter_use_op, + double join_filter_rate) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(op) || OB_ISNULL(join_filter_create_op) || OB_ISNULL(join_filter_use_op)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("null operator", K(op), K(join_filter_create_op)); + } else if (get_plan()->get_optimizer_context().enable_in_filter() && + OB_FAIL(generate_runtime_filter_expr(op, join_filter_create_op, + join_filter_use_op, join_filter_rate, RuntimeFilterType::IN))) { + LOG_WARN("fail to generate range filter expr", K(ret)); + } else if (get_plan()->get_optimizer_context().enable_range_filter() && + OB_FAIL(generate_runtime_filter_expr(op, join_filter_create_op, + join_filter_use_op, join_filter_rate, RuntimeFilterType::RANGE))) { + LOG_WARN("fail to generate range filter expr", K(ret)); + } else if (get_plan()->get_optimizer_context().enable_bloom_filter() && + OB_FAIL(generate_runtime_filter_expr(op, join_filter_create_op, + join_filter_use_op, join_filter_rate, RuntimeFilterType::BLOOM_FILTER))) { + LOG_WARN("fail to generate range filter expr", K(ret)); + } else { + ObLogJoin *join_op = static_cast(this); + ObLogJoinFilter *join_filter_create = static_cast(join_filter_create_op); + if (LOG_JOIN != get_type()) { + //do nothing + } else { + for (int i = 0; i < join_op->get_join_conditions().count(); ++i) { + if (OB_FAIL(join_filter_create->get_is_null_safe_cmps().push_back( + T_OP_NSEQ == join_op->get_join_conditions().at(i)->get_expr_type()))) { + LOG_WARN("fail to push back is null safe flag", K(ret)); } } } @@ -4499,7 +4606,7 @@ int ObLogicalOperator::allocate_partition_join_filter(const ObIArray(this)->is_shared_hash_join(); scan_op = static_cast(node); @@ -4534,16 +4644,20 @@ int ObLogicalOperator::allocate_partition_join_filter(const ObIArrayset_parent(this); set_child(first_child, join_filter_create); join_filter_create->set_filter_length(info.sharding_->get_part_cnt() * 2); - if (is_shared_hash_join) { - join_filter_create->set_is_shared_partition_join_filter(); + join_filter_create->set_is_use_filter_shuffle(right_has_exchange); + if (is_partition_wise_ && !right_has_exchange) { + join_filter_create->set_is_no_shared_partition_join_filter(); } else { - join_filter_create->set_is_no_shared_partition_join_filter(); + join_filter_create->set_is_shared_partition_join_filter(); } join_filter_create->set_tablet_id_expr(info.calc_part_id_expr_); - join_filter_create->set_is_use_filter_shuffle(false); OZ(join_filter_create->compute_property()); OZ(bf_info.init(get_plan()->get_optimizer_context().get_session_info()->get_effective_tenant_id(), - filter_id, GCTX.server_id_, is_shared_hash_join, info.skip_subpart_)); + filter_id, GCTX.server_id_, + join_filter_create->is_shared_join_filter(), + info.skip_subpart_, + join_filter_create->get_p2p_sequence_ids().at(0), + right_has_exchange)); scan_op->set_join_filter_info(bf_info); filter_id++; for (int j = 0; j < info.lexprs_.count() && OB_SUCC(ret); ++j) { @@ -4616,10 +4730,6 @@ int ObLogicalOperator::allocate_normal_join_filter(const ObIArrayset_is_use_filter_shuffle(true); join_filter_use->set_is_use_filter_shuffle(true); - // receive op record the id of bloom filter that will be sent by him - if (OB_FAIL(mark_bloom_filter_id_to_receive_op(join_filter_use, filter_id))) { - LOG_WARN("failed to mark bloom filter id to receive op", K(filter_id), K(join_filter_use)); - } } if (is_partition_wise_ && !right_has_exchange) { join_filter_create->set_is_non_shared_join_filter(); @@ -4655,37 +4765,15 @@ int ObLogicalOperator::allocate_normal_join_filter(const ObIArrayget_join_exprs().push_back(lexpr)); OZ(join_filter_use->get_join_exprs().push_back(rexpr)); } - OZ(push_down_bloom_filter_expr(node, join_filter_use, info.join_filter_selectivity_)); + OZ(create_runtime_filter_info(node, + join_filter_create, join_filter_use, info.join_filter_selectivity_)); } } } return ret; } -int ObLogicalOperator::mark_bloom_filter_id_to_receive_op(ObLogicalOperator *filter_use, int64_t filter_id) -{ - int ret = OB_SUCCESS; - ObLogicalOperator *parent = NULL; - if (OB_ISNULL(filter_use)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpect null", K(ret)); - } else if (FALSE_IT(parent = filter_use->get_parent())) { - } else if (NULL != parent) { - if (log_op_def::LOG_EXCHANGE == parent->get_type() && - static_cast(parent)->is_consumer()) { - if (OB_FAIL(static_cast(parent)->get_bloom_filter_ids().push_back(filter_id))) { - LOG_WARN("ObLogExchange failed to record bloom filter id", K(parent), K(filter_id), K(ret)); - } else { - LOG_DEBUG("ObLogExchange succ to record bloom filter id", K(parent), K(filter_id), K(parent->get_name()), KP(parent), K(parent->get_op_id()), K(filter_id)); - } - } else if (OB_FAIL(SMART_CALL(mark_bloom_filter_id_to_receive_op(parent, filter_id)))) { - LOG_WARN("mark bloom filter id to receive op failed", K(ret)); - } - } - return ret; -} - -int ObLogicalOperator::allocate_bf_node_for_hash_join(AllocBloomFilterContext &ctx) +int ObLogicalOperator::allocate_runtime_filter_for_hash_join(AllocBloomFilterContext &ctx) { int ret = OB_SUCCESS; ObLogJoin *join_op = static_cast(this); diff --git a/src/sql/optimizer/ob_logical_operator.h b/src/sql/optimizer/ob_logical_operator.h index 562483c6d..41d55897c 100644 --- a/src/sql/optimizer/ob_logical_operator.h +++ b/src/sql/optimizer/ob_logical_operator.h @@ -1367,9 +1367,9 @@ public: */ int check_has_exchange_below(bool &has_exchange) const; /** - * Allocate bloom filter operator. + * Allocate runtime filter operator. */ - int allocate_bf_node_for_hash_join(AllocBloomFilterContext &ctx); + int allocate_runtime_filter_for_hash_join(AllocBloomFilterContext &ctx); static int check_is_table_scan(const ObLogicalOperator &op, bool &is_table_scan); @@ -1778,9 +1778,27 @@ private: int64_t &filter_id); int allocate_normal_join_filter(const ObIArray &infos, int64_t &filter_id); - int mark_bloom_filter_id_to_receive_op(ObLogicalOperator *filter_use, int64_t filter_id); - int push_down_bloom_filter_expr(ObLogicalOperator *op, - ObLogicalOperator *join_filter_op, double join_filter_rate); + int create_runtime_filter_info( + ObLogicalOperator *op, + ObLogicalOperator *join_filter_creare_op, + ObLogicalOperator *join_filter_use_op, + double join_filter_rate); + int add_join_filter_info( + ObLogicalOperator *join_filter_creare_op, + ObLogicalOperator *join_filter_use_op, + ObRawExpr *join_filter_expr, + RuntimeFilterType type); + int add_partition_join_filter_info( + ObLogicalOperator *join_filter_creare_op, + RuntimeFilterType type); + int generate_runtime_filter_expr( + ObLogicalOperator *op, + ObLogicalOperator *join_filter_creare_op, + ObLogicalOperator *join_filter_use_op, + double join_filter_rate, + RuntimeFilterType type); + + /* manual set dop for each dfo */ int refine_dop_by_hint(); int check_has_temp_table_access(ObLogicalOperator *cur, bool &has_temp_table_access); diff --git a/src/sql/optimizer/ob_optimizer.h b/src/sql/optimizer/ob_optimizer.h index 0dd32f162..732173d21 100644 --- a/src/sql/optimizer/ob_optimizer.h +++ b/src/sql/optimizer/ob_optimizer.h @@ -83,7 +83,7 @@ namespace sql PX_RESCAN, ALLOC_MONITORING_DUMP, PX_ESTIMATE_SIZE, - BLOOM_FILTER, + RUNTIME_FILTER, ALLOC_STARTUP_EXPR, ADJUST_SHARED_EXPR, diff --git a/src/sql/optimizer/ob_optimizer_context.h b/src/sql/optimizer/ob_optimizer_context.h index 32c6d33d5..c517c72ff 100644 --- a/src/sql/optimizer/ob_optimizer_context.h +++ b/src/sql/optimizer/ob_optimizer_context.h @@ -23,8 +23,11 @@ #include "observer/omt/ob_tenant_config_mgr.h" #include "sql/optimizer/ob_sharding_info.h" #include "sql/optimizer/ob_opt_est_cost.h" +#include "sql/engine/expr/ob_expr_join_filter.h" #include "sql/engine/aggregate/ob_adaptive_bypass_ctrl.h" #include "sql/optimizer/ob_dynamic_sampling.h" +#include "share/config/ob_config_helper.h" + namespace oceanbase { @@ -126,7 +129,7 @@ ObOptimizerContext(ObSQLSessionInfo *session_info, enable_batch_opt_(-1), force_default_stat_(false), eval_plan_cost_(false), - enable_bloom_filter_(-1), + runtime_filter_type_(-1), batch_size_(0), root_stmt_(root_stmt), enable_px_batch_rescan_(-1), @@ -304,15 +307,32 @@ ObOptimizerContext(ObSQLSessionInfo *session_info, void set_use_default_stat() { force_default_stat_ = true; } void set_cost_evaluation() { eval_plan_cost_ = true; } bool is_cost_evaluation() { return eval_plan_cost_; } - bool enable_bloom_filter() { - if (-1 == enable_bloom_filter_) { - if (session_info_->is_enable_bloom_filter()) { - enable_bloom_filter_ = 1; - } else { - enable_bloom_filter_ = 0; - } + bool enable_runtime_filter() { + if (0 > runtime_filter_type_) { + get_runtime_filter_type(); } - return 1 == enable_bloom_filter_; + return 0 != runtime_filter_type_ && GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_2_0_0; + } + bool enable_bloom_filter() { + if (0 > runtime_filter_type_) { + get_runtime_filter_type(); + } + return 0 != (runtime_filter_type_ & (1 << RuntimeFilterType::BLOOM_FILTER)); + } + bool enable_range_filter() { + if (0 > runtime_filter_type_) { + get_runtime_filter_type(); + } + return 0 != (runtime_filter_type_ & (1 << RuntimeFilterType::RANGE)); + } + bool enable_in_filter() { + if (0 > runtime_filter_type_) { + get_runtime_filter_type(); + } + return 0 != (runtime_filter_type_ & (1 << RuntimeFilterType::IN)); + } + void get_runtime_filter_type() { + runtime_filter_type_ = session_info_->get_runtime_filter_type(); } int64_t get_batch_size() const { return batch_size_; } @@ -509,7 +529,8 @@ private: int enable_batch_opt_; bool force_default_stat_; bool eval_plan_cost_; - int enable_bloom_filter_; //租户配置项是否打开join filter. + // for runtime filter + int64_t runtime_filter_type_; // batch row count for vectorized execution, 0 for no vectorize. int64_t batch_size_; ObDMLStmt *root_stmt_; diff --git a/src/sql/plan_cache/ob_plan_cache_util.cpp b/src/sql/plan_cache/ob_plan_cache_util.cpp index fd1a83713..8e35881b1 100644 --- a/src/sql/plan_cache/ob_plan_cache_util.cpp +++ b/src/sql/plan_cache/ob_plan_cache_util.cpp @@ -395,6 +395,7 @@ int ObConfigInfoInPC::load_influence_plan_config() // here to add value of configs that can influence execution plan. enable_px_ordered_coord_ = GCONF._enable_px_ordered_coord; enable_newsort_ = GCONF._enable_newsort; + is_enable_px_fast_reclaim_ = GCONF._enable_px_fast_reclaim; // For Tenant configs // tenant config use tenant_config to get configs @@ -444,8 +445,11 @@ int ObConfigInfoInPC::serialize_configs(char *buf, int buf_len, int64_t &pos) SQL_PC_LOG(WARN, "failed to databuff_printf", K(ret), K(px_join_skew_minfreq_)); } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, - "%lu", min_cluster_version_))) { + "%lu,", min_cluster_version_))) { SQL_PC_LOG(WARN, "failed to databuff_printf", K(ret), K(min_cluster_version_)); + } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, + "%d", is_enable_px_fast_reclaim_))) { + SQL_PC_LOG(WARN, "failed to databuff_printf", K(ret), K(is_enable_px_fast_reclaim_)); } else { // do nothing } diff --git a/src/sql/plan_cache/ob_plan_cache_util.h b/src/sql/plan_cache/ob_plan_cache_util.h index 25bad1cb4..f2cbe51cb 100644 --- a/src/sql/plan_cache/ob_plan_cache_util.h +++ b/src/sql/plan_cache/ob_plan_cache_util.h @@ -976,6 +976,7 @@ public: px_join_skew_handling_(true), px_join_skew_minfreq_(30), min_cluster_version_(0), + is_enable_px_fast_reclaim_(false), cluster_config_version_(-1), tenant_config_version_(-1), tenant_id_(0) @@ -1015,6 +1016,7 @@ public: bool px_join_skew_handling_; int8_t px_join_skew_minfreq_; uint64_t min_cluster_version_; + bool is_enable_px_fast_reclaim_; private: // current cluster config version_ diff --git a/src/sql/resolver/expr/ob_raw_expr.cpp b/src/sql/resolver/expr/ob_raw_expr.cpp index bf6876d60..380a8c1b1 100644 --- a/src/sql/resolver/expr/ob_raw_expr.cpp +++ b/src/sql/resolver/expr/ob_raw_expr.cpp @@ -56,7 +56,7 @@ ObRawExpr *USELESS_POINTER = NULL; SQL_RESV_LOG(WARN, "failed to create raw expr", K(ret), K(expr_type)); \ } \ } - + template <> int ObRawExprFactory::create_raw_expr(ObItemType expr_type, ObSysFunRawExpr *&raw_expr) { @@ -2361,9 +2361,21 @@ int ObOpRawExpr::get_name_internal(char *buf, const int64_t buf_len, int64_t &po LOG_WARN("fail to BUF_PRINTF", K(ret)); } } - } else if (T_OP_JOIN_BLOOM_FILTER == get_expr_type()) { - if (OB_FAIL(BUF_PRINTF("SYS_OP_BLOOM_FILTER("))) { - LOG_WARN("fail to BUF_PRINTF", K(ret)); + } else if (T_OP_RUNTIME_FILTER == get_expr_type()) { + if (RuntimeFilterType::BLOOM_FILTER == runtime_filter_type_) { + if (OB_FAIL(BUF_PRINTF("RF_BLOOM_FILTER("))) { + LOG_WARN("fail to BUF_PRINTF", K(ret)); + } + } else if (RuntimeFilterType::RANGE == runtime_filter_type_) { + if (OB_FAIL(BUF_PRINTF("RF_RANGE_FILTER("))) { + LOG_WARN("fail to BUF_PRINTF", K(ret)); + } + } else if (RuntimeFilterType::IN == runtime_filter_type_) { + if (OB_FAIL(BUF_PRINTF("RF_IN_FILTER("))) { + LOG_WARN("fail to BUF_PRINTF", K(ret)); + } + } + if (OB_FAIL(ret)) { } else { for (int64_t i = 0; OB_SUCC(ret) && i < get_param_count() ; ++i) { if (OB_ISNULL(get_param_expr(i))) { diff --git a/src/sql/resolver/expr/ob_raw_expr.h b/src/sql/resolver/expr/ob_raw_expr.h index c28262265..d24fa02c9 100644 --- a/src/sql/resolver/expr/ob_raw_expr.h +++ b/src/sql/resolver/expr/ob_raw_expr.h @@ -41,6 +41,7 @@ #include "sql/code_generator/ob_static_engine_expr_cg.h" #include "pl/ob_pl_type.h" #include "share/schema/ob_trigger_info.h" +#include "sql/engine/expr/ob_expr_join_filter.h" #include "sql/engine/expr/ob_expr_calc_partition_id.h" #include "sql/resolver/dml/ob_raw_expr_sets.h" namespace oceanbase @@ -1658,7 +1659,8 @@ public: is_calculated_(false), is_deterministic_(true), partition_id_calc_type_(CALC_INVALID), - may_add_interval_part_(MayAddIntervalPart::NO) + may_add_interval_part_(MayAddIntervalPart::NO), + runtime_filter_type_(NOT_INIT_RUNTIME_FILTER_TYPE) { } virtual ~ObRawExpr(); @@ -1710,9 +1712,9 @@ public: /** +-is_static_scalar_const_expr * (1、1+2、sysdate) | (1,not for[1,2,3]) * +-is_static_const_expr-+ - * | - * is_const_or_calculable_expr-+ - * (1、1+2、2+?、sysdate) | + * | + * is_const_or_calculable_expr-+ + * (1、1+2、2+?、sysdate) | * +-is_dynamic_const_expr * (2 + ?) */ @@ -1856,6 +1858,8 @@ public: } MayAddIntervalPart get_may_add_interval_part() const { return may_add_interval_part_;} + RuntimeFilterType get_runtime_filter_type() const { return runtime_filter_type_; } + void set_runtime_filter_type(RuntimeFilterType type) { runtime_filter_type_ = type; } VIRTUAL_TO_STRING_KV(N_ITEM_TYPE, type_, N_RESULT_TYPE, result_type_, N_EXPR_INFO, info_, @@ -1904,7 +1908,8 @@ protected: bool is_calculated_; // 用于在新引擎 cg 中检查 raw expr 是否被重复计算 bool is_deterministic_; //expr is deterministic, given the same inputs, returns the same result PartitionIdCalcType partition_id_calc_type_; //for calc_partition_id func to mark calc part type - MayAddIntervalPart may_add_interval_part_; // for calc_partition_id + MayAddIntervalPart may_add_interval_part_; // for calc_partition_id + RuntimeFilterType runtime_filter_type_; // for runtime filter private: DISALLOW_COPY_AND_ASSIGN(ObRawExpr); }; @@ -1971,7 +1976,7 @@ inline bool ObRawExpr::is_const_expr() const inline bool ObRawExpr::is_static_const_expr() const { - return is_const_expr() && + return is_const_expr() && !has_flag(CNT_DYNAMIC_PARAM); } @@ -4745,7 +4750,7 @@ public: inline void set_stmt_type(stmt::StmtType type) { type_ = type; } inline void set_route_sql(const common::ObString &sql) { route_sql_ = sql; } inline void set_subquery_result_type(const sql::ObExprResType &type) - { + { subquery_result_type_ = type; } inline const common::ObString &get_ps_sql() const { return ps_sql_; } diff --git a/src/sql/session/ob_basic_session_info.cpp b/src/sql/session/ob_basic_session_info.cpp index 366150dbd..2fb97aeea 100644 --- a/src/sql/session/ob_basic_session_info.cpp +++ b/src/sql/session/ob_basic_session_info.cpp @@ -546,7 +546,7 @@ int ObBasicSessionInfo::switch_tenant(uint64_t effective_tenant_id) ret = OB_NOT_SUPPORTED; // only inner-SQL goes switch_tenant and may fall into such state // print out error to easy trouble-shot - LOG_USER_ERROR(OB_NOT_SUPPORTED, + LOG_USER_ERROR(OB_NOT_SUPPORTED, "try to switch to another tenant without commit/rollback in a transaction"); LOG_ERROR("try to switch another tenant while session has active txn," " you must commit/rollback first", K(ret), @@ -2576,6 +2576,35 @@ OB_INLINE int ObBasicSessionInfo::process_session_variable(ObSysVarClassType var OX (sys_vars_cache_.set_ob_max_read_stale_time(int_val)); break; } + case SYS_VAR_RUNTIME_FILTER_TYPE: { + ObString str; + if (OB_FAIL(val.get_string(str))) { + LOG_WARN("fail to get str value", K(ret), K(var)); + } else { + int64_t run_time_filter_type = ObConfigRuntimeFilterChecker:: + get_runtime_filter_type(str.ptr(), str.length()); + sys_vars_cache_.set_runtime_filter_type(run_time_filter_type); + } + break; + } + case SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS: { + int64_t int_val = 0; + OZ (val.get_int(int_val), val); + OX (sys_vars_cache_.set_runtime_filter_wait_time_ms(int_val)); + break; + } + case SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM: { + int64_t int_val = 0; + OZ (val.get_int(int_val), val); + OX (sys_vars_cache_.set_runtime_filter_max_in_num(int_val)); + break; + } + case SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE: { + int64_t int_val = 0; + OZ (val.get_int(int_val), val); + OX (sys_vars_cache_.set_runtime_bloom_filter_max_size(int_val)); + break; + } default: { //do nothing } @@ -2972,6 +3001,35 @@ int ObBasicSessionInfo::fill_sys_vars_cache_base_value( OX (sys_vars_cache.set_base_ob_max_read_stale_time(int_val)); break; } + case SYS_VAR_RUNTIME_FILTER_TYPE: { + ObString str; + if (OB_FAIL(val.get_string(str))) { + LOG_WARN("fail to get str value", K(ret), K(var)); + } else { + int64_t run_time_filter_type = ObConfigRuntimeFilterChecker:: + get_runtime_filter_type(str.ptr(), str.length()); + sys_vars_cache.set_base_runtime_filter_type(run_time_filter_type); + } + break; + } + case SYS_VAR_RUNTIME_FILTER_WAIT_TIME_MS: { + int64_t int_val = 0; + OZ (val.get_int(int_val), val); + OX (sys_vars_cache.set_base_runtime_filter_wait_time_ms(int_val)); + break; + } + case SYS_VAR_RUNTIME_FILTER_MAX_IN_NUM: { + int64_t int_val = 0; + OZ (val.get_int(int_val), val); + OX (sys_vars_cache.set_base_runtime_filter_max_in_num(int_val)); + break; + } + case SYS_VAR_RUNTIME_BLOOM_FILTER_MAX_SIZE: { + int64_t int_val = 0; + OZ (val.get_int(int_val), val); + OX (sys_vars_cache.set_base_runtime_bloom_filter_max_size(int_val)); + break; + } default: { //do nothing } @@ -3949,7 +4007,11 @@ OB_DEF_SERIALIZE(ObBasicSessionInfo::SysVarsCacheData) ob_trx_lock_timeout_, ob_trace_info_, ob_plsql_ccflags_, - ob_max_read_stale_time_); + ob_max_read_stale_time_, + runtime_filter_type_, + runtime_filter_wait_time_ms_, + runtime_filter_max_in_num_, + runtime_bloom_filter_max_size_); return ret; } @@ -3975,7 +4037,11 @@ OB_DEF_DESERIALIZE(ObBasicSessionInfo::SysVarsCacheData) ob_trx_lock_timeout_, ob_trace_info_, ob_plsql_ccflags_, - ob_max_read_stale_time_); + ob_max_read_stale_time_, + runtime_filter_type_, + runtime_filter_wait_time_ms_, + runtime_filter_max_in_num_, + runtime_bloom_filter_max_size_); set_nls_date_format(nls_formats_[NLS_DATE]); set_nls_timestamp_format(nls_formats_[NLS_TIMESTAMP]); set_nls_timestamp_tz_format(nls_formats_[NLS_TIMESTAMP_TZ]); @@ -4006,7 +4072,11 @@ OB_DEF_SERIALIZE_SIZE(ObBasicSessionInfo::SysVarsCacheData) ob_trx_lock_timeout_, ob_trace_info_, ob_plsql_ccflags_, - ob_max_read_stale_time_); + ob_max_read_stale_time_, + runtime_filter_type_, + runtime_filter_wait_time_ms_, + runtime_filter_max_in_num_, + runtime_bloom_filter_max_size_); return len; } diff --git a/src/sql/session/ob_basic_session_info.h b/src/sql/session/ob_basic_session_info.h index 06220e804..e7c3cd502 100644 --- a/src/sql/session/ob_basic_session_info.h +++ b/src/sql/session/ob_basic_session_info.h @@ -1176,6 +1176,11 @@ public: return sys_vars_cache_.get_cursor_sharing_mode() == ObCursorSharingMode::EXACT_MODE; } + int64_t get_runtime_filter_type() const { return sys_vars_cache_.get_runtime_filter_type(); } + int64_t get_runtime_filter_wait_time_ms() const { return sys_vars_cache_.get_runtime_filter_wait_time_ms(); } + int64_t get_runtime_filter_max_in_num() const { return sys_vars_cache_.get_runtime_filter_max_in_num(); } + int64_t get_runtime_bloom_filter_max_size() const { return sys_vars_cache_.get_runtime_bloom_filter_max_size(); } + const ObString &get_app_trace_id() const { return app_trace_id_; } void set_app_trace_id(common::ObString trace_id) { @@ -1497,7 +1502,12 @@ public: nls_nation_collation_(CS_TYPE_INVALID), ob_trace_info_(), ob_plsql_ccflags_(), - ob_max_read_stale_time_(0) + ob_max_read_stale_time_(0), + runtime_filter_type_(0), + runtime_filter_wait_time_ms_(0), + runtime_filter_max_in_num_(0), + runtime_bloom_filter_max_size_(INT_MAX32) + { for (int64_t i = 0; i < ObNLSFormatEnum::NLS_MAX; ++i) { MEMSET(nls_formats_buf_[i], 0, MAX_NLS_FORMAT_STR_LEN); @@ -1552,6 +1562,10 @@ public: ob_plsql_ccflags_.reset(); log_row_value_option_.reset(); ob_max_read_stale_time_ = 0; + runtime_filter_type_ = 0; + runtime_filter_wait_time_ms_ = 0; + runtime_filter_max_in_num_ = 0; + runtime_bloom_filter_max_size_ = INT32_MAX; } inline bool operator==(const SysVarsCacheData &other) const { @@ -1758,6 +1772,10 @@ public: ObString ob_plsql_ccflags_; char plsql_ccflags_[OB_TMP_BUF_SIZE_256]; int64_t ob_max_read_stale_time_; + int64_t runtime_filter_type_; + int64_t runtime_filter_wait_time_ms_; + int64_t runtime_filter_max_in_num_; + int64_t runtime_bloom_filter_max_size_; private: char nls_formats_buf_[ObNLSFormatEnum::NLS_MAX][MAX_NLS_FORMAT_STR_LEN]; }; @@ -1865,6 +1883,10 @@ private: DEF_SYS_VAR_CACHE_FUNCS_STR(iso_nls_currency); DEF_SYS_VAR_CACHE_FUNCS_STR(log_row_value_option); DEF_SYS_VAR_CACHE_FUNCS(int64_t, ob_max_read_stale_time); + DEF_SYS_VAR_CACHE_FUNCS(int64_t, runtime_filter_type); + DEF_SYS_VAR_CACHE_FUNCS(int64_t, runtime_filter_wait_time_ms); + DEF_SYS_VAR_CACHE_FUNCS(int64_t, runtime_filter_max_in_num); + DEF_SYS_VAR_CACHE_FUNCS(int64_t, runtime_bloom_filter_max_size); void set_autocommit_info(bool inc_value) { inc_data_.autocommit_ = inc_value; @@ -1928,6 +1950,10 @@ private: bool inc_iso_nls_currency_:1; bool inc_log_row_value_option_:1; bool inc_ob_max_read_stale_time_:1; + bool inc_runtime_filter_type_:1; + bool inc_runtime_filter_wait_time_ms_:1; + bool inc_runtime_filter_max_in_num_:1; + bool inc_runtime_bloom_filter_max_size_:1; }; }; }; diff --git a/src/storage/direct_load/ob_direct_load_multi_map.h b/src/storage/direct_load/ob_direct_load_multi_map.h index f0c7606b7..89b3a833d 100644 --- a/src/storage/direct_load/ob_direct_load_multi_map.h +++ b/src/storage/direct_load/ob_direct_load_multi_map.h @@ -8,6 +8,7 @@ #include "lib/hash/ob_concurrent_hash_map.h" #include "lib/lock/ob_mutex.h" #include "lib/container/ob_array.h" +#include "share/rc/ob_tenant_base.h" #include "share/ob_errno.h" namespace oceanbase diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result index aff57d162..6d6ec50d2 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result @@ -268,6 +268,7 @@ _enable_plan_cache_mem_diagnosis _enable_protocol_diagnose _enable_px_batch_rescan _enable_px_bloom_filter_sync +_enable_px_fast_reclaim _enable_px_ordered_coord _enable_reserved_user_dcl_restriction _enable_resource_limit_spec diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result index 4c209bd75..3931295a9 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/inner_table_overall.result @@ -595,6 +595,7 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 12384 __all_virtual_thread 2 201001 1 12393 __all_virtual_virtual_long_ops_status_mysql_sys_agent 2 201001 1 12395 __all_virtual_timestamp_service 2 201001 1 +12397 __all_virtual_px_p2p_datahub 2 201001 1 20001 GV$OB_PLAN_CACHE_STAT 1 201001 1 20002 GV$OB_PLAN_CACHE_PLAN_STAT 1 201001 1 20003 SCHEMATA 1 201002 1 @@ -893,6 +894,10 @@ select 0xffffffffff & table_id, table_name, table_type, database_id, part_num fr 21404 V$OB_TIMESTAMP_SERVICE 1 201001 1 21417 DBA_OB_EXTERNAL_TABLE_FILES 1 201001 1 21418 ALL_OB_EXTERNAL_TABLE_FILES 1 201001 1 +21419 GV$OB_PX_P2P_DATAHUB 1 201001 1 +21420 V$OB_PX_P2P_DATAHUB 1 201001 1 +21421 GV$SQL_JOIN_FILTER 1 201001 1 +21422 V$SQL_JOIN_FILTER 1 201001 1 21423 DBA_OB_TABLE_STAT_STALE_INFO 1 201001 1 21425 CDB_OB_EXTERNAL_TABLE_FILES 1 201001 1 check sys table count and table_id range success diff --git a/tools/deploy/mysql_test/test_suite/px/r/mysql/add_material.result b/tools/deploy/mysql_test/test_suite/px/r/mysql/add_material.result index 01da56408..3360ba924 100644 --- a/tools/deploy/mysql_test/test_suite/px/r/mysql/add_material.result +++ b/tools/deploy/mysql_test/test_suite/px/r/mysql/add_material.result @@ -11,19 +11,20 @@ select /*+ USE_PX parallel(3) */* from (select c1,count(c2) c2 from t1 group by c1)c) a join (select c1,count(*) c2 from t2 group by c1)b on a.c1=b.c1 ; Query Plan -=============================================== -|ID|OPERATOR |NAME | ------------------------------------------------ -|0 |PX COORDINATOR | | -|1 |└─EXCHANGE OUT DISTR |:EX10001| -|2 | └─HASH JOIN | | -|3 | ├─PX PARTITION ITERATOR | | -|4 | │ └─TABLE FULL SCAN |t1 | -|5 | └─EXCHANGE IN DISTR | | -|6 | └─EXCHANGE OUT DISTR (PKEY)|:EX10000| -|7 | └─PX BLOCK ITERATOR | | -|8 | └─TABLE FULL SCAN |t2 | -=============================================== +================================================= +|ID|OPERATOR |NAME | +------------------------------------------------- +|0 |PX COORDINATOR | | +|1 |└─EXCHANGE OUT DISTR |:EX10001| +|2 | └─HASH JOIN | | +|3 | ├─PART JOIN FILTER CREATE |:RF0000 | +|4 | │ └─PX PARTITION ITERATOR | | +|5 | │ └─TABLE FULL SCAN |t1 | +|6 | └─EXCHANGE IN DISTR | | +|7 | └─EXCHANGE OUT DISTR (PKEY) |:EX10000| +|8 | └─PX BLOCK HASH JOIN-FILTER|:RF0000 | +|9 | └─TABLE FULL SCAN |t2 | +================================================= Outputs & filters: ------------------------------------- 0 - output([INTERNAL_FUNCTION(t1.c1, 1, t2.c1, 1)]), filter(nil), rowset=256 @@ -32,16 +33,18 @@ Outputs & filters: 2 - output([t1.c1], [t2.c1]), filter(nil), rowset=256 equal_conds([t1.c1 = t2.c1]), other_conds(nil) 3 - output([t1.c1]), filter(nil), rowset=256 - affinitize + RF_TYPE(bloom), RF_EXPR[t1.c1] 4 - output([t1.c1]), filter(nil), rowset=256 + affinitize + 5 - output([t1.c1]), filter(nil), rowset=256 access([t1.c1]), partitions(p[0-9]) is_index_back=false, is_global_index=false, range_key([t1.c1]), range(MIN ; MAX)always true - 5 - output([t2.c1]), filter(nil), rowset=256 6 - output([t2.c1]), filter(nil), rowset=256 - (#keys=1, [t2.c1]), dop=3 7 - output([t2.c1]), filter(nil), rowset=256 + (#keys=1, [t2.c1]), dop=3 8 - output([t2.c1]), filter(nil), rowset=256 + 9 - output([t2.c1], [PARTITION_ID]), filter(nil), rowset=256 access([t2.c1]), partitions(p[0-5]) is_index_back=false, is_global_index=false, range_key([t2.c1]), range(MIN ; MAX)always true @@ -73,12 +76,12 @@ Query Plan |4 | └─EXCHANGE OUT DISTR (HASH) |:EX10001| |5 | └─HASH GROUP BY | | |6 | └─HASH JOIN | | -|7 | ├─PART JOIN FILTER CREATE |:BF0000 | +|7 | ├─PART JOIN FILTER CREATE |:RF0000 | |8 | │ └─EXCHANGE IN DISTR | | |9 | │ └─EXCHANGE OUT DISTR (PKEY)|:EX10000| |10| │ └─PX BLOCK ITERATOR | | |11| │ └─TABLE FULL SCAN |b | -|12| └─PX PARTITION HASH JOIN-FILTER|:BF0000 | +|12| └─PX PARTITION HASH JOIN-FILTER|:RF0000 | |13| └─TABLE FULL SCAN |a | ========================================================= Outputs & filters: @@ -96,6 +99,7 @@ Outputs & filters: 6 - output([a.c2]), filter(nil), rowset=256 equal_conds([a.c1 = b.c1]), other_conds(nil) 7 - output([b.c1]), filter(nil), rowset=256 + RF_TYPE(bloom), RF_EXPR[b.c1] 8 - output([b.c1]), filter(nil), rowset=256 9 - output([b.c1]), filter(nil), rowset=256 (#keys=1, [b.c1]), dop=3 diff --git a/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/px_basic.result b/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/px_basic.result index 8a60c2df2..d2f275b13 100644 --- a/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/px_basic.result +++ b/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/px_basic.result @@ -283,20 +283,21 @@ select /*+ parallel(2) pq_distribute(a hash hash) NO_USE_HASH_AGGREGATION */ a.c ## broadcast shuffle EXPLAIN BASIC select /*+ parallel(2) pq_distribute(b broadcast none) */ * from t1 a, t2 b where a.c1=b.c2 order by a.c1,b.c1; Query Plan -================================================= -|ID|OPERATOR |NAME | -------------------------------------------------- -|0 |PX COORDINATOR MERGE SORT | | -|1 |└─EXCHANGE OUT DISTR |:EX10001| -|2 | └─SORT | | -|3 | └─HASH JOIN | | -|4 | ├─PX PARTITION ITERATOR | | -|5 | │ └─TABLE FULL SCAN |b | -|6 | └─EXCHANGE IN DISTR | | -|7 | └─EXCHANGE OUT DISTR (PKEY)|:EX10000| -|8 | └─PX BLOCK ITERATOR | | -|9 | └─TABLE FULL SCAN |a | -================================================= +=================================================== +|ID|OPERATOR |NAME | +--------------------------------------------------- +|0 |PX COORDINATOR MERGE SORT | | +|1 |└─EXCHANGE OUT DISTR |:EX10001| +|2 | └─SORT | | +|3 | └─HASH JOIN | | +|4 | ├─PART JOIN FILTER CREATE |:RF0000 | +|5 | │ └─PX PARTITION ITERATOR | | +|6 | │ └─TABLE FULL SCAN |b | +|7 | └─EXCHANGE IN DISTR | | +|8 | └─EXCHANGE OUT DISTR (PKEY) |:EX10000| +|9 | └─PX BLOCK HASH JOIN-FILTER|:RF0000 | +|10| └─TABLE FULL SCAN |a | +=================================================== Outputs & filters: ------------------------------------- 0 - output([INTERNAL_FUNCTION(a.c1, a.c2, b.c1, b.c2)]), filter(nil), rowset=256 @@ -307,17 +308,19 @@ Outputs & filters: sort_keys([a.c1, ASC], [b.c1, ASC]) 3 - output([a.c1], [b.c1], [b.c2], [a.c2]), filter(nil), rowset=256 equal_conds([a.c1 = b.c2]), other_conds(nil) - 4 - output([b.c2], [b.c1]), filter(nil), rowset=256 - affinitize + 4 - output([b.c1], [b.c2]), filter(nil), rowset=256 + RF_TYPE(bloom), RF_EXPR[b.c2] 5 - output([b.c2], [b.c1]), filter(nil), rowset=256 + affinitize + 6 - output([b.c2], [b.c1]), filter(nil), rowset=256 access([b.c2], [b.c1]), partitions(p[0-5]) is_index_back=false, is_global_index=false, range_key([b.__pk_increment]), range(MIN ; MAX)always true - 6 - output([a.c1], [a.c2]), filter(nil), rowset=256 7 - output([a.c1], [a.c2]), filter(nil), rowset=256 - (#keys=1, [a.c1]), dop=2 8 - output([a.c1], [a.c2]), filter(nil), rowset=256 + (#keys=1, [a.c1]), dop=2 9 - output([a.c1], [a.c2]), filter(nil), rowset=256 + 10 - output([a.c1], [a.c2], [PARTITION_ID]), filter(nil), rowset=256 access([a.c1], [a.c2]), partitions(p[0-4]) is_index_back=false, is_global_index=false, range_key([a.__pk_increment]), range(MIN ; MAX)always true diff --git a/unittest/share/CMakeLists.txt b/unittest/share/CMakeLists.txt index 12f2991e6..16aec2896 100644 --- a/unittest/share/CMakeLists.txt +++ b/unittest/share/CMakeLists.txt @@ -81,3 +81,4 @@ add_subdirectory(schema) add_subdirectory(interrupt) add_subdirectory(tablet) add_subdirectory(location_cache) +add_subdirectory(detect) diff --git a/unittest/share/detect/CMakeLists.txt b/unittest/share/detect/CMakeLists.txt new file mode 100644 index 000000000..c410cca66 --- /dev/null +++ b/unittest/share/detect/CMakeLists.txt @@ -0,0 +1 @@ +ob_unittest(test_ob_detect_manager) diff --git a/unittest/share/detect/test_ob_detect_manager.cpp b/unittest/share/detect/test_ob_detect_manager.cpp new file mode 100644 index 000000000..f10166128 --- /dev/null +++ b/unittest/share/detect/test_ob_detect_manager.cpp @@ -0,0 +1,216 @@ +/** + * 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 SQL + +#include +#define private public +#define protected public +#include "share/detect/ob_detect_manager.h" +#include "lib/oblog/ob_log_module.h" +#include "lib/thread/thread_mgr.h" +#include "lib/alloc/memory_dump.h" +#include "share/detect/ob_detect_manager_utils.h" + +using namespace oceanbase; +using namespace oceanbase::common; +using namespace oceanbase::lib; +using namespace std; + +static const uint64_t ACTIVATE_DELAY_TIME_SEC = 5; + +static uint64_t s_tenant_id = 500; +ObDetectManager *dm = new ObDetectManager(s_tenant_id); +ObDetectManagerThread &dm_thr = ObDetectManagerThread::instance(); + +// only detect local, mock an invalid transport; +rpc::frame::ObReqTransport mock_transport(nullptr, nullptr); + +static int init_dm() +{ + int ret = OB_SUCCESS; + GCTX.server_id_ = 10086; + ObAddr self; + self.set_ip_addr("127.0.0.1", 8086); + if (OB_FAIL(dm->init(self, 1))) { + LOG_WARN("failed to init dm"); + } else if (OB_FAIL(dm_thr.init(self, &mock_transport))) { + LOG_WARN("failed to init dm thr"); + } + return ret; +} + +// check memory具有延迟性,从测试的结果来看,大致阈值是5s +// 调用 check_memory_leak 前需要前置sleep +int check_memory_leak() +{ + sleep(15); + int ret = OB_SUCCESS; + uint64_t tenant_id = OB_SERVER_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_, "ReqMapTempContext") == 0 || 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 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 &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; +} + +class ObDetectManagerTest : public ::testing::Test +{ +}; + +TEST_F(ObDetectManagerTest, init_dm) +{ + ASSERT_EQ(OB_SUCCESS, init_dm()); +} + +TEST_F(ObDetectManagerTest, test_register_detectable_id) +{ + // register + ObDetectableId id1; + ObDetectManagerUtils::generate_detectable_id(id1, s_tenant_id); + ObDetectableId id2; + ObDetectManagerUtils::generate_detectable_id(id2, s_tenant_id); + ObDetectableId id3; + ObDetectManagerUtils::generate_detectable_id(id3, s_tenant_id); + ObDetectableId id4; + ASSERT_EQ(OB_SUCCESS, dm->register_detectable_id(id1)); + ASSERT_EQ(OB_HASH_EXIST, dm->register_detectable_id(id1)); + ASSERT_EQ(OB_SUCCESS, dm->register_detectable_id(id2)); + ASSERT_EQ(OB_SUCCESS, dm->register_detectable_id(id3)); + ASSERT_EQ(OB_INVALID_ARGUMENT, dm->register_detectable_id(id4)); + ASSERT_EQ(3, dm->detectable_ids_.size()); + + // seek task alive + ASSERT_TRUE(dm->is_task_alive(id1)); + ASSERT_TRUE(dm->is_task_alive(id2)); + ASSERT_TRUE(dm->is_task_alive(id3)); + + // unregister + ASSERT_EQ(OB_SUCCESS, dm->unregister_detectable_id(id1)); + ASSERT_EQ(OB_HASH_NOT_EXIST, dm->unregister_detectable_id(id1)); + ASSERT_EQ(OB_SUCCESS, dm->unregister_detectable_id(id2)); + ASSERT_EQ(OB_SUCCESS, dm->unregister_detectable_id(id3)); + ASSERT_EQ(OB_INVALID_ARGUMENT, dm->unregister_detectable_id(id4)); + ASSERT_EQ(0, dm->detectable_ids_.size()); + + ASSERT_FALSE(dm->is_task_alive(id1)); + ASSERT_FALSE(dm->is_task_alive(id2)); + ASSERT_FALSE(dm->is_task_alive(id3)); + ASSERT_FALSE(dm->is_task_alive(id4)); +} + +TEST_F(ObDetectManagerTest, test_register_check_item) +{ + ObArray peer_states; + // one id with one callback + ObDetectableId id1; + ObDetectManagerUtils::generate_detectable_id(id1, s_tenant_id); + ObMockDetectCB *cb1 = nullptr; + uint64_t node_sequence_id1 = 0; + auto rsc1 = allocate_mock_resource(); + 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)); + + // one id with multiple callbacks + ObDetectableId id2; + ObDetectManagerUtils::generate_detectable_id(id2, s_tenant_id); + auto rsc2_1 = allocate_mock_resource(); + auto rsc2_2 = allocate_mock_resource(); + ObMockDetectCB *cb2_1 = nullptr; + ObMockDetectCB *cb2_2 = nullptr; + uint64_t node_sequence_id2_1 = 0; + uint64_t node_sequence_id2_2 = 0; + ASSERT_EQ(OB_SUCCESS, dm->register_check_item(id2, cb2_1, node_sequence_id2_1, true/* need_ref */, peer_states, rsc2_1)); + ASSERT_EQ(OB_SUCCESS, dm->register_check_item(id2, cb2_2, node_sequence_id2_2, true/* need_ref */, peer_states, rsc2_2)); + ASSERT_EQ(2, dm->all_check_items_.size()); + + // unregister + cb1->dec_ref_count(); + cb2_1->dec_ref_count(); + cb2_2->dec_ref_count(); + ASSERT_EQ(OB_SUCCESS, dm->unregister_check_item(id1, node_sequence_id1)); + ASSERT_EQ(OB_HASH_NOT_EXIST, dm->unregister_check_item(id1, node_sequence_id1)); + ASSERT_EQ(OB_SUCCESS, dm->unregister_check_item(id2, node_sequence_id2_2)); + ASSERT_EQ(OB_SUCCESS, dm->unregister_check_item(id2, node_sequence_id2_1)); + ASSERT_EQ(0, dm->all_check_items_.size()); +} + +int main(int argc, char **argv) +{ + int ret = OB_SUCCESS; + ObMemoryDump::get_instance().init(); + system("rm -f test_ob_detect_manager.log*"); + OB_LOGGER.set_file_name("test_ob_detect_manager.log", true, true); + OB_LOGGER.set_log_level("TRACE", "TRACE"); + ::testing::InitGoogleTest(&argc, argv); + ret = RUN_ALL_TESTS(); + TG_STOP(lib::TGDefIDs::DetectManager); + return ret; +}