[FEAT MERGE]4_1_sql_feature
Co-authored-by: leslieyuchen <leslieyuchen@gmail.com> Co-authored-by: Charles0429 <xiezhenjiang@gmail.com> Co-authored-by: raywill <hustos@gmail.com>
This commit is contained in:
@ -132,7 +132,9 @@ OB_SERIALIZE_MEMBER((ObGranuleIteratorSpec, ObOpSpec),
|
||||
gi_attri_flag_,
|
||||
bf_info_,
|
||||
hash_func_,
|
||||
tablet_id_expr_);
|
||||
tablet_id_expr_,
|
||||
pw_dml_tsc_ids_,
|
||||
repart_pruning_tsc_idx_);
|
||||
|
||||
ObGranuleIteratorSpec::ObGranuleIteratorSpec(ObIAllocator &alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type),
|
||||
@ -143,12 +145,13 @@ ObGranuleIteratorSpec::ObGranuleIteratorSpec(ObIAllocator &alloc, const ObPhyOpe
|
||||
access_all_(false),
|
||||
nlj_with_param_down_(false),
|
||||
pw_op_tscs_(alloc),
|
||||
pw_dml_tsc_ids_(alloc),
|
||||
gi_attri_flag_(0),
|
||||
dml_op_(NULL),
|
||||
bf_info_(),
|
||||
hash_func_(),
|
||||
tablet_id_expr_(NULL)
|
||||
|
||||
tablet_id_expr_(NULL),
|
||||
repart_pruning_tsc_idx_(OB_INVALID_ID)
|
||||
{}
|
||||
|
||||
ObGranuleIteratorOp::ObGranuleIteratorOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input)
|
||||
@ -169,7 +172,8 @@ ObGranuleIteratorOp::ObGranuleIteratorOp(ObExecContext &exec_ctx, const ObOpSpec
|
||||
bf_key_(),
|
||||
bloom_filter_ptr_(NULL),
|
||||
tablet2part_id_map_(),
|
||||
real_child_(NULL)
|
||||
real_child_(NULL),
|
||||
is_parallel_runtime_filtered_(false)
|
||||
{
|
||||
op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::FILTERED_GRANULE_COUNT;
|
||||
op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::TOTAL_GRANULE_COUNT;
|
||||
@ -206,7 +210,7 @@ int ObGranuleIteratorOp::parameters_init()
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleIteratorOp::try_pruning_partition(
|
||||
int ObGranuleIteratorOp::try_pruning_repart_partition(
|
||||
const ObGITaskSet &taskset,
|
||||
int64_t &pos,
|
||||
bool &partition_pruned)
|
||||
@ -218,10 +222,8 @@ int ObGranuleIteratorOp::try_pruning_partition(
|
||||
LOG_WARN("pruning id is not set", K(ret));
|
||||
} else if (OB_FAIL(taskset.get_task_at_pos(info, pos))) {
|
||||
LOG_WARN("get task info failed", K(ret));
|
||||
} else if (info.tablet_loc_->tablet_id_.id() == ctx_.get_gi_pruning_info().get_part_id()) {
|
||||
partition_pruned = false;
|
||||
} else {
|
||||
partition_pruned = true;
|
||||
partition_pruned = repart_partition_pruned(info);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -256,8 +258,8 @@ int ObGranuleIteratorOp::try_fetch_task(ObGranuleTaskInfo &info)
|
||||
} else {
|
||||
taskset = rescan_taskset_;
|
||||
pos = rescan_tasks_.at(rescan_task_idx_++);
|
||||
if (MY_SPEC.enable_partition_pruning()
|
||||
&& OB_FAIL(try_pruning_partition(*taskset, pos, partition_pruned))) {
|
||||
if (ObGranuleUtil::enable_partition_pruning(MY_SPEC.gi_attri_flag_)
|
||||
&& OB_FAIL(try_pruning_repart_partition(*taskset, pos, partition_pruned))) {
|
||||
LOG_WARN("fail try prune partition", K(ret));
|
||||
} else if (partition_pruned) {
|
||||
// next task
|
||||
@ -306,6 +308,11 @@ int ObGranuleIteratorOp::rescan()
|
||||
int ret = ObOperator::inner_rescan();
|
||||
CK(NULL != pump_);
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (!ObGranuleUtil::is_partition_task_mode(MY_SPEC.gi_attri_flag_) &&
|
||||
ObGranuleUtil::partition_filter(MY_SPEC.gi_attri_flag_)) {
|
||||
// no this plan
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("partition filter GI cannot rescan", K(ret));
|
||||
} else if (pump_version_ != pump_->get_pump_version()) {
|
||||
// We can not reused the processed tasks when task regenerated (pump version changed).
|
||||
// e.g.: px batch rescan.
|
||||
@ -328,7 +335,7 @@ int ObGranuleIteratorOp::rescan()
|
||||
rescan_task_idx_ = 0;
|
||||
state_ = GI_GET_NEXT_GRANULE_TASK;
|
||||
}
|
||||
} else if (!MY_SPEC.partition_wise()) {
|
||||
} else if (!MY_SPEC.full_partition_wise()) {
|
||||
if (GI_UNINITIALIZED == state_) {
|
||||
// NJ call rescan before iterator rows, need to nothing for the first scan.
|
||||
} else if (GI_PREPARED == state_) {
|
||||
@ -386,7 +393,7 @@ int ObGranuleIteratorOp::inner_open()
|
||||
if (OB_FAIL(parameters_init())) {
|
||||
LOG_WARN("parameters init failed", K(ret));
|
||||
} else {
|
||||
if (!MY_SPEC.partition_wise()) {
|
||||
if (!MY_SPEC.full_partition_wise()) {
|
||||
if (OB_FAIL(get_gi_task_consumer_node(this, real_child))) {
|
||||
LOG_WARN("Failed to get real child", K(ret));
|
||||
} else {
|
||||
@ -559,10 +566,17 @@ int ObGranuleIteratorOp::do_get_next_granule_task(bool &partition_pruning)
|
||||
GIPrepareTaskMap *gi_prepare_map = nullptr;
|
||||
if (OB_FAIL(ctx_.get_gi_task_map(gi_prepare_map))) {
|
||||
LOG_WARN("Failed to get gi task map", K(ret));
|
||||
} else if (!MY_SPEC.partition_wise()) {
|
||||
} else if (!MY_SPEC.full_partition_wise()) {
|
||||
ObGranuleTaskInfo gi_task_info;
|
||||
/* non-partition wise join */
|
||||
if (OB_FAIL(try_fetch_task(gi_task_info))) {
|
||||
if (enable_parallel_runtime_filter_pruning() &&
|
||||
OB_FAIL(do_parallel_runtime_filter_pruning())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to do parallel runtime filter pruning", K(ret));
|
||||
} else {
|
||||
state_ = GI_END;
|
||||
}
|
||||
} else if (OB_FAIL(try_fetch_task(gi_task_info))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("try fetch task failed", K(ret));
|
||||
} else {
|
||||
@ -583,46 +597,52 @@ int ObGranuleIteratorOp::do_get_next_granule_task(bool &partition_pruning)
|
||||
LOG_DEBUG("produce a gi task", K(tsc_op_id_), K(gi_task_info));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (MY_SPEC.bf_info_.is_inited_ &&
|
||||
OB_FAIL(do_join_filter_partition_pruning(gi_task_info, partition_pruning))) {
|
||||
LOG_WARN("fail to do join filter partition pruning", K(ret));
|
||||
} else if (!partition_pruning && OB_FAIL(do_partition_pruning(gi_task_info, partition_pruning))) {
|
||||
if (enable_single_runtime_filter_pruning() &&
|
||||
OB_FAIL(do_single_runtime_filter_pruning(gi_task_info, partition_pruning))) {
|
||||
LOG_WARN("fail to do join filter partition pruning", K(ret));
|
||||
} else if (!partition_pruning && OB_FAIL(do_dynamic_partition_pruning(gi_task_info, partition_pruning))) {
|
||||
LOG_WARN("fail to do dynamic partition pruning", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
/* partition wise join */
|
||||
ObSEArray<int64_t, 4> op_ids;
|
||||
if (OB_NOT_NULL(MY_SPEC.dml_op_)) {
|
||||
// GI对INSERT表进行任务划分,获取对应的INSERT的op id
|
||||
if (OB_FAIL(op_ids.push_back(MY_SPEC.dml_op_->id_))) {
|
||||
LOG_WARN("failed to push back op ids", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
// GI对TSCs进行划分,获得对应的TSC的op id
|
||||
for (int i = 0 ; i < MY_SPEC.pw_op_tscs_.count() && OB_SUCC(ret); i++) {
|
||||
const ObTableScanSpec *tsc = MY_SPEC.pw_op_tscs_.at(i);
|
||||
if (OB_FAIL(op_ids.push_back(tsc->id_))) {
|
||||
const ObIArray<int64_t> *op_ids_pointer = NULL;
|
||||
if (MY_SPEC.pw_dml_tsc_ids_.count() > 0) {
|
||||
op_ids_pointer = &(MY_SPEC.pw_dml_tsc_ids_);
|
||||
} else {
|
||||
op_ids_pointer = &op_ids;
|
||||
if (OB_NOT_NULL(MY_SPEC.dml_op_)) {
|
||||
//GI对INSERT表进行任务划分,获取对应的INSERT的op id
|
||||
if (OB_FAIL(op_ids.push_back(MY_SPEC.dml_op_->id_))) {
|
||||
LOG_WARN("failed to push back op ids", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
// GI对TSCs进行划分,获得对应的TSC的op id
|
||||
for (int i = 0 ; i < MY_SPEC.pw_op_tscs_.count() && OB_SUCC(ret); i++) {
|
||||
const ObTableScanSpec *tsc = MY_SPEC.pw_op_tscs_.at(i);
|
||||
if (OB_FAIL(op_ids.push_back(tsc->id_))) {
|
||||
LOG_WARN("failed to push back op ids", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// 获得gi tasks:
|
||||
// 每一个`op_id`都会对应一个`gi_task_info`
|
||||
if (OB_SUCC(ret)) {
|
||||
if (is_rescan_) {
|
||||
if (OB_FAIL(fetch_rescan_pw_task_infos(op_ids, gi_prepare_map, gi_task_infos))) {
|
||||
if (OB_FAIL(fetch_rescan_pw_task_infos(*op_ids_pointer, gi_prepare_map, gi_task_infos))) {
|
||||
LOG_WARN("fail to fetch rescan pw task infos", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(fetch_normal_pw_task_infos(op_ids, gi_prepare_map, gi_task_infos))) {
|
||||
} else if (OB_FAIL(fetch_normal_pw_task_infos(*op_ids_pointer, gi_prepare_map, gi_task_infos))) {
|
||||
LOG_WARN("fail to fetch normal pw task infos", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
// 动态分区裁剪
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(do_partition_pruning(gi_task_infos, partition_pruning))) {
|
||||
if (OB_FAIL(do_dynamic_partition_pruning(gi_task_infos, partition_pruning))) {
|
||||
LOG_WARN("fail to do partition pruning", K(ret));
|
||||
}
|
||||
}
|
||||
@ -636,28 +656,33 @@ int ObGranuleIteratorOp::do_get_next_granule_task(bool &partition_pruning)
|
||||
|
||||
|
||||
int ObGranuleIteratorOp::do_join_filter_partition_pruning(
|
||||
const ObGranuleTaskInfo &gi_task_info,
|
||||
int64_t tablet_id,
|
||||
bool &partition_pruning)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_match = false;
|
||||
if (OB_SUCC(ret) && OB_ISNULL(bloom_filter_ptr_)) {
|
||||
while (OB_SUCC(ret) && OB_ISNULL(bloom_filter_ptr_)) {
|
||||
if (OB_FAIL(ObPxBloomFilterManager::instance().get_px_bloom_filter(bf_key_,
|
||||
bloom_filter_ptr_))) {
|
||||
if (OB_HASH_NOT_EXIST == ret) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
LOG_WARN("fail to get px bloom filter", K(ret), K(bf_key_));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ctx_.fast_check_status())) {
|
||||
LOG_WARN("fail to check status", K(ret));
|
||||
}
|
||||
}
|
||||
LOG_WARN("fail to get px bloom filter", K(ret), K(bf_key_));
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && OB_NOT_NULL(bloom_filter_ptr_)) {
|
||||
uint64_t hash_val = ObExprJoinFilter::JOIN_FILTER_SEED;
|
||||
ObDatum &datum = MY_SPEC.tablet_id_expr_->locate_expr_datum(eval_ctx_);
|
||||
int64_t tablet_id = gi_task_info.tablet_loc_->tablet_id_.id();
|
||||
if (MY_SPEC.bf_info_.skip_subpart_) {
|
||||
int64_t part_id = OB_INVALID_ID;
|
||||
if (OB_FAIL(try_build_tablet2part_id_map(gi_task_info.tablet_loc_))) {
|
||||
if (OB_FAIL(try_build_tablet2part_id_map())) {
|
||||
LOG_WARN("fail to build tablet2part id map", K(ret));
|
||||
} else if (OB_FAIL(tablet2part_id_map_.get_refactored(tablet_id, part_id))) {
|
||||
LOG_WARN("fail to get refactored part id", K(ret), K(tablet_id), K(part_id));
|
||||
@ -667,6 +692,7 @@ int ObGranuleIteratorOp::do_join_filter_partition_pruning(
|
||||
}
|
||||
if (OB_SUCC(ret) && !is_match) {
|
||||
datum.int_ = &tablet_id;
|
||||
datum.len_ = sizeof(tablet_id);
|
||||
hash_val = MY_SPEC.hash_func_.hash_func_(datum, hash_val);
|
||||
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));
|
||||
@ -755,7 +781,6 @@ int ObGranuleIteratorOp::set_tscs(ObIArray<const ObTableScanSpec *> &tscs)
|
||||
LOG_DEBUG("Set table scan to GI", K(tscs), K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleIteratorOp::set_dml_op(const ObTableModifySpec *dml_op)
|
||||
{
|
||||
const_cast<ObGranuleIteratorSpec &>(MY_SPEC).dml_op_ = const_cast<ObTableModifySpec *>(dml_op);
|
||||
@ -787,7 +812,7 @@ int ObGranuleIteratorOp::get_gi_task_consumer_node(ObOperator *cur,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleIteratorOp::do_partition_pruning(const ObGranuleTaskInfo &gi_task_info, bool &partition_pruning)
|
||||
int ObGranuleIteratorOp::do_dynamic_partition_pruning(const ObGranuleTaskInfo &gi_task_info, bool &partition_pruning)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
partition_pruning = false;
|
||||
@ -826,7 +851,7 @@ int ObGranuleIteratorOp::do_partition_pruning(const ObGranuleTaskInfo &gi_task_i
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleIteratorOp::do_partition_pruning(const common::ObIArray<ObGranuleTaskInfo> &gi_task_infos,
|
||||
int ObGranuleIteratorOp::do_dynamic_partition_pruning(const common::ObIArray<ObGranuleTaskInfo> &gi_task_infos,
|
||||
bool &partition_pruning)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -895,6 +920,29 @@ int ObGranuleIteratorOp::fetch_rescan_pw_task_infos(const common::ObIArray<int64
|
||||
ret = OB_ITER_END;
|
||||
state_ = GI_END;
|
||||
} else {
|
||||
if (ObGranuleUtil::enable_partition_pruning(MY_SPEC.gi_attri_flag_)) {
|
||||
bool partition_pruned = false;
|
||||
int64_t repart_idx = MY_SPEC.repart_pruning_tsc_idx_;
|
||||
if (OB_UNLIKELY(repart_idx < 0 || repart_idx >= op_ids.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected repart pruning tsc index", K(ret), K(repart_idx));
|
||||
}
|
||||
do {
|
||||
if (rescan_task_idx_ >= pwj_rescan_task_infos_.count()) {
|
||||
ret = OB_ITER_END;
|
||||
state_ = GI_END;
|
||||
} else if (OB_UNLIKELY(rescan_task_idx_ + repart_idx >= pwj_rescan_task_infos_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexected pwj_rescan_task_infos_ count", K(ret), K(rescan_task_idx_),
|
||||
K(repart_idx), K(pwj_rescan_task_infos_.count()), K(op_ids));
|
||||
} else {
|
||||
partition_pruned = repart_partition_pruned(pwj_rescan_task_infos_.at(rescan_task_idx_ + repart_idx));
|
||||
if (partition_pruned) {
|
||||
rescan_task_idx_ += op_ids.count();
|
||||
}
|
||||
}
|
||||
} while (OB_SUCC(ret) && partition_pruned);
|
||||
}
|
||||
ARRAY_FOREACH_X(op_ids, idx, cnt, OB_SUCC(ret)) {
|
||||
// GI在向Map中塞任务的时候,需要尝试清理上一次塞入的任务
|
||||
if (OB_NOT_NULL(gi_prepare_map->get(op_ids.at(idx)))) {
|
||||
@ -930,7 +978,7 @@ int ObGranuleIteratorOp::fetch_normal_pw_task_infos(const common::ObIArray<int64
|
||||
int ret = OB_SUCCESS;
|
||||
if (op_ids.empty() || !gi_task_infos.empty() || OB_ISNULL(gi_prepare_map)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected error", K(ret), K(op_ids.count()), K(gi_task_infos.count()));
|
||||
LOG_WARN("unexpected error", K(ret), K(get_spec().id_), K(op_ids.count()), K(gi_task_infos.count()));
|
||||
} else if (OB_FAIL(fetch_full_pw_tasks(gi_task_infos, op_ids))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("try fetch task failed", K(ret));
|
||||
@ -961,10 +1009,10 @@ int ObGranuleIteratorOp::fetch_normal_pw_task_infos(const common::ObIArray<int64
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleIteratorOp::try_build_tablet2part_id_map(ObDASTabletLoc *loc)
|
||||
int ObGranuleIteratorOp::try_build_tablet2part_id_map()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(loc)) {
|
||||
if (OB_INVALID_ID == MY_SPEC.ref_table_id_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("loc is unexpected", K(ret));
|
||||
} else if (tablet2part_id_map_.created()) {
|
||||
@ -974,7 +1022,7 @@ int ObGranuleIteratorOp::try_build_tablet2part_id_map(ObDASTabletLoc *loc)
|
||||
LOG_WARN("get unexpected null", K(ret), K(ctx_));
|
||||
} else {
|
||||
const ObTableSchema *table_schema = NULL;
|
||||
int64_t ref_table_id = loc->loc_meta_->ref_table_id_;
|
||||
int64_t ref_table_id = MY_SPEC.ref_table_id_;
|
||||
if (OB_FAIL(ctx_.get_sql_ctx()->schema_guard_->get_table_schema(
|
||||
MTL_ID(), ref_table_id, table_schema))) {
|
||||
LOG_WARN("fail to get table schema", K(ret));
|
||||
@ -1010,5 +1058,107 @@ int ObGranuleIteratorOp::try_build_tablet2part_id_map(ObDASTabletLoc *loc)
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObGranuleIteratorOp::enable_parallel_runtime_filter_pruning()
|
||||
{
|
||||
return !is_parallel_runtime_filtered_ && MY_SPEC.bf_info_.is_inited_ &&
|
||||
!ObGranuleUtil::is_partition_task_mode(MY_SPEC.gi_attri_flag_);
|
||||
}
|
||||
|
||||
bool ObGranuleIteratorOp::enable_single_runtime_filter_pruning()
|
||||
{
|
||||
return MY_SPEC.bf_info_.is_inited_ && ObGranuleUtil::is_partition_task_mode(MY_SPEC.gi_attri_flag_);
|
||||
}
|
||||
|
||||
int ObGranuleIteratorOp::do_single_runtime_filter_pruning(
|
||||
const ObGranuleTaskInfo &gi_task_info, bool &partition_pruning)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(gi_task_info.tablet_loc_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected tablet loc", K(ret));
|
||||
} else {
|
||||
int64_t tablet_id = gi_task_info.tablet_loc_->tablet_id_.id();
|
||||
if (OB_FAIL(do_join_filter_partition_pruning(tablet_id, partition_pruning))) {
|
||||
LOG_WARN("fail to do join filter partition pruning", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleIteratorOp::do_parallel_runtime_filter_pruning()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(pump_) || 1 != pump_->get_pump_args().count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("pump is unexpected", K(pump_), K(ret));
|
||||
} else {
|
||||
ObGranulePumpArgs &args = pump_->get_pump_args().at(0);
|
||||
int64_t cur_tablet_idx = -1;
|
||||
int64_t finish_tablet_idx = -1;
|
||||
int64_t tablet_cnt = args.run_time_pruning_flags_.count();
|
||||
bool is_pruning = false;
|
||||
CK(0 != tablet_cnt);
|
||||
// 1 Firstly, all threads(not all is also ok)
|
||||
// come to do the parallel pruning together via aotimic operations
|
||||
// 2 The last worker to finish pruning will be responsible for regenerating the gi task
|
||||
// 3 Other threads need to sleep to wait to be woken up
|
||||
while (OB_SUCC(ret) && args.cur_tablet_idx_ < tablet_cnt) {
|
||||
// Concurrency controls the tablet idx of the current operation
|
||||
cur_tablet_idx = ATOMIC_FAA(&args.cur_tablet_idx_, 1);
|
||||
if (cur_tablet_idx >= tablet_cnt) {
|
||||
} else {
|
||||
DASTabletLocArray &tablet_array = args.tablet_arrays_.at(0);
|
||||
is_pruning = false;
|
||||
if (OB_FAIL(do_join_filter_partition_pruning(
|
||||
tablet_array.at(cur_tablet_idx)->tablet_id_.id(), is_pruning))) {
|
||||
LOG_WARN("fail to do join filter partition pruning", K(ret));
|
||||
} else if (is_pruning) {
|
||||
args.run_time_pruning_flags_.at(cur_tablet_idx) = true;
|
||||
}
|
||||
// Record the current finish_tablet_idx
|
||||
// To find who is the last finished worker
|
||||
finish_tablet_idx = ATOMIC_AAF(&args.finish_pruning_tablet_idx_, 1);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
is_parallel_runtime_filtered_ = true;
|
||||
if (finish_tablet_idx == tablet_cnt) {
|
||||
DASTabletLocArray pruning_remain_tablets;
|
||||
DASTabletLocArray &tablet_array = args.tablet_arrays_.at(0);
|
||||
for (int i = 0; OB_SUCC(ret) && i < tablet_array.count(); ++i) {
|
||||
if (!args.run_time_pruning_flags_.at(i)) {
|
||||
OZ(pruning_remain_tablets.push_back(tablet_array.at(i)));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (pruning_remain_tablets.empty()) {
|
||||
ret = OB_ITER_END;
|
||||
args.sharing_iter_end_ = true;
|
||||
} else if (pruning_remain_tablets.count() == tablet_array.count()) {
|
||||
/* do nothing*/
|
||||
} else {
|
||||
args.tablet_arrays_.reset();
|
||||
OZ(args.tablet_arrays_.push_back(pruning_remain_tablets));
|
||||
OZ(pump_->regenerate_gi_task());
|
||||
}
|
||||
args.set_finish_pruning();
|
||||
}
|
||||
} else {
|
||||
while (OB_SUCC(ret) && !args.is_finish_pruning()) {
|
||||
if (OB_FAIL(ctx_.fast_check_status())) {
|
||||
LOG_WARN("fail to fast check status", K(ret));
|
||||
} else {
|
||||
ob_usleep(100);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && args.sharing_iter_end_) {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
Reference in New Issue
Block a user