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