[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:
obdev
2023-01-28 16:01:26 +08:00
committed by ob-robot
parent 3080f2b66f
commit 2d19a9d8f5
846 changed files with 161957 additions and 116661 deletions

File diff suppressed because it is too large Load Diff

View File

@ -15,6 +15,7 @@
#include "lib/container/ob_array.h"
#include "lib/container/ob_fixed_array.h"
#include "lib/lock/ob_scond.h"
#include "objit/common/ob_item_type.h"
#include "sql/resolver/expr/ob_raw_expr.h"
#include "sql/engine/basic/ob_ra_datum_store.h"
@ -22,6 +23,7 @@
#include "sql/engine/ob_operator.h"
#include "sql/engine/px/datahub/ob_dh_msg.h"
#include "sql/engine/px/datahub/components/ob_dh_winbuf.h"
#include "sql/engine/px/datahub/components/ob_dh_second_stage_reporting_wf.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
namespace oceanbase
@ -62,7 +64,8 @@ public:
is_ignore_null_(false),
is_from_first_(false),
remove_type_(common::REMOVE_INVALID),
expr_(NULL)
expr_(NULL),
can_push_down_(false)
{
}
@ -96,7 +99,8 @@ public:
TO_STRING_KV(K_(win_type), K_(func_type), K_(is_ignore_null), K_(is_from_first), K_(remove_type),
KPC_(expr), K_(aggr_info), K_(upper), K_(lower), K_(param_exprs),
K_(partition_exprs), K_(sort_exprs), K_(sort_collations), K_(sort_cmp_funcs));
K_(partition_exprs), K_(sort_exprs), K_(sort_collations), K_(sort_cmp_funcs),
K_(can_push_down));
WindowType win_type_;
ObItemType func_type_;
bool is_ignore_null_;
@ -114,15 +118,103 @@ public:
ExprFixedArray sort_exprs_;
ObSortCollations sort_collations_;
ObSortFuncs sort_cmp_funcs_;
bool can_push_down_;
};
typedef common::ObFixedArray<WinFuncInfo, common::ObIAllocator> WFInfoFixedArray;
// TODO:@xiaofeng.lby, add a max/min row, use it to decide whether following rows can by pass
typedef common::hash::ObHashSet<uint64_t, common::hash::NoPthreadDefendMode> ReportingWFHashSet;
// for sync_wait to reset whole_msg_provider in get_participator_whole_msg
struct ObWFParticipatorSharedInfo
{
int ret_;
int64_t sqc_thread_count_;
int64_t process_cnt_;
ObSpinLock lock_;
common::SimpleCond cond_;
};
// set task_count to ObWindowFunctionOpInput for wf pushdown participator
class ObWindowFunctionOpInput : public ObOpInput
{
OB_UNIS_VERSION_V(1);
public:
ObWindowFunctionOpInput(ObExecContext &ctx, const ObOpSpec &spec)
: ObOpInput(ctx, spec), local_task_count_(1), total_task_count_(1),
wf_participator_shared_info_(0) {};
virtual ~ObWindowFunctionOpInput() = default;
virtual int init(ObTaskInfo &task_info) override { UNUSED(task_info); return common::OB_SUCCESS; }
virtual void reset() override { local_task_count_ = 1; total_task_count_ = 1;}
void set_local_task_count(uint64_t task_count) { local_task_count_ = task_count; }
int64_t get_local_task_count() const { return local_task_count_; }
void set_total_task_count(uint64_t task_count) { total_task_count_ = task_count; }
int64_t get_total_task_count() const { return total_task_count_; }
ObWFParticipatorSharedInfo *get_wf_participator_shared_info()
{
return reinterpret_cast<ObWFParticipatorSharedInfo *>(wf_participator_shared_info_);
}
int64_t &get_sqc_thread_count()
{
return reinterpret_cast<ObWFParticipatorSharedInfo *>(
wf_participator_shared_info_)->sqc_thread_count_;
}
int64_t &get_process_cnt()
{
return reinterpret_cast<ObWFParticipatorSharedInfo *>(
wf_participator_shared_info_)->process_cnt_;
}
void set_error_code(int in_ret)
{
ObWFParticipatorSharedInfo *shared_info = reinterpret_cast<ObWFParticipatorSharedInfo *>(
wf_participator_shared_info_);
ATOMIC_SET(&shared_info->ret_, in_ret);
}
int init_wf_participator_shared_info(ObIAllocator &alloc, int64_t task_cnt)
{
int ret = OB_SUCCESS;
ObWFParticipatorSharedInfo *shared_info = nullptr;
if (OB_ISNULL(shared_info = reinterpret_cast<ObWFParticipatorSharedInfo *>(
alloc.alloc(sizeof(ObWFParticipatorSharedInfo))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
SQL_ENG_LOG(WARN, "fail to alloc memory", K(ret));
} else {
wf_participator_shared_info_ = reinterpret_cast<int64_t>(shared_info);
shared_info->sqc_thread_count_ = task_cnt;
shared_info->process_cnt_ = 0;
shared_info->ret_ = OB_SUCCESS;
new (&shared_info->cond_)common::SimpleCond(
common::ObWaitEventIds::SQL_WF_PARTICIPATOR_COND_WAIT);
new (&shared_info->lock_)ObSpinLock(
common::ObLatchIds::SQL_WF_PARTICIPATOR_COND_LOCK);
}
return ret;
}
int sync_wait(ObExecContext &ctx, ObReportingWFWholeMsg::WholeMsgProvider *whole_msg_provider);
protected:
DISALLOW_COPY_AND_ASSIGN(ObWindowFunctionOpInput);
public:
int64_t local_task_count_;
int64_t total_task_count_;
uint64_t wf_participator_shared_info_;
};
class ObWindowFunctionSpec : public ObOpSpec
{
public:
OB_UNIS_VERSION_V(1);
public:
enum WindowFunctionRoleType
{
NORMAL = 0,
PARTICIPATOR = 1, // for wf adaptive pushdown
CONSOLIDATOR = 2 // for wf adaptive pushdown
};
ObWindowFunctionSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type)
: ObOpSpec(alloc, type),
wf_infos_(alloc),
@ -133,7 +225,9 @@ public:
rd_coord_exprs_(alloc),
rd_sort_collations_(alloc),
rd_sort_cmp_funcs_(alloc),
rd_pby_sort_cnt_(0)
rd_pby_sort_cnt_(0),
role_type_(0),
wf_aggr_status_expr_(NULL)
{
}
DECLARE_VIRTUAL_TO_STRING;
@ -168,6 +262,11 @@ public:
return rd_sort_cmp(l, r, 0, rd_sort_collations_.count());
}
int64_t get_role_type() const { return role_type_; }
bool is_push_down() const { return PARTICIPATOR == role_type_|| CONSOLIDATOR == role_type_; }
bool is_participator() const { return PARTICIPATOR == role_type_; }
bool is_consolidator() const { return CONSOLIDATOR == role_type_; }
public:
WFInfoFixedArray wf_infos_;
ExprFixedArray all_expr_; //child output + all sort expr
@ -183,11 +282,11 @@ public:
// all exprs seed to PX COORD in range distribution window function, organized as:
// PBY + OBY + rd_wfs_
ExprFixedArray rd_coord_exprs_;
ObSortCollations rd_sort_collations_;
ObSortFuncs rd_sort_cmp_funcs_;
int64_t rd_pby_sort_cnt_;
int64_t role_type_;
ObExpr *wf_aggr_status_expr_;
private:
// disallow copy
@ -198,6 +297,8 @@ private:
class ObWindowFunctionOp : public ObOperator
{
public:
typedef common::hash::ObHashMap<int64_t, int64_t> WFPbyExprCntIdxHashMap;
typedef common::ObSArray<uint64_t> PbyHashValueArray;
enum class ProcessStatus
{
PARTIAL,
@ -226,8 +327,13 @@ public:
class RowsStore
{
public:
RowsStore() : ra_rs_(NULL /*allocator*/), begin_idx_(0), row_cnt_(0),
stored_row_cnt_(0), output_row_idx_(0) {}
RowsStore() :
ra_rs_(NULL /*allocator*/),
begin_idx_(0),
row_cnt_(0),
stored_row_cnt_(0),
output_row_idx_(0),
need_output_(false) {}
~RowsStore() { destroy(); }
void destroy() { ra_rs_.reset(); }
inline int add_row(const common::ObIArray<ObExpr*> &exprs,
@ -238,7 +344,7 @@ public:
int ret = ra_rs_.add_row(exprs, ctx, stored_row);
stored_row_cnt_++;
row_cnt_ += add_row_cnt;
SQL_ENG_LOG(DEBUG, "add_row", K_(row_cnt), K(ret));
SQL_ENG_LOG(DEBUG, "add_row", K(ret), K_(row_cnt), K_(stored_row_cnt));
return ret;
}
inline int add_row(const common::ObIArray<common::ObDatum> &datums,
@ -248,7 +354,7 @@ public:
int ret = ra_rs_.add_row(datums, stored_row);
row_cnt_ += add_row_cnt;
++stored_row_cnt_;
SQL_ENG_LOG(DEBUG, "add_row", K_(row_cnt), K(static_cast<int64_t>(add_row_cnt)), K(ret));
SQL_ENG_LOG(DEBUG, "add_row", K(ret), K_(row_cnt), K_(stored_row_cnt), K(add_row_cnt));
return ret;
}
inline int add_row_with_index(int64_t idx,
@ -258,18 +364,20 @@ public:
bool add_row_cnt = true)
{
int ret = OB_SUCCESS;
UNUSED(idx);
//ctx->batch_idx_ = idx;
UNUSED(idx); // use BatchInfoScopeGuard instead
SQL_ENG_LOG(DEBUG, "add row with index", K(idx), K((void*)this),
K(begin_idx_), K(output_row_idx_),
K(row_cnt_), K(stored_row_cnt_), K(ObToStringExprRow(*ctx, exprs)));
ret = add_row(exprs, ctx, stored_row, add_row_cnt);
//ctx->batch_idx_ = 0;
if (OB_FAIL(add_row(exprs, ctx, stored_row, add_row_cnt))) {
LOG_WARN("fail to add_row", K(ret));
}
return ret;
}
inline int64_t count() const { return row_cnt_; }
// return row count which computed but not outputed
inline int64_t to_output_rows() const { return row_cnt_ - output_row_idx_; }
inline int64_t to_output_rows() const {
return row_cnt_ - output_row_idx_;
}
// return row count which window function not computed
inline int64_t to_compute_rows() const { return stored_row_cnt_ - row_cnt_; }
inline bool is_empty() const { return stored_row_cnt_ == begin_idx_; }
@ -309,11 +417,13 @@ public:
}
return ret;
}
TO_STRING_KV(K_(begin_idx), K_(output_row_idx), K_(row_cnt), K(stored_row_cnt_), K_(ra_rs));
TO_STRING_KV(K_(begin_idx), K_(output_row_idx), K_(row_cnt), K_(stored_row_cnt),
K_(need_output), K_(ra_rs));
public:
ObRADatumStore ra_rs_;
// record begin idx of current partition. always zero for rows_store_
int64_t begin_idx_;
// cnt of rows computed
int64_t row_cnt_;
/* In non-batch execution, rows of only one partition is stored in rows_store_.
* If get row of next partition from child, store it in next_row_ and compute current_part.
@ -327,6 +437,8 @@ public:
// [output_row_idx_, row_cnt_) => rows computed already but not output
// [row_cnt_, stored_row_cnt_) => rows not computed yet
int64_t output_row_idx_;
// record whether the input row of consolidator need to output
bool need_output_;
};
struct Stores
@ -618,7 +730,14 @@ public:
patch_last_(false),
first_row_same_order_cache_(SAME_ORDER_CACHE_DEFAULT),
last_row_same_order_cache_(SAME_ORDER_CACHE_DEFAULT),
last_computed_part_rows_(0)
last_computed_part_rows_(0),
last_aggr_status_(0),
pby_set_count_(0),
next_wf_pby_expr_cnt_to_transmit_(common::OB_INVALID_COUNT),
pby_expr_cnt_idx_array_(),
pby_hash_values_(),
participator_whole_msg_array_(),
pby_hash_values_sets_()
{
}
virtual ~ObWindowFunctionOp() {}
@ -656,9 +775,7 @@ protected:
int partial_next_row();
int do_partial_next_batch(const int64_t max_row_cnt, bool &do_output);
int partial_next_batch(const int64_t max_row_cnt);
int coordinate();
int final_next_row();
int final_next_batch(const int64_t max_row_cnt);
@ -684,13 +801,16 @@ protected:
{
static_assert(-1 == static_cast<int64_t>(common::OB_INVALID_INDEX),
"invalid index is not -1");
return input_rows_.cur_->count() == static_cast<int64_t>(last_output_row_idx_) + 1;
bool is_all_outputed =
input_rows_.cur_->count() == static_cast<int64_t>(last_output_row_idx_) + 1;
return is_all_outputed;
}
int fetch_child_row();
int input_one_row(WinFuncCell &func_ctx, bool &part_end);
int compute(RowsReader &row_reader, WinFuncCell &wf_cell, const int64_t row_idx,
common::ObDatum &val);
int compute_push_down_by_pass(WinFuncCell &wf_cell, common::ObDatum &val);
int check_same_partition(const ExprFixedArray &other_exprs,
bool &is_same_part,
const ExprFixedArray *curr_exprs = NULL);
@ -710,19 +830,29 @@ protected:
void restore_all_expr_datums();
int store_all_expr_datums(int64_t store_begin_idx, int64_t store_num);
int get_next_partition(int64_t &check_times);
int process_child_batch(
const int64_t batch_idx,
const ObBatchRows *child_brs,
int64_t &check_times);
int64_t next_nonskip_row_index(int64_t cur_idx, const ObBatchRows &child_brs);
int get_next_batch_from_child(int64_t batch_size, const ObBatchRows *&child_brs);
int compute_wf_values(const WinFuncCell *end, int64_t &check_times);
int check_wf_same_partition(WinFuncCell *&end);
int save_partition_by_exprs_and_part_idx();
int save_partition_by_exprs();
int save_part_first_row_idx();
// for pushdown
int detect_computing_method(const int64_t row_idx, const int64_t wf_idx,
bool &is_result_datum_null, bool &is_pushdown_bypass);
// For participator, add aggr result row to input rows
int found_part_end(const WinFuncCell *end, RowsStore *rows_store, bool add_row_cnt = true);
int found_new_part(const bool update_part_first_row_idx);
int save_part_first_row_idx();
int output_row();
int output_row(const int64_t idx,
int output_row(int64_t idx,
StoreMemberPtr store_member = &Stores::cur_,
const ObRADatumStore::StoredRow **all_expr_row = NULL);
int output_rows_store_rows(const int64_t output_row_cnt,
RowsStore &rows_store, RowsStore &part_rows_store,
ObEvalCtx::BatchInfoScopeGuard &guard);
@ -752,6 +882,20 @@ protected:
// Send partial aggregation info to PX COORD and get the patch info of partial partitions,
// in range distribution parallelism.
int rd_fetch_patch();
int set_compute_result_for_invalid_frame(WinFuncCell &wf_cell, ObDatum &val);
// Send the first dop part values of each pushdown wfs to PX COORD
// and get all the part values of each pushdown wfs to be caculate
int participator_coordinate(const int64_t pushdown_wf_idx, const ExprFixedArray &pby_exprs);
int get_participator_whole_msg(
const PbyHashValueArray &pby_hash_value_array, ObReportingWFWholeMsg &whole);
int build_pby_hash_values_for_transmit();
int build_participator_whole_msg_array();
int send_empty_piece_data();
int calc_part_exprs_hash(const common::ObIArray<ObExpr *> *exprs_,
const ObChunkDatumStore::StoredRow *row_, uint64_t &hash_value);
int detect_aggr_status();
bool skip_calc(const int64_t wf_idx);
private:
// disallow copy
@ -767,7 +911,7 @@ private:
Stores input_rows_;
WinFuncCellList wf_list_;
// shadow copy the next and restore it before get next row from child.
ObChunkDatumStore::ShadowStoredRow next_row_;
ObChunkDatumStore::ShadowStoredRow next_row_; // for backup & restore datum ptr of child output
DatumFixedArray curr_row_collect_values_;
common::ObFixedArray<ObDatum*, common::ObIAllocator> all_expr_datums_copy_;
common::ObFixedArray<ObDatum*, common::ObIAllocator> all_expr_datums_;
@ -802,11 +946,39 @@ private:
int64_t last_computed_part_rows_;
// row store iteration age to prevent output row datum released dring the same batch
ObRADatumStore::IterationAge output_rows_it_age_;
// Members for reporting wf push down, use for pushdown paricipator transmit pieces to datahub begin
int64_t last_aggr_status_; // aggr_status of last input row for participator
// Use for wf participator, the count of different pby_set in wf op
// ( pby1(c1, c2, c3), pby2(c1, c2), pby3(c1, c2), pby4(c1), pby5(c1)) is 3
int64_t pby_set_count_;
// next part expr count of pieces to send to datahub for wf pushdown participator
int64_t next_wf_pby_expr_cnt_to_transmit_;
// Use for wf participator, the idx of different pby_set of pushdown wf
// index of array : wf_idx - 1 (because wf_idx is start from 1)
// value of array : the idx of different pby_set of pushdown wf (value is -1 if isn't pushdown wf)
ObArray<int64_t> pby_expr_cnt_idx_array_;
// Use for wf participator, to transmit pieces to datahub
ObArray<PbyHashValueArray *> pby_hash_values_;
// Use to store msg recieved from datahub
ObArray<ObReportingWFWholeMsg *> participator_whole_msg_array_;
// Use to decide whether compute or bypass, generated from ObReportingWFWholeMsg
ObArray<ReportingWFHashSet *> pby_hash_values_sets_;
// Members for reporting wf push down, use for pushdown paricipator transmit pieces to datahub end
};
template <typename STORE_ROW_L, typename STORE_ROW_R>
int ObWindowFunctionSpec::rd_sort_cmp(const STORE_ROW_L *l,
const STORE_ROW_R*r,
const STORE_ROW_R *r,
const int64_t begin,
const int64_t end) const
{