init push

This commit is contained in:
oceanbase-admin
2021-05-31 22:56:52 +08:00
commit cea7de1475
7020 changed files with 5689869 additions and 0 deletions

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,378 @@
/**
* 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_AGGREGATE_FUNCTION_H
#define _OB_AGGREGATE_FUNCTION_H
#include "lib/string/ob_sql_string.h"
#include "lib/container/ob_array.h"
#include "lib/container/ob_se_array.h"
#include "lib/container/ob_vector.h"
#include "lib/list/ob_dlist.h"
#include "lib/hash/ob_hashset.h"
#include "common/row/ob_row_store.h"
#include "sql/engine/expr/ob_sql_expression.h"
#include "lib/container/ob_fixed_array.h"
#include "sql/engine/sort/ob_sort_impl.h"
namespace oceanbase {
namespace sql {
typedef common::ObDList<ObSqlExpression> ObAggrExprList;
class ObAggregateDistinctItem {
public:
ObAggregateDistinctItem()
: group_id_(0), col_idx_(0), cs_type_(common::CS_TYPE_INVALID), cells_(NULL), cs_type_list_(NULL)
{}
uint64_t hash() const;
bool operator==(const ObAggregateDistinctItem& other) const;
inline bool operator!=(const ObAggregateDistinctItem& other) const
{
return !operator==(other);
}
TO_STRING_KV(K_(group_id), K_(col_idx), KP_(cells), K_(cs_type_list));
int64_t group_id_;
int64_t col_idx_;
common::ObCollationType cs_type_;
// size of %cells_ is $cs_type_list_->count()
common::ObObj* cells_;
const common::ObIArray<common::ObCollationType>* cs_type_list_;
};
// Context structure for one aggregation function of one group, only some functions need this:
// with distinct: need this for distinct calculate
// group concat: need context to hold the input cells.
// rank: need context to hold the input cells.
// percent_rank: need context to hold the input cells.
// dense_rank: need context to hold the input cells.
// cume_dist: need context to hold the input cells.
// max keep(): need context to hold the input cells.
// min keep(): need context to hold the input cells.
// sum keep(): need context to hold the input cells.
// count keep(): need context to hold the input cells.
//
//
// We only implement distinct logic here, no common interface abstracted here,
// use the derived class directly.
class ObAggCellCtx {
public:
// %alloc is used to initialize the structures, can not be used to hold the data
explicit ObAggCellCtx(common::ObIAllocator& alloc);
virtual ~ObAggCellCtx();
virtual void reuse();
public:
// for distinct calculate may be replace by hash based distinct in the future.
ObUniqueSort* distinct_set_;
common::ObArrayHelper<ObSortColumn> sort_columns_;
int init_distinct_set(const uint64_t tenant_id, const common::ObIArray<common::ObCollationType>& cs_types,
const int64_t sort_col_cnt, const bool need_rewind);
protected:
common::ObIAllocator& alloc_;
};
class ObGroupConcatRowStore {
public:
ObGroupConcatRowStore();
~ObGroupConcatRowStore();
int init(const uint64_t tenant_id, const common::ObIArray<ObSortColumn>& sort_columns,
const ObSortImpl::SortExtraInfos* extra_infos, const bool rewind);
void reuse();
int add_row(const common::ObNewRow& row)
{
rows_++;
return need_sort_ ? sort_.add_row(row) : rs_.add_row(row);
}
int get_next_row(const common::ObNewRow*& row)
{
int ret = need_sort_ ? sort_.get_next_row(row) : rs_it_.get_next_row(*const_cast<common::ObNewRow**>(&row));
if (OB_SUCCESS == ret) {
iter_idx_++;
}
return ret;
}
int finish_add_row();
// iterate twice with rollup process, rewind is needed.
int rewind();
bool is_last_row() const
{
return iter_idx_ == rows_;
}
bool empty() const
{
return 0 == rows_;
}
bool is_iterated() const
{
return iter_idx_ > 0;
}
int get_row_count() const
{
return rows_;
}
int get_sort_columns(common::ObIArray<ObSortColumn>& sort_columns)
{
return sort_.get_sort_columns(sort_columns);
}
bool is_allocate_sort() const
{
return need_sort_;
}
private:
bool need_sort_;
int64_t rows_;
int64_t iter_idx_;
ObChunkRowStore rs_;
ObChunkRowStore::Iterator rs_it_;
ObSortImpl sort_;
};
class ObGroupConcatCtx : public ObAggCellCtx {
public:
ObGroupConcatCtx(common::ObIAllocator& alloc) : ObAggCellCtx(alloc), gc_rs_(NULL)
{}
virtual ~ObGroupConcatCtx();
virtual void reuse() override;
public:
ObGroupConcatRowStore* gc_rs_;
};
enum {
// processing row from lower op uses memory allocated from calc_buf of this level (Group by),
// we need to put a limit to it, otherwise too much memory can be wasted. .
AGGR_CALC_BUF_LIMIT = 2 * common::OB_MALLOC_NORMAL_BLOCK_SIZE
};
class ObAggregateFunction {
public:
struct GroupRow {
GroupRow() : row_(NULL), ctx_(NULL)
{}
common::ObRowStore::StoredRow* row_;
ObAggCellCtx** ctx_;
TO_STRING_KV(K(row_), KP(ctx_));
};
struct ExprCtxIdx {
ExprCtxIdx() : expr_(NULL), ctx_idx_(-1)
{}
const ObAggregateExpression* expr_;
int64_t ctx_idx_;
TO_STRING_KV(K(expr_), K(ctx_idx_));
};
public:
ObAggregateFunction();
~ObAggregateFunction();
void set_int_div_as_double(bool did);
bool get_int_div_as_double() const;
void set_sort_based_gby()
{
is_sort_based_gby_ = true;
}
void set_in_window_func()
{
set_sort_based_gby();
in_window_func_ = true;
}
int init(const int64_t input_column_count, const ObAggrExprList* aggr_columns, common::ObExprCtx& expr_ctx,
int32_t prepare_row_num, int64_t distinct_set_bucket_num);
int init_agg_udf(const ObIArray<ObAggUdfMeta>& udf_metas);
void destroy();
void reuse();
int get_cur_row(const common::ObRowStore::StoredRow*& stored_row, const int64_t group_id = 0) const;
inline int64_t get_output_column_cnt() const
{
return output_column_count_;
}
int prepare(
const common::ObNewRow& row, const int64_t group_id = 0, common::ObRowStore::StoredRow** output_row = NULL);
int process(const common::ObNewRow& row, const common::ObTimeZoneInfo* tz_info, const int64_t group_id = 0);
int get_result(common::ObNewRow& row, const common::ObTimeZoneInfo* tz_info, const int64_t group_id = 0);
// used by ScalarAggregate operator when there's no input rows
int get_result_for_empty_set(common::ObNewRow& row);
int64_t get_used_mem_size() const;
int64_t get_hold_mem_size() const;
// added by wangguoping to support rollup
// these functions are only called by merge group by
int rollup_init(const int64_t num_group_col);
int rollup_process(const common::ObTimeZoneInfo* tz_info, const int64_t group_id1, const int64_t group_id2,
const int64_t diff_col_idx, bool set_grouping = false);
int reuse_group(const int64_t group_id);
int init_first_rollup_cols(
common::ObIAllocator* alloc, const ObIArray<ObColumnInfo>& group_idxs, const ObIArray<ObColumnInfo>& rollup_idxs);
void set_tenant_id(const uint64_t tenant_id)
{
stored_row_buf_.set_tenant_id(tenant_id);
agg_cell_ctx_alloc_.set_tenant_id(tenant_id);
}
ObArenaAllocator& get_stored_row_buf()
{
return stored_row_buf_;
}
bool has_distinct() const
{
return has_distinct_;
}
bool has_sort() const
{
return has_sort_;
}
int check_rows_equal(const ObAggregateExpression* cexpr, common::ObIArray<ObSortColumn>& sort_columns,
const ObNewRow* prev_row, const ObNewRow* curr_row, bool& is_equal);
int deep_copy_cur_row(ObNewRow*& prev_row, const ObNewRow cur_row);
int compare_calc(ObObj& obj1, ObObj& obj2, const ObOpSchemaObj& extra_info, int32_t& compare_result, bool& is_asc);
private:
inline ObAggCellCtx* get_agg_cell_ctx(const int64_t group_id, const int64_t cell_idx)
{
return cell_idx >= 0 ? row_array_.at(group_id).ctx_[cell_idx] : NULL;
}
// function members
int init_aggr_cell(const ObItemType aggr_fun, const common::ObNewRow& oprands, common::ObObj& res1,
common::ObObj* res2, ObAggCellCtx* agg_cell_ctx, common::ObCollationType cs_type, int64_t agg_udf_id,
ObAggUdfMeta* agg_udf_meta);
int calc_aggr_cell(const ObItemType aggr_fun, const common::ObNewRow& oprands, common::ObObj& res1,
common::ObObj* res2, const common::ObTimeZoneInfo* tz_info, common::ObCollationType cs_type,
ObAggCellCtx* cell_ctx, int64_t offset = 0);
int calc_distinct_item(const ObItemType aggr_fun, const common::ObNewRow& input_row,
const common::ObTimeZoneInfo* tz_info, const ObAggregateExpression* cexpr, ObAggregateDistinctItem& distinct_item,
common::ObObj& res1, common::ObObj* res2, ObAggCellCtx* cell_ctx, const bool should_fill_item = true,
const bool should_calc = true);
int aggr_distinct_cell(const ObItemType aggr_fun, ObAggCellCtx* cell_ctx, common::ObObj& res1, common::ObObj* res2,
const common::ObTimeZoneInfo* tz_info, common::ObCollationType cs_type, int64_t agg_udf_id,
ObAggUdfMeta* agg_udf_meta);
int clone_cell(const common::ObObj& src_cell, common::ObObj& target_cell);
int clone_number_cell(const common::ObObj& src_cell, common::ObObj& target_cell);
int fill_distinct_item_cell_list(
const ObAggregateExpression* cexpr, const common::ObNewRow& input_row, ObAggregateDistinctItem& distinct_item);
int max_calc(common::ObObj& base, const common::ObObj& other, common::ObCollationType cs_type);
int min_calc(common::ObObj& base, const common::ObObj& other, common::ObCollationType cs_type);
int add_calc(
common::ObObj& res, const common::ObObj& left, const common::ObObj& right, const common::ObTimeZoneInfo* tz_info);
int linear_inter_calc(common::ObObj& res, const common::ObObj& left, const common::ObObj& right,
const common::number::ObNumber& factor, const int16_t scale);
int get_percentile_param(const ObItemType aggr_fun, const common::ObObj& param_obj, const int64_t not_null_start_loc,
const int64_t total_row_count, int64_t& dest_loc, bool& need_linear_inter, common::number::ObNumber& factor,
common::ObDataBuffer& allocator);
// added by wangguoping to support rollup
int get_stored_row(const int64_t group_id, common::ObRowStore::StoredRow*& stored_row);
int rollup_aggregation(const ObItemType aggr_fun, const bool is_first_rollup, const common::ObTimeZoneInfo* tz_info,
common::ObCollationType cs_type, common::ObObj& res1, common::ObObj& res2, common::ObObj& aux_res1,
common::ObObj& aux_res2, ObGroupConcatRowStore* group_concat_row_store1,
ObGroupConcatRowStore* group_concat_row_store2, bool set_grouping);
int init_one_group(const int64_t group_id);
// HyperLogLogCount-related functions
int llc_init(common::ObObj* llc_bitmap);
static uint64_t llc_calc_hash_value(
const common::ObNewRow& oprands, common::ObCollationType cs_type, bool& has_null_cell);
static int llc_add_value(const uint64_t value, common::ObObj* llc_bitmap);
static int llc_add(common::ObObj& res, const common::ObObj& left, const common::ObObj& right);
int get_wm_concat_result(
const ObAggregateExpression*& cexpr, ObGroupConcatCtx*& cell_ctx, bool is_keep_group_concat, ObObj& concat_obj);
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObAggregateFunction);
typedef common::hash::ObHashSet<ObAggregateDistinctItem, common::hash::NoPthreadDefendMode> AggrDistinctBucket;
typedef common::ObSEArray<GroupRow, 1> RowArray;
const static int64_t CONCAT_STR_BUF_LEN = common::OB_MAX_VARCHAR_LENGTH;
const static int64_t STORED_ROW_MAGIC_NUM = 0xaaaabbbbccccdddd;
private:
// data members
bool has_distinct_;
bool has_sort_;
bool is_sort_based_gby_;
bool in_window_func_;
bool aggr_fun_need_cell_ctx_;
ObSEArray<ExprCtxIdx, 4> agg_columns_;
int64_t agg_cell_ctx_cnt_;
int64_t output_column_count_;
int64_t full_column_count_;
bool did_int_div_as_double_;
common::ObArenaAllocator stored_row_buf_;
common::ObExprCtx expr_ctx_;
int64_t child_column_count_;
AggrDistinctBucket aggr_distinct_set_;
RowArray row_array_;
char concat_str_buf_[CONCAT_STR_BUF_LEN];
// allocator for aggregation cell context
common::ObArenaAllocator agg_cell_ctx_alloc_;
uint64_t group_concat_max_len_;
// common allocator, can not be reset in reuse() or reset()
common::ObArenaAllocator alloc_;
common::ObObj* input_cells_;
int64_t gconcat_cur_row_num_; // for log warning, the rows which are ignored is exclude.
// HyperLogLogCount-related data members
static const int8_t LLC_BUCKET_BITS = 10;
static const int64_t LLC_NUM_BUCKETS = (1 << LLC_BUCKET_BITS);
common::ObFixedArray<bool, common::ObIAllocator> first_rollup_cols_;
common::ObArenaAllocator agg_udf_buf_;
common::ObSEArray<ObAggUdfMeta, 16> agg_udf_metas_;
common::hash::ObHashMap<int64_t, ObAggUdfExeUnit, common::hash::NoPthreadDefendMode> agg_udf_;
};
inline void ObAggregateFunction::set_int_div_as_double(bool did)
{
did_int_div_as_double_ = did;
}
inline bool ObAggregateFunction::get_int_div_as_double() const
{
return did_int_div_as_double_;
}
inline int ObAggregateFunction::get_cur_row(
const common::ObRowStore::StoredRow*& stored_row, const int64_t group_id) const
{
int ret = common::OB_SUCCESS;
GroupRow tmp;
ret = row_array_.at(group_id, tmp);
stored_row = tmp.row_;
return ret;
}
inline int64_t ObAggregateFunction::get_hold_mem_size() const
{
return stored_row_buf_.total() + agg_cell_ctx_alloc_.total() + alloc_.total() + agg_udf_buf_.total();
}
inline int64_t ObAggregateFunction::get_used_mem_size() const
{
return stored_row_buf_.used() + agg_cell_ctx_alloc_.used() + alloc_.used() + agg_udf_buf_.used();
}
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_AGGREGATE_FUNCTION_H */

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,593 @@
/**
* 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_SQL_ENGINE_AGGREGATE_PROCESSOR_H
#define OCEANBASE_SQL_ENGINE_AGGREGATE_PROCESSOR_H
#include "lib/string/ob_sql_string.h"
#include "lib/container/ob_array.h"
#include "lib/container/ob_se_array.h"
#include "lib/container/ob_vector.h"
#include "lib/list/ob_dlist.h"
#include "lib/hash/ob_hashset.h"
#include "sql/engine/expr/ob_expr.h"
#include "lib/container/ob_fixed_array.h"
#include "sql/engine/sort/ob_sort_op_impl.h"
#include "sql/engine/ob_operator.h"
#include "sql/engine/aggregate/ob_exec_hash_struct.h"
#include "lib/string/ob_fixed_length_string.h"
namespace oceanbase {
namespace sql {
struct ObAggrInfo {
public:
OB_UNIS_VERSION_V(1);
public:
ObAggrInfo()
: expr_(NULL),
real_aggr_type_(T_INVALID),
has_distinct_(false),
is_implicit_first_aggr_(false),
has_order_by_(false),
param_exprs_(),
distinct_collations_(),
distinct_cmp_funcs_(),
group_concat_param_count_(common::OB_INVALID_COUNT),
sort_collations_(),
sort_cmp_funcs_(),
separator_expr_(NULL),
separator_datum_(nullptr),
linear_inter_expr_(NULL)
{}
virtual ~ObAggrInfo();
inline ObObjType get_first_child_type() const;
inline bool is_number() const;
inline bool is_implicit_first_aggr() const
{
return is_implicit_first_aggr_;
}
int64_t get_child_output_count() const
{
return is_implicit_first_aggr()
? 1
: ((T_FUN_COUNT == get_expr_type() && param_exprs_.empty()) ? 0 : param_exprs_.count());
}
int eval_aggr(ObChunkDatumStore::ShadowStoredRow<>& curr_row_results, ObEvalCtx& ctx) const;
inline void set_implicit_first_aggr()
{
is_implicit_first_aggr_ = true;
}
inline ObExprOperatorType get_expr_type() const
{
return (T_INVALID == real_aggr_type_ && NULL != expr_) ? expr_->type_ : real_aggr_type_;
}
inline void set_allocator(common::ObIAllocator* alloc)
{
param_exprs_.set_allocator(alloc);
distinct_collations_.set_allocator(alloc);
distinct_cmp_funcs_.set_allocator(alloc);
sort_collations_.set_allocator(alloc);
sort_cmp_funcs_.set_allocator(alloc);
}
int64_t to_string(char* buf, const int64_t buf_len) const;
ObExpr* expr_;
ObExprOperatorType real_aggr_type_; // used for wf
bool has_distinct_;
bool is_implicit_first_aggr_;
bool has_order_by_;
// distinct exprs or group concat/group aggr/keep aggr exprs can use multi exprs.
ExprFixedArray param_exprs_; // sort expr is also in param_exprs
ObSortCollations distinct_collations_;
ObSortFuncs distinct_cmp_funcs_;
// used for group concat/group aggr/keep aggr
int64_t group_concat_param_count_;
ObSortCollations sort_collations_;
ObSortFuncs sort_cmp_funcs_;
ObExpr* separator_expr_;
ObDatum* separator_datum_;
// used for median/percentile_cont aggr
ObExpr* linear_inter_expr_;
};
typedef common::ObFixedArray<ObAggrInfo, common::ObIAllocator> AggrInfoFixedArray;
inline ObObjType ObAggrInfo::get_first_child_type() const
{
// OB_ASSERT(param_exprs_.count() == 1);
return param_exprs_.at(0)->datum_meta_.type_;
}
inline bool ObAggrInfo::is_number() const
{
// OB_ASSERT(param_exprs_.count() == 1);
const ObObjType tmp_type = param_exprs_.at(0)->datum_meta_.type_;
return (ObNumberType == tmp_type || ObUNumberType == tmp_type || ObNumberFloatType == tmp_type);
}
typedef common::ObFixedArray<ObDatum, common::ObIAllocator> DatumFixedArray;
class ObAggregateProcessor {
public:
// Context structure for one aggregation function of one group, only some functions need this:
// with distinct: need this for distinct calculate
// group concat: need context to hold the input cells.
// rank: need context to hold the input cells.
// percent_rank: need context to hold the input cells.
// dense_rank: need context to hold the input cells.
// cume_dist: need context to hold the input cells.
// max keep(): need context to hold the input cells.
// min keep(): need context to hold the input cells.
// sum keep(): need context to hold the input cells.
// count keep(): need context to hold the input cells.
//
//
// We only implement distinct logic here, no common interface abstracted here,
// use the derived class directly.
class ExtraResult {
public:
// %alloc is used to initialize the structures, can not be used to hold the data
explicit ExtraResult(common::ObIAllocator& alloc) : alloc_(alloc), unique_sort_op_(NULL)
{}
virtual ~ExtraResult();
virtual void reuse();
int init_distinct_set(
const uint64_t tenant_id, const ObAggrInfo& aggr_info, ObEvalCtx& eval_ctx, const bool need_rewind);
DECLARE_VIRTUAL_TO_STRING;
protected:
common::ObIAllocator& alloc_;
public:
// for distinct calculate may be replace by hash based distinct in the future.
ObUniqueSortImpl* unique_sort_op_;
};
class GroupConcatExtraResult : public ExtraResult {
public:
explicit GroupConcatExtraResult(common::ObIAllocator& alloc)
: ExtraResult(alloc), row_count_(0), iter_idx_(0), sort_op_(NULL)
{}
virtual ~GroupConcatExtraResult();
void reuse_self();
virtual void reuse() override;
int init(const uint64_t tenant_id, const ObAggrInfo& aggr_info, ObEvalCtx& eval_ctx, const bool need_rewind);
int add_row(const ObIArray<ObExpr*>& expr, ObEvalCtx& eval_ctx)
{
int ret = sort_op_ != NULL ? sort_op_->add_row(expr) : row_store_.add_row(expr, &eval_ctx);
if (OB_SUCC(ret)) {
row_count_++;
}
return ret;
}
int add_row(const ObChunkDatumStore::StoredRow& sr)
{
int ret = sort_op_ != NULL ? sort_op_->add_stored_row(sr) : row_store_.add_row(sr);
if (OB_SUCC(ret)) {
row_count_++;
}
return ret;
}
int get_next_row(const ObChunkDatumStore::StoredRow*& sr)
{
int ret = sort_op_ != NULL ? sort_op_->get_next_row(sr) : row_store_iter_.get_next_row(sr);
if (OB_SUCC(ret)) {
iter_idx_++;
}
return ret;
}
int finish_add_row();
// iterate twice with rollup process, rewind is needed.
int rewind();
bool empty() const
{
return 0 == row_count_;
}
bool is_iterated() const
{
return iter_idx_ > 0;
}
int get_row_count() const
{
return row_count_;
}
DECLARE_VIRTUAL_TO_STRING;
private:
int64_t row_count_;
int64_t iter_idx_;
ObChunkDatumStore row_store_;
ObChunkDatumStore::Iterator row_store_iter_;
ObSortOpImpl* sort_op_;
};
class AggrCell {
public:
AggrCell()
: curr_row_results_(),
row_count_(0),
tiny_num_int_(0),
is_tiny_num_used_(false),
llc_bitmap_(),
iter_result_(),
alloc_(NULL),
extra_(NULL)
{
iter_result_.set_null();
}
~AggrCell();
void destroy();
void inc_row_count()
{
++row_count_;
}
int64_t get_row_count()
{
return row_count_;
}
void add_row_count(const int64_t value)
{
row_count_ += value;
}
ObDatum& get_iter_result()
{
return iter_result_;
}
int64_t get_tiny_num_int()
{
return tiny_num_int_;
}
uint64_t get_tiny_num_uint()
{
return tiny_num_uint_;
}
void set_iter_result(const ObDatum& value)
{
iter_result_ = value;
}
ObDatum& get_llc_bitmap()
{
return llc_bitmap_;
}
void set_llc_bitmap(const ObDatum& value)
{
llc_bitmap_ = value;
}
void set_tiny_num_int(const int64_t value)
{
tiny_num_int_ = value;
}
void set_tiny_num_uint(const uint64_t value)
{
tiny_num_uint_ = value;
}
void set_tiny_num_used()
{
is_tiny_num_used_ = true;
}
bool is_tiny_num_used() const
{
return is_tiny_num_used_;
}
int collect_result(const ObObjTypeClass tc, ObEvalCtx& eval_ctx, const ObAggrInfo& aggr_info);
ExtraResult* get_extra()
{
return extra_;
}
void set_extra(ExtraResult* extra)
{
extra_ = extra;
}
int64_t to_string(char* buf, const int64_t buf_len) const;
inline void reuse()
{
row_count_ = 0;
tiny_num_int_ = 0;
is_tiny_num_used_ = false;
iter_result_.reset();
iter_result_.set_null();
llc_bitmap_.reset();
if (NULL != extra_) {
extra_->reuse();
}
curr_row_results_.reset();
}
inline void set_allocator(common::ObIAllocator* alloc)
{
alloc_ = alloc;
;
}
public:
ObChunkDatumStore::ShadowStoredRow<> curr_row_results_;
private:
// for avg/count
int64_t row_count_;
// for int fast path
union {
int64_t tiny_num_int_;
uint64_t tiny_num_uint_;
};
bool is_tiny_num_used_;
// for T_FUN_APPROX_COUNT_DISTINCT
ObDatum llc_bitmap_;
ObDatum iter_result_;
common::ObIAllocator* alloc_;
ExtraResult* extra_;
};
struct GroupRow {
GroupRow() : aggr_cells_(nullptr), n_cells_(0), groupby_store_row_(nullptr)
{}
~GroupRow()
{
destroy();
}
TO_STRING_KV(K_(n_cells));
inline void destroy()
{
for (int64_t i = 0; i < n_cells_; ++i) {
aggr_cells_[i].destroy();
}
groupby_store_row_ = nullptr;
}
inline void reuse()
{
for (int64_t i = 0; i < n_cells_; ++i) {
aggr_cells_[i].reuse();
}
groupby_store_row_ = nullptr;
}
AggrCell* aggr_cells_;
int32_t n_cells_;
ObChunkDatumStore::StoredRow* groupby_store_row_;
};
public:
ObAggregateProcessor(ObEvalCtx& eval_ctx, ObIArray<ObAggrInfo>& aggr_infos);
~ObAggregateProcessor()
{
destroy();
};
int init();
void destroy();
void reuse();
int prepare(GroupRow& group_row);
int process(GroupRow& group_row);
int collect(const int64_t group_id = 0, const ObExpr* diff_expr = NULL);
// used by ScalarAggregate operator when there's no input rows
int collect_for_empty_set();
inline void set_in_window_func()
{
in_window_func_ = true;
}
inline bool has_distinct() const
{
return has_distinct_;
}
inline bool has_order_by() const
{
return has_order_by_;
}
inline int64_t get_aggr_used_size() const
{
return aggr_alloc_.used();
}
inline int64_t get_aggr_hold_size() const
{
return aggr_alloc_.total();
}
inline common::ObIAllocator& get_aggr_alloc()
{
return aggr_alloc_;
}
inline void set_tenant_id(const uint64_t tenant_id)
{
aggr_alloc_.set_tenant_id(tenant_id);
}
int init_one_group(const int64_t group_id = 0);
int init_group_rows(const int64_t num_group_col);
int rollup_process(const int64_t group_id, const ObExpr* diff_expr = NULL);
int reuse_group(const int64_t group_id);
inline int get_group_row(const int64_t group_id, GroupRow*& group_row)
{
return group_rows_.at(group_id, group_row);
}
int clone_cell(ObDatum& target_cell, const ObDatum& src_cell, const bool is_number = false);
static int get_llc_size();
private:
int extend_concat_str_buf(const ObString& pad_str, const int64_t pos, const int64_t group_concat_cur_row_num,
int64_t& append_len, bool& buf_is_full);
OB_INLINE bool need_extra_info(const ObExprOperatorType expr_type);
// function members
int prepare_aggr_result(const ObChunkDatumStore::StoredRow& stored_row, const ObIArray<ObExpr*>* param_exprs,
AggrCell& aggr_cell, const ObAggrInfo& aggr_info);
int process_aggr_result(const ObChunkDatumStore::StoredRow& stored_row, const ObIArray<ObExpr*>* param_exprs,
AggrCell& aggr_cell, const ObAggrInfo& aggr_info);
int collect_aggr_result(AggrCell& aggr_cell, const ObExpr* diff_expr, const ObAggrInfo& aggr_info);
int process_aggr_result_from_distinct(AggrCell& aggr_cell, const ObAggrInfo& aggr_info);
int clone_number_cell(const number::ObNumber& src_cell, ObDatum& target_cell);
int max_calc(ObDatum& base, const ObDatum& other, common::ObDatumCmpFuncType cmp_func, const bool is_number);
int min_calc(ObDatum& base, const ObDatum& other, common::ObDatumCmpFuncType cmp_func, const bool is_number);
int prepare_add_calc(const ObDatum& iter_value, AggrCell& aggr_cell, const ObAggrInfo& aggr_info);
int add_calc(const ObDatum& iter_value, AggrCell& aggr_cell, const ObAggrInfo& aggr_info);
int rollup_add_calc(AggrCell& aggr_cell, AggrCell& rollup_cell, const ObAggrInfo& aggr_info);
int search_op_expr(ObExpr* upper_expr, const ObItemType dst_op, ObExpr*& res_expr);
int linear_inter_calc(const ObAggrInfo& aggr_info, const ObDatum& prev_datum, const ObDatum& curr_datum,
const number::ObNumber& factor, ObDatum& res);
int get_percentile_param(const ObAggrInfo& aggr_info, const ObDatum& param, const int64_t not_null_start_loc,
const int64_t total_row_count, int64_t& dest_loc, bool& need_linear_inter, number::ObNumber& factor,
ObDataBuffer& allocator);
int rollup_add_calc(AggrCell& aggr_cell, AggrCell& rollup_cell);
int rollup_add_number_calc(ObDatum& aggr_result, ObDatum& rollup_result);
int rollup_aggregation(
AggrCell& aggr_cell, AggrCell& rollup_cell, const ObExpr* diff_expr, const ObAggrInfo& aggr_info);
int rollup_distinct(AggrCell& aggr_cell, AggrCell& rollup_cell);
int compare_calc(const ObDatum& left_value, const ObDatum& right_value, const ObAggrInfo& aggr_info, int64_t index,
int& compare_result, bool& is_asc);
int check_rows_equal(const ObChunkDatumStore::LastStoredRow<>& prev_row, const ObChunkDatumStore::StoredRow& cur_row,
const ObAggrInfo& aggr_info, bool& is_equal);
int get_wm_concat_result(
const ObAggrInfo& aggr_info, GroupConcatExtraResult*& extra, bool is_keep_group_concat, ObDatum& concat_result);
// HyperLogLogCount-related functions
int llc_init(ObDatum& datum);
int llc_init_empty(ObExpr& expr, ObEvalCtx& eval_ctx);
static uint64_t llc_calc_hash_value(
const ObChunkDatumStore::StoredRow& stored_row, const ObIArray<ObExpr*>& param_exprs, bool& has_null_cell);
static int llc_add_value(const uint64_t value, const common::ObString& llc_bitmap_buf);
static int llc_add(ObDatum& result, const ObDatum& new_value);
static const int8_t LLC_BUCKET_BITS = 10;
static const int64_t LLC_NUM_BUCKETS = (1 << LLC_BUCKET_BITS);
// const static int64_t CONCAT_STR_BUF_LEN = common::OB_MAX_VARCHAR_LENGTH;
const static int64_t STORED_ROW_MAGIC_NUM = 0xaaaabbbbccccdddd;
// typedef common::hash::ObHashSet<AggrDistinctItem, common::hash::NoPthreadDefendMode> AggrDistinctBucket;
//
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObAggregateProcessor);
private:
static const int64_t GROUP_ROW_SIZE = sizeof(GroupRow);
static const int64_t GROUP_CELL_SIZE = sizeof(AggrCell);
// data members
bool has_distinct_;
bool has_order_by_;
bool has_group_concat_;
bool in_window_func_;
bool has_extra_;
ObEvalCtx& eval_ctx_;
common::ObArenaAllocator aggr_alloc_;
ObIArray<ObAggrInfo>& aggr_infos_;
common::ObSEArray<GroupRow*, 1> group_rows_;
uint64_t concat_str_max_len_;
uint64_t cur_concat_buf_len_;
char* concat_str_buf_;
// common::ObArenaAllocator aggr_udf_buf_;
// common::ObSEArray<ObAggUdfMeta, 16> aggr_udf_metas_;
// common::hash::ObHashMap<int64_t, ObAggUdfExeUnit, common::hash::NoPthreadDefendMode> aggr_udf_;
};
// Used for calc hash for columns
class ObGroupRowItem {
public:
ObGroupRowItem() : group_id_(0), group_row_ptr_(NULL), groupby_datums_hash_(0), next_(NULL)
{}
~ObGroupRowItem()
{}
inline uint64_t hash() const
{
return groupby_datums_hash_;
}
ObGroupRowItem*& next()
{
return *reinterpret_cast<ObGroupRowItem**>(&next_);
};
TO_STRING_KV(K_(group_id), KPC_(group_row), K_(groupby_datums_hash), KP_(group_exprs), KP_(next));
public:
int64_t group_id_;
union {
void* group_row_ptr_;
ObAggregateProcessor::GroupRow* group_row_;
ExprFixedArray* group_exprs_;
};
uint64_t groupby_datums_hash_;
void* next_;
};
class ObGroupRowHashTable : public ObExtendHashTable<ObGroupRowItem> {
public:
ObGroupRowHashTable() : ObExtendHashTable(), eval_ctx_(nullptr), cmp_funcs_(nullptr)
{}
const ObGroupRowItem* get(const ObGroupRowItem& item) const;
int init(ObIAllocator* allocator, lib::ObMemAttr& mem_attr, ObEvalCtx* eval_ctx,
const common::ObIArray<ObCmpFunc>* cmp_funcs, int64_t initial_size = INITIAL_SIZE);
private:
bool compare(const ObGroupRowItem& left, const ObGroupRowItem& right) const;
private:
ObEvalCtx* eval_ctx_;
const common::ObIArray<ObCmpFunc>* cmp_funcs_;
};
struct ObAggregateCalcFunc {
const static int64_t STORED_ROW_MAGIC_NUM = 0xaaaabbbbccccdddd;
static int add_calc(const ObDatum& left_value, const ObDatum& right_value, ObDatum& result_datum,
const ObObjTypeClass type, ObIAllocator& allocator);
static int clone_number_cell(const number::ObNumber& src_cell, ObDatum& target_cell, ObIAllocator& allocator);
};
OB_INLINE bool ObAggregateProcessor::need_extra_info(const ObExprOperatorType expr_type)
{
bool need_extra = false;
switch (expr_type) {
case T_FUN_GROUP_CONCAT:
case T_FUN_GROUP_RANK:
case T_FUN_GROUP_DENSE_RANK:
case T_FUN_GROUP_PERCENT_RANK:
case T_FUN_GROUP_CUME_DIST:
case T_FUN_GROUP_PERCENTILE_CONT:
case T_FUN_GROUP_PERCENTILE_DISC:
case T_FUN_MEDIAN:
case T_FUN_KEEP_MAX:
case T_FUN_KEEP_MIN:
case T_FUN_KEEP_SUM:
case T_FUN_KEEP_COUNT:
case T_FUN_KEEP_WM_CONCAT:
case T_FUN_WM_CONCAT: {
need_extra = true;
break;
}
default:
break;
}
return need_extra;
}
} // end namespace sql
} // end namespace oceanbase
#endif /* OCEANBASE_SQL_ENGINE_AGGREGATE_PROCESSOR_H */

View File

@ -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 "lib/allocator/ob_malloc.h"
#include "lib/utility/utility.h"
#include "common/row/ob_row.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/aggregate/ob_distinct.h"
namespace oceanbase {
using namespace common;
namespace sql {
ObDistinct::ObDistinct(common::ObIAllocator& alloc)
: ObSingleChildPhyOperator(alloc), distinct_columns_(alloc), is_block_mode_(false)
{}
ObDistinct::~ObDistinct()
{}
void ObDistinct::reset()
{
distinct_columns_.reset();
ObSingleChildPhyOperator::reset();
}
void ObDistinct::reuse()
{
distinct_columns_.reuse();
ObSingleChildPhyOperator::reuse();
}
int ObDistinct::add_distinct_column(const int64_t column_index, ObCollationType cs_type)
{
ObDistinctColumn distinct_column;
distinct_column.index_ = column_index;
distinct_column.cs_type_ = cs_type;
return distinct_columns_.push_back(distinct_column);
}
int64_t ObDistinct::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV(N_DISTINCT, distinct_columns_);
return pos;
}
OB_SERIALIZE_MEMBER((ObDistinct, ObSingleChildPhyOperator), distinct_columns_, is_block_mode_);
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,57 @@
/**
* 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_ENGINE_AGGREGATE_OB_DISTINCT
#define SQL_ENGINE_AGGREGATE_OB_DISTINCT
#include "lib/string/ob_string.h"
#include "share/ob_define.h"
#include "common/row/ob_row.h"
#include "sql/engine/ob_single_child_phy_operator.h"
#include "lib/container/ob_fixed_array.h"
namespace oceanbase {
namespace sql {
typedef common::ObColumnInfo ObDistinctColumn;
class ObDistinct : public ObSingleChildPhyOperator {
OB_UNIS_VERSION_V(1);
public:
explicit ObDistinct(common::ObIAllocator& alloc);
// ObDistinct();
virtual ~ObDistinct();
virtual void reset();
virtual void reuse();
// not always whole row is compared
// example: SELECT distinct c1 FROM t order by c2;
int add_distinct_column(const int64_t column_index, common::ObCollationType cs_type);
inline void set_block_mode(bool is_block_mode)
{
is_block_mode_ = is_block_mode;
}
// TODO distinct operator need re_est_cost function
int init(int64_t distinct_count)
{
return ObPhyOperator::init_array_size<>(distinct_columns_, distinct_count);
}
private:
DISALLOW_COPY_AND_ASSIGN(ObDistinct);
protected:
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
// data members
common::ObFixedArray<ObDistinctColumn, common::ObIAllocator> distinct_columns_;
bool is_block_mode_;
};
} // namespace sql
} // namespace oceanbase
#endif

View File

@ -0,0 +1,32 @@
/**
* 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/aggregate/ob_distinct_op.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/px/ob_px_util.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
ObDistinctSpec::ObDistinctSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
: ObOpSpec(alloc, type), distinct_exprs_(), cmp_funcs_(alloc), is_block_mode_(false)
{}
OB_SERIALIZE_MEMBER((ObDistinctSpec, ObOpSpec), distinct_exprs_, cmp_funcs_, is_block_mode_);
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,39 @@
/**
* 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_SRC_SQL_ENGINE_AGGREGATE_OB_DISTINCT_OP_H_
#define OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_DISTINCT_OP_H_
#include "sql/engine/ob_operator.h"
#include "share/datum/ob_datum_funcs.h"
namespace oceanbase {
namespace sql {
class ObDistinctSpec : public ObOpSpec {
OB_UNIS_VERSION_V(1);
public:
ObDistinctSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(distinct_exprs), K_(is_block_mode), K_(cmp_funcs));
// data members
common::ObSEArray<ObExpr*, 4> distinct_exprs_;
common::ObCmpFuncs cmp_funcs_;
bool is_block_mode_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_DISTINCT_OP_H_ */

View File

@ -0,0 +1,105 @@
/**
* 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 "ob_exec_hash_struct.h"
#include "common/row/ob_row_store.h"
namespace oceanbase {
using namespace common;
namespace sql {
// When there's stored_row_ and reserved_cells_, use store_row's reserved_cells_ for calc hash.
// Other, use row_ for calc hash
uint64_t ObHashCols::inner_hash() const
{
uint64_t result = 99194853094755497L;
if (hash_col_idx_ != NULL) {
int64_t group_col_count = hash_col_idx_->count();
if (stored_row_ != NULL && stored_row_->reserved_cells_count_ > 0) {
const ObObj* cells = stored_row_->reserved_cells_;
for (int32_t i = 0; i < group_col_count; ++i) {
if (hash_col_idx_->at(i).index_ < stored_row_->reserved_cells_count_) {
const ObObj& cell = cells[hash_col_idx_->at(i).index_];
result = cell.is_string_type() ? cell.varchar_murmur_hash(hash_col_idx_->at(i).cs_type_, result)
: cell.hash(result);
}
}
} else if (row_ != NULL && row_->is_valid()) {
const ObObj* cells = row_->cells_;
const int32_t* projector = row_->projector_;
for (int64_t i = 0; i < group_col_count; ++i) {
int64_t real_index =
row_->projector_size_ > 0 ? projector[hash_col_idx_->at(i).index_] : hash_col_idx_->at(i).index_;
const ObObj& cell = cells[real_index];
result =
cell.is_string_type() ? cell.varchar_murmur_hash(hash_col_idx_->at(i).cs_type_, result) : cell.hash(result);
}
}
}
return result;
}
// When there is stored_row_ reserved_cells, use stored_row_'s reserved_cells_ for calc equal.
// Other use row_.
bool ObHashCols::operator==(const ObHashCols& other) const
{
bool result = true;
const ObObj* lcell = NULL;
const ObObj* rcell = NULL;
if (OB_ISNULL(hash_col_idx_)) {
result = false;
} else {
int64_t group_col_count = hash_col_idx_->count();
// When group_col_count<=0 is the case of hash group by const,
// it should be considered as a group,
// so result should be initialized to true
for (int32_t i = 0; i < group_col_count && result; ++i) {
int64_t group_idx = hash_col_idx_->at(i).index_;
if (stored_row_ != NULL) {
if (group_idx < stored_row_->reserved_cells_count_) {
lcell = &stored_row_->reserved_cells_[group_idx];
}
} else if (row_ != NULL) {
if (row_->is_valid()) {
int64_t real_idx = row_->projector_size_ > 0 ? row_->projector_[group_idx] : group_idx;
lcell = &row_->cells_[real_idx];
}
}
if (other.stored_row_ != NULL) {
if (group_idx < other.stored_row_->reserved_cells_count_) {
rcell = &other.stored_row_->reserved_cells_[group_idx];
}
} else if (other.row_ != NULL) {
if (other.row_->is_valid()) {
int64_t real_idx = other.row_->projector_size_ > 0 ? other.row_->projector_[group_idx] : group_idx;
rcell = &other.row_->cells_[real_idx];
}
}
if (NULL == lcell || NULL == rcell) {
result = false;
} else {
result = lcell->is_equal(*rcell, hash_col_idx_->at(i).cs_type_);
}
}
}
return result;
}
void ObHashCols::set_stored_row(const ObRowStore::StoredRow* stored_row)
{
stored_row_ = stored_row;
row_ = NULL;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,453 @@
/**
* 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_ENGINE_AGGREGATE_OB_HASH_STRUCT
#define SQL_ENGINE_AGGREGATE_OB_HASH_STRUCT
#include "lib/hash/ob_hashmap.h"
#include "lib/hash/ob_hashutils.h"
#include "common/row/ob_row_store.h"
#include "lib/utility/utility.h"
#include "lib/ob_define.h"
#include "sql/engine/basic/ob_chunk_row_store.h"
#include "lib/container/ob_2d_array.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
namespace oceanbase {
namespace common {
class ObNewRow;
class ObRowStore;
} // namespace common
namespace sql {
// Auto extended hash table, extend to double buckets size if hash table is quarter filled.
template <typename Item>
class ObExtendHashTable {
public:
const static int64_t INITIAL_SIZE = 128;
const static int64_t SIZE_BUCKET_SCALE = 4;
ObExtendHashTable() : initial_bucket_num_(0), size_(0), buckets_(NULL), allocator_(NULL)
{}
~ObExtendHashTable()
{
destroy();
}
int init(ObIAllocator* allocator, lib::ObMemAttr& mem_attr, int64_t initial_size = INITIAL_SIZE);
bool is_inited() const
{
return NULL != buckets_;
}
// return the first item which equal to, NULL for none exist.
const Item* get(const Item& item) const;
// Link item to hash table, extend buckets if needed.
// (Do not check item is exist or not)
int set(Item& item);
int64_t size() const
{
return size_;
}
void reuse()
{
int ret = common::OB_SUCCESS;
if (nullptr != buckets_) {
int64_t bucket_num = get_bucket_num();
buckets_->reuse();
if (OB_FAIL(buckets_->init(bucket_num))) {
SQL_ENG_LOG(ERROR, "resize bucket array failed", K(size_), K(bucket_num), K(get_bucket_num()));
}
}
size_ = 0;
}
int resize(ObIAllocator* allocator, int64_t bucket_num);
void destroy()
{
if (NULL != buckets_) {
buckets_->destroy();
allocator_.free(buckets_);
buckets_ = NULL;
}
allocator_.set_allocator(nullptr);
size_ = 0;
initial_bucket_num_ = 0;
}
int64_t mem_used() const
{
return NULL == buckets_ ? 0 : buckets_->mem_used();
}
inline int64_t get_bucket_num() const
{
return NULL == buckets_ ? 0 : buckets_->count();
}
template <typename CB>
int foreach (CB& cb) const
{
int ret = common::OB_SUCCESS;
if (OB_ISNULL(buckets_)) {
ret = OB_INVALID_ARGUMENT;
SQL_ENG_LOG(WARN, "invalid null buckets", K(ret), K(buckets_));
}
for (int64_t i = 0; OB_SUCC(ret) && i < get_bucket_num(); i++) {
Item* item = buckets_->at(i);
while (NULL != item && OB_SUCC(ret)) {
if (OB_FAIL(cb(*item))) {
SQL_ENG_LOG(WARN, "call back failed", K(ret));
} else {
item = item->next();
}
}
}
return ret;
}
protected:
DISALLOW_COPY_AND_ASSIGN(ObExtendHashTable);
int extend();
protected:
lib::ObMemAttr mem_attr_;
int64_t initial_bucket_num_;
int64_t size_;
using BucketArray = common::ObSegmentArray<Item*, OB_MALLOC_BIG_BLOCK_SIZE, common::ModulePageAllocator>;
BucketArray* buckets_;
common::ModulePageAllocator allocator_;
};
template <typename Item>
int ObExtendHashTable<Item>::init(
ObIAllocator* allocator, lib::ObMemAttr& mem_attr, const int64_t initial_size /* INITIAL_SIZE */)
{
int ret = common::OB_SUCCESS;
if (initial_size < 2) {
ret = common::OB_INVALID_ARGUMENT;
SQL_ENG_LOG(WARN, "invalid argument", K(ret));
} else {
mem_attr_ = mem_attr;
allocator_.set_allocator(allocator);
allocator_.set_label(mem_attr.label_);
void* buckets_buf = NULL;
if (OB_ISNULL(buckets_buf = allocator_.alloc(sizeof(BucketArray), mem_attr))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
SQL_ENG_LOG(WARN, "failed to allocate memory", K(ret));
} else {
buckets_ = new (buckets_buf) BucketArray(allocator_);
initial_bucket_num_ = common::next_pow2(initial_size * SIZE_BUCKET_SCALE);
size_ = 0;
}
if (OB_FAIL(ret)) {
// do nothing
} else if (OB_FAIL(extend())) {
SQL_ENG_LOG(WARN, "extend failed", K(ret));
}
}
return ret;
}
template <typename Item>
int ObExtendHashTable<Item>::resize(ObIAllocator* allocator, int64_t bucket_num)
{
int ret = OB_SUCCESS;
if (bucket_num < get_bucket_num() / 2) {
destroy();
if (OB_FAIL(init(allocator, mem_attr_, bucket_num))) {
SQL_ENG_LOG(WARN, "failed to reuse with bucket", K(bucket_num), K(ret));
}
} else {
reuse();
}
return ret;
}
template <typename Item>
const Item* ObExtendHashTable<Item>::get(const Item& item) const
{
Item* res = NULL;
if (NULL == buckets_) {
// do nothing
} else {
common::hash::hash_func<Item> hf;
common::hash::equal_to<Item> eqf;
const uint64_t hash_val = hf(item);
Item* bucket = buckets_->at(hash_val & (get_bucket_num() - 1));
while (NULL != bucket) {
if (hash_val == hf(*bucket) && eqf(*bucket, item)) {
res = bucket;
break;
}
bucket = bucket->next();
}
}
return res;
}
template <typename Item>
int ObExtendHashTable<Item>::set(Item& item)
{
common::hash::hash_func<Item> hf;
int ret = common::OB_SUCCESS;
if (size_ * SIZE_BUCKET_SCALE >= get_bucket_num()) {
if (OB_FAIL(extend())) {
SQL_ENG_LOG(WARN, "extend failed", K(ret));
}
}
if (OB_FAIL(ret)) {
// do nothing
} else if (OB_ISNULL(buckets_)) {
ret = OB_INVALID_ARGUMENT;
SQL_ENG_LOG(WARN, "invalid argument", K(ret), K(buckets_));
} else {
Item*& bucket = buckets_->at(hf(item) & (get_bucket_num() - 1));
item.next() = bucket;
bucket = &item;
size_ += 1;
}
return ret;
}
template <typename Item>
int ObExtendHashTable<Item>::extend()
{
common::hash::hash_func<Item> hf;
int ret = common::OB_SUCCESS;
const int64_t new_bucket_num =
0 == get_bucket_num() ? (0 == initial_bucket_num_ ? INITIAL_SIZE : initial_bucket_num_) : get_bucket_num() * 2;
BucketArray* new_buckets = NULL;
void* buckets_buf = NULL;
if (OB_ISNULL(buckets_buf = allocator_.alloc(sizeof(BucketArray), mem_attr_))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
SQL_ENG_LOG(WARN, "failed to allocate memory", K(ret));
} else {
new_buckets = new (buckets_buf) BucketArray(allocator_);
}
if (OB_FAIL(ret)) {
// do nothing
} else if (OB_ISNULL(buckets_)) {
ret = OB_INVALID_ARGUMENT;
SQL_ENG_LOG(WARN, "invalid argument", K(ret), K(buckets_));
} else if (OB_FAIL(new_buckets->init(new_bucket_num))) {
SQL_ENG_LOG(WARN, "resize bucket array failed", K(ret), K(new_bucket_num));
} else {
for (int64_t i = 0; i < get_bucket_num(); i++) {
Item* bucket = buckets_->at(i);
while (bucket != NULL) {
Item* item = bucket;
bucket = bucket->next();
Item*& new_bucket = new_buckets->at(hf(*item) & (new_bucket_num - 1));
item->next() = new_bucket;
new_bucket = item;
}
}
buckets_->destroy();
allocator_.free(buckets_);
buckets_ = new_buckets;
}
if (OB_FAIL(ret)) {
if (buckets_ == new_buckets) {
SQL_ENG_LOG(ERROR, "unexpected status: failed allocate new bucket", K(ret));
} else if (nullptr != new_buckets) {
new_buckets->destroy();
allocator_.free(new_buckets);
new_buckets = nullptr;
}
}
return ret;
}
// Used for calc hash for columns
class ObHashCols {
public:
ObHashCols() : row_(NULL), stored_row_(NULL), hash_col_idx_(NULL), next_(NULL), hash_val_(0)
{}
ObHashCols(const common::ObNewRow* row, const common::ObIArray<common::ObColumnInfo>* hash_col_idx)
: row_(row), stored_row_(NULL), hash_col_idx_(hash_col_idx), next_(NULL), hash_val_(0)
{}
~ObHashCols()
{}
int init(const common::ObNewRow* row, const common::ObIArray<common::ObColumnInfo>* hash_col_idx,
const uint64_t hash_val = 0)
{
row_ = row;
stored_row_ = NULL;
hash_col_idx_ = hash_col_idx;
hash_val_ = hash_val;
return common::OB_SUCCESS;
}
uint64_t hash() const
{
if (hash_val_ == 0) {
hash_val_ = inner_hash();
}
return hash_val_;
}
uint64_t inner_hash() const;
bool operator==(const ObHashCols& other) const;
void set_stored_row(const common::ObRowStore::StoredRow* stored_row);
ObHashCols*& next()
{
return *reinterpret_cast<ObHashCols**>(&next_);
};
TO_STRING_KV(K_(row), K_(stored_row), K_(hash_col_idx), K_(next), K_(hash_val));
public:
const common::ObNewRow* row_;
const common::ObRowStore::StoredRow* stored_row_;
const common::ObIArray<common::ObColumnInfo>* hash_col_idx_;
void* next_;
mutable uint64_t hash_val_;
};
class ObGbyHashCols : public ObHashCols {
public:
using ObHashCols::ObHashCols;
ObGbyHashCols*& next()
{
return *reinterpret_cast<ObGbyHashCols**>(&next_);
};
public:
int64_t group_id_ = 0;
};
// This class not inherit ObPhyOperatorCtx, as multi-inheritance
// Used for build hash group row.
template <typename Item>
class ObHashCtx {
public:
explicit ObHashCtx() : group_rows_(), started_(false), bkt_created_(false)
{}
virtual ~ObHashCtx()
{
group_rows_.destroy();
}
virtual void destroy()
{
group_rows_.~ObExtendHashTable();
}
int64_t get_used_mem_size() const
{
return group_rows_.mem_used();
}
private:
DISALLOW_COPY_AND_ASSIGN(ObHashCtx);
protected:
ObExtendHashTable<Item> group_rows_;
bool started_;
bool bkt_created_;
};
//
// We use 8 bit for one element and 2 hash functions, false positive probability is:
// (1/8 + 1/8 - 1/8 * 1/8)^2 ~= 5.5%, good enough for us.
//
// To reuse the hash value (hash_val) of hash table && partition, we restrict bit count (bit_cnt)
// to power of 2, and construct bloom filter hashes (h1, h2) from hash_val:
// h1: return hash_val directly, only the low log2(bit_cnt) will be used.
// h2: bswap high 32 bit of hash_val, then right shift log2(bit_cnt) bits.
// bswap is needed here because the low bit of hight 32 bit may be used for partition, they
// are the same in one partition.
//
class ObGbyBloomFilter {
public:
explicit ObGbyBloomFilter(const ModulePageAllocator& alloc) : bits_(alloc), cnt_(0), h2_shift_(0)
{}
int init(const int64_t size, int64_t ratio = 8)
{
int ret = common::OB_SUCCESS;
if (size <= 0 || ratio <= 0) {
ret = common::OB_INVALID_ARGUMENT;
SQL_ENG_LOG(WARN, "invalid bit cnt", K(ret), K(size), K(ratio));
} else {
cnt_ = next_pow2(ratio * size);
if (OB_FAIL(bits_.reserve(cnt_))) {
SQL_ENG_LOG(WARN, "bit set reserve failed", K(ret));
} else {
h2_shift_ = sizeof(cnt_) * CHAR_BIT - __builtin_clzl(cnt_);
}
}
return ret;
}
void reuse()
{
bits_.reuse();
cnt_ = 0;
h2_shift_ = 0;
}
void reset()
{
bits_.reset();
cnt_ = 0;
h2_shift_ = 0;
}
private:
inline uint64_t h1(const uint64_t hash_val)
{
return hash_val;
}
inline uint64_t h2(const uint64_t hash_val)
{
#ifdef OB_LITTLE_ENDIAN
const static int64_t idx = 1;
#else
const static int64_t idx = 0;
#endif
uint64_t v = hash_val;
reinterpret_cast<uint32_t*>(&v)[idx] = __builtin_bswap32(reinterpret_cast<const uint32_t*>(&hash_val)[idx]);
return v >> h2_shift_;
}
public:
int set(const uint64_t hash_val)
{
int ret = common::OB_SUCCESS;
if (0 == cnt_) {
ret = OB_ERR_UNEXPECTED;
SQL_ENG_LOG(WARN, "invalied cnt", K(ret), K(cnt_));
} else if (OB_FAIL(bits_.add_member(h1(hash_val) & (cnt_ - 1))) ||
OB_FAIL(bits_.add_member(h2(hash_val) & (cnt_ - 1)))) {
SQL_ENG_LOG(WARN, "bit set add member failed", K(ret), K(cnt_));
}
return ret;
}
bool exist(const uint64_t hash_val)
{
return bits_.has_member(h1(hash_val) & (cnt_ - 1)) && bits_.has_member(h2(hash_val) & (cnt_ - 1));
}
private:
ObSegmentBitSet<> bits_;
int64_t cnt_; // power of 2
int64_t h2_shift_;
};
} // namespace sql
} // namespace oceanbase
#endif

View File

@ -0,0 +1,342 @@
/**
* 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/aggregate/ob_groupby.h"
#include "lib/utility/utility.h"
#include "sql/ob_sql_utils.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/expr/ob_sql_expression.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
using namespace serialization;
namespace sql {
ObGroupBy::ObGroupBy(common::ObIAllocator& alloc)
: ObSingleChildPhyOperator(alloc),
mem_size_limit_(0),
prepare_row_num_(1),
distinct_set_bucket_num_(100),
group_col_idxs_(alloc),
has_rollup_(false),
est_group_cnt_(0),
rollup_col_idxs_(alloc)
{
UNUSED(alloc);
}
ObGroupBy::~ObGroupBy()
{}
void ObGroupBy::reset()
{
aggr_columns_.reset();
mem_size_limit_ = 0;
prepare_row_num_ = 1;
distinct_set_bucket_num_ = 100;
group_col_idxs_.reset();
est_group_cnt_ = 0;
rollup_col_idxs_.reset();
ObSingleChildPhyOperator::reset();
}
void ObGroupBy::reuse()
{
aggr_columns_.reset();
mem_size_limit_ = 0;
prepare_row_num_ = 1;
distinct_set_bucket_num_ = 100;
group_col_idxs_.reuse();
est_group_cnt_ = 0;
rollup_col_idxs_.reuse();
ObSingleChildPhyOperator::reuse();
}
void ObGroupBy::set_mem_size_limit(const int64_t limit)
{
mem_size_limit_ = limit;
}
void ObGroupBy::set_rollup(const bool has_rollup)
{
has_rollup_ = has_rollup;
}
int ObGroupBy::add_aggr_column(ObAggregateExpression* expr)
{
int ret = ObSqlExpressionUtil::add_expr_to_list(aggr_columns_, expr);
return ret;
}
int ObGroupBy::init_group_by(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
int64_t child_column_count = 0;
int64_t est_bucket_size = 0;
int64_t optimizer_est_input_rows = 0;
ObExprCtx expr_ctx;
ObGroupByCtx* groupby_ctx = NULL;
if (OB_FAIL(init_op_ctx(ctx))) {
LOG_WARN("init operator context failed", K(ret));
} else if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
LOG_WARN("wrap expr context failed", K(ret));
} else if (OB_ISNULL(groupby_ctx = GET_PHY_OPERATOR_CTX(ObGroupByCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get physical operator context failed");
} else if (OB_ISNULL(my_phy_plan_) || OB_ISNULL(child_op_)) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K_(my_phy_plan), K_(child_op));
} else if (OB_FAIL(ObSQLUtils::get_default_cast_mode(my_phy_plan_, expr_ctx.my_session_, expr_ctx.cast_mode_))) {
LOG_WARN("set cast mode failed", K(ret));
} else if (OB_ISNULL(get_child(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the child of group by can not be null", K(ret));
} else {
child_column_count =
(child_op_->get_projector_size() > 0 ? child_op_->get_projector_size() : child_op_->get_column_count());
optimizer_est_input_rows = get_child(0)->get_rows();
est_bucket_size = optimizer_est_input_rows;
est_bucket_size = est_bucket_size < MIN_BUCKET_COUNT ? MIN_BUCKET_COUNT : est_bucket_size;
est_bucket_size = est_bucket_size > MAX_BUCKET_COUNT ? MAX_BUCKET_COUNT : est_bucket_size;
// LOG_DEBUG("Init hash distinct bucket size", K(optimizer_est_input_rows), K(est_distinct_input_rows));
if (OB_FAIL(groupby_ctx->get_aggr_func().init_first_rollup_cols(
&ctx.get_allocator(), group_col_idxs_, rollup_col_idxs_))) {
LOG_WARN("failed to init group cols.", K(ret));
} else if (OB_FAIL(groupby_ctx->get_aggr_func().init(
child_column_count, &aggr_columns_, expr_ctx, prepare_row_num_, est_bucket_size))) {
LOG_WARN("failed to construct row desc", K(ret));
} else if (OB_FAIL(groupby_ctx->get_aggr_func().init_agg_udf(agg_udf_meta_))) {
LOG_WARN("failed to init udf meta", K(ret));
} else {
groupby_ctx->get_aggr_func().set_tenant_id(ctx.get_my_session()->get_effective_tenant_id());
}
}
return ret;
}
int ObGroupBy::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_FAIL(inner_create_operator_ctx(ctx, op_ctx))) {
LOG_WARN("create operator context failed", K(ret), K_(id));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op_ctx is null");
} else if (OB_FAIL(init_cur_row(*op_ctx, true /* need create row cells */))) {
LOG_WARN("create current row failed", K(ret));
}
return ret;
}
int ObGroupBy::is_same_group(
const ObRowStore::StoredRow& row1, const ObNewRow& row2, bool& result, int64_t& first_diff_pos) const
{
int ret = OB_SUCCESS;
const ObObj* lcell = NULL;
const ObObj* rcell = NULL;
result = true;
for (int64_t i = 0; OB_SUCC(ret) && result && i < group_col_idxs_.count(); ++i) {
int64_t group_idx = group_col_idxs_[i].index_;
if (OB_UNLIKELY(group_idx >= row1.reserved_cells_count_) || OB_UNLIKELY(row2.is_invalid()) ||
OB_UNLIKELY(group_idx >= row2.get_count())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(row1), K(row2), K(group_idx));
} else {
lcell = &row1.reserved_cells_[group_idx];
rcell = &row2.get_cell(group_idx); // read through projector
if (lcell->compare(*rcell, group_col_idxs_[i].cs_type_) != 0) {
first_diff_pos = i;
result = false;
}
}
}
for (int64_t j = 0; OB_SUCC(ret) && result && j < rollup_col_idxs_.count(); ++j) {
int64_t rollup_idx = rollup_col_idxs_[j].index_;
if (OB_INVALID_INDEX == rollup_idx)
continue;
if (OB_UNLIKELY(rollup_idx >= row1.reserved_cells_count_) || OB_UNLIKELY(row2.is_invalid()) ||
OB_UNLIKELY(rollup_idx >= row2.get_count())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(row1), K(row2), K(rollup_idx));
} else {
lcell = &row1.reserved_cells_[rollup_idx];
rcell = &row2.get_cell(rollup_idx); // read through projector
if (lcell->compare(*rcell, rollup_col_idxs_[j].cs_type_) != 0) {
first_diff_pos = j + group_col_idxs_.count();
result = false;
}
}
}
return ret;
}
int64_t ObGroupBy::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV(N_AGGR_COLUMN,
aggr_columns_,
N_MEM_LIMIT,
mem_size_limit_,
N_GROUP_BY_IDX,
group_col_idxs_,
"rollup",
has_rollup_,
N_ROLLUP_IDX,
rollup_col_idxs_);
return pos;
}
OB_DEF_SERIALIZE(ObGroupBy)
{
int ret = OB_SUCCESS;
int32_t aggr_col_count = aggr_columns_.get_size();
OB_UNIS_ENCODE(aggr_col_count);
DLIST_FOREACH(node, aggr_columns_)
{
if (OB_ISNULL(node)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("node or expr is null");
} else if (OB_FAIL(node->serialize(buf, buf_len, pos))) {
LOG_WARN("serialize aggregate column expression failed", K(ret));
}
}
OB_UNIS_ENCODE(group_col_idxs_);
OB_UNIS_ENCODE(mem_size_limit_);
OB_UNIS_ENCODE(prepare_row_num_);
OB_UNIS_ENCODE(distinct_set_bucket_num_);
if (OB_SUCC(ret)) {
if (OB_FAIL(ObSingleChildPhyOperator::serialize(buf, buf_len, pos))) {
LOG_WARN("serialize single child operator failed", K(ret));
}
}
OB_UNIS_ENCODE(has_rollup_);
OB_UNIS_ENCODE(agg_udf_meta_);
for (int64_t i = 0; i < agg_udf_meta_.count() && OB_SUCC(ret); ++i) {
const ObAggUdfMeta& udf_meta = agg_udf_meta_.at(i);
for (int64_t j = 0; j < udf_meta.calculable_results_.count() && OB_SUCC(ret); ++j) {
const ObUdfConstArgs& args = udf_meta.calculable_results_.at(i);
const ObSqlExpression* expr = args.sql_calc_;
if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr is null");
} else if (OB_FAIL(expr->serialize(buf, buf_len, pos))) {
LOG_WARN("serialize aggregate column expression failed", K(ret));
}
}
}
OB_UNIS_ENCODE(est_group_cnt_);
OB_UNIS_ENCODE(rollup_col_idxs_);
return ret;
}
OB_DEF_DESERIALIZE(ObGroupBy)
{
int ret = OB_SUCCESS;
int32_t aggr_col_count = 0;
if (OB_ISNULL(my_phy_plan_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("phy_plan_ is null");
}
OB_UNIS_DECODE(aggr_col_count);
for (int32_t i = 0; OB_SUCC(ret) && i < aggr_col_count; ++i) {
ObAggregateExpression* expr = NULL;
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, expr))) {
LOG_WARN("failed to make expr", K(ret));
} else if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr is null");
} else if (OB_FAIL(expr->deserialize(buf, data_len, pos))) {
LOG_WARN("fail to deserialize expression", K(ret));
} else if (OB_FAIL(add_aggr_column(expr))) {
LOG_WARN("fail to add aggregate function", K(ret), K(buf), K(data_len), K(pos));
}
}
OB_UNIS_DECODE(group_col_idxs_);
OB_UNIS_DECODE(mem_size_limit_);
OB_UNIS_DECODE(prepare_row_num_);
OB_UNIS_DECODE(distinct_set_bucket_num_);
if (OB_SUCC(ret) && OB_FAIL(ObSingleChildPhyOperator::deserialize(buf, data_len, pos))) {
LOG_WARN("deserialize group by child operator failed", K(ret));
}
OB_UNIS_DECODE(has_rollup_);
OB_UNIS_DECODE(agg_udf_meta_);
for (int64_t i = 0; i < agg_udf_meta_.count() && OB_SUCC(ret); ++i) {
ObAggUdfMeta& udf_meta = agg_udf_meta_.at(i);
for (int64_t j = 0; j < udf_meta.calculable_results_.count() && OB_SUCC(ret); ++j) {
ObUdfConstArgs& args = udf_meta.calculable_results_.at(i);
ObSqlExpression* expr = nullptr;
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, expr))) {
LOG_WARN("failed to make expr", K(ret));
} else if (OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr is null");
} else if (OB_FAIL(expr->deserialize(buf, data_len, pos))) {
LOG_WARN("fail to deserialize expression", K(ret));
} else {
args.sql_calc_ = expr;
}
}
}
if (pos < data_len) {
OB_UNIS_DECODE(est_group_cnt_);
}
OB_UNIS_DECODE(rollup_col_idxs_);
return ret;
}
OB_DEF_SERIALIZE_SIZE(ObGroupBy)
{
int64_t len = 0;
int32_t aggr_col_count = aggr_columns_.get_size();
OB_UNIS_ADD_LEN(aggr_col_count);
DLIST_FOREACH_NORET(node, aggr_columns_)
{
if (node != NULL) {
const ObAggregateExpression* p = static_cast<const ObAggregateExpression*>(node);
len += p->get_serialize_size();
}
}
OB_UNIS_ADD_LEN(group_col_idxs_);
OB_UNIS_ADD_LEN(mem_size_limit_);
OB_UNIS_ADD_LEN(prepare_row_num_);
OB_UNIS_ADD_LEN(distinct_set_bucket_num_);
len += ObSingleChildPhyOperator::get_serialize_size();
OB_UNIS_ADD_LEN(has_rollup_);
OB_UNIS_ADD_LEN(agg_udf_meta_);
for (int64_t i = 0; i < agg_udf_meta_.count(); ++i) {
const ObAggUdfMeta& udf_meta = agg_udf_meta_.at(i);
for (int64_t j = 0; j < udf_meta.calculable_results_.count(); ++j) {
const ObUdfConstArgs& args = udf_meta.calculable_results_.at(i);
const ObSqlExpression* expr = args.sql_calc_;
if (OB_ISNULL(expr)) {
LOG_ERROR("udf agg expr is null");
} else {
len += expr->get_serialize_size();
}
}
}
OB_UNIS_ADD_LEN(est_group_cnt_);
OB_UNIS_ADD_LEN(rollup_col_idxs_);
return len;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,177 @@
/**
* 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_GROUPBY_H
#define _OB_GROUPBY_H
#include "lib/container/ob_array.h"
#include "lib/charset/ob_charset.h"
#include "sql/engine/ob_single_child_phy_operator.h"
#include "sql/engine/expr/ob_sql_expression.h"
#include "sql/engine/aggregate/ob_aggregate_function.h"
namespace oceanbase {
namespace sql {
typedef common::ObColumnInfo ObGroupColumn;
// struct ObGroupColumn
//{
// int64_t index_;
// common::ObCollationType cs_id_;
// ObGroupColumn() : index_(common::OB_INVALID_INDEX), cs_id_(common::COLLATION_TYPE_NONE) {}
// NEED_SERIALIZE_AND_DESERIALIZE;
// TO_STRING_KV2(N_INDEX_ID, index_,
// N_COLLATION_TYPE, common::ObCharset::collation_name(cs_id_));
//};
class ObGroupBy : public ObSingleChildPhyOperator {
public:
static const int64_t MIN_BUCKET_COUNT = 100;
static const int64_t MAX_BUCKET_COUNT = 100000;
static const int64_t LOCAL_COLUMN_COUNT = 16;
protected:
class ObGroupByCtx : public ObPhyOperatorCtx {
public:
explicit ObGroupByCtx(ObExecContext& exec_ctx) : ObPhyOperatorCtx(exec_ctx), aggr_func_(), child_column_count_(0)
{}
virtual ~ObGroupByCtx()
{}
virtual void destroy()
{
aggr_func_.~ObAggregateFunction();
ObPhyOperatorCtx::destroy_base();
}
inline ObAggregateFunction& get_aggr_func()
{
return aggr_func_;
}
inline int64_t get_child_column_count() const
{
return child_column_count_;
}
inline void set_child_column_count(int64_t child_column_count)
{
child_column_count_ = child_column_count;
}
protected:
ObAggregateFunction aggr_func_;
int64_t child_column_count_;
};
OB_UNIS_VERSION_V(1);
public:
explicit ObGroupBy(common::ObIAllocator& alloc);
virtual ~ObGroupBy();
virtual void reset();
virtual void reuse();
virtual int add_aggr_column(ObAggregateExpression* expr);
/// set memory limit
void set_mem_size_limit(const int64_t limit);
// set rollup statement
void set_rollup(const bool has_rollup);
/**
* @brief add group column index
* @param column_idx[in], the group column index in the row from child operator,
* such as select c1, c2 group by c2, c3,
* the row is ROW(c1, c2, c3) from child operator,
* column_idx of c2 is 1, column_idx of c3 is 2
*/
inline virtual int add_group_column_idx(
int64_t column_idx, common::ObCollationType cs_type /* = common::COLLATION_TYPE_NONE*/)
{
ObGroupColumn group_column;
group_column.index_ = column_idx;
group_column.cs_type_ = cs_type;
return group_col_idxs_.push_back(group_column);
}
/**
* @brief add rolup column index
*/
inline virtual int add_rollup_column_idx(
int64_t column_idx, common::ObCollationType cs_type /* = common::COLLATION_TYPE_NONE*/)
{
ObGroupColumn group_column;
group_column.index_ = column_idx;
group_column.cs_type_ = cs_type;
return rollup_col_idxs_.push_back(group_column);
}
inline virtual const ObAggrExprList& get_aggr_columns() const
{
return aggr_columns_;
}
int init(int64_t count)
{
return init_array_size<>(group_col_idxs_, count);
}
int init_rollup(int64_t count)
{
return init_array_size<>(rollup_col_idxs_, count);
}
int add_udf_meta(ObAggUdfMeta& meta)
{
return agg_udf_meta_.push_back(meta);
}
void set_est_group_cnt(const int64_t cnt)
{
est_group_cnt_ = cnt;
}
protected:
int is_same_group(const common::ObRowStore::StoredRow& row1, const common::ObNewRow& row2, bool& result,
int64_t& first_diff_pos) const;
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const = 0;
int init_group_by(ObExecContext& ctx) const;
/**
* @brief for specified phy operator to print it's member variable with json key-value format
* @param buf[in] to string buffer
* @param buf_len[in] buffer length
* @return if success, return the length used by print string, otherwise return 0
*/
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
/**
* @brief create operator context, only child operator can know it's specific operator type,
* so must be overwrited by child operator,
* @param ctx[in], execute context
* @param op_ctx[out], the pointer of operator context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const = 0;
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObGroupBy);
protected:
ObAggrExprList aggr_columns_;
int64_t mem_size_limit_; // discarded, never assigned
int32_t prepare_row_num_;
int32_t distinct_set_bucket_num_;
common::ObFixedArray<ObGroupColumn, common::ObIAllocator> group_col_idxs_;
bool has_rollup_;
common::ObSEArray<ObAggUdfMeta, 16> agg_udf_meta_;
int64_t est_group_cnt_;
common::ObFixedArray<ObGroupColumn, common::ObIAllocator> rollup_col_idxs_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_GROUPBY_H */

View File

@ -0,0 +1,102 @@
/**
* 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/aggregate/ob_groupby_op.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
//#include "sql/engine/expr/ob_sql_expression.h"
//#include "sql/ob_sql_utils.h"
namespace oceanbase {
namespace sql {
using namespace oceanbase::common;
using namespace oceanbase::share::schema;
OB_SERIALIZE_MEMBER((ObGroupBySpec, ObOpSpec), aggr_infos_);
DEF_TO_STRING(ObGroupBySpec)
{
int64_t pos = 0;
J_OBJ_START();
J_NAME("op_spec");
J_COLON();
pos += ObOpSpec::to_string(buf + pos, buf_len - pos);
J_COMMA();
J_KV(K_(aggr_infos));
J_OBJ_END();
return pos;
}
int ObGroupByOp::inner_open()
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObOperator::inner_open())) {
LOG_WARN("failed to inner_open", K(ret));
} else if (OB_FAIL(aggr_processor_.init())) {
LOG_WARN("failed to init", K(ret));
} else {
LOG_DEBUG("finish inner_open");
}
return ret;
}
int ObGroupByOp::rescan()
{
int ret = OB_SUCCESS;
aggr_processor_.reuse();
if (OB_FAIL(ObOperator::rescan())) {
LOG_WARN("failed to rescan", K(ret));
} else if (OB_FAIL(aggr_processor_.init())) {
LOG_WARN("failed to init", K(ret));
} else {
LOG_DEBUG("finish rescan");
}
return ret;
}
int ObGroupByOp::switch_iterator()
{
int ret = OB_SUCCESS;
aggr_processor_.reuse();
if (OB_FAIL(ObOperator::switch_iterator())) {
LOG_WARN("failed to switch_iterator", K(ret));
} else if (OB_FAIL(aggr_processor_.init())) {
LOG_WARN("failed to init", K(ret));
} else {
LOG_DEBUG("finish switch_iterator");
}
return ret;
}
int ObGroupByOp::inner_close()
{
int ret = OB_SUCCESS;
aggr_processor_.reuse();
if (OB_FAIL(ObOperator::inner_close())) {
LOG_WARN("failed to inner_close", K(ret));
} else {
LOG_DEBUG("finish inner_close");
}
return ret;
}
void ObGroupByOp::destroy()
{
aggr_processor_.~ObAggregateProcessor();
ObOperator::destroy();
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,74 @@
/**
* 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_BASIC_OB_GROUPBY_OP_H_
#define OCEANBASE_BASIC_OB_GROUPBY_OP_H_
#include "sql/engine/ob_operator.h"
#include "sql/engine/aggregate/ob_aggregate_processor.h"
namespace oceanbase {
namespace sql {
// constant
class ObGroupBySpec : public ObOpSpec {
public:
OB_UNIS_VERSION_V(1);
public:
ObGroupBySpec(common::ObIAllocator& alloc, const ObPhyOperatorType type) : ObOpSpec(alloc, type), aggr_infos_(alloc)
{}
DECLARE_VIRTUAL_TO_STRING;
// int add_udf_meta(ObAggUDFDatumMeta &meta) { return agg_udf_meta_.push_back(meta); }
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObGroupBySpec);
public:
AggrInfoFixedArray aggr_infos_; // aggr column + non-aggr column
// common::ObSEArray<ObAggUdfMeta, 16> agg_udf_meta_;
};
// modifiable
class ObGroupByOp : public ObOperator {
public:
ObGroupByOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObOperator(exec_ctx, spec, input),
aggr_processor_(eval_ctx_, (static_cast<ObGroupBySpec&>(const_cast<ObOpSpec&>(spec))).aggr_infos_)
{}
inline ObAggregateProcessor& get_aggr_processor()
{
return aggr_processor_;
}
virtual int inner_open() override;
virtual int inner_get_next_row() = 0;
virtual int rescan() override;
virtual int switch_iterator() override;
virtual int inner_close() override;
virtual void destroy() override;
private:
void reset_default();
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObGroupByOp);
protected:
ObAggregateProcessor aggr_processor_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_BASIC_OB_GROUPBY_OP_H_

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,278 @@
/**
* 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_ENGINE_AGGREGATE_OB_HASH_DISTINCT
#define SQL_ENGINE_AGGREGATE_OB_HASH_DISTINCT
#include "lib/string/ob_string.h"
#include "share/ob_define.h"
#include "common/row/ob_row.h"
#include "sql/engine/aggregate/ob_distinct.h"
#include "lib/container/ob_fixed_array.h"
#include "sql/engine/basic/ob_chunk_row_store.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/ob_sql_mem_mgr_processor.h"
#include "lib/container/ob_2d_array.h"
namespace oceanbase {
namespace sql {
class ObHashDistinct : public ObDistinct {
OB_UNIS_VERSION_V(1);
public:
class ObHashDistinctCtx : public ObPhyOperatorCtx {
public:
static const int64_t MIN_PART_COUNT = 8;
static const int64_t MAX_PART_COUNT = 256;
enum HDState { SCAN_CHILD = 1, OUTPUT_HA = 2, PROCESS_IN_MEM = 3, GET_PARTITION = 4, SCAN_PARTITION = 5 };
friend class ObHashDistinct;
class PartitionLinkNode : public common::ObDLinkBase<PartitionLinkNode> {
public:
PartitionLinkNode(ObHashDistinctCtx* data, ObHashDistinctCtx* parent)
: ObDLinkBase<PartitionLinkNode>(), data_(data), parent_(parent)
{}
ObHashDistinctCtx* data_;
ObHashDistinctCtx* parent_;
};
struct HashRow {
HashRow() : stored_row_(NULL), next_tuple_(NULL)
{}
const ObChunkRowStore::StoredRow* stored_row_;
HashRow* next_tuple_;
TO_STRING_KV(K_(stored_row), K(static_cast<void*>(next_tuple_)));
};
struct HashTable {
const static int16_t BUCKET_BUF_SIZE = 1024;
const static int16_t BUCKET_SHIFT = 10;
const static int16_t BUCKET_MASK = 1023;
HashTable() : mem_context_(NULL), buckets_(), nbuckets_(0), buckets_buf_(), buf_cnt_(0), cur_(0)
{}
void reuse()
{
buckets_.set_all(NULL);
cur_ = 0;
}
void reset()
{
buckets_.reset();
buckets_buf_.reset();
nbuckets_ = 0;
cur_ = 0;
}
TO_STRING_KV(K_(nbuckets), K_(buf_cnt), K_(cur));
lib::MemoryContext* mem_context_;
using RowArray = common::ObSegmentArray<HashRow*, OB_MALLOC_BIG_BLOCK_SIZE, common::ModulePageAllocator>;
RowArray buckets_;
int64_t nbuckets_;
RowArray buckets_buf_;
int64_t buf_cnt_;
int64_t cur_;
};
OB_INLINE int64_t get_part_idx(const uint64_t hash_value)
{
int64_t part_idx = ((hash_value & part_mask_) >> bucket_bits_) % part_count_;
return part_idx < part_count_ ? part_idx : part_count_ - 1;
}
OB_INLINE uint64_t get_bucket_idx(const uint64_t hash_value)
{
return hash_value & bucket_mask_;
}
explicit ObHashDistinctCtx(ObExecContext& exec_ctx);
virtual ~ObHashDistinctCtx()
{}
virtual void destroy()
{
reset();
ObPhyOperatorCtx::destroy_base();
}
void reset();
void reuse();
int estimate_memory_usage(int64_t input, int64_t mem_limit, int64_t& ha_mem_limit);
int estimate_memory_usage_for_partition(int64_t input, int64_t mem_limit, int64_t& ha_mem_limit);
int init_hash_table(const ObSQLSessionInfo* session, int64_t bucket_size, int64_t mem_limit);
int init_partitions_info(ObSQLSessionInfo* session);
int set_hash_info(int64_t part_count, int64_t n_bucket, int64_t level = 0);
inline bool equal_with_hashrow(
const ObHashDistinct* dist, const common::ObNewRow* row, uint64_t hash_val, const void* other) const;
bool in_hash_tab(const ObHashDistinct* dist, const common::ObNewRow* row, const uint64_t hash_val,
const uint64_t bucket_idx) const;
int insert_hash_tab(const ObChunkRowStore::StoredRow* sr, const uint64_t bucket_idx);
int process_rows_from_child(const ObHashDistinct* dist, const ObNewRow*& row, bool& got_distinct_row);
int process_one_partition(const ObHashDistinct* dist, ObChunkRowStore* row_store, ObSQLSessionInfo* my_session,
const ObNewRow*& row, bool& got_distinct_row);
// get cur sub partition from list
inline bool get_cur_sub_partition(ObHashDistinctCtx* top_ctx, PartitionLinkNode*& link_node)
{
bool got = false;
while (!got && top_ctx->sub_list_.get_size() > 0) {
link_node = top_ctx->sub_list_.get_first();
got = true;
}
return got;
}
int init_sql_mem_profile(ObIAllocator* allocator, int64_t max_mem_size, int64_t cache_size);
void destroy_sql_mem_profile();
int update_used_mem_bound(int64_t mem_used);
int update_used_mem_size();
int update_mem_status_periodically();
int add_row(const ObNewRow* row, ObChunkRowStore::StoredRow** sr, bool& ha_is_full_);
inline int64_t get_mem_used()
{
return nullptr == mem_context_ ? 0 : mem_context_->used();
}
inline int64_t get_data_size()
{
int64_t data_size = 0;
if (OB_NOT_NULL(sql_mem_processor_)) {
data_size = sql_mem_processor_->get_data_size();
}
return data_size;
}
void calc_data_ratio();
int64_t get_total_based_mem_used();
int get_next_partition(ObHashDistinctCtx* top_ctx, PartitionLinkNode*& link_node);
int out_put_ha_rows(const ObNewRow*& row);
int get_dist_row_from_in_mem_partitions(const ObHashDistinct* dist, const ObNewRow*& row, bool& got_distinct_row);
int clean_up_partitions();
inline int64_t get_partitions_mem_usage();
int get_next_mem_partition();
int open_cur_partition(ObChunkRowStore* row_store);
int assign_sub_ctx(ObHashDistinctCtx* parent, ObChunkRowStore* parent_row_stores, ObHashDistinctCtx* top_ctx);
DISALLOW_COPY_AND_ASSIGN(ObHashDistinctCtx);
private:
int64_t mem_limit_;
// n_bucket must be pow2, so the last N bits of hash_value can used as bucket_idx directly
// N = bucket_bits_ = log2(n_buckets_)
// and last M to N + 1 bits can used to calculate part_idx
//(M - N + 1) = next_pow2(part_count_ - 1) - 1
// 64.. M N 1
// hash_value: |----------|---|-------|
// bucket_mask: 0xF..F000..000
// part_mask: 0xFFF..FFF
int64_t part_count_;
int64_t part_mem_limit_;
int64_t estimated_mem_usage_; // estimate according to card and the avg_width of tuples
// int64_t n_bucket_;
int64_t bucket_bits_;
int64_t part_mask_;
int64_t bucket_mask_;
// part_it_ will go through part_row_stores_ twice, one for rs in mem, one for rs has dumped
ObChunkRowStore* part_row_stores_;
ObChunkRowStore::Iterator part_it_;
int64_t cur_it_part_;
// bool it_in_mem_; //iterate rows in mem
HashTable* hash_tab_;
ObChunkRowStore* ha_row_store_;
lib::MemoryContext* mem_context_ = nullptr;
bool ha_is_full_;
bool bkt_created_;
bool child_finished_;
ObDList<PartitionLinkNode> sub_list_;
int64_t state_;
int64_t part_level_;
int64_t cur_rows_;
int64_t pre_mem_used_;
ObSqlWorkAreaProfile* profile_;
ObSqlMemMgrProcessor* sql_mem_processor_;
ObHashDistinctCtx* top_ctx_;
bool is_top_ctx_;
ObPhyOperatorType op_type_;
uint64_t op_id_;
bool enable_sql_dumped_;
bool first_read_part_;
friend class ObHashDistinct;
};
public:
static const int64_t MIN_BUCKET_COUNT = 1L << 14; // 16384;
static const int64_t MAX_BUCKET_COUNT = 1L << 19; // 524288;
static const int64_t HASH_DISTINCT_BUCKET_RATIO = 2;
explicit ObHashDistinct(common::ObIAllocator& alloc);
// ObHashDistinct();
virtual ~ObHashDistinct();
virtual void set_mem_limit(int64_t mem_limit)
{
mem_limit_ = mem_limit;
}
virtual void reset();
virtual void reuse();
virtual int rescan(ObExecContext& ctx) const;
private:
int estimate_memory_usage(
ObHashDistinctCtx* hash_ctx, int64_t memory_limit, int64_t& ha_mem, int64_t& mem_need) const;
int get_hash_value(const common::ObNewRow* row, uint64_t& hash_value) const;
int init_ha(const ObSQLSessionInfo* session, ObHashDistinctCtx* hash_ctx) const;
// bool hashrows_are_equal(const ObHashDistinctCtx::HashRow *one) const;
inline bool equal_with_hashrow(const common::ObNewRow* row, uint64_t hash_val, const void* other) const;
/* bool in_hash_tab(const ObHashDistinctCtx* hash_ctx,
const common::ObNewRow *row,
const uint64_t hash_val,
const uint64_t bucket_idx) const;
*/
// Get bucket size to create
static int get_bucket_size(int64_t est_bucket, int64_t max_bound_size, double extra_ratio, int64_t& bucket_size);
static int64_t estimate_mem_size_by_rows(int64_t rows)
{
return next_pow2(rows * HASH_DISTINCT_BUCKET_RATIO);
}
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
/**
* @brief: called by get_next_row(), get a row from the child operator or row_store
* @param: ctx[in], execute context
* @param: row[out], ObSqlRow an obj array and row_size
*/
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
/**
* @brief open operator, not including children operators.
* called by open.
* Every op should implement this method.
*/
virtual int inner_open(ObExecContext& ctx) const;
/**
* @brief close operator, not including children operators.
* Every op should implement this method.
*/
virtual int inner_close(ObExecContext& ctx) const;
/**
* @brief for specified phy operator to print it's member variable with json key-value format
* @param buf[in] to string buffer
* @param buf_len[in] buffer length
* @return if success, return the length used by print string, otherwise return 0
*/
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
int64_t mem_limit_;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObHashDistinct);
};
} // namespace sql
} // namespace oceanbase
#endif

View File

@ -0,0 +1,306 @@
/**
* 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/aggregate/ob_hash_distinct_op.h"
#include "sql/engine/px/ob_px_util.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
ObHashDistinctSpec::ObHashDistinctSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
: ObDistinctSpec(alloc, type), sort_collations_(alloc), hash_funcs_(alloc)
{}
OB_SERIALIZE_MEMBER((ObHashDistinctSpec, ObDistinctSpec), sort_collations_, hash_funcs_);
ObHashDistinctOp::ObHashDistinctOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObOperator(exec_ctx, spec, input),
first_got_row_(true),
has_got_part_(false),
iter_end_(false),
get_next_row_func_(nullptr),
profile_(ObSqlWorkAreaType::HASH_WORK_AREA),
sql_mem_processor_(profile_),
hp_infras_(),
group_cnt_(0)
{
enable_sql_dumped_ = GCONF.is_sql_operator_dump_enabled() && !(GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_2250);
}
int ObHashDistinctOp::inner_open()
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(nullptr == left_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected status: left is null", K(ret));
} else if (OB_FAIL(ObOperator::inner_open())) {
LOG_WARN("failed to inner open", K(ret));
} else {
first_got_row_ = true;
if (MY_SPEC.is_block_mode_) {
get_next_row_func_ = &ObHashDistinctOp::do_block_distinct;
} else {
get_next_row_func_ = &ObHashDistinctOp::do_unblock_distinct;
}
LOG_TRACE("trace block mode", K(MY_SPEC.is_block_mode_), K(spec_.id_));
}
return ret;
}
int ObHashDistinctOp::inner_close()
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObOperator::inner_close())) {
LOG_WARN("failed to inner close", K(ret));
} else {
reset();
sql_mem_processor_.unregister_profile();
}
return ret;
}
void ObHashDistinctOp::reset()
{
first_got_row_ = true;
has_got_part_ = false;
group_cnt_ = 0;
hp_infras_.reset();
if (MY_SPEC.is_block_mode_) {
get_next_row_func_ = &ObHashDistinctOp::do_block_distinct;
} else {
get_next_row_func_ = &ObHashDistinctOp::do_unblock_distinct;
}
LOG_TRACE("trace block mode", K(MY_SPEC.is_block_mode_), K(spec_.id_));
}
int ObHashDistinctOp::rescan()
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObOperator::rescan())) {
LOG_WARN("failed to rescan child operator", K(ret));
} else {
reset();
iter_end_ = false;
}
return ret;
}
void ObHashDistinctOp::destroy()
{
hp_infras_.~ObHashPartInfrastructure();
ObOperator::destroy();
}
int ObHashDistinctOp::init_hash_partition_infras()
{
int ret = OB_SUCCESS;
int64_t est_rows = MY_SPEC.rows_;
if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(&ctx_, MY_SPEC.px_est_size_factor_, est_rows, est_rows))) {
LOG_WARN("failed to get px size", K(ret));
} else if (OB_FAIL(sql_mem_processor_.init(&ctx_.get_allocator(),
ctx_.get_my_session()->get_effective_tenant_id(),
est_rows * MY_SPEC.width_,
MY_SPEC.type_,
MY_SPEC.id_,
&ctx_))) {
LOG_WARN("failed to init sql mem processor", K(ret));
} else if (OB_FAIL(hp_infras_.init(ctx_.get_my_session()->get_effective_tenant_id(),
enable_sql_dumped_,
true,
true,
2,
&sql_mem_processor_))) {
LOG_WARN("failed to init hash partition infrastructure", K(ret));
} else {
int64_t est_bucket_num = hp_infras_.est_bucket_count(est_rows, MY_SPEC.width_, MIN_BUCKET_COUNT, MAX_BUCKET_COUNT);
if (OB_FAIL(
hp_infras_.set_funcs(&MY_SPEC.hash_funcs_, &MY_SPEC.sort_collations_, &MY_SPEC.cmp_funcs_, &eval_ctx_))) {
LOG_WARN("failed to set funcs", K(ret));
} else if (OB_FAIL(hp_infras_.start_round())) {
LOG_WARN("failed to start round", K(ret));
} else if (OB_FAIL(hp_infras_.init_hash_table(est_bucket_num, MIN_BUCKET_COUNT, MAX_BUCKET_COUNT))) {
LOG_WARN("failed to init hash table", K(ret));
} else {
op_monitor_info_.otherstat_1_value_ = est_bucket_num;
op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::HASH_INIT_BUCKET_COUNT;
op_monitor_info_.otherstat_4_value_ = MY_SPEC.is_block_mode_;
op_monitor_info_.otherstat_4_id_ = ObSqlMonitorStatIds::DISTINCT_BLOCK_MODE;
}
}
return ret;
}
int ObHashDistinctOp::build_distinct_data(bool is_block)
{
int ret = OB_SUCCESS;
bool got_row = false;
bool has_exists = false;
bool inserted = false;
const ObChunkDatumStore::StoredRow* store_row = nullptr;
while (OB_SUCC(ret) && !got_row) {
if (!has_got_part_) {
clear_evaluated_flag();
ret = child_->get_next_row();
} else {
ret = hp_infras_.get_left_next_row(store_row, MY_SPEC.distinct_exprs_);
}
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
// get dumped partition
if (OB_FAIL(hp_infras_.finish_insert_row())) {
LOG_WARN("failed to finish to insert row", K(ret));
} else if (!has_got_part_) {
has_got_part_ = true;
} else {
if (OB_FAIL(hp_infras_.close_cur_part(InputSide::LEFT))) {
LOG_WARN("failed to close cur part", K(ret));
}
}
if (OB_FAIL(ret)) {
} else if (is_block) {
// return data after handling this batch of data in block mode.
LOG_TRACE("trace block", K(is_block));
break;
} else if (OB_FAIL(hp_infras_.end_round())) {
LOG_WARN("failed to end round", K(ret));
} else if (OB_FAIL(hp_infras_.start_round())) {
LOG_WARN("failed to open round", K(ret));
} else if (OB_FAIL(hp_infras_.get_next_partition(InputSide::LEFT))) {
if (OB_ITER_END != ret) {
LOG_WARN("failed to create dumped partitions", K(ret));
}
} else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) {
LOG_WARN("failed to open cur part");
} else if (OB_FAIL(hp_infras_.resize(hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) {
LOG_WARN("failed to init hash table", K(ret));
}
} else if (OB_FAIL(ret)) {
} else if (OB_FAIL(try_check_status())) {
LOG_WARN("failed to check status", K(ret));
} else if (OB_FAIL(hp_infras_.insert_row(MY_SPEC.distinct_exprs_, has_exists, inserted))) {
LOG_WARN("failed to insert row", K(ret));
} else if (has_exists) {
// Already in hash map, do nothing
} else if (inserted && !is_block) {
got_row = true;
++group_cnt_;
}
} // end of while
return ret;
}
int ObHashDistinctOp::do_unblock_distinct()
{
int ret = OB_SUCCESS;
if (first_got_row_) {
if (OB_FAIL(init_hash_partition_infras())) {
LOG_WARN("failed to get next row", K(ret));
}
first_got_row_ = false;
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(build_distinct_data(false))) {
if (OB_ITER_END != ret) {
LOG_WARN("failed to build distinct data", K(ret));
} else {
LOG_TRACE("trace iter end", K(ret));
}
}
return ret;
}
int ObHashDistinctOp::do_block_distinct()
{
int ret = OB_SUCCESS;
const ObChunkDatumStore::StoredRow* store_row = nullptr;
if (first_got_row_) {
if (OB_FAIL(init_hash_partition_infras())) {
LOG_WARN("failed to get next row", K(ret));
} else if (OB_FAIL(build_distinct_data(true))) {
if (OB_ITER_END == ret) {
ret = OB_ERR_UNEXPECTED;
}
LOG_WARN("failed to build distinct data", K(ret));
} else if (OB_FAIL(hp_infras_.open_hash_table_part())) {
LOG_WARN("failed to open hash table part", K(ret));
}
clear_evaluated_flag();
first_got_row_ = false;
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(hp_infras_.get_next_hash_table_row(store_row, &MY_SPEC.distinct_exprs_))) {
if (OB_ITER_END == ret) {
// begin to handle next batch data after return the previous batch.
if (OB_FAIL(hp_infras_.end_round())) {
LOG_WARN("failed to end round", K(ret));
} else if (OB_FAIL(hp_infras_.start_round())) {
LOG_WARN("failed to open round", K(ret));
} else if (OB_FAIL(hp_infras_.get_next_partition(InputSide::LEFT))) {
if (OB_ITER_END != ret) {
LOG_WARN("failed to create dumped partitions", K(ret));
}
} else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) {
LOG_WARN("failed to open cur part");
} else if (OB_FAIL(hp_infras_.resize(hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) {
LOG_WARN("failed to init hash table", K(ret));
} else if (OB_FAIL(build_distinct_data(true))) {
if (OB_ITER_END == ret) {
ret = OB_ERR_UNEXPECTED;
}
LOG_WARN("failed to build distinct data", K(ret));
} else if (OB_FAIL(hp_infras_.open_hash_table_part())) {
LOG_WARN("failed to open hash table part", K(ret));
} else if (OB_FAIL(hp_infras_.get_next_hash_table_row(store_row, &MY_SPEC.distinct_exprs_))) {
LOG_WARN("failed to get next row in hash table", K(ret));
}
} else {
LOG_WARN("failed to get next row in hash table", K(ret));
}
} else {
++group_cnt_;
}
return ret;
}
int ObHashDistinctOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
clear_evaluated_flag();
if (OB_UNLIKELY(iter_end_)) {
ret = OB_ITER_END;
} else if (OB_UNLIKELY(nullptr == get_next_row_func_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get next row func is null", K(ret));
} else if (OB_FAIL((this->*get_next_row_func_)())) {
if (OB_ITER_END != ret) {
LOG_WARN("failed to get next row", K(ret));
} else {
op_monitor_info_.otherstat_2_value_ = group_cnt_;
op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::HASH_ROW_COUNT;
op_monitor_info_.otherstat_3_value_ = hp_infras_.get_bucket_num();
op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::HASH_BUCKET_COUNT;
reset();
iter_end_ = true;
}
} else {
LOG_DEBUG("trace output row", K(ROWEXPR2STR(eval_ctx_, MY_SPEC.output_)));
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,72 @@
/**
* 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_SRC_SQL_ENGINE_AGGREGATE_OB_HASH_DISTINCT_OP_H_
#define OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_HASH_DISTINCT_OP_H_
#include "sql/engine/ob_operator.h"
#include "sql/engine/aggregate/ob_distinct_op.h"
#include "share/datum/ob_datum_funcs.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
#include "sql/engine/basic/ob_hash_partitioning_infrastructure_op.h"
namespace oceanbase {
namespace sql {
class ObHashDistinctSpec : public ObDistinctSpec {
OB_UNIS_VERSION_V(1);
public:
ObHashDistinctSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
ObSortCollations sort_collations_;
ObHashFuncs hash_funcs_;
};
class ObHashDistinctOp : public ObOperator {
public:
ObHashDistinctOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
~ObHashDistinctOp()
{}
virtual int inner_open() override;
virtual int inner_close() override;
virtual int rescan() override;
virtual int inner_get_next_row() override;
virtual void destroy() override;
private:
void reset();
int do_unblock_distinct();
int do_block_distinct();
int init_hash_partition_infras();
int build_distinct_data(bool is_block);
private:
typedef int (ObHashDistinctOp::*GetNextRowFunc)();
static const int64_t MIN_BUCKET_COUNT = 1L << 14; // 16384;
static const int64_t MAX_BUCKET_COUNT = 1L << 19; // 524288;
bool enable_sql_dumped_;
bool first_got_row_;
bool has_got_part_;
bool iter_end_;
GetNextRowFunc get_next_row_func_;
ObSqlWorkAreaProfile profile_;
ObSqlMemMgrProcessor sql_mem_processor_;
ObHashPartInfrastructure<ObHashPartCols, ObHashPartStoredRow> hp_infras_;
int64_t group_cnt_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_HASH_DISTINCT_OP_H_ */

View File

@ -0,0 +1,851 @@
/**
* 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/aggregate/ob_hash_groupby.h"
#include "share/ob_define.h"
#include "lib/utility/serialization.h"
#include "lib/list/ob_list.h"
#include "common/row/ob_row_util.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/aggregate/ob_exec_hash_struct.h"
#include "sql/engine/ob_exec_context.h"
#include "sql/engine/aggregate/ob_aggregate_function.h"
#include "sql/engine/expr/ob_sql_expression.h"
#include "lib/list/ob_dlink_node.h"
#include "sql/engine/basic/ob_chunk_row_store.h"
#include "lib/cpu/ob_endian.h"
#include "sql/engine/ob_sql_mem_mgr_processor.h"
#include "sql/engine/px/ob_px_util.h"
namespace oceanbase {
using namespace common;
using namespace common::hash;
namespace sql {
struct ObGbyPartition : public common::ObDLinkBase<ObGbyPartition> {
public:
ObGbyPartition(common::ObIAllocator* alloc = nullptr) : row_store_(alloc), level_(0)
{}
ObChunkRowStore row_store_;
int64_t level_;
};
class ObHashGroupBy::ObHashGroupByCtx : public ObGroupByCtx, public ObHashCtx<ObGbyHashCols> {
public:
explicit ObHashGroupByCtx(ObExecContext& exec_ctx)
: ObGroupByCtx(exec_ctx),
ObHashCtx(),
group_idx_(0),
mem_context_(NULL),
agged_group_cnt_(0),
agged_row_cnt_(0),
profile_(ObSqlWorkAreaType::HASH_WORK_AREA),
sql_mem_processor_(profile_)
{}
~ObHashGroupByCtx()
{}
void destroy_all_parts()
{
if (NULL != mem_context_) {
while (!all_parts_.is_empty()) {
ObGbyPartition* p = all_parts_.remove_first();
if (NULL != p) {
p->~ObGbyPartition();
mem_context_->get_malloc_allocator().free(p);
}
}
}
}
int init_mem_context()
{
int ret = OB_SUCCESS;
if (NULL == mem_context_) {
lib::ContextParam param;
param.set_mem_attr(exec_ctx_.get_my_session()->get_effective_tenant_id(),
ObModIds::OB_HASH_NODE_GROUP_ROWS,
ObCtxIds::WORK_AREA);
if (OB_FAIL(CURRENT_CONTEXT.CREATE_CONTEXT(mem_context_, param))) {
LOG_WARN("memory entity create failed", K(ret));
}
}
return ret;
}
virtual void destroy()
{
sql_mem_processor_.destroy();
if (NULL != mem_context_) {
destroy_all_parts();
}
group_idx_ = 0;
ObHashCtx::destroy();
ObGroupByCtx::destroy();
// last free mem,ObHashCtx depends on mem_context
if (NULL != mem_context_) {
DESTROY_CONTEXT(mem_context_);
mem_context_ = NULL;
}
}
void calc_data_mem_ratio(int64_t part_cnt, double& data_ratio);
void adjust_part_cnt(int64_t& part_cnt);
int64_t get_ctx_mem_size()
{
return ObHashCtx::get_used_mem_size();
}
int64_t get_part_mem_size()
{
return (NULL == mem_context_ ? 0 : mem_context_->used());
}
int64_t get_part_hold_size()
{
return (NULL == mem_context_ ? 0 : mem_context_->hold());
}
int64_t get_used_mem_size()
{
return aggr_func_.get_used_mem_size() + ObHashCtx::get_used_mem_size() +
(NULL == mem_context_ ? 0 : mem_context_->used());
}
int64_t estimate_hash_bucket_cnt_by_mem_size(int64_t bucket_cnt, int64_t max_mem_size, double extra_ratio);
int64_t estimate_hash_bucket_size(int64_t bucket_cnt);
int64_t get_data_hold_size()
{
return aggr_func_.get_hold_mem_size();
}
int64_t get_data_size()
{
return aggr_func_.get_used_mem_size();
}
int64_t get_extra_size()
{
return ObHashCtx::get_used_mem_size() + (NULL == mem_context_ ? 0 : mem_context_->used());
}
int64_t get_mem_bound()
{
return sql_mem_processor_.get_mem_bound();
}
private:
DISALLOW_COPY_AND_ASSIGN(ObHashGroupByCtx);
friend class ObHashGroupBy;
private:
static const int64_t FIX_SIZE_PER_PART = sizeof(ObGbyPartition) + ObChunkRowStore::BLOCK_SIZE;
int64_t group_idx_;
// memory allocator for group by partitions
lib::MemoryContext* mem_context_;
ObDList<ObGbyPartition> all_parts_;
int64_t agged_group_cnt_;
int64_t agged_row_cnt_;
ObSqlWorkAreaProfile profile_;
ObSqlMemMgrProcessor sql_mem_processor_;
};
// Calculate data and extra memory ratio
// Here, the memory occupied by the chunk row store that needs to be dumped + ctx (hash table) memory is used as extra
// memory and the memory involved in the aggregate function is data memory Calculate the ratio of the two here, and then
// determine how much data can be stored according to the mem bound If you directly use data_size/(extra_size +
// data_size), the extra is very large, but the data is very small, which will lead to inserting a few rows of data to
// estimate the data_ratio is very small, there is a problem
void ObHashGroupBy::ObHashGroupByCtx::calc_data_mem_ratio(int64_t part_cnt, double& data_ratio)
{
int64_t extra_size = (get_ctx_mem_size() + part_cnt * FIX_SIZE_PER_PART) * (1 + EXTRA_MEM_RATIO);
int64_t data_size = max(get_data_size(), (get_mem_bound() - extra_size) * 0.8);
data_ratio = data_size * 1.0 / (extra_size + data_size);
sql_mem_processor_.set_data_ratio(data_ratio);
LOG_TRACE("trace calc data ratio", K(data_ratio), K(extra_size), K(part_cnt), K(data_size), K(get_data_size()));
}
void ObHashGroupBy::ObHashGroupByCtx::adjust_part_cnt(int64_t& part_cnt)
{
int64_t mem_used = get_used_mem_size();
int64_t mem_bound = get_mem_bound();
int64_t dumped_remain_mem_size = mem_bound - mem_used;
int64_t max_part_cnt = dumped_remain_mem_size / FIX_SIZE_PER_PART;
if (max_part_cnt <= MIN_PARTITION_CNT) {
part_cnt = MIN_PARTITION_CNT;
} else {
while (part_cnt > max_part_cnt) {
part_cnt >>= 1;
}
}
LOG_TRACE(
"trace adjust part cnt", K(part_cnt), K(max_part_cnt), K(dumped_remain_mem_size), K(mem_bound), K(mem_used));
}
int64_t ObHashGroupBy::ObHashGroupByCtx::estimate_hash_bucket_size(int64_t bucket_cnt)
{
return next_pow2(ObExtendHashTable<ObGbyHashCols>::SIZE_BUCKET_SCALE * bucket_cnt) * sizeof(void*);
}
int64_t ObHashGroupBy::ObHashGroupByCtx::estimate_hash_bucket_cnt_by_mem_size(
int64_t bucket_cnt, int64_t max_mem_size, double extra_ratio)
{
int64_t mem_size = estimate_hash_bucket_size(bucket_cnt);
int64_t max_hash_size = max_mem_size * extra_ratio;
if (0 < max_hash_size) {
while (mem_size > max_hash_size) {
mem_size >>= 1;
}
}
return mem_size / sizeof(void*) / ObExtendHashTable<ObGbyHashCols>::SIZE_BUCKET_SCALE;
}
ObHashGroupBy::ObHashGroupBy(ObIAllocator& alloc) : ObGroupBy(alloc)
{}
ObHashGroupBy::~ObHashGroupBy()
{}
void ObHashGroupBy::reset()
{
ObGroupBy::reset();
distinct_set_bucket_num_ = DISTINCT_SET_NUM;
prepare_row_num_ = PREPARE_ROW_NUM;
}
void ObHashGroupBy::reuse()
{
ObGroupBy::reuse();
distinct_set_bucket_num_ = DISTINCT_SET_NUM;
prepare_row_num_ = PREPARE_ROW_NUM;
}
int ObHashGroupBy::get_hash_groupby_row_count(ObExecContext& exec_ctx, int64_t& hash_groupby_row_count) const
{
int ret = OB_SUCCESS;
ObHashGroupByCtx* hash_groupby_ctx = NULL;
if (OB_ISNULL(hash_groupby_ctx = GET_PHY_OPERATOR_CTX(ObHashGroupByCtx, exec_ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
SQL_ENG_LOG(WARN, "get hash_groupby ctx failed", K(ret));
} else {
hash_groupby_row_count = hash_groupby_ctx->group_rows_.size();
}
return ret;
}
int ObHashGroupBy::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
{
return CREATE_PHY_OPERATOR_CTX(ObHashGroupByCtx, ctx, get_id(), get_type(), op_ctx);
}
int64_t ObHashGroupBy::to_string_kv(char* buf, const int64_t buf_len) const
{
return ObGroupBy::to_string_kv(buf, buf_len);
}
int ObHashGroupBy::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObGroupBy::init_group_by(ctx))) {
LOG_WARN("failed to init hash group by", K(ret));
}
return ret;
}
int ObHashGroupBy::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObHashGroupByCtx* groupby_ctx = NULL;
if (OB_ISNULL(groupby_ctx = GET_PHY_OPERATOR_CTX(ObHashGroupByCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get operator context failed");
} else {
groupby_ctx->sql_mem_processor_.reset();
groupby_ctx->started_ = false;
// Why is reuse not reset or destroy here?
// This is because group_rows_ will not be init again after rescan, only resize
groupby_ctx->group_rows_.reuse();
groupby_ctx->aggr_func_.reuse();
groupby_ctx->destroy_all_parts();
if (OB_FAIL(ObGroupBy::rescan(ctx))) {
LOG_WARN("rescan child operator failed", K(ret));
}
}
return ret;
}
inline bool ObHashGroupBy::need_start_dump(
ObHashGroupByCtx& gby_ctx, const int64_t input_rows, int64_t& est_part_cnt, bool check_dump) const
{
bool dump = false;
const int64_t mem_used = gby_ctx.get_used_mem_size();
const int64_t mem_bound = gby_ctx.get_mem_bound();
double data_ratio = gby_ctx.sql_mem_processor_.get_data_ratio();
if (OB_UNLIKELY(0 == est_part_cnt)) {
est_part_cnt = detect_part_cnt(input_rows, gby_ctx);
gby_ctx.calc_data_mem_ratio(est_part_cnt, data_ratio);
}
// We continue do aggregation after we start dumping, reserve 1/8 memory for it.
if (gby_ctx.get_data_size() > data_ratio * mem_bound || check_dump) {
int ret = OB_SUCCESS;
dump = true;
// When you think a dump is about to happen,
// try to expand to get more memory, then decide whether to dump
if (OB_FAIL(gby_ctx.sql_mem_processor_.extend_max_memory_size(
&gby_ctx.exec_ctx_.get_allocator(),
[&](int64_t max_memory_size) { return gby_ctx.get_data_size() > data_ratio * max_memory_size; },
dump,
mem_used))) {
dump = true;
LOG_WARN("failed to extend max memory size", K(ret), K(dump));
} else if (OB_FAIL(gby_ctx.sql_mem_processor_.update_used_mem_size(mem_used))) {
LOG_WARN("failed to update used memory size", K(ret), K(mem_used), K(mem_bound), K(dump));
} else {
est_part_cnt = detect_part_cnt(input_rows, gby_ctx);
gby_ctx.calc_data_mem_ratio(est_part_cnt, data_ratio);
LOG_TRACE("trace extend max memory size",
K(ret),
K(data_ratio),
K(gby_ctx.get_data_size()),
K(gby_ctx.get_used_mem_size()),
K(gby_ctx.get_extra_size()),
K(mem_used),
K(mem_bound),
K(dump),
K(est_part_cnt));
}
}
return dump;
}
int ObHashGroupBy::update_mem_status_periodically(
ObHashGroupByCtx* gby_ctx, int64_t nth_cnt, const int64_t input_row, int64_t& est_part_cnt, bool& need_dump) const
{
int ret = OB_SUCCESS;
bool updated = false;
need_dump = false;
if (OB_FAIL(gby_ctx->sql_mem_processor_.update_max_available_mem_size_periodically(
&gby_ctx->exec_ctx_.get_allocator(), [&](int64_t cur_cnt) { return nth_cnt > cur_cnt; }, updated))) {
LOG_WARN("failed to update usable memory size periodically", K(ret));
} else if (updated) {
if (OB_FAIL(gby_ctx->sql_mem_processor_.update_used_mem_size(gby_ctx->get_used_mem_size()))) {
LOG_WARN("failed to update used memory size", K(ret));
} else {
double data_ratio = gby_ctx->sql_mem_processor_.get_data_ratio();
;
est_part_cnt = detect_part_cnt(input_row, *gby_ctx);
gby_ctx->calc_data_mem_ratio(est_part_cnt, data_ratio);
need_dump = (gby_ctx->get_data_size() > gby_ctx->get_mem_bound() * data_ratio);
}
}
return ret;
}
int ObHashGroupBy::load_data(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
const ObChunkRowStore::StoredRow* srow = NULL;
const ObNewRow* row = NULL;
ObGbyHashCols gby_cols;
const ObGbyHashCols* exist_gby_cols = NULL;
ObHashGroupByCtx* gby_ctx = NULL;
ObGbyPartition* cur_part = NULL;
ObChunkRowStore::Iterator row_store_iter;
int64_t level = 0;
int64_t part_shift = sizeof(uint64_t) * CHAR_BIT / 2; // low half bits for hash table lookup.
ObGbyPartition* parts[MAX_PARTITION_CNT];
int64_t part_cnt = 0;
int64_t est_part_cnt = 0;
bool start_dump = false;
bool check_dump = false;
bool is_dump_enabled = GCONF.is_sql_operator_dump_enabled();
int64_t input_rows = child_op_->get_rows();
int64_t input_size = child_op_->get_width() * input_rows;
ObGbyBloomFilter* bloom_filter = NULL;
static_assert(MAX_PARTITION_CNT <= (1 << (CHAR_BIT)), "max partition cnt is too big");
if (OB_ISNULL(gby_ctx = GET_PHY_OPERATOR_CTX(ObHashGroupByCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get operator context failed");
} else {
if (gby_ctx->aggr_func_.has_distinct() || gby_ctx->aggr_func_.has_sort()) {
// We use sort based group by for aggregation which need distinct or sort right now,
// disable operator dump for compatibility.
is_dump_enabled = false;
}
if (!gby_ctx->all_parts_.is_empty()) {
gby_ctx->aggr_func_.reuse();
cur_part = gby_ctx->all_parts_.remove_first();
if (OB_ISNULL(cur_part)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("pop head partition failed", K(ret));
} else if (OB_FAIL(row_store_iter.init(&cur_part->row_store_, ObChunkRowStore::BLOCK_SIZE))) {
LOG_WARN("init chunk row store iterator failed", K(ret));
} else {
input_rows = cur_part->row_store_.get_row_cnt();
level = cur_part->level_;
part_shift = part_shift + level * CHAR_BIT;
input_size = cur_part->row_store_.get_file_size();
if (OB_FAIL(gby_ctx->group_rows_.resize(&gby_ctx->mem_context_->get_malloc_allocator(), max(2, input_rows)))) {
LOG_WARN("failed to reuse extended hash table", K(ret));
} else if (OB_FAIL(init_sql_mem_mgr(gby_ctx, input_size))) {
LOG_WARN("failed to init sql mem manager", K(ret));
}
}
LOG_TRACE("scan new partition",
K(level),
K(input_rows),
K(input_size),
K(gby_ctx->group_rows_.size()),
K(gby_ctx->get_used_mem_size()),
K(gby_ctx->aggr_func_.get_used_mem_size()),
K(gby_ctx->get_mem_bound()));
}
}
int64_t nth_cnt = 0;
for (int64_t loop_cnt = 1; OB_SUCC(ret); ++loop_cnt, ++nth_cnt) {
if (NULL == cur_part) {
ret = child_op_->get_next_row(ctx, row);
} else {
if (OB_FAIL(row_store_iter.get_next_row(srow))) {
} else if (NULL == srow) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("return NULL store row", K(ret));
} else if (OB_FAIL(row_store_iter.convert_to_row(srow, *const_cast<ObNewRow**>(&row)))) {
LOG_WARN("convert store row to new row failed", K(ret));
}
}
if (OB_SUCCESS != ret || OB_ISNULL(row)) {
if (OB_SUCCESS == ret) {
ret = OB_ERR_UNEXPECTED;
}
if (OB_ITER_END != ret) {
LOG_WARN("get input row failed", K(ret), KP(row));
}
break;
}
if (loop_cnt % 1024 == 0) {
if (OB_FAIL(ctx.check_status())) {
LOG_WARN("check status failed", K(ret));
}
}
ObSQLSessionInfo* my_session = gby_ctx->exec_ctx_.get_my_session();
const ObTimeZoneInfo* tz_info = (NULL != my_session) ? my_session->get_timezone_info() : NULL;
if (OB_FAIL(ret)) {
} else if (!start_dump &&
OB_FAIL(update_mem_status_periodically(gby_ctx, nth_cnt, input_rows, est_part_cnt, check_dump))) {
LOG_WARN("failed to update usable memory size periodically", K(ret));
} else if (OB_FAIL(gby_cols.init(
row, &group_col_idxs_, NULL == srow ? 0 : *static_cast<uint64_t*>(srow->get_extra_payload())))) {
LOG_WARN("init group by hash columns failed", K(ret));
} else if ((NULL == bloom_filter || bloom_filter->exist(gby_cols.hash())) &&
NULL != (exist_gby_cols = gby_ctx->group_rows_.get(gby_cols))) {
gby_ctx->agged_row_cnt_ += 1;
if (OB_FAIL(gby_ctx->aggr_func_.process(*row, tz_info, exist_gby_cols->group_id_))) {
LOG_WARN("fail to process row", K(ret), K(*row), K(exist_gby_cols->group_id_));
}
} else {
if (!is_dump_enabled || gby_ctx->group_rows_.size() < MIN_INMEM_GROUPS ||
(!start_dump && !need_start_dump(*gby_ctx, input_rows, est_part_cnt, check_dump))) {
gby_ctx->agged_row_cnt_ += 1;
gby_ctx->agged_group_cnt_ += 1;
ObGbyHashCols* store_gby_cols =
static_cast<ObGbyHashCols*>(gby_ctx->aggr_func_.get_stored_row_buf().alloc(sizeof(ObGbyHashCols)));
if (OB_ISNULL(store_gby_cols)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
*store_gby_cols = gby_cols;
store_gby_cols->group_id_ = gby_ctx->group_rows_.size();
ObRowStore::StoredRow* output_row = NULL;
if (OB_FAIL(gby_ctx->aggr_func_.prepare(*row, store_gby_cols->group_id_, &output_row))) {
LOG_WARN("fail to prepare row", K(ret), K(*row), K(store_gby_cols->group_id_));
} else {
store_gby_cols->set_stored_row(output_row);
if (OB_FAIL(gby_ctx->group_rows_.set(*store_gby_cols))) {
LOG_WARN("hash table set failed", K(ret));
} else {
ret = OB_SUCCESS;
}
}
}
} else {
if (OB_UNLIKELY(!start_dump)) {
if (OB_FAIL(setup_dump_env(level, max(input_rows, nth_cnt), parts, part_cnt, bloom_filter, *gby_ctx))) {
LOG_WARN("setup dump environment failed", K(ret));
} else {
start_dump = true;
gby_ctx->sql_mem_processor_.set_number_pass(level + 1);
}
}
if (OB_SUCC(ret)) {
const int64_t part_idx = (gby_cols.hash() >> part_shift) & (part_cnt - 1);
// TODO : copy store row to new row store directly.
ObChunkRowStore::StoredRow* stored_row = NULL;
if (OB_FAIL(parts[part_idx]->row_store_.add_row(*row, &stored_row))) {
LOG_WARN("add row failed", K(ret), K(part_cnt));
} else {
*static_cast<uint64_t*>(stored_row->get_extra_payload()) = gby_cols.hash();
}
}
}
}
}
// must be reset first, otherwise the memory will be released when the automatic destruction occurs,
// and there will be problems
row_store_iter.reset();
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
if (NULL != gby_ctx) {
// cleanup_dump_env() must be called whether success or not
int tmp_ret = cleanup_dump_env(common::OB_SUCCESS == ret, level, parts, part_cnt, bloom_filter, *gby_ctx);
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("cleanup dump environment failed", K(ret));
ret = OB_SUCCESS == ret ? tmp_ret : ret;
}
if (NULL != gby_ctx->mem_context_ && NULL != cur_part) {
cur_part->~ObGbyPartition();
gby_ctx->mem_context_->get_malloc_allocator().free(cur_part);
cur_part = NULL;
}
IGNORE_RETURN gby_ctx->sql_mem_processor_.update_used_mem_size(gby_ctx->get_used_mem_size());
}
return ret;
}
int64_t ObHashGroupBy::detect_part_cnt(const int64_t rows, ObHashGroupByCtx& gby_ctx) const
{
const double group_mem_avg = (double)gby_ctx.get_data_size() / gby_ctx.group_rows_.size();
int64_t data_size = rows * ((double)gby_ctx.agged_group_cnt_ / gby_ctx.agged_row_cnt_) * group_mem_avg;
int64_t mem_bound = gby_ctx.get_mem_bound();
const double part_skew_factor = 1.2;
data_size = data_size * part_skew_factor;
int64_t part_cnt = (data_size + mem_bound) / mem_bound;
part_cnt = next_pow2(part_cnt);
// There is only 75% utilization,
// and then change it to segment array to see if it can be removed
int64_t availble_mem_size = min(mem_bound - gby_ctx.get_data_hold_size(), mem_bound * MAX_PART_MEM_RATIO);
int64_t est_dump_size = part_cnt * ObChunkRowStore::BLOCK_SIZE;
if (0 < availble_mem_size) {
while (est_dump_size > availble_mem_size) {
est_dump_size >>= 1;
}
part_cnt = est_dump_size / ObChunkRowStore::BLOCK_SIZE;
} else {
// Excessive memory usage
part_cnt = MIN_PARTITION_CNT;
}
part_cnt = next_pow2(part_cnt);
part_cnt = std::max(part_cnt, (int64_t)MIN_PARTITION_CNT);
part_cnt = std::min(part_cnt, (int64_t)MAX_PARTITION_CNT);
LOG_TRACE("trace detect partition cnt",
K(data_size),
K(group_mem_avg),
K(gby_ctx.get_used_mem_size()),
K(gby_ctx.get_mem_bound()),
K(part_skew_factor),
K(gby_ctx.agged_group_cnt_),
K(gby_ctx.agged_row_cnt_),
K(gby_ctx.group_rows_.size()),
K(part_cnt),
K(gby_ctx.get_data_size()),
K(gby_ctx.get_ctx_mem_size()),
K(gby_ctx.get_part_mem_size()),
K(gby_ctx.get_data_hold_size()),
K(gby_ctx.get_part_hold_size()),
K(rows),
K(gby_ctx.mem_context_),
K(availble_mem_size),
K(est_dump_size));
return part_cnt;
}
int ObHashGroupBy::setup_dump_env(const int64_t level, const int64_t input_rows, ObGbyPartition** parts,
int64_t& part_cnt, ObGbyBloomFilter*& bloom_filter, ObHashGroupByCtx& gby_ctx) const
{
int ret = OB_SUCCESS;
if (input_rows < 0 || OB_ISNULL(parts)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(input_rows), K(parts));
} else {
int64_t pre_part_cnt = 0;
part_cnt = pre_part_cnt = detect_part_cnt(input_rows, gby_ctx);
gby_ctx.adjust_part_cnt(part_cnt);
MEMSET(parts, 0, sizeof(parts[0]) * part_cnt);
const int64_t max_level = sizeof(uint64_t) / 2 - 1; // half for partition half for hash table
if (level > max_level) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("reach max recursion depth", K(ret), K(level));
}
if (OB_SUCC(ret) && NULL == bloom_filter) {
void* mem = gby_ctx.mem_context_->get_malloc_allocator().alloc(sizeof(ObGbyBloomFilter));
if (OB_ISNULL(mem)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
ModulePageAllocator mod_alloc(ObModIds::OB_HASH_NODE_GROUP_ROWS,
gby_ctx.exec_ctx_.get_my_session()->get_effective_tenant_id(),
ObCtxIds::WORK_AREA);
mod_alloc.set_allocator(&gby_ctx.mem_context_->get_malloc_allocator());
bloom_filter = new (mem) ObGbyBloomFilter(mod_alloc);
if (OB_FAIL(bloom_filter->init(gby_ctx.group_rows_.size()))) {
LOG_WARN("bloom filter init failed", K(ret));
} else {
auto cb = [&](ObGbyHashCols& cols) {
int ret = OB_SUCCESS;
if (OB_FAIL(bloom_filter->set(cols.hash()))) {
LOG_WARN("add hash value to bloom failed", K(ret));
}
return ret;
};
if (OB_FAIL(gby_ctx.group_rows_.foreach (cb))) {
LOG_WARN("fill bloom filter failed", K(ret));
}
}
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt; i++) {
void* mem = gby_ctx.mem_context_->get_malloc_allocator().alloc(sizeof(ObGbyPartition));
if (OB_ISNULL(mem)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
parts[i] = new (mem) ObGbyPartition(&gby_ctx.mem_context_->get_malloc_allocator());
parts[i]->level_ = level + 1;
const int64_t extra_size = sizeof(uint64_t); // for hash value
if (OB_FAIL(parts[i]->row_store_.init(1 /* memory limit, dump immediately */,
gby_ctx.exec_ctx_.get_my_session()->get_effective_tenant_id(),
ObCtxIds::WORK_AREA,
ObModIds::OB_HASH_NODE_GROUP_ROWS,
true /* enable dump */,
ObChunkRowStore::FULL,
extra_size))) {
LOG_WARN("init chunk row store failed");
} else {
parts[i]->row_store_.set_dir_id(gby_ctx.sql_mem_processor_.get_dir_id());
parts[i]->row_store_.set_callback(&gby_ctx.sql_mem_processor_);
}
}
}
// If the hash group is dumped, each subsequent partition will allocate a buffer page,about 64K.
// If there are 32 partitions, 2M memory is required
// At the same time, the dump logic needs some memory
if (OB_FAIL(ret)) {
} else if (OB_FAIL(gby_ctx.sql_mem_processor_.get_max_available_mem_size(&gby_ctx.exec_ctx_.get_allocator()))) {
LOG_WARN("failed to get max available memory size", K(ret));
} else if (OB_FAIL(gby_ctx.sql_mem_processor_.update_used_mem_size(gby_ctx.get_used_mem_size()))) {
LOG_WARN("failed to update mem size", K(ret));
}
LOG_TRACE("trace setup dump", K(part_cnt), K(pre_part_cnt), K(level));
}
return ret;
}
int ObHashGroupBy::cleanup_dump_env(const bool dump_success, const int64_t level, ObGbyPartition** parts,
int64_t& part_cnt, ObGbyBloomFilter*& bloom_filter, ObHashGroupByCtx& gby_ctx) const
{
int ret = OB_SUCCESS;
// add spill partitions to partition list
if (dump_success && part_cnt > 0) {
int64_t part_rows[part_cnt];
int64_t part_file_size[part_cnt];
for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt; i++) {
ObGbyPartition*& p = parts[i];
if (p->row_store_.get_row_cnt() > 0) {
if (OB_FAIL(p->row_store_.dump(false, true))) {
LOG_WARN("failed to dump partition", K(ret), K(i));
} else if (OB_FAIL(p->row_store_.finish_add_row(true /* do dump */))) {
LOG_WARN("do dump failed", K(ret));
} else {
part_rows[i] = p->row_store_.get_row_cnt();
part_file_size[i] = p->row_store_.get_file_size();
if (!gby_ctx.all_parts_.add_first(p)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("add to list failed", K(ret));
} else {
p = NULL;
}
}
} else {
part_rows[i] = 0;
part_file_size[i] = 0;
}
}
LOG_TRACE("hash group by dumped",
K(level),
K(gby_ctx.group_rows_.size()),
K(gby_ctx.get_used_mem_size()),
K(gby_ctx.aggr_func_.get_used_mem_size()),
K(gby_ctx.get_mem_bound()),
K(part_cnt),
"part rows",
ObArrayWrap<int64_t>(part_rows, part_cnt),
"part file sizes",
ObArrayWrap<int64_t>(part_file_size, part_cnt));
}
LOG_TRACE("trace hash group by info",
K(level),
K(gby_ctx.group_rows_.size()),
K(gby_ctx.get_used_mem_size()),
K(gby_ctx.aggr_func_.get_used_mem_size()),
K(gby_ctx.get_mem_bound()));
ObArrayWrap<ObGbyPartition*> part_array(parts, part_cnt);
if (NULL != gby_ctx.mem_context_) {
FOREACH_CNT(p, part_array)
{
if (NULL != *p) {
(*p)->~ObGbyPartition();
gby_ctx.mem_context_->get_malloc_allocator().free(*p);
*p = NULL;
}
}
if (NULL != bloom_filter) {
bloom_filter->~ObGbyBloomFilter();
gby_ctx.mem_context_->get_malloc_allocator().free(bloom_filter);
bloom_filter = NULL;
}
}
return ret;
}
int ObHashGroupBy::inner_close(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObHashGroupByCtx* groupby_ctx = NULL;
if (OB_ISNULL(groupby_ctx = GET_PHY_OPERATOR_CTX(ObHashGroupByCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get hash groupby context failed", K_(id));
} else {
groupby_ctx->sql_mem_processor_.unregister_profile();
}
return ret;
}
int ObHashGroupBy::init_sql_mem_mgr(ObHashGroupByCtx* gby_ctx, int64_t input_size) const
{
int ret = OB_SUCCESS;
gby_ctx->sql_mem_processor_.reset();
if (OB_FAIL(gby_ctx->sql_mem_processor_.init(&gby_ctx->exec_ctx_.get_allocator(),
gby_ctx->exec_ctx_.get_my_session()->get_effective_tenant_id(),
input_size,
get_type(),
get_id(),
&gby_ctx->exec_ctx_))) {
LOG_WARN("failed to init sql mem processor", K(ret));
}
return ret;
}
int ObHashGroupBy::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObHashGroupByCtx* groupby_ctx = NULL;
ObSQLSessionInfo* my_session = ctx.get_my_session();
const ObTimeZoneInfo* tz_info = (my_session != NULL) ? my_session->get_timezone_info() : NULL;
if (OB_ISNULL(groupby_ctx = GET_PHY_OPERATOR_CTX(ObHashGroupByCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get hash groupby context failed", K_(id));
} else {
if (!groupby_ctx->started_) {
if (OB_FAIL(groupby_ctx->init_mem_context())) {
LOG_WARN("failed to create entity", K(ret));
} else if (!groupby_ctx->bkt_created_) {
// create bucket
int64_t est_group_cnt = est_group_cnt_;
if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(&ctx, px_est_size_factor_, est_group_cnt_, est_group_cnt))) {
LOG_WARN("failed to get px size", K(ret));
}
int64_t est_hash_mem_size = groupby_ctx->estimate_hash_bucket_size(est_group_cnt);
int64_t estimate_mem_size = est_hash_mem_size + get_width() * est_group_cnt;
if (OB_FAIL(ret)) {
} else if (OB_FAIL(init_sql_mem_mgr(groupby_ctx, estimate_mem_size))) {
LOG_WARN("failed to init sql mem manager", K(ret));
} else {
int64_t init_size = groupby_ctx->estimate_hash_bucket_cnt_by_mem_size(est_group_cnt,
groupby_ctx->sql_mem_processor_.get_mem_bound(),
est_hash_mem_size * 1. / estimate_mem_size);
init_size = std::max((int64_t)MIN_GROUP_HT_INIT_SIZE, init_size);
init_size = std::min((int64_t)MAX_GROUP_HT_INIT_SIZE, init_size);
ObMemAttr attr(
ctx.get_my_session()->get_effective_tenant_id(), ObModIds::OB_HASH_NODE_GROUP_ROWS, ObCtxIds::WORK_AREA);
if (OB_FAIL(
groupby_ctx->group_rows_.init(&groupby_ctx->mem_context_->get_malloc_allocator(), attr, init_size))) {
LOG_WARN("fail to init hash map", K(ret));
} else {
groupby_ctx->op_monitor_info_.otherstat_1_value_ = init_size;
groupby_ctx->op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::HASH_INIT_BUCKET_COUNT;
IGNORE_RETURN groupby_ctx->sql_mem_processor_.update_used_mem_size(groupby_ctx->get_used_mem_size());
groupby_ctx->bkt_created_ = true;
LOG_TRACE("trace init hash table",
K(init_size),
K(est_group_cnt_),
K(est_group_cnt),
K(est_hash_mem_size),
K(estimate_mem_size),
K(groupby_ctx->profile_.get_expect_size()),
K(groupby_ctx->profile_.get_cache_size()),
K(groupby_ctx->sql_mem_processor_.get_mem_bound()));
}
}
}
// load data
if (OB_FAIL(ret)) {
} else if (OB_FAIL(load_data(ctx))) {
LOG_WARN("load data to hash group by map failed", K(ret));
} else {
groupby_ctx->group_idx_ = 0;
groupby_ctx->started_ = true;
}
} else {
++groupby_ctx->group_idx_;
}
if (OB_FAIL(ret)) {
} else if (OB_UNLIKELY(groupby_ctx->group_idx_ >= groupby_ctx->group_rows_.size())) {
if (groupby_ctx->all_parts_.is_empty()) {
ret = OB_ITER_END;
groupby_ctx->op_monitor_info_.otherstat_2_value_ = groupby_ctx->agged_group_cnt_;
groupby_ctx->op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::HASH_ROW_COUNT;
groupby_ctx->op_monitor_info_.otherstat_3_value_ =
max(groupby_ctx->group_rows_.get_bucket_num(), groupby_ctx->op_monitor_info_.otherstat_3_value_);
groupby_ctx->op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::HASH_BUCKET_COUNT;
} else {
if (OB_FAIL(load_data(ctx))) {
LOG_WARN("load partitioned data failed", K(ret));
} else {
groupby_ctx->op_monitor_info_.otherstat_3_value_ =
max(groupby_ctx->group_rows_.get_bucket_num(), groupby_ctx->op_monitor_info_.otherstat_3_value_);
groupby_ctx->op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::HASH_BUCKET_COUNT;
groupby_ctx->group_idx_ = 0;
}
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(groupby_ctx->aggr_func_.get_result(groupby_ctx->get_cur_row(), tz_info, groupby_ctx->group_idx_))) {
LOG_WARN("failed to get aggr result", K(ret));
}
}
}
if (OB_SUCC(ret)) {
row = &groupby_ctx->get_cur_row();
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,108 @@
/**
* 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_HASH_GROUPBY_H
#define _OB_HASH_GROUPBY_H 1
#include "sql/engine/aggregate/ob_groupby.h"
namespace oceanbase {
namespace sql {
class ObGbyBloomFilter;
class ObGbyPartition;
class ObHashGroupBy : public ObGroupBy {
private:
class ObHashGroupByCtx;
public:
static const int64_t MIN_PARTITION_CNT = 8;
static const int64_t MAX_PARTITION_CNT = 256;
// min in memory groups
static const int64_t MIN_INMEM_GROUPS = 4;
static const int64_t MIN_GROUP_HT_INIT_SIZE = 1024;
static const int64_t MAX_GROUP_HT_INIT_SIZE = 1 << 20; // 1048576
static const int64_t DISTINCT_SET_NUM = 100000;
static const int64_t PREPARE_ROW_NUM = 100000;
static constexpr const double MAX_PART_MEM_RATIO = 0.5;
static constexpr const double EXTRA_MEM_RATIO = 0.25;
explicit ObHashGroupBy(common::ObIAllocator& alloc);
virtual ~ObHashGroupBy();
virtual void reset();
virtual void reuse();
virtual int rescan(ObExecContext& ctx) const;
int get_hash_groupby_row_count(ObExecContext& exec_ctx, int64_t& hash_groupby_row_count) const;
private:
DISALLOW_COPY_AND_ASSIGN(ObHashGroupBy);
private:
/**
* @brief create operator context, only child operator can know it's specific operator type,
* so must be overwrited by child operator,
* @param ctx[in], execute context
* @param op_ctx[out], the pointer of operator context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
/**
* @brief: called by get_next_row(), get a row from the child operator or row_store
* @param: ctx[in], execute context
* @param: row[out], ObSqlRow an obj array and row_size
*/
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
/**
* @brief open operator, not including children operators.
* called by open.
* Every op should implement this method.
*/
virtual int inner_open(ObExecContext& ctx) const;
/**
* @brief close operator, not including children operators.
* Every op should implement this method.
*/
virtual int inner_close(ObExecContext& ctx) const;
/**
* @brief load data to hash group by map
*/
virtual int load_data(ObExecContext& ctx) const;
/**
* @brief for specified phy operator to print it's member variable with json key-value format
* @param buf[in] to string buffer
* @param buf_len[in] buffer length
* @return if success, return the length used by print string, otherwise return 0
*/
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
int init_sql_mem_mgr(ObHashGroupByCtx* gby_ctx, int64_t input_size) const;
int update_mem_status_periodically(ObHashGroupByCtx* gby_ctx, int64_t nth_cnt, const int64_t input_row,
int64_t& est_part_cnt, bool& need_dump) const;
int64_t detect_part_cnt(const int64_t rows, ObHashGroupByCtx& gby_ctx) const;
// check is dump needed right now, update %est_part_cnt on demand
OB_INLINE bool need_start_dump(
ObHashGroupByCtx& gby_ctx, const int64_t input_rows, int64_t& est_part_cnt, bool check_dump) const;
// Setup: memory entity, bloom filter, spill partitions
int setup_dump_env(const int64_t level, const int64_t input_rows, ObGbyPartition** parts, int64_t& part_cnt,
ObGbyBloomFilter*& bloom_filter, ObHashGroupByCtx& gby_ctx) const;
int cleanup_dump_env(const bool dump_success, const int64_t level, ObGbyPartition** parts, int64_t& part_cnt,
ObGbyBloomFilter*& bloom_filter, ObHashGroupByCtx& gby_ctx) const;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_HASH_GROUPBY_H */

View File

@ -0,0 +1,788 @@
/**
* 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/aggregate/ob_hash_groupby_op.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
#include "sql/engine/px/ob_px_util.h"
namespace oceanbase {
using namespace common;
using namespace common::hash;
namespace sql {
OB_SERIALIZE_MEMBER((ObHashGroupBySpec, ObGroupBySpec), group_exprs_, cmp_funcs_, est_group_cnt_);
DEF_TO_STRING(ObHashGroupBySpec)
{
int64_t pos = 0;
J_OBJ_START();
J_NAME("groupby_spec");
J_COLON();
pos += ObGroupBySpec::to_string(buf + pos, buf_len - pos);
J_COMMA();
J_KV(K_(group_exprs));
J_OBJ_END();
return pos;
}
int ObHashGroupBySpec::add_group_expr(ObExpr* expr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(expr)) {
ret = OB_INVALID_ARGUMENT;
OB_LOG(WARN, "invalid argument", K(ret));
} else if (OB_FAIL(group_exprs_.push_back(expr))) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("failed to push_back expr");
}
return ret;
}
void ObHashGroupByOp::reset()
{
curr_group_id_ = common::OB_INVALID_INDEX;
local_group_rows_.reuse();
sql_mem_processor_.reset();
destroy_all_parts();
group_store_.reset();
}
int ObHashGroupByOp::inner_open()
{
int ret = OB_SUCCESS;
reset();
if (OB_FAIL(ObGroupByOp::inner_open())) {
LOG_WARN("failed to inner_open", K(ret));
} else if (OB_FAIL(init_mem_context())) {
LOG_WARN("init memory entity failed", K(ret));
} else if (!local_group_rows_.is_inited()) {
// create bucket
int64_t est_group_cnt = MY_SPEC.est_group_cnt_;
int64_t est_hash_mem_size = 0;
int64_t estimate_mem_size = 0;
int64_t init_size = 0;
ObMemAttr attr(
ctx_.get_my_session()->get_effective_tenant_id(), ObModIds::OB_HASH_NODE_GROUP_ROWS, ObCtxIds::WORK_AREA);
if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(
&ctx_, MY_SPEC.px_est_size_factor_, MY_SPEC.est_group_cnt_, est_group_cnt))) {
LOG_WARN("failed to get px size", K(ret));
} else if (FALSE_IT(est_hash_mem_size = estimate_hash_bucket_size(est_group_cnt))) {
} else if (FALSE_IT(estimate_mem_size = est_hash_mem_size + MY_SPEC.width_ * est_group_cnt)) {
} else if (OB_FAIL(sql_mem_processor_.init(&ctx_.get_allocator(),
ctx_.get_my_session()->get_effective_tenant_id(),
estimate_mem_size,
MY_SPEC.type_,
MY_SPEC.id_,
&ctx_))) {
LOG_WARN("failed to init sql mem processor", K(ret));
} else if (FALSE_IT(init_size = estimate_hash_bucket_cnt_by_mem_size(est_group_cnt,
sql_mem_processor_.get_mem_bound(),
est_hash_mem_size * 1. / estimate_mem_size))) {
} else if (FALSE_IT(init_size = std::max((int64_t)MIN_GROUP_HT_INIT_SIZE, init_size))) {
} else if (FALSE_IT(init_size = std::min((int64_t)MAX_GROUP_HT_INIT_SIZE, init_size))) {
} else if (OB_FAIL(local_group_rows_.init(
&mem_context_->get_malloc_allocator(), attr, &eval_ctx_, &MY_SPEC.cmp_funcs_, init_size))) {
LOG_WARN("fail to init hash map", K(ret));
} else if (OB_FAIL(sql_mem_processor_.update_used_mem_size(get_mem_used_size()))) {
LOG_WARN("fail to update_used_mem_size", "size", get_mem_used_size(), K(ret));
} else if (OB_FAIL(group_store_.init(0,
ctx_.get_my_session()->get_effective_tenant_id(),
ObCtxIds::WORK_AREA,
ObModIds::OB_HASH_NODE_GROUP_ROWS,
false /* disable dump */,
0))) {
LOG_WARN("failed to init group store", K(ret));
} else {
group_store_.set_dir_id(sql_mem_processor_.get_dir_id());
group_store_.set_callback(&sql_mem_processor_);
group_store_.set_allocator(mem_context_->get_malloc_allocator());
op_monitor_info_.otherstat_1_value_ = init_size;
op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::HASH_INIT_BUCKET_COUNT;
LOG_TRACE("trace init hash table",
K(init_size),
K(MY_SPEC.est_group_cnt_),
K(est_group_cnt),
K(est_hash_mem_size),
K(estimate_mem_size),
K(profile_.get_expect_size()),
K(profile_.get_cache_size()),
K(sql_mem_processor_.get_mem_bound()));
}
}
return ret;
}
int ObHashGroupByOp::inner_close()
{
sql_mem_processor_.unregister_profile();
curr_group_id_ = common::OB_INVALID_INDEX;
return ObGroupByOp::inner_close();
}
int ObHashGroupByOp::init_mem_context()
{
int ret = OB_SUCCESS;
if (NULL == mem_context_) {
lib::ContextParam param;
param.set_mem_attr(
ctx_.get_my_session()->get_effective_tenant_id(), ObModIds::OB_HASH_NODE_GROUP_ROWS, ObCtxIds::WORK_AREA);
if (OB_FAIL(CURRENT_CONTEXT.CREATE_CONTEXT(mem_context_, param))) {
LOG_WARN("memory entity create failed", K(ret));
}
}
return ret;
}
void ObHashGroupByOp::destroy()
{
local_group_rows_.destroy();
sql_mem_processor_.destroy();
if (NULL != mem_context_) {
destroy_all_parts();
}
group_store_.reset();
ObGroupByOp::destroy();
if (NULL != mem_context_) {
DESTROY_CONTEXT(mem_context_);
mem_context_ = NULL;
}
}
int ObHashGroupByOp::switch_iterator()
{
int ret = OB_SUCCESS;
reset();
if (OB_FAIL(ObGroupByOp::switch_iterator())) {
LOG_WARN("failed to rescan", K(ret));
}
return ret;
}
int ObHashGroupByOp::rescan()
{
int ret = OB_SUCCESS;
reset();
if (OB_FAIL(ObGroupByOp::rescan())) {
LOG_WARN("failed to rescan", K(ret));
} else {
iter_end_ = false;
}
return ret;
}
int ObHashGroupByOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
LOG_DEBUG("before inner_get_next_row",
K(get_aggr_used_size()),
K(get_aggr_used_size()),
K(get_local_hash_used_size()),
K(get_dumped_part_used_size()),
K(get_dump_part_hold_size()),
K(get_mem_used_size()),
K(get_mem_bound_size()),
K(agged_dumped_cnt_),
K(agged_group_cnt_),
K(agged_row_cnt_));
if (iter_end_) {
ret = OB_ITER_END;
} else if (curr_group_id_ < 0) {
if (OB_FAIL(load_data())) {
LOG_WARN("load data failed", K(ret));
} else {
curr_group_id_ = 0;
}
} else {
++curr_group_id_;
}
if (OB_SUCC(ret)) {
if (OB_UNLIKELY(curr_group_id_ >= local_group_rows_.size())) {
if (dumped_group_parts_.is_empty()) {
op_monitor_info_.otherstat_2_value_ = agged_group_cnt_;
op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::HASH_ROW_COUNT;
op_monitor_info_.otherstat_3_value_ =
max(local_group_rows_.get_bucket_num(), op_monitor_info_.otherstat_3_value_);
op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::HASH_BUCKET_COUNT;
ret = OB_ITER_END;
iter_end_ = true;
reset();
} else {
if (OB_FAIL(load_data())) {
LOG_WARN("load data failed", K(ret));
} else {
op_monitor_info_.otherstat_3_value_ =
max(local_group_rows_.get_bucket_num(), op_monitor_info_.otherstat_3_value_);
op_monitor_info_.otherstat_3_id_ = ObSqlMonitorStatIds::HASH_BUCKET_COUNT;
curr_group_id_ = 0;
}
}
}
}
if (OB_SUCC(ret)) {
clear_evaluated_flag();
if (OB_FAIL(aggr_processor_.collect(curr_group_id_))) {
LOG_WARN("failed to collect result", K(ret));
} else if (OB_FAIL(restore_groupby_datum())) {
LOG_WARN("failed to restore_groupby_datum", K(ret));
}
}
LOG_DEBUG("after inner_get_next_row",
K(get_aggr_used_size()),
K(get_aggr_used_size()),
K(get_local_hash_used_size()),
K(get_dumped_part_used_size()),
K(get_dump_part_hold_size()),
K(get_mem_used_size()),
K(get_mem_bound_size()),
K(agged_dumped_cnt_),
K(agged_group_cnt_),
K(agged_row_cnt_));
return ret;
}
int ObHashGroupByOp::load_data()
{
int ret = OB_SUCCESS;
ObChunkDatumStore::Iterator row_store_iter;
DatumStoreLinkPartition* cur_part = NULL;
int64_t part_id = 0;
int64_t part_shift = sizeof(uint64_t) * CHAR_BIT / 2; // low half bits for hash table lookup.
int64_t input_rows = child_->get_spec().rows_;
int64_t input_size = child_->get_spec().width_ * input_rows;
static_assert(MAX_PARTITION_CNT <= (1 << (CHAR_BIT)), "max partition cnt is too big");
if (!dumped_group_parts_.is_empty()) {
aggr_processor_.reuse();
sql_mem_processor_.reset();
cur_part = dumped_group_parts_.remove_first();
if (OB_ISNULL(cur_part)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("pop head partition failed", K(ret));
} else if (OB_FAIL(row_store_iter.init(&cur_part->datum_store_))) {
LOG_WARN("init chunk row store iterator failed", K(ret));
} else {
input_rows = cur_part->datum_store_.get_row_cnt();
part_id = cur_part->part_id_;
part_shift = part_shift + part_id * CHAR_BIT;
input_size = cur_part->datum_store_.get_file_size();
if (OB_FAIL(local_group_rows_.resize(&mem_context_->get_malloc_allocator(), max(2, input_rows)))) {
LOG_WARN("failed to reuse extended hash table", K(ret));
} else if (OB_FAIL(sql_mem_processor_.init(&ctx_.get_allocator(),
ctx_.get_my_session()->get_effective_tenant_id(),
input_size,
MY_SPEC.type_,
MY_SPEC.id_,
&ctx_))) {
LOG_WARN("failed to init sql mem processor", K(ret));
} else {
LOG_TRACE("scan new partition",
K(part_id),
K(input_rows),
K(input_size),
K(local_group_rows_.size()),
K(get_mem_used_size()),
K(get_aggr_used_size()),
K(get_mem_bound_size()));
}
}
}
// We use sort based group by for aggregation which need distinct or sort right now,
// disable operator dump for compatibility.
const bool is_dump_enabled =
(!(aggr_processor_.has_distinct() || aggr_processor_.has_order_by()) && GCONF.is_sql_operator_dump_enabled());
ObGroupRowItem curr_gr_item;
curr_gr_item.group_exprs_ = const_cast<ExprFixedArray*>(&(MY_SPEC.group_exprs_));
const ObGroupRowItem* exist_curr_gr_item = NULL;
DatumStoreLinkPartition* parts[MAX_PARTITION_CNT] = {};
int64_t part_cnt = 0;
int64_t est_part_cnt = 0;
bool check_dump = false;
ObGbyBloomFilter* bloom_filter = NULL;
const ObChunkDatumStore::StoredRow* srow = NULL;
for (int64_t loop_cnt = 0; OB_SUCC(ret); ++loop_cnt) {
if (NULL == cur_part) {
ret = child_->get_next_row();
} else {
ret = row_store_iter.get_next_row(child_->get_spec().output_, eval_ctx_, &srow);
}
clear_evaluated_flag();
if (common::OB_SUCCESS != ret) {
if (OB_ITER_END != ret) {
LOG_WARN("get input row failed", K(ret));
}
break;
}
if (loop_cnt % 1024 == 1023) {
if (OB_FAIL(ctx_.check_status())) {
LOG_WARN("check status failed", K(ret));
}
}
const bool start_dump = (bloom_filter != NULL);
if (OB_SUCC(ret)) {
if (!start_dump && OB_FAIL(update_mem_status_periodically(loop_cnt, input_rows, est_part_cnt, check_dump))) {
LOG_WARN("failed to update usable memory size periodically", K(ret));
}
}
if (OB_SUCC(ret)) {
if (NULL != srow) {
curr_gr_item.groupby_datums_hash_ = *static_cast<uint64_t*>(srow->get_extra_payload());
} else {
if (OB_FAIL(calc_groupby_exprs_hash(curr_gr_item.groupby_datums_hash_))) {
LOG_WARN("failed to get_groupby_exprs_hash", K(ret));
}
}
LOG_DEBUG("finish calc_groupby_exprs_hash", K(curr_gr_item));
}
if (OB_FAIL(ret)) {
} else if ((!start_dump || bloom_filter->exist(curr_gr_item.hash())) &&
NULL != (exist_curr_gr_item = local_group_rows_.get(curr_gr_item))) {
agged_row_cnt_ += 1;
if (OB_ISNULL(exist_curr_gr_item->group_row_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group_row is null", K(exist_curr_gr_item->group_id_), K(ret));
} else if (OB_FAIL(aggr_processor_.process(*exist_curr_gr_item->group_row_))) {
LOG_WARN("fail to process row", K(ret), KPC(exist_curr_gr_item));
}
} else {
if (!is_dump_enabled || local_group_rows_.size() < MIN_INMEM_GROUPS ||
(!start_dump && !need_start_dump(input_rows, est_part_cnt, check_dump))) {
++agged_row_cnt_;
++agged_group_cnt_;
ObGroupRowItem* tmp_gr_item = NULL;
if (OB_FAIL(init_group_row_item(curr_gr_item, tmp_gr_item))) {
LOG_WARN("failed to init_group_row_item", K(ret));
} else if (OB_FAIL(aggr_processor_.prepare(*tmp_gr_item->group_row_))) {
LOG_WARN("fail to prepare row", K(ret), KPC(tmp_gr_item->group_row_));
} else if (OB_FAIL(local_group_rows_.set(*tmp_gr_item))) {
LOG_WARN("hash table set failed", K(ret));
}
} else {
if (OB_UNLIKELY(!start_dump)) {
if (OB_FAIL(setup_dump_env(part_id, max(input_rows, loop_cnt), parts, part_cnt, bloom_filter))) {
LOG_WARN("setup dump environment failed", K(ret));
} else {
sql_mem_processor_.set_number_pass(part_id + 1);
}
}
if (OB_SUCC(ret)) {
++agged_dumped_cnt_;
const int64_t part_idx = (curr_gr_item.hash() >> part_shift) & (part_cnt - 1);
// TODO : copy store row to new row store directly.
ObChunkDatumStore::StoredRow* stored_row = NULL;
if (OB_FAIL(parts[part_idx]->datum_store_.add_row(child_->get_spec().output_, &eval_ctx_, &stored_row))) {
LOG_WARN("add row failed", K(ret));
} else {
*static_cast<uint64_t*>(stored_row->get_extra_payload()) = curr_gr_item.hash();
LOG_DEBUG("finish dump", K(part_idx), K(curr_gr_item), KPC(stored_row));
}
}
}
}
}
// must reset. can't delay to destruction(memory released)
row_store_iter.reset();
if (OB_ITER_END == ret) {
ret = OB_SUCCESS;
}
// cleanup_dump_env() must be called whether success or not
int tmp_ret = cleanup_dump_env(common::OB_SUCCESS == ret, part_id, parts, part_cnt, bloom_filter);
if (OB_SUCCESS != tmp_ret) {
LOG_WARN("cleanup dump environment failed", K(tmp_ret), K(ret));
ret = OB_SUCCESS == ret ? tmp_ret : ret;
}
if (NULL != mem_context_ && NULL != cur_part) {
cur_part->~DatumStoreLinkPartition();
mem_context_->get_malloc_allocator().free(cur_part);
cur_part = NULL;
}
IGNORE_RETURN sql_mem_processor_.update_used_mem_size(get_mem_used_size());
return ret;
}
int ObHashGroupByOp::init_group_row_item(const ObGroupRowItem& curr_item, ObGroupRowItem*& gr_row_item)
{
int ret = common::OB_SUCCESS;
ObAggregateProcessor::GroupRow* group_row = NULL;
gr_row_item = NULL;
void* buf = NULL;
if (OB_ISNULL(buf = (aggr_processor_.get_aggr_alloc().alloc(sizeof(ObGroupRowItem))))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
gr_row_item = new (buf) ObGroupRowItem();
gr_row_item->group_id_ = local_group_rows_.size();
gr_row_item->groupby_datums_hash_ = curr_item.hash();
if (OB_FAIL(aggr_processor_.init_one_group(gr_row_item->group_id_))) {
LOG_WARN("failed to init one group", K(gr_row_item->group_id_), K(ret));
} else if (OB_FAIL(aggr_processor_.get_group_row(gr_row_item->group_id_, group_row))) {
LOG_WARN("failed to get group_row", K(gr_row_item->group_id_), K(ret));
} else if (OB_ISNULL(group_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group_row is null", K(gr_row_item->group_id_), K(ret));
} else {
gr_row_item->group_row_ = group_row;
}
}
if (OB_SUCC(ret)) {
ObChunkDatumStore::StoredRow* groupby_store_row = nullptr;
if (OB_FAIL(group_store_.add_row(MY_SPEC.group_exprs_, &eval_ctx_, &groupby_store_row))) {
LOG_WARN("failed to add row", K(ret));
} else {
group_row->groupby_store_row_ = groupby_store_row;
}
}
return ret;
}
int ObHashGroupByOp::update_mem_status_periodically(
const int64_t nth_cnt, const int64_t input_row, int64_t& est_part_cnt, bool& need_dump)
{
int ret = common::OB_SUCCESS;
bool updated = false;
need_dump = false;
if (OB_FAIL(sql_mem_processor_.update_max_available_mem_size_periodically(
&ctx_.get_allocator(), [&](int64_t cur_cnt) { return nth_cnt > cur_cnt; }, updated))) {
LOG_WARN("failed to update usable memory size periodically", K(ret));
} else if (updated) {
if (OB_FAIL(sql_mem_processor_.update_used_mem_size(get_mem_used_size()))) {
LOG_WARN("failed to update used memory size", K(ret));
} else {
double data_ratio = sql_mem_processor_.get_data_ratio();
;
est_part_cnt = detect_part_cnt(input_row);
calc_data_mem_ratio(est_part_cnt, data_ratio);
need_dump = (get_aggr_used_size() > get_mem_bound_size() * data_ratio);
}
}
return ret;
}
int64_t ObHashGroupByOp::detect_part_cnt(const int64_t rows) const
{
const double group_mem_avg = (double)get_aggr_used_size() / local_group_rows_.size();
int64_t data_size = rows * ((double)agged_group_cnt_ / agged_row_cnt_) * group_mem_avg;
int64_t mem_bound = get_mem_bound_size();
const double part_skew_factor = 1.2;
data_size = data_size * part_skew_factor;
int64_t part_cnt = (data_size + mem_bound) / mem_bound;
part_cnt = next_pow2(part_cnt);
int64_t availble_mem_size = min(mem_bound - get_aggr_hold_size(), mem_bound * MAX_PART_MEM_RATIO);
int64_t est_dump_size = part_cnt * ObChunkRowStore::BLOCK_SIZE;
if (0 < availble_mem_size) {
while (est_dump_size > availble_mem_size) {
est_dump_size >>= 1;
}
part_cnt = est_dump_size / ObChunkRowStore::BLOCK_SIZE;
} else {
part_cnt = MIN_PARTITION_CNT;
}
part_cnt = next_pow2(part_cnt);
part_cnt = std::max(part_cnt, (int64_t)MIN_PARTITION_CNT);
part_cnt = std::min(part_cnt, (int64_t)MAX_PARTITION_CNT);
LOG_TRACE("trace detect partition cnt",
K(data_size),
K(group_mem_avg),
K(get_mem_used_size()),
K(get_mem_bound_size()),
K(part_skew_factor),
K(agged_group_cnt_),
K(agged_row_cnt_),
K(local_group_rows_.size()),
K(part_cnt),
K(get_aggr_used_size()),
K(get_local_hash_used_size()),
K(get_dumped_part_used_size()),
K(get_aggr_hold_size()),
K(get_dump_part_hold_size()),
K(rows),
K(mem_context_),
K(availble_mem_size),
K(est_dump_size));
return part_cnt;
}
void ObHashGroupByOp::calc_data_mem_ratio(const int64_t part_cnt, double& data_ratio)
{
int64_t extra_size = (get_local_hash_used_size() + part_cnt * FIX_SIZE_PER_PART) * (1 + EXTRA_MEM_RATIO);
int64_t data_size = max(get_aggr_used_size(), (get_mem_bound_size() - extra_size) * 0.8);
data_ratio = data_size * 1.0 / (extra_size + data_size);
sql_mem_processor_.set_data_ratio(data_ratio);
LOG_TRACE("trace calc data ratio", K(data_ratio), K(extra_size), K(part_cnt), K(data_size), K(get_aggr_used_size()));
}
void ObHashGroupByOp::adjust_part_cnt(int64_t& part_cnt)
{
int64_t mem_used = get_mem_used_size();
int64_t mem_bound = get_mem_bound_size();
int64_t dumped_remain_mem_size = mem_bound - mem_used;
int64_t max_part_cnt = dumped_remain_mem_size / FIX_SIZE_PER_PART;
if (max_part_cnt <= MIN_PARTITION_CNT) {
part_cnt = MIN_PARTITION_CNT;
} else {
while (part_cnt > max_part_cnt) {
part_cnt >>= 1;
}
}
LOG_TRACE(
"trace adjust part cnt", K(part_cnt), K(max_part_cnt), K(dumped_remain_mem_size), K(mem_bound), K(mem_used));
}
int ObHashGroupByOp::calc_groupby_exprs_hash(uint64_t& hash_value)
{
int ret = OB_SUCCESS;
hash_value = 99194853094755497L;
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.group_exprs_.count(); ++i) {
ObExpr* expr = MY_SPEC.group_exprs_.at(i);
ObDatum* result = NULL;
if (OB_ISNULL(expr) || OB_ISNULL(expr->basic_funcs_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr node is null", K(ret));
} else if (OB_FAIL(expr->eval(eval_ctx_, result))) {
LOG_WARN("eval failed", K(ret));
} else {
ObExprHashFuncType hash_func = expr->basic_funcs_->murmur_hash_;
hash_value = hash_func(*result, hash_value);
}
}
return ret;
}
bool ObHashGroupByOp::need_start_dump(const int64_t input_rows, int64_t& est_part_cnt, const bool check_dump)
{
bool need_dump = false;
const int64_t mem_used = get_mem_used_size();
const int64_t mem_bound = get_mem_bound_size();
double data_ratio = sql_mem_processor_.get_data_ratio();
if (OB_UNLIKELY(0 == est_part_cnt)) {
est_part_cnt = detect_part_cnt(input_rows);
calc_data_mem_ratio(est_part_cnt, data_ratio);
}
// We continue do aggregation after we start dumping, reserve 1/8 memory for it.
if (get_aggr_used_size() > data_ratio * mem_bound || check_dump) {
int ret = OB_SUCCESS;
need_dump = true;
if (OB_FAIL(sql_mem_processor_.extend_max_memory_size(
&ctx_.get_allocator(),
[&](int64_t max_memory_size) { return get_aggr_used_size() > data_ratio * max_memory_size; },
need_dump,
mem_used))) {
need_dump = true;
LOG_WARN("failed to extend max memory size", K(ret), K(need_dump));
} else if (OB_FAIL(sql_mem_processor_.update_used_mem_size(mem_used))) {
LOG_WARN("failed to update used memory size", K(ret), K(mem_used), K(mem_bound), K(need_dump));
} else {
est_part_cnt = detect_part_cnt(input_rows);
calc_data_mem_ratio(est_part_cnt, data_ratio);
LOG_TRACE("trace extend max memory size",
K(ret),
K(data_ratio),
K(get_aggr_used_size()),
K(get_extra_size()),
K(mem_used),
K(mem_bound),
K(need_dump),
K(est_part_cnt));
}
}
return need_dump;
}
int ObHashGroupByOp::setup_dump_env(const int64_t part_id, const int64_t input_rows, DatumStoreLinkPartition** parts,
int64_t& part_cnt, ObGbyBloomFilter*& bloom_filter)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(input_rows < 0) || OB_ISNULL(parts)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(input_rows), KP(parts));
} else {
int64_t pre_part_cnt = 0;
part_cnt = pre_part_cnt = detect_part_cnt(input_rows);
adjust_part_cnt(part_cnt);
MEMSET(parts, 0, sizeof(parts[0]) * part_cnt);
const int64_t max_part_id = sizeof(uint64_t) / 2 - 1; // half for partition half for hash table
if (OB_UNLIKELY(part_id > max_part_id)) { // part_id means level, max_part_id means max_level
ret = OB_ERR_UNEXPECTED;
LOG_WARN("reach max recursion depth", K(ret), K(part_id), K(max_part_id));
}
if (OB_SUCC(ret) && NULL == bloom_filter) {
ModulePageAllocator mod_alloc(
ObModIds::OB_HASH_NODE_GROUP_ROWS, ctx_.get_my_session()->get_effective_tenant_id(), ObCtxIds::WORK_AREA);
mod_alloc.set_allocator(&mem_context_->get_malloc_allocator());
void* mem = mem_context_->get_malloc_allocator().alloc(sizeof(ObGbyBloomFilter));
if (OB_ISNULL(mem)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else if (FALSE_IT(bloom_filter = new (mem) ObGbyBloomFilter(mod_alloc))) {
} else if (OB_FAIL(bloom_filter->init(local_group_rows_.size()))) {
LOG_WARN("bloom filter init failed", K(ret));
} else {
auto cb_func = [&](ObGroupRowItem& item) {
int ret = OB_SUCCESS;
if (OB_FAIL(bloom_filter->set(item.hash()))) {
LOG_WARN("add hash value to bloom failed", K(ret));
}
return ret;
};
if (OB_FAIL(local_group_rows_.foreach (cb_func))) {
LOG_WARN("fill bloom filter failed", K(ret));
}
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt; i++) {
void* mem = mem_context_->get_malloc_allocator().alloc(sizeof(DatumStoreLinkPartition));
if (OB_ISNULL(mem)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate memory failed", K(ret));
} else {
parts[i] = new (mem) DatumStoreLinkPartition(&mem_context_->get_malloc_allocator());
parts[i]->part_id_ = part_id + 1;
const int64_t extra_size = sizeof(uint64_t); // for hash value
if (OB_FAIL(parts[i]->datum_store_.init(1 /* memory limit, dump immediately */,
ctx_.get_my_session()->get_effective_tenant_id(),
ObCtxIds::WORK_AREA,
ObModIds::OB_HASH_NODE_GROUP_ROWS,
true /* enable dump */,
extra_size))) {
LOG_WARN("init chunk row store failed", K(ret));
} else {
parts[i]->datum_store_.set_dir_id(sql_mem_processor_.get_dir_id());
parts[i]->datum_store_.set_callback(&sql_mem_processor_);
}
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(sql_mem_processor_.get_max_available_mem_size(&ctx_.get_allocator()))) {
LOG_WARN("failed to get max available memory size", K(ret));
} else if (OB_FAIL(sql_mem_processor_.update_used_mem_size(get_mem_used_size()))) {
LOG_WARN("failed to update mem size", K(ret));
}
LOG_TRACE("trace setup dump", K(part_cnt), K(pre_part_cnt), K(part_id));
}
return ret;
}
int ObHashGroupByOp::cleanup_dump_env(const bool dump_success, const int64_t part_id, DatumStoreLinkPartition** parts,
int64_t& part_cnt, ObGbyBloomFilter*& bloom_filter)
{
int ret = OB_SUCCESS;
// add spill partitions to partition list
if (dump_success && part_cnt > 0) {
int64_t part_rows[part_cnt];
int64_t part_file_size[part_cnt];
for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt; i++) {
DatumStoreLinkPartition*& p = parts[i];
if (p->datum_store_.get_row_cnt() > 0) {
if (OB_FAIL(p->datum_store_.dump(false, true))) {
LOG_WARN("failed to dump partition", K(ret), K(i));
} else if (OB_FAIL(p->datum_store_.finish_add_row(true /* do dump */))) {
LOG_WARN("do dump failed", K(ret));
} else {
part_rows[i] = p->datum_store_.get_row_cnt();
part_file_size[i] = p->datum_store_.get_file_size();
if (!dumped_group_parts_.add_first(p)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("add to list failed", K(ret));
} else {
p = NULL;
}
}
} else {
part_rows[i] = 0;
part_file_size[i] = 0;
}
}
LOG_TRACE("hash group by dumped",
K(part_id),
K(local_group_rows_.size()),
K(get_mem_used_size()),
K(get_aggr_used_size()),
K(get_mem_bound_size()),
K(part_cnt),
"part rows",
ObArrayWrap<int64_t>(part_rows, part_cnt),
"part file sizes",
ObArrayWrap<int64_t>(part_file_size, part_cnt));
}
LOG_TRACE("trace hash group by info",
K(part_id),
K(local_group_rows_.size()),
K(get_mem_used_size()),
K(get_aggr_used_size()),
K(get_mem_bound_size()));
ObArrayWrap<DatumStoreLinkPartition*> part_array(parts, part_cnt);
if (NULL != mem_context_) {
FOREACH_CNT(p, part_array)
{
if (NULL != *p) {
(*p)->~DatumStoreLinkPartition();
mem_context_->get_malloc_allocator().free(*p);
*p = NULL;
}
}
if (NULL != bloom_filter) {
bloom_filter->~ObGbyBloomFilter();
mem_context_->get_malloc_allocator().free(bloom_filter);
bloom_filter = NULL;
}
}
return ret;
}
void ObHashGroupByOp::destroy_all_parts()
{
if (NULL != mem_context_) {
while (!dumped_group_parts_.is_empty()) {
DatumStoreLinkPartition* p = dumped_group_parts_.remove_first();
if (NULL != p) {
p->~DatumStoreLinkPartition();
mem_context_->get_malloc_allocator().free(p);
}
}
}
}
int ObHashGroupByOp::restore_groupby_datum()
{
int ret = OB_SUCCESS;
ObAggregateProcessor::GroupRow* group_row = NULL;
if (OB_FAIL(aggr_processor_.get_group_row(curr_group_id_, group_row))) {
LOG_WARN("failed to get group_row", K(curr_group_id_), K(ret));
} else if (OB_ISNULL(group_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group_row or groupby_datums_ is null", K(curr_group_id_), KP(group_row), K(ret));
} else if (OB_FAIL(group_row->groupby_store_row_->to_expr(MY_SPEC.group_exprs_, eval_ctx_))) {
LOG_WARN("failed to convert store row to expr", K(ret));
}
LOG_DEBUG("finish restore_groupby_datum", K(MY_SPEC.group_exprs_), K(ret));
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,197 @@
/**
* 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_BASIC_OB_HASH_GROUPBY_OP_H_
#define OCEANBASE_BASIC_OB_HASH_GROUPBY_OP_H_
#include "common/row/ob_row_store.h"
#include "sql/engine/aggregate/ob_groupby_op.h"
#include "sql/engine/aggregate/ob_exec_hash_struct.h"
#include "lib/list/ob_list.h"
#include "lib/list/ob_dlink_node.h"
#include "sql/engine/ob_sql_mem_mgr_processor.h"
namespace oceanbase {
namespace sql {
class ObHashGroupBySpec : public ObGroupBySpec {
OB_UNIS_VERSION_V(1);
public:
ObHashGroupBySpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
: ObGroupBySpec(alloc, type), group_exprs_(alloc), cmp_funcs_(alloc), est_group_cnt_(0)
{}
DECLARE_VIRTUAL_TO_STRING;
inline int init_group_exprs(const int64_t count)
{
return group_exprs_.init(count);
}
int add_group_expr(ObExpr* expr);
inline void set_est_group_cnt(const int64_t cnt)
{
est_group_cnt_ = cnt;
}
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObHashGroupBySpec);
public:
ExprFixedArray group_exprs_; // group by column
ObCmpFuncs cmp_funcs_;
int64_t est_group_cnt_;
};
// input rows is already sorted by groupby columns
class ObHashGroupByOp : public ObGroupByOp {
public:
struct DatumStoreLinkPartition : public common::ObDLinkBase<DatumStoreLinkPartition> {
public:
DatumStoreLinkPartition(common::ObIAllocator* alloc = nullptr) : datum_store_(alloc), part_id_(0)
{}
ObChunkDatumStore datum_store_;
int64_t part_id_;
};
public:
static const int64_t MIN_PARTITION_CNT = 8;
static const int64_t MAX_PARTITION_CNT = 256;
// min in memory groups
static const int64_t MIN_INMEM_GROUPS = 4;
static const int64_t MIN_GROUP_HT_INIT_SIZE = 1 << 10; // 1024
static const int64_t MAX_GROUP_HT_INIT_SIZE = 1 << 20; // 1048576
static constexpr const double MAX_PART_MEM_RATIO = 0.5;
static constexpr const double EXTRA_MEM_RATIO = 0.25;
static const int64_t FIX_SIZE_PER_PART = sizeof(DatumStoreLinkPartition) + ObChunkRowStore::BLOCK_SIZE;
public:
ObHashGroupByOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObGroupByOp(exec_ctx, spec, input),
curr_group_id_(common::OB_INVALID_INDEX),
mem_context_(NULL),
agged_group_cnt_(0),
agged_row_cnt_(0),
agged_dumped_cnt_(0),
profile_(ObSqlWorkAreaType::HASH_WORK_AREA),
sql_mem_processor_(profile_),
iter_end_(false)
{}
void reset();
virtual int inner_open() override;
virtual int inner_close() override;
virtual int rescan() override;
virtual int switch_iterator() override;
virtual int inner_get_next_row() override;
virtual void destroy() override;
int load_data();
int check_same_group(int64_t& diff_pos);
int restore_groupby_datum(const int64_t diff_pos);
int rollup_and_calc_results(const int64_t group_id);
int64_t get_hash_groupby_row_count() const
{
return local_group_rows_.size();
}
OB_INLINE int64_t get_aggr_used_size() const
{
return aggr_processor_.get_aggr_used_size();
}
OB_INLINE int64_t get_aggr_hold_size() const
{
return aggr_processor_.get_aggr_hold_size();
}
OB_INLINE int64_t get_local_hash_used_size() const
{
return local_group_rows_.mem_used();
}
OB_INLINE int64_t get_dumped_part_used_size() const
{
return (NULL == mem_context_ ? 0 : mem_context_->used());
}
OB_INLINE int64_t get_dump_part_hold_size() const
{
return (NULL == mem_context_ ? 0 : mem_context_->hold());
}
OB_INLINE int64_t get_extra_size() const
{
return get_local_hash_used_size() + get_dumped_part_used_size();
}
OB_INLINE int64_t get_mem_used_size() const
{
return get_aggr_used_size() + get_extra_size();
}
OB_INLINE int64_t get_mem_bound_size() const
{
return sql_mem_processor_.get_mem_bound();
}
OB_INLINE int64_t estimate_hash_bucket_size(const int64_t bucket_cnt) const
{
return next_pow2(ObGroupRowHashTable::SIZE_BUCKET_SCALE * bucket_cnt) * sizeof(void*);
}
OB_INLINE int64_t estimate_hash_bucket_cnt_by_mem_size(
const int64_t bucket_cnt, const int64_t max_mem_size, const double extra_ratio) const
{
int64_t mem_size = estimate_hash_bucket_size(bucket_cnt);
int64_t max_hash_size = max_mem_size * extra_ratio;
if (0 < max_hash_size) {
while (mem_size > max_hash_size) {
mem_size >>= 1;
}
}
return (mem_size / sizeof(void*) / ObGroupRowHashTable::SIZE_BUCKET_SCALE);
}
int update_mem_status_periodically(
const int64_t nth_cnt, const int64_t input_row, int64_t& est_part_cnt, bool& need_dump);
int64_t detect_part_cnt(const int64_t rows) const;
void calc_data_mem_ratio(const int64_t part_cnt, double& data_ratio);
void adjust_part_cnt(int64_t& part_cnt);
int calc_groupby_exprs_hash(uint64_t& hash_value);
int init_group_row_item(const ObGroupRowItem& curr_item, ObGroupRowItem*& gr_row_item);
bool need_start_dump(const int64_t input_rows, int64_t& est_part_cnt, const bool check_dump);
// Setup: memory entity, bloom filter, spill partitions
int setup_dump_env(const int64_t part_id, const int64_t input_rows, DatumStoreLinkPartition** parts,
int64_t& part_cnt, ObGbyBloomFilter*& bloom_filter);
int cleanup_dump_env(const bool dump_success, const int64_t part_id, DatumStoreLinkPartition** parts,
int64_t& part_cnt, ObGbyBloomFilter*& bloom_filter);
void destroy_all_parts();
int restore_groupby_datum();
int init_mem_context(void);
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObHashGroupByOp);
private:
ObGroupRowHashTable local_group_rows_;
int64_t curr_group_id_;
// memory allocator for group by partitions
lib::MemoryContext* mem_context_;
ObDList<DatumStoreLinkPartition> dumped_group_parts_;
ObChunkDatumStore group_store_;
int64_t agged_group_cnt_;
int64_t agged_row_cnt_;
int64_t agged_dumped_cnt_;
ObSqlWorkAreaProfile profile_;
ObSqlMemMgrProcessor sql_mem_processor_;
bool iter_end_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_BASIC_OB_HASH_GROUPBY_OP_H_

View File

@ -0,0 +1,284 @@
/**
* 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 "lib/allocator/ob_malloc.h"
#include "lib/utility/utility.h"
#include "sql/engine/aggregate/ob_merge_distinct.h"
#include "common/row/ob_row.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
class ObMergeDistinct::ObMergeDistinctCtx : public ObPhyOperatorCtx {
public:
explicit ObMergeDistinctCtx(ObExecContext& ctx)
: ObPhyOperatorCtx(ctx),
got_first_row_(false),
last_row_(),
last_row_buf_(NULL),
last_row_buf_size_(OB_MAX_ROW_LENGTH)
{}
int alloc_last_row_buf(const int64_t column_count)
{
int ret = OB_SUCCESS;
int64_t row_size = column_count * sizeof(ObObj);
void* ptr = NULL;
if (OB_UNLIKELY(NULL == (last_row_buf_ = exec_ctx_.get_allocator().alloc(last_row_buf_size_)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc last row buffer failed", K_(last_row_buf_size));
} else if (OB_UNLIKELY(NULL == (ptr = exec_ctx_.get_allocator().alloc(row_size)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_ERROR("alloc last row cell memory failed", K(row_size));
} else {
last_row_.cells_ = new (ptr) common::ObObj[column_count];
last_row_.count_ = column_count;
}
return ret;
}
int store_last_row(const ObNewRow& row)
{
int ret = OB_SUCCESS;
ObDataBuffer data_buf(static_cast<char*>(last_row_buf_), OB_ROW_BUF_SIZE);
if (OB_UNLIKELY(row.is_invalid()) || OB_UNLIKELY(last_row_.is_invalid()) ||
OB_UNLIKELY(row.get_count() != last_row_.get_count())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid_argument", K(row), K(last_row_));
}
for (int64_t i = 0; OB_SUCC(ret) && i < row.get_count(); ++i) {
if (OB_FAIL(ob_write_obj(data_buf, row.get_cell(i), last_row_.cells_[i]))) {
LOG_WARN("write obj failed", K(ret), K(i), K(row));
}
}
// give second chance
if (OB_NOT_NULL(last_row_buf_) && OB_ALLOCATE_MEMORY_FAILED == ret && OB_MAX_ROW_LENGTH == last_row_buf_size_) {
exec_ctx_.get_allocator().free(last_row_buf_);
last_row_buf_size_ = OB_MAX_ROW_LENGTH_IN_MEMTABLE;
if (OB_ISNULL(last_row_buf_ = exec_ctx_.get_allocator().alloc(last_row_buf_size_))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
SQL_ENG_LOG(ERROR, "alloc last row buffer failed", K_(last_row_buf_size));
} else {
ret = store_last_row(row);
}
}
return ret;
}
void free_last_row_buf()
{
if (last_row_buf_) {
exec_ctx_.get_allocator().free(last_row_buf_);
last_row_buf_ = NULL;
}
if (last_row_.cells_) {
exec_ctx_.get_allocator().free(last_row_.cells_);
last_row_.cells_ = NULL;
}
}
virtual void destroy()
{
ObPhyOperatorCtx::destroy_base();
}
private:
static const int64_t OB_ROW_BUF_SIZE = OB_MAX_ROW_LENGTH;
private:
bool got_first_row_;
ObNewRow last_row_;
void* last_row_buf_;
int64_t last_row_buf_size_;
friend class ObMergeDistinct;
};
ObMergeDistinct::ObMergeDistinct(common::ObIAllocator& alloc) : ObDistinct(alloc)
{}
ObMergeDistinct::~ObMergeDistinct()
{}
void ObMergeDistinct::reset()
{
ObDistinct::reset();
}
void ObMergeDistinct::reuse()
{
ObDistinct::reuse();
}
int ObMergeDistinct::init_op_ctx(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObPhyOperatorCtx* op_ctx = NULL;
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObMergeDistinctCtx, ctx, get_id(), get_type(), op_ctx))) {
LOG_WARN("create physical operator context failed", K(ret));
} else if (OB_ISNULL(op_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("op_ctx is null", K(ret));
} else {
if (OB_FAIL(init_cur_row(*op_ctx, need_copy_row_for_compute()))) {
LOG_WARN("init current row failed", K(ret));
}
}
return ret;
}
int ObMergeDistinct::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObMergeDistinctCtx* distinct_ctx = NULL;
if (OB_FAIL(init_op_ctx(ctx))) {
LOG_WARN("initialize operator context failed", K(ret));
} else if (OB_ISNULL(child_op_) ||
OB_ISNULL(distinct_ctx = GET_PHY_OPERATOR_CTX(ObMergeDistinctCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get physical operator context failed", K_(child_op));
} else {
int64_t real_col_cnt = child_op_->get_output_count();
if (OB_FAIL(distinct_ctx->alloc_last_row_buf(real_col_cnt))) {
LOG_WARN("alloc last row buffer failed", K(ret), "column_count", real_col_cnt);
}
}
return ret;
}
int ObMergeDistinct::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObMergeDistinctCtx* distinct_ctx = NULL;
if (OB_FAIL(ObSingleChildPhyOperator::rescan(ctx))) {
LOG_WARN("rescan child operator failed", K(ret));
} else if (OB_ISNULL(distinct_ctx = GET_PHY_OPERATOR_CTX(ObMergeDistinctCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("distinct ctx is null");
} else {
distinct_ctx->got_first_row_ = false;
}
return ret;
}
int ObMergeDistinct::inner_close(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObMergeDistinctCtx* phy_op_ctx = NULL;
if (OB_ISNULL(phy_op_ctx = GET_PHY_OPERATOR_CTX(ObMergeDistinctCtx, ctx, get_id()))) {
LOG_DEBUG("The operator has not been opened.", K(ret), K_(id), "op_type", ob_phy_operator_type_str(get_type()));
} else {
phy_op_ctx->free_last_row_buf();
}
return ret;
}
int ObMergeDistinct::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObMergeDistinctCtx* phy_op_ctx = NULL;
const ObNewRow* this_row = NULL;
bool got_distinct_row = false;
bool equal = false;
if (OB_ISNULL(child_op_)) {
ret = OB_NOT_INIT;
LOG_WARN("child_op is null");
} else if (OB_ISNULL(phy_op_ctx = GET_PHY_OPERATOR_CTX(ObMergeDistinctCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get physical operator context failed", K(ret));
} else if (phy_op_ctx->got_first_row_) {
while (OB_SUCC(ret) && !got_distinct_row) {
if (OB_FAIL(child_op_->get_next_row(ctx, this_row))) {
if (ret != OB_ITER_END) {
LOG_WARN("child operator get next row failed", K(ret));
}
} else if (OB_ISNULL(this_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("this row is null");
} else {
// compare current_row and last_row
if (OB_FAIL(compare_equal(*this_row, phy_op_ctx->last_row_, equal))) {
LOG_WARN("fail to compare this_row and last_row_", K(ret), K(*this_row), K_(phy_op_ctx->last_row));
} else if (!equal) {
got_distinct_row = true;
/* return value */
row = this_row;
/* save this row to local buffer. last_row_buf_ reused */
if (OB_FAIL(phy_op_ctx->store_last_row(*this_row))) {
LOG_WARN("fail to storage row to operator context", K(ret), K(*this_row));
}
}
}
} /* end while */
} else { /* first row, always output */
if (OB_FAIL(child_op_->get_next_row(ctx, row))) {
if (ret != OB_ITER_END) {
LOG_WARN("fail to get next row", K(ret));
}
} else if (OB_FAIL(phy_op_ctx->store_last_row(*row))) {
LOG_WARN("fail to storage row to operator context", K(ret), K(*row));
} else {
phy_op_ctx->got_first_row_ = true;
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(copy_cur_row(*phy_op_ctx, row))) {
LOG_WARN("copy current row failed", K(ret));
}
}
return ret;
}
int ObMergeDistinct::compare_equal(const ObNewRow& this_row, const ObNewRow& last_row, bool& result) const
{
int ret = OB_SUCCESS;
int64_t column_count = 0;
int64_t col_index = 0;
ObCollationType cs_type = CS_TYPE_INVALID;
bool cmp_val = true;
result = false;
if (OB_UNLIKELY(this_row.is_invalid()) || OB_UNLIKELY(last_row.is_invalid())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("compared row invalid", K(this_row), K(last_row));
} else {
column_count = distinct_columns_.count();
for (int64_t i = 0; OB_SUCC(ret) && cmp_val && i < column_count; ++i) {
col_index = distinct_columns_.at(i).index_;
cs_type = distinct_columns_.at(i).cs_type_;
if (col_index < 0 || col_index >= this_row.get_count() || col_index >= last_row.get_count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("col_index is invalid", K(col_index), K(this_row.get_count()), K(last_row.get_count()));
} else if (this_row.get_cell(col_index).compare(last_row.get_cell(col_index), cs_type) != 0) {
cmp_val = false;
}
} /* end for */
}
if (OB_SUCC(ret)) {
result = cmp_val;
}
return ret;
}
int64_t ObMergeDistinct::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV(N_DISTINCT, distinct_columns_);
return pos;
}
OB_SERIALIZE_MEMBER((ObMergeDistinct, ObSingleChildPhyOperator), distinct_columns_);
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,77 @@
/**
* 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_MERGE_DISTINCT_H
#define _OB_MERGE_DISTINCT_H
#include "lib/string/ob_string.h"
#include "share/ob_define.h"
#include "lib/container/ob_fixed_array.h"
#include "common/row/ob_row.h"
#include "sql/engine/ob_single_child_phy_operator.h"
#include "sql/engine/aggregate/ob_distinct.h"
namespace oceanbase {
namespace sql {
class ObMergeDistinct : public ObDistinct {
OB_UNIS_VERSION_V(1);
private:
class ObMergeDistinctCtx;
public:
explicit ObMergeDistinct(common::ObIAllocator& alloc);
// ObMergeDistinct();
virtual ~ObMergeDistinct();
virtual void reset();
virtual void reuse();
virtual int rescan(ObExecContext& ctx) const;
private:
// member function
int compare_equal(const common::ObNewRow& this_row, const common::ObNewRow& last_row, bool& result) const;
/**
* @brief init operator context, will create a physical operator context (and a current row space)
* @param ctx[in], execute context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int init_op_ctx(ObExecContext& ctx) const;
/**
* @brief: called by get_next_row(), get a row from the child operator or row_store
* @param: ctx[in], execute context
* @param: row[out], ObSqlRow an obj array and row_size
*/
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
/**
* @brief open operator, not including children operators.
* called by open.
* Every op should implement this method.
*/
virtual int inner_open(ObExecContext& ctx) const;
/**
* @brief close operator, not including children operators.
* Every op should implement this method.
*/
virtual int inner_close(ObExecContext& ctx) const;
/**
* @brief for specified phy operator to print it's member variable with json key-value format
* @param buf[in] to string buffer
* @param buf_len[in] buffer length
* @return if success, return the length used by print string, otherwise return 0
*/
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObMergeDistinct);
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_MERGE_DISTINCT_H */

View File

@ -0,0 +1,164 @@
/**
* 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/aggregate/ob_merge_distinct_op.h"
#include "sql/engine/px/ob_px_util.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
ObMergeDistinctSpec::ObMergeDistinctSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
: ObDistinctSpec(alloc, type)
{}
OB_SERIALIZE_MEMBER((ObMergeDistinctSpec, ObDistinctSpec));
ObMergeDistinctOp::ObMergeDistinctOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObOperator(exec_ctx, spec, input),
first_got_row_(true),
alloc_(ObModIds::OB_SQL_MERGE_GROUPBY, OB_MALLOC_NORMAL_BLOCK_SIZE, OB_SERVER_TENANT_ID, ObCtxIds::WORK_AREA),
last_row_(alloc_)
{}
int ObMergeDistinctOp::inner_open()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(child_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("child is null", K(ret));
} else if (MY_SPEC.is_block_mode_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("merge distinct not support block mode", K(ret));
} else {
last_row_.reuse_ = true;
}
return ret;
}
int ObMergeDistinctOp::rescan()
{
int ret = OB_SUCCESS;
first_got_row_ = true;
last_row_.reset();
alloc_.reset();
if (OB_FAIL(ObOperator::rescan())) {}
return ret;
}
int ObMergeDistinctOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
bool got_distinct_row = false;
bool equal = false;
if (!first_got_row_) {
while (OB_SUCC(ret) && !got_distinct_row) {
clear_evaluated_flag();
if (OB_FAIL(child_->get_next_row())) {
if (ret != OB_ITER_END) {
LOG_WARN("child operator get next row failed", K(ret));
}
} else {
// compare current_row and last_row
if (!cmp_.equal(&MY_SPEC.distinct_exprs_, last_row_.store_row_)) {
got_distinct_row = true;
ret = cmp_.ret_code_;
/* save this row to local buffer. last_row_buf_ reused */
if (OB_SUCC(ret) && OB_FAIL(last_row_.save_store_row(MY_SPEC.distinct_exprs_, eval_ctx_, 0))) {
LOG_WARN("failed to storage row to operator context", K(ret));
}
}
}
} /* end while */
} else { /* first row, always output */
if (OB_FAIL(cmp_.init(&eval_ctx_, &MY_SPEC.cmp_funcs_))) {
LOG_WARN("failed to init compare functions", K(ret));
} else if (OB_FAIL(child_->get_next_row())) {
if (ret != OB_ITER_END) {
LOG_WARN("failed to get next row", K(ret));
}
} else if (OB_FAIL(last_row_.save_store_row(MY_SPEC.distinct_exprs_, eval_ctx_, 0))) {
LOG_WARN("failed to store row", K(ret));
} else {
first_got_row_ = false;
}
}
return ret;
}
void ObMergeDistinctOp::destroy()
{
last_row_.reset();
alloc_.reset();
ObOperator::destroy();
}
int ObMergeDistinctOp::inner_close()
{
int ret = OB_SUCCESS;
last_row_.reset();
alloc_.reset();
if (OB_FAIL(ObOperator::inner_close())) {}
return ret;
}
int ObMergeDistinctOp::Compare::init(ObEvalCtx* eval_ctx, const ObIArray<ObCmpFunc>* cmp_funcs)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(eval_ctx) || OB_ISNULL(cmp_funcs)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to init compare funcs", K(ret));
} else {
eval_ctx_ = eval_ctx;
cmp_funcs_ = cmp_funcs;
ret_code_ = OB_SUCCESS;
}
return ret;
}
bool ObMergeDistinctOp::Compare::equal(const ObIArray<ObExpr*>* l, const ObChunkDatumStore::StoredRow* r)
{
int ret = OB_SUCCESS;
bool equal = false;
if (OB_ISNULL(l) || OB_ISNULL(eval_ctx_) || OB_ISNULL(cmp_funcs_)) {
ret = OB_ERR_UNEXPECTED;
} else if (OB_ISNULL(r)) {
if (0 != l->count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected status: it must be distinct const", K(ret));
} else {
// case: select distinct 1 from t1;
equal = true;
}
} else {
const ObDatum* rcells = r->cells();
int cmp = 0;
ObDatum* other_datum = nullptr;
for (int64_t i = 0; 0 == cmp && i < cmp_funcs_->count() && OB_SUCC(ret); i++) {
if (OB_FAIL(l->at(i)->eval(*eval_ctx_, other_datum))) {
LOG_WARN("failed to get expr value", K(ret), K(i));
} else {
cmp = cmp_funcs_->at(i).cmp_func_(*other_datum, rcells[i]);
}
}
equal = OB_SUCC(ret) ? (0 == cmp) : false;
}
ret_code_ = ret;
return equal;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,63 @@
/**
* 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_SRC_SQL_ENGINE_AGGREGATE_OB_MERGE_DISTINCT_OP_H_
#define OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_MERGE_DISTINCT_OP_H_
#include "sql/engine/ob_operator.h"
#include "sql/engine/aggregate/ob_distinct_op.h"
#include "sql/engine/basic/ob_chunk_datum_store.h"
namespace oceanbase {
namespace sql {
class ObMergeDistinctSpec : public ObDistinctSpec {
OB_UNIS_VERSION_V(1);
public:
ObMergeDistinctSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
};
class ObMergeDistinctOp : public ObOperator {
public:
ObMergeDistinctOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
virtual int inner_open() override;
virtual int inner_close() override;
virtual int rescan() override;
virtual int inner_get_next_row() override;
virtual void destroy() override;
class Compare {
public:
Compare() : eval_ctx_(nullptr), cmp_funcs_(nullptr), ret_code_(common::OB_SUCCESS)
{}
int init(ObEvalCtx* eval_ctx, const common::ObIArray<ObCmpFunc>* cmp_funcs);
bool equal(const common::ObIArray<ObExpr*>* l, const ObChunkDatumStore::StoredRow* r);
ObEvalCtx* eval_ctx_;
const common::ObIArray<ObCmpFunc>* cmp_funcs_;
int ret_code_;
};
private:
typedef ObChunkDatumStore::LastStoredRow<ObChunkDatumStore::StoredRow> LastStoreRow;
bool first_got_row_;
common::ObArenaAllocator alloc_;
LastStoreRow last_row_;
Compare cmp_;
};
} // end namespace sql
} // end namespace oceanbase
#endif /* OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_MERGE_DISTINCT_OP_H_ */

View File

@ -0,0 +1,241 @@
/**
* 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/aggregate/ob_merge_groupby.h"
#include "lib/utility/utility.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
// REGISTER_PHY_OPERATOR(ObMergeGroupBy, PHY_MERGE_GROUP_BY);
class ObMergeGroupBy::ObMergeGroupByCtx : public ObGroupByCtx {
public:
explicit ObMergeGroupByCtx(ObExecContext& exec_ctx)
: ObGroupByCtx(exec_ctx), last_input_row_(NULL), is_end_(false), cur_output_group_id(-1), first_output_group_id(0)
{}
virtual void destroy()
{
ObGroupByCtx::destroy();
}
private:
DISALLOW_COPY_AND_ASSIGN(ObMergeGroupByCtx);
private:
const ObNewRow* last_input_row_;
bool is_end_;
// added to support groupby with rollup
int64_t cur_output_group_id;
int64_t first_output_group_id;
friend class ObMergeGroupBy;
};
ObMergeGroupBy::ObMergeGroupBy(ObIAllocator& alloc) : ObGroupBy(alloc)
{}
ObMergeGroupBy::~ObMergeGroupBy()
{}
int ObMergeGroupBy::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
{
return CREATE_PHY_OPERATOR_CTX(ObMergeGroupByCtx, ctx, get_id(), get_type(), op_ctx);
}
int ObMergeGroupBy::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObGroupBy::init_group_by(ctx))) {
LOG_WARN("init group by failed", K(ret));
} else {
ObMergeGroupByCtx* groupby_ctx = GET_PHY_OPERATOR_CTX(ObMergeGroupByCtx, ctx, get_id());
if (OB_ISNULL(groupby_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group by ctx is NULL", K(ret));
} else {
groupby_ctx->get_aggr_func().set_sort_based_gby();
}
}
return ret;
}
int ObMergeGroupBy::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObMergeGroupByCtx* groupby_ctx = NULL;
if (OB_FAIL(ObGroupBy::rescan(ctx))) {
LOG_WARN("rescan child operator failed", K(ret));
} else if (OB_ISNULL(groupby_ctx = GET_PHY_OPERATOR_CTX(ObMergeGroupByCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("groupby_ctx is null");
} else {
groupby_ctx->aggr_func_.reuse();
groupby_ctx->last_input_row_ = NULL;
groupby_ctx->is_end_ = false;
groupby_ctx->cur_output_group_id = -1;
groupby_ctx->first_output_group_id = 0;
}
return ret;
}
int ObMergeGroupBy::inner_close(ObExecContext& ctx) const
{
UNUSED(ctx);
int ret = OB_SUCCESS;
return ret;
}
int ObMergeGroupBy::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
ObMergeGroupByCtx* groupby_ctx = NULL;
int64_t stop_output_group_id = group_col_idxs_.count();
int64_t col_count = group_col_idxs_.count() + rollup_col_idxs_.count();
if (OB_ISNULL(child_op_) || OB_ISNULL(groupby_ctx = GET_PHY_OPERATOR_CTX(ObMergeGroupByCtx, ctx, get_id()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get ObMergeGroupByCtx failed", K_(child_op));
} else if (has_rollup_ && groupby_ctx->cur_output_group_id >= groupby_ctx->first_output_group_id &&
groupby_ctx->cur_output_group_id >= stop_output_group_id) {
// output roll-up results here
if (OB_FAIL(rollup_and_calc_results(groupby_ctx->cur_output_group_id, groupby_ctx))) {
LOG_WARN("failed to rollup and calculate results", K(groupby_ctx->cur_output_group_id), K(ret));
} else {
--groupby_ctx->cur_output_group_id;
}
} else {
// output group by results here
if (groupby_ctx->is_end_) {
ret = OB_ITER_END;
} else if (NULL == groupby_ctx->last_input_row_) {
// get the first input row
if (OB_FAIL(child_op_->get_next_row(ctx, groupby_ctx->last_input_row_))) {
if (ret != OB_ITER_END) {
LOG_WARN("failed to get next row", K(ret));
}
} else if (has_rollup_ && OB_FAIL(groupby_ctx->aggr_func_.rollup_init(col_count))) {
LOG_WARN("failed to initialize roll up", K(ret));
}
}
int64_t group_id = 0;
if (OB_SUCC(ret) && has_rollup_) {
group_id = col_count;
}
if (OB_SUCC(ret) && groupby_ctx->last_input_row_ != NULL) {
if (OB_FAIL(groupby_ctx->aggr_func_.prepare(*groupby_ctx->last_input_row_, group_id))) {
LOG_WARN("failed to init aggr cells", K(ret));
} else {
groupby_ctx->last_input_row_ = NULL;
}
}
if (OB_SUCC(ret)) {
bool same_group = false;
const ObNewRow* input_row = NULL;
const ObRowStore::StoredRow* stored_row = NULL;
ObSQLSessionInfo* my_session = groupby_ctx->exec_ctx_.get_my_session();
const ObTimeZoneInfo* tz_info = (my_session != NULL) ? my_session->get_timezone_info() : NULL;
bool is_break = false;
int64_t first_diff_pos = 0;
while (OB_SUCC(ret) && !is_break && OB_SUCC(child_op_->get_next_row(ctx, input_row))) {
if (OB_FAIL(try_check_status(ctx))) {
LOG_WARN("check status failed", K(ret));
} else if (OB_FAIL(groupby_ctx->aggr_func_.get_cur_row(stored_row, group_id))) {
LOG_WARN("fail to get cur row from aggr_func", K(ret));
} else if (OB_ISNULL(stored_row) || OB_ISNULL(input_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("the stored row is NULL");
} else if (OB_FAIL(is_same_group(*stored_row, *input_row, same_group, first_diff_pos))) {
LOG_WARN("failed to check group", K(ret));
} else if (same_group) {
if (OB_FAIL(groupby_ctx->aggr_func_.process(*input_row, tz_info, group_id))) {
LOG_WARN("failed to calc aggr", K(ret));
} else if (mem_size_limit_ > 0 && mem_size_limit_ < groupby_ctx->aggr_func_.get_used_mem_size()) {
ret = OB_EXCEED_MEM_LIMIT;
LOG_WARN("merge group by has exceeded the mem limit",
K_(mem_size_limit),
"aggr mem size",
groupby_ctx->aggr_func_.get_used_mem_size());
}
} else if (OB_FAIL(rollup_and_calc_results(group_id, groupby_ctx))) {
// ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to rollup and calculate results", K(group_id), K(ret));
} else {
groupby_ctx->last_input_row_ = input_row;
is_break = true;
if (has_rollup_) {
groupby_ctx->first_output_group_id = first_diff_pos + 1;
groupby_ctx->cur_output_group_id = group_id - 1;
}
}
} // end while
if (OB_ITER_END == ret) {
// the last group
groupby_ctx->is_end_ = true;
if (OB_FAIL(rollup_and_calc_results(group_id, groupby_ctx))) {
LOG_WARN("failed to rollup and calculate results", K(group_id), K(ret));
} else if (has_rollup_) {
groupby_ctx->first_output_group_id = 0;
groupby_ctx->cur_output_group_id = group_id - 1;
}
}
}
}
if (OB_SUCC(ret)) {
row = &groupby_ctx->get_cur_row();
}
return ret;
}
int ObMergeGroupBy::rollup_and_calc_results(const int64_t group_id, ObMergeGroupByCtx* groupby_ctx) const
{
int ret = OB_SUCCESS;
int64_t col_count = group_col_idxs_.count() + rollup_col_idxs_.count();
if (OB_UNLIKELY(OB_ISNULL(groupby_ctx) || group_id < 0 || group_id > col_count)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(group_id), K(ret));
} else {
ObSQLSessionInfo* my_session = groupby_ctx->exec_ctx_.get_my_session();
const ObTimeZoneInfo* tz_info = (my_session != NULL) ? my_session->get_timezone_info() : NULL;
// for: SELECT GROUPING(z0_test0) FROM Z0CASE GROUP BY z0_test0, ROLLUP(z0_test0);
bool set_grouping = true;
if (group_id > group_col_idxs_.count()) {
int64_t rollup_id = rollup_col_idxs_[group_id - group_col_idxs_.count() - 1].index_;
for (int64_t i = 0; set_grouping && i < group_col_idxs_.count(); ++i) {
if (rollup_id == group_col_idxs_[i].index_) {
set_grouping = false;
}
}
}
if (has_rollup_ && group_id > 0 &&
OB_FAIL(groupby_ctx->aggr_func_.rollup_process(tz_info,
group_id - 1,
group_id,
group_id <= group_col_idxs_.count() ? group_col_idxs_[group_id - 1].index_
: rollup_col_idxs_[group_id - group_col_idxs_.count() - 1].index_,
set_grouping))) {
// ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to rollup aggregation results", K(ret));
} else if (OB_FAIL(groupby_ctx->aggr_func_.get_result(groupby_ctx->get_cur_row(), tz_info, group_id))) {
LOG_WARN("failed to get aggr result", K(group_id), K(ret));
} else if (OB_FAIL(groupby_ctx->aggr_func_.reuse_group(group_id))) {
LOG_WARN("failed to reuse group", K(group_id), K(ret));
} else {
// do nothing
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -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.
*/
#ifndef _OB_MERGE_GROUPBY_H
#define _OB_MERGE_GROUPBY_H
#include "sql/engine/aggregate/ob_groupby.h"
namespace oceanbase {
namespace sql {
class ObMergeGroupBy : public ObGroupBy {
protected:
class ObMergeGroupByCtx;
public:
explicit ObMergeGroupBy(common::ObIAllocator& alloc);
virtual ~ObMergeGroupBy();
virtual int rescan(ObExecContext& ctx) const;
private:
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
/**
* @brief open operator, not including children operators.
* called by open.
* Every op should implement this method.
*/
virtual int inner_open(ObExecContext& ctx) const;
/**
* @brief close operator, not including children operators.
* Every op should implement this method.
*/
virtual int inner_close(ObExecContext& ctx) const;
/**
* @brief create operator context, only child operator can know it's specific operator type,
* so must be overwrited by child operator,
* @param ctx[in], execute context
* @param op_ctx[out], the pointer of operator context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
int rollup_and_calc_results(const int64_t group_id, ObMergeGroupByCtx* groupby_ctx) const;
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObMergeGroupBy);
};
} // end namespace sql
} // end namespace oceanbase
#endif /* _OB_MERGE_GROUPBY_H */

View File

@ -0,0 +1,438 @@
/**
* 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/aggregate/ob_merge_groupby_op.h"
#include "lib/number/ob_number_v2.h"
namespace oceanbase {
namespace sql {
OB_SERIALIZE_MEMBER(
(ObMergeGroupBySpec, ObGroupBySpec), group_exprs_, rollup_exprs_, is_distinct_rollup_expr_, has_rollup_);
DEF_TO_STRING(ObMergeGroupBySpec)
{
int64_t pos = 0;
J_OBJ_START();
J_NAME("groupby_spec");
J_COLON();
pos += ObGroupBySpec::to_string(buf + pos, buf_len - pos);
J_COMMA();
J_KV(K_(group_exprs), K_(rollup_exprs), K_(is_distinct_rollup_expr), K_(has_rollup));
J_OBJ_END();
return pos;
}
int ObMergeGroupBySpec::add_group_expr(ObExpr* expr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(expr)) {
ret = OB_INVALID_ARGUMENT;
OB_LOG(WARN, "invalid argument", K(ret));
} else if (OB_FAIL(group_exprs_.push_back(expr))) {
LOG_ERROR("failed to push_back expr");
}
return ret;
}
int ObMergeGroupBySpec::add_rollup_expr(ObExpr* expr)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(expr)) {
ret = OB_INVALID_ARGUMENT;
OB_LOG(WARN, "invalid argument", K(ret));
} else if (OB_FAIL(rollup_exprs_.push_back(expr))) {
LOG_ERROR("failed to push_back expr");
}
return ret;
}
void ObMergeGroupByOp::reset()
{
is_end_ = false;
cur_output_group_id_ = OB_INVALID_INDEX;
first_output_group_id_ = 0;
last_child_output_.reset();
curr_groupby_datums_.reset();
}
int ObMergeGroupByOp::init()
{
int ret = OB_SUCCESS;
const int64_t col_count =
(MY_SPEC.has_rollup_ ? (MY_SPEC.group_exprs_.count() + MY_SPEC.rollup_exprs_.count() + 1) : 0);
if (OB_FAIL(aggr_processor_.init_group_rows(col_count))) {
LOG_WARN("failed to initialize init_group_rows", K(ret));
} else if (OB_FAIL(
curr_groupby_datums_.prepare_allocate(MY_SPEC.group_exprs_.count() + MY_SPEC.rollup_exprs_.count()))) {
LOG_WARN("failed to initialize curr_groupby_datums_", K(ret));
} else {
last_child_output_.reuse_ = true;
}
return ret;
}
int ObMergeGroupByOp::inner_open()
{
int ret = OB_SUCCESS;
reset();
if (OB_FAIL(ObGroupByOp::inner_open())) {
LOG_WARN("failed to inner_open", K(ret));
} else if (OB_FAIL(init())) {
LOG_WARN("failed to init", K(ret));
}
return ret;
}
int ObMergeGroupByOp::inner_close()
{
reset();
return ObGroupByOp::inner_close();
}
void ObMergeGroupByOp::destroy()
{
reset();
ObGroupByOp::destroy();
}
int ObMergeGroupByOp::switch_iterator()
{
int ret = OB_SUCCESS;
reset();
if (OB_FAIL(ObGroupByOp::switch_iterator())) {
LOG_WARN("failed to switch_iterator", K(ret));
} else if (OB_FAIL(init())) {
LOG_WARN("failed to init", K(ret));
}
return ret;
}
int ObMergeGroupByOp::rescan()
{
int ret = OB_SUCCESS;
reset();
if (OB_FAIL(ObGroupByOp::rescan())) {
LOG_WARN("failed to rescan", K(ret));
} else if (OB_FAIL(init())) {
LOG_WARN("failed to init", K(ret));
}
return ret;
}
int ObMergeGroupByOp::rewrite_rollup_column(ObExpr*& diff_expr)
{
int ret = OB_SUCCESS;
// output roll-up results here
const bool is_distinct_expr =
(cur_output_group_id_ < MY_SPEC.group_exprs_.count()
? true
: MY_SPEC.is_distinct_rollup_expr_.at(cur_output_group_id_ - MY_SPEC.group_exprs_.count()));
if (!is_distinct_expr) {
ObExpr* null_expr =
const_cast<ObExpr*>(cur_output_group_id_ < MY_SPEC.group_exprs_.count()
? MY_SPEC.group_exprs_[cur_output_group_id_]
: MY_SPEC.rollup_exprs_[cur_output_group_id_ - MY_SPEC.group_exprs_.count()]);
null_expr->locate_expr_datum(eval_ctx_).set_null();
null_expr->get_eval_info(eval_ctx_).evaluated_ = true;
}
diff_expr = const_cast<ObExpr*>(cur_output_group_id_ < MY_SPEC.group_exprs_.count()
? MY_SPEC.group_exprs_[cur_output_group_id_]
: MY_SPEC.rollup_exprs_[cur_output_group_id_ - MY_SPEC.group_exprs_.count()]);
// for SELECT GROUPING(z0_test0) FROM Z0CASE GROUP BY z0_test0, ROLLUP(z0_test0);
if (cur_output_group_id_ >= MY_SPEC.group_exprs_.count()) {
for (int64_t i = 0; diff_expr != NULL && i < MY_SPEC.group_exprs_.count(); ++i) {
if (MY_SPEC.group_exprs_[i] == diff_expr) {
diff_expr = NULL;
}
}
}
return ret;
}
int ObMergeGroupByOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
const int64_t stop_output_group_id = MY_SPEC.group_exprs_.count();
const int64_t col_count = MY_SPEC.group_exprs_.count() + MY_SPEC.rollup_exprs_.count();
const int64_t group_id = MY_SPEC.has_rollup_ ? col_count : 0;
LOG_DEBUG("before inner_get_next_row",
"aggr_hold_size",
aggr_processor_.get_aggr_hold_size(),
"aggr_used_size",
aggr_processor_.get_aggr_used_size());
if (MY_SPEC.has_rollup_ && cur_output_group_id_ >= first_output_group_id_ &&
cur_output_group_id_ >= stop_output_group_id) {
ObExpr* diff_expr = NULL;
if (OB_FAIL(rewrite_rollup_column(diff_expr))) {
LOG_WARN("failed to rewrite_rollup_column", K(ret));
} else if (OB_FAIL(rollup_and_calc_results(cur_output_group_id_, diff_expr))) {
LOG_WARN("failed to rollup and calculate results", K(cur_output_group_id_), K(ret));
} else {
--cur_output_group_id_;
LOG_DEBUG("finish ouput rollup row", K(cur_output_group_id_), K(first_output_group_id_), K(ret));
}
} else if (is_end_) {
ret = OB_ITER_END;
} else {
// output group results here
bool is_break = false;
int64_t first_diff_pos = OB_INVALID_INDEX;
ObAggregateProcessor::GroupRow* group_row = NULL;
if (OB_FAIL(aggr_processor_.get_group_row(group_id, group_row))) {
LOG_WARN("failed to get_group_row", K(ret));
} else if (OB_ISNULL(group_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group_row is null", K(ret));
} else if (NULL != last_child_output_.store_row_) {
if (OB_FAIL(last_child_output_.store_row_->to_expr(child_->get_spec().output_, eval_ctx_))) {
LOG_WARN("Failed to get next row", K(ret));
}
} else {
if (OB_FAIL(child_->get_next_row())) {
if (ret != OB_ITER_END) {
LOG_WARN("failed to get next row", K(ret));
}
}
}
LOG_DEBUG("finish merge prepare 1", K(child_->get_spec().output_), KPC(last_child_output_.store_row_));
if (OB_SUCC(ret)) {
clear_evaluated_flag();
if (OB_FAIL(aggr_processor_.prepare(*group_row))) {
LOG_WARN("failed to prepare", K(ret));
} else if (OB_FAIL(prepare_curr_groupby_datum())) {
LOG_WARN("failed to prepare_curr_groupby_datum", K(ret));
}
while (OB_SUCC(ret) && !is_break && OB_SUCC(child_->get_next_row())) {
clear_evaluated_flag();
if (OB_FAIL(try_check_status())) {
LOG_WARN("check status failed", K(ret));
} else if (OB_FAIL(check_same_group(first_diff_pos))) {
LOG_WARN("failed to check group", K(ret));
} else if (OB_INVALID_INDEX == first_diff_pos) {
// same group
if (OB_FAIL(aggr_processor_.process(*group_row))) {
LOG_WARN("failed to calc aggr", K(ret));
}
} else {
// different group
if (OB_FAIL(last_child_output_.save_store_row(child_->get_spec().output_, eval_ctx_, 0))) {
LOG_WARN("failed to store child output", K(ret));
} else if (OB_FAIL(restore_groupby_datum(first_diff_pos))) {
LOG_WARN("failed to restore_groupby_datum", K(ret));
} else if (OB_FAIL(rollup_and_calc_results(group_id))) {
LOG_WARN("failed to rollup and calculate results", K(group_id), K(ret));
} else {
is_break = true;
if (MY_SPEC.has_rollup_) {
first_output_group_id_ = first_diff_pos + 1;
cur_output_group_id_ = group_id - 1;
}
}
}
} // end while
if (OB_ITER_END == ret) {
// the last group
is_end_ = true;
if (OB_FAIL(restore_groupby_datum(0))) {
LOG_WARN("failed to restore_groupby_datum", K(ret));
} else if (OB_FAIL(rollup_and_calc_results(group_id))) {
LOG_WARN("failed to rollup and calculate results", K(group_id), K(ret));
} else {
if (MY_SPEC.has_rollup_) {
first_output_group_id_ = 0;
cur_output_group_id_ = group_id - 1;
}
}
LOG_DEBUG("finish iter end", K(first_output_group_id_), K(cur_output_group_id_));
}
}
}
LOG_TRACE("after inner_get_next_row",
"aggr_hold_size",
aggr_processor_.get_aggr_hold_size(),
"aggr_used_size",
aggr_processor_.get_aggr_used_size());
return ret;
}
int ObMergeGroupByOp::prepare_curr_groupby_datum()
{
int ret = OB_SUCCESS;
ExprFixedArray& group_exprs = const_cast<ExprFixedArray&>(MY_SPEC.group_exprs_);
ExprFixedArray& rollup_exprs = const_cast<ExprFixedArray&>(MY_SPEC.rollup_exprs_);
const int64_t group_count = group_exprs.count();
const int64_t rollup_count = rollup_exprs.count();
if (OB_UNLIKELY(curr_groupby_datums_.count() != (group_count + rollup_count))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("last_groupby_datum is unexpected ",
"count",
curr_groupby_datums_.count(),
K(group_count),
K(rollup_count),
K(ret));
}
for (int64_t i = 0; OB_SUCC(ret) && i < group_count; ++i) {
ObDatum& last_datum = curr_groupby_datums_.at(i);
ObDatum* result = NULL;
ObExpr* expr = group_exprs.at(i);
if (OB_FAIL(expr->eval(eval_ctx_, result))) {
LOG_WARN("eval failed", K(ret));
} else if (OB_FAIL(aggr_processor_.clone_cell(last_datum, *result))) {
LOG_WARN("clone_cell failed", K(ret));
} else {
LOG_DEBUG("clone_cell succ", K(i), KPC(result), K(last_datum));
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < rollup_count; ++i) {
ObDatum& last_datum = curr_groupby_datums_.at(i + group_count);
ObDatum* result = NULL;
ObExpr* expr = rollup_exprs.at(i);
if (OB_FAIL(expr->eval(eval_ctx_, result))) {
LOG_WARN("eval failed", K(ret));
} else if (OB_FAIL(aggr_processor_.clone_cell(last_datum, *result))) {
LOG_WARN("clone_cell failed", K(ret));
} else {
LOG_DEBUG("clone_cell succ", K(i), KPC(result), K(last_datum));
}
}
LOG_DEBUG("finish prepare_curr_groupby_datum", K(curr_groupby_datums_), K(group_exprs), K(rollup_exprs), K(ret));
return ret;
}
int ObMergeGroupByOp::check_same_group(int64_t& diff_pos)
{
int ret = OB_SUCCESS;
diff_pos = OB_INVALID_INDEX;
ExprFixedArray& group_exprs = const_cast<ExprFixedArray&>(MY_SPEC.group_exprs_);
ExprFixedArray& rollup_exprs = const_cast<ExprFixedArray&>(MY_SPEC.rollup_exprs_);
const int64_t group_count = group_exprs.count();
const int64_t rollup_count = rollup_exprs.count();
ObDatum* result = NULL;
for (int64_t i = 0; OB_SUCC(ret) && i < group_count; ++i) {
const ObDatum& last_datum = curr_groupby_datums_.at(i);
ObExpr* expr = group_exprs.at(i);
if (OB_ISNULL(expr) || OB_ISNULL(expr->basic_funcs_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr node is null", K(ret));
} else if (OB_FAIL(expr->eval(eval_ctx_, result))) {
LOG_WARN("eval failed", K(ret));
} else if (last_datum.is_null() && result->is_null()) {
continue;
} else if (last_datum.is_null() || result->is_null() ||
expr->basic_funcs_->null_first_cmp_(last_datum, *result) != 0) {
diff_pos = i;
break;
}
}
// need a new expr for distinct??@
if (OB_INVALID_INDEX == diff_pos) {
result = NULL;
for (int64_t i = 0; OB_SUCC(ret) && i < rollup_exprs.count(); ++i) {
const ObDatum& last_datum = curr_groupby_datums_.at(i + group_count);
ObExprCmpFuncType cmp_func = rollup_exprs.at(i)->basic_funcs_->null_first_cmp_;
ObExpr* expr = rollup_exprs.at(i);
if (OB_ISNULL(expr) || OB_ISNULL(expr->basic_funcs_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("expr node is null", K(ret));
} else if (OB_FAIL(expr->eval(eval_ctx_, result))) {
LOG_WARN("eval failed", K(ret));
} else if (last_datum.is_null() && result->is_null()) {
continue;
} else if (last_datum.is_null() || result->is_null() ||
expr->basic_funcs_->null_first_cmp_(last_datum, *result) != 0) {
diff_pos = i + group_count;
break;
}
}
}
LOG_DEBUG("finish check_same_group",
K(diff_pos),
K(curr_groupby_datums_),
KPC(result),
K(group_exprs),
K(rollup_exprs),
K(ret));
return ret;
}
int ObMergeGroupByOp::restore_groupby_datum(const int64_t diff_pos)
{
int ret = OB_SUCCESS;
ExprFixedArray& group_exprs = const_cast<ExprFixedArray&>(MY_SPEC.group_exprs_);
ExprFixedArray& rollup_exprs = const_cast<ExprFixedArray&>(MY_SPEC.rollup_exprs_);
const int64_t group_count = group_exprs.count();
const int64_t rollup_count = rollup_exprs.count();
for (int64_t i = 0; OB_SUCC(ret) && i < group_count; ++i) {
ObDatum& last_datum = curr_groupby_datums_.at(i);
ObExpr* expr = group_exprs.at(i);
ObDatum& result = expr->locate_expr_datum(eval_ctx_);
result.set_datum(last_datum);
expr->get_eval_info(eval_ctx_).evaluated_ = true;
LOG_DEBUG("succ to restore", K(i), KPC(expr), K(result), K(last_datum));
}
for (int64_t i = std::max(0L, diff_pos - group_count); OB_SUCC(ret) && i < rollup_count; ++i) {
ObDatum& last_datum = curr_groupby_datums_.at(i + group_count);
ObExpr* expr = rollup_exprs.at(i);
ObDatum& result = expr->locate_expr_datum(eval_ctx_);
result.set_datum(last_datum);
expr->get_eval_info(eval_ctx_).evaluated_ = true;
LOG_DEBUG("succ to restore", K(i), KPC(expr), K(result), K(last_datum));
}
LOG_DEBUG(
"finish restore_groupby_datum", K(curr_groupby_datums_), K(group_exprs), K(rollup_exprs), K(diff_pos), K(ret));
return ret;
}
int ObMergeGroupByOp::rollup_and_calc_results(const int64_t group_id, const ObExpr* diff_expr /*= NULL*/)
{
int ret = OB_SUCCESS;
const int64_t col_count = MY_SPEC.group_exprs_.count() + MY_SPEC.rollup_exprs_.count();
if (OB_UNLIKELY(group_id < 0 || group_id > col_count)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(group_id), K(col_count), K(ret));
} else if (MY_SPEC.has_rollup_ && group_id > 0) {
if (OB_FAIL(aggr_processor_.rollup_process(group_id, diff_expr))) {
LOG_WARN("failed to rollup aggregation results", K(ret));
}
diff_expr = 0;
}
if (OB_SUCC(ret)) {
clear_evaluated_flag();
if (OB_FAIL(aggr_processor_.collect(group_id, diff_expr))) {
LOG_WARN("failed to collect aggr result", K(group_id), K(ret));
} else if (OB_FAIL(aggr_processor_.reuse_group(group_id))) {
LOG_WARN("failed to reuse group", K(group_id), K(ret));
} else {
LOG_DEBUG(
"finish rollup_and_calc_results", K(group_id), K(is_end_), "row", ROWEXPR2STR(eval_ctx_, MY_SPEC.output_));
}
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,106 @@
/**
* 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_BASIC_OB_MERGE_GROUPBY_OP_H_
#define OCEANBASE_BASIC_OB_MERGE_GROUPBY_OP_H_
#include "common/row/ob_row_store.h"
#include "sql/engine/aggregate/ob_groupby_op.h"
#include "sql/engine/aggregate/ob_aggregate_function.h"
namespace oceanbase {
namespace sql {
class ObMergeGroupBySpec : public ObGroupBySpec {
OB_UNIS_VERSION_V(1);
public:
ObMergeGroupBySpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
: ObGroupBySpec(alloc, type),
group_exprs_(alloc),
rollup_exprs_(alloc),
is_distinct_rollup_expr_(alloc),
has_rollup_(false)
{}
DECLARE_VIRTUAL_TO_STRING;
inline void set_rollup(const bool has_rollup)
{
has_rollup_ = has_rollup;
}
inline int init_group_exprs(const int64_t count)
{
return group_exprs_.init(count);
}
inline int init_rollup_exprs(const int64_t count)
{
return rollup_exprs_.init(count);
}
inline int init_distinct_rollup_expr(const int64_t count)
{
return is_distinct_rollup_expr_.init(count);
}
int add_group_expr(ObExpr* expr);
int add_rollup_expr(ObExpr* expr);
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObMergeGroupBySpec);
public:
ExprFixedArray group_exprs_; // group by column
ExprFixedArray rollup_exprs_; // rollup column
common::ObFixedArray<bool, common::ObIAllocator> is_distinct_rollup_expr_;
bool has_rollup_;
};
// The input data has been sorted according to the groupby column
class ObMergeGroupByOp : public ObGroupByOp {
public:
ObMergeGroupByOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObGroupByOp(exec_ctx, spec, input),
is_end_(false),
cur_output_group_id_(common::OB_INVALID_INDEX),
first_output_group_id_(0),
last_child_output_(aggr_processor_.get_aggr_alloc()),
curr_groupby_datums_(aggr_processor_.get_aggr_alloc())
{}
void reset();
virtual int inner_open() override;
virtual int inner_close() override;
virtual int rescan() override;
virtual int switch_iterator() override;
virtual int inner_get_next_row() override;
virtual void destroy() override;
int init();
int prepare_curr_groupby_datum();
int check_same_group(int64_t& diff_pos);
int restore_groupby_datum(const int64_t diff_pos);
int rollup_and_calc_results(const int64_t group_id, const ObExpr* diff_expr = NULL);
int rewrite_rollup_column(ObExpr*& diff_expr);
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObMergeGroupByOp);
private:
bool is_end_;
// added to support groupby with rollup
int64_t cur_output_group_id_;
int64_t first_output_group_id_;
ObChunkDatumStore::LastStoredRow<> last_child_output_;
DatumFixedArray curr_groupby_datums_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_BASIC_OB_MERGE_GROUPBY_OP_H_

View File

@ -0,0 +1,190 @@
/**
* 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/aggregate/ob_scalar_aggregate.h"
#include "common/row/ob_row.h"
#include "sql/ob_sql_utils.h"
#include "sql/session/ob_sql_session_info.h"
#include "sql/engine/ob_physical_plan.h"
#include "sql/engine/ob_exec_context.h"
namespace oceanbase {
using namespace common;
namespace sql {
class ObScalarAggregate::ObScalarAggregateCtx : public ObGroupByCtx {
public:
explicit ObScalarAggregateCtx(ObExecContext& exec_ctx) : ObGroupByCtx(exec_ctx), started_(false)
{}
virtual void destroy()
{
ObGroupByCtx::destroy();
}
private:
bool started_;
friend class ObScalarAggregate;
};
ObScalarAggregate::ObScalarAggregate(ObIAllocator& alloc) : ObGroupBy(alloc)
{}
ObScalarAggregate::~ObScalarAggregate()
{}
void ObScalarAggregate::reset()
{
ObGroupBy::reset();
}
void ObScalarAggregate::reuse()
{
reset();
}
int ObScalarAggregate::inner_open(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(group_col_idxs_.count() > 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("scalar aggregate has no group", K(ret));
} else if (OB_FAIL(ObGroupBy::init_group_by(ctx))) {
LOG_WARN("failed to init group by", K(ret));
} else {
ObScalarAggregateCtx* groupby_ctx = GET_PHY_OPERATOR_CTX(ObScalarAggregateCtx, ctx, get_id());
if (OB_ISNULL(groupby_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group by ctx is NULL", K(ret));
} else {
// treat scalar group by as a one group merge group by
groupby_ctx->get_aggr_func().set_sort_based_gby();
}
}
return ret;
}
int ObScalarAggregate::switch_iterator(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObScalarAggregateCtx* agg_ctx = NULL;
if (OB_FAIL(ObGroupBy::switch_iterator(ctx))) {
if (OB_ITER_END != ret) {
LOG_WARN("switch iterator ObGroupBy failed", K(ret));
}
} else if (OB_ISNULL(agg_ctx = GET_PHY_OPERATOR_CTX(ObScalarAggregateCtx, ctx, id_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get scalar aggregate ctx failed");
} else {
agg_ctx->started_ = false;
agg_ctx->aggr_func_.reuse();
}
return ret;
}
int ObScalarAggregate::rescan(ObExecContext& ctx) const
{
int ret = OB_SUCCESS;
ObScalarAggregateCtx* agg_ctx = NULL;
if (OB_FAIL(ObGroupBy::rescan(ctx))) {
LOG_WARN("rescan ObGroupBy failed", K(ret));
} else if (OB_ISNULL(agg_ctx = GET_PHY_OPERATOR_CTX(ObScalarAggregateCtx, ctx, id_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get scalar aggregate ctx failed");
} else {
agg_ctx->started_ = false;
agg_ctx->aggr_func_.reuse();
}
return ret;
}
int ObScalarAggregate::inner_close(ObExecContext& ctx) const
{
UNUSED(ctx);
return OB_SUCCESS; // ObGroupBy::close(ctx);
}
int ObScalarAggregate::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
{
int ret = OB_SUCCESS;
const ObNewRow* input_row = NULL;
ObScalarAggregateCtx* scalar_aggregate_ctx = NULL;
// no group columns and only aggr functions
if (OB_ISNULL(scalar_aggregate_ctx = GET_PHY_OPERATOR_CTX(ObScalarAggregateCtx, ctx, id_))) {
ret = OB_ERR_NULL_VALUE;
LOG_WARN("get physical operator context failed", K_(id));
} else if (OB_ISNULL(child_op_)) {
ret = OB_NOT_INIT;
LOG_WARN("child operator is null");
} else if (!scalar_aggregate_ctx->started_) {
scalar_aggregate_ctx->started_ = true;
if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to get next row", K(ret));
} else {
// empty result, should return a row with info which indicates empty result set
ret = scalar_aggregate_ctx->aggr_func_.get_result_for_empty_set(scalar_aggregate_ctx->get_cur_row());
}
} else if (OB_ISNULL(input_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("input row is null");
} else if ((OB_FAIL(scalar_aggregate_ctx->aggr_func_.prepare(*input_row)))) {
LOG_WARN("fail to prepare the aggr func", K(ret), K(*input_row));
} else {
ObSQLSessionInfo* my_session = scalar_aggregate_ctx->exec_ctx_.get_my_session();
const ObTimeZoneInfo* tz_info = (NULL != my_session) ? my_session->get_timezone_info() : NULL;
while (OB_SUCC(ret) && OB_SUCC(child_op_->get_next_row(ctx, input_row))) {
if (OB_FAIL(try_check_status(ctx))) {
LOG_WARN("failed to check status", K(ret));
} else if (OB_FAIL(scalar_aggregate_ctx->aggr_func_.process(*input_row, tz_info))) {
LOG_WARN("fail to process the aggr func", K(ret), K(*input_row));
} else if (mem_size_limit_ > 0 && mem_size_limit_ < scalar_aggregate_ctx->aggr_func_.get_used_mem_size()) {
ret = OB_EXCEED_MEM_LIMIT;
LOG_WARN("merge group by has exceeded the mem limit",
K_(mem_size_limit),
"mem_used",
scalar_aggregate_ctx->aggr_func_.get_used_mem_size());
}
}
if (OB_ITER_END == ret) {
ret = scalar_aggregate_ctx->aggr_func_.get_result(scalar_aggregate_ctx->get_cur_row(), tz_info);
}
}
} else {
ret = OB_ITER_END;
}
if (OB_SUCC(ret)) {
row = &scalar_aggregate_ctx->get_cur_row();
}
return ret;
}
int ObScalarAggregate::inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const
{
return CREATE_PHY_OPERATOR_CTX(ObScalarAggregateCtx, ctx, id_, get_type(), op_ctx);
}
int64_t ObScalarAggregate::to_string_kv(char* buf, const int64_t buf_len) const
{
int64_t pos = 0;
J_KV(N_AGGR_COLUMN, aggr_columns_, N_MEM_LIMIT, mem_size_limit_);
return pos;
}
int ObScalarAggregate::add_group_column_idx(int64_t column_idx, common::ObCollationType cs_type)
{
UNUSED(column_idx);
UNUSED(cs_type);
return OB_NOT_IMPLEMENT;
}
} // namespace sql
} // namespace oceanbase

View File

@ -0,0 +1,74 @@
/**
* 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_SRC_SQL_ENGINE_AGGREGATE_OB_SCALAR_AGGREGATE_H_
#define OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_SCALAR_AGGREGATE_H_
#include "common/row/ob_row_store.h"
#include "sql/engine/aggregate/ob_groupby.h"
#include "sql/engine/aggregate/ob_aggregate_function.h"
namespace oceanbase {
namespace common {
class ObNewRow;
} // namespace common
namespace sql {
class ObScalarAggregate : public ObGroupBy {
protected:
class ObScalarAggregateCtx;
public:
explicit ObScalarAggregate(common::ObIAllocator& alloc);
virtual ~ObScalarAggregate();
virtual void reset();
virtual void reuse();
virtual int rescan(ObExecContext& ctx) const;
virtual int switch_iterator(ObExecContext& ctx) const override;
protected:
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
/**
* @brief open operator, not including children operators.
* called by open.
* Every op should implement this method.
*/
virtual int inner_open(ObExecContext& ctx) const;
/**
* @brief close operator, not including children operators.
* Every op should implement this method.
*/
virtual int inner_close(ObExecContext& ctx) const;
/**
* @brief create operator context, only child operator can know it's specific operator type,
* so must be overwrited by child operator,
* @param ctx[in], execute context
* @param op_ctx[out], the pointer of operator context
* @return if success, return OB_SUCCESS, otherwise, return errno
*/
virtual int inner_create_operator_ctx(ObExecContext& ctx, ObPhyOperatorCtx*& op_ctx) const;
/**
* @brief for specified phy operator to print it's member variable with json key-value format
* @param buf[in] to string buffer
* @param buf_len[in] buffer length
* @return if success, return the length used by print string, otherwise return 0
*/
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
virtual int add_group_column_idx(int64_t column_idx, common::ObCollationType cs_type);
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObScalarAggregate);
};
} // namespace sql
} // namespace oceanbase
#endif /* OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_SCALAR_AGGREGATE_H_ */

View File

@ -0,0 +1,131 @@
/**
* 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/aggregate/ob_scalar_aggregate_op.h"
#include "lib/number/ob_number_v2.h"
namespace oceanbase {
namespace sql {
OB_SERIALIZE_MEMBER((ObScalarAggregateSpec, ObGroupBySpec));
int ObScalarAggregateOp::inner_open()
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObGroupByOp::inner_open())) {
LOG_WARN("failed to inner_open", K(ret));
} else if (OB_FAIL(aggr_processor_.init_one_group())) {
LOG_WARN("failed to init one group", K(ret));
} else {
started_ = false;
}
return ret;
}
int ObScalarAggregateOp::inner_close()
{
started_ = false;
return ObGroupByOp::inner_close();
}
void ObScalarAggregateOp::destroy()
{
started_ = false;
ObGroupByOp::destroy();
}
int ObScalarAggregateOp::switch_iterator()
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObGroupByOp::switch_iterator())) {
LOG_WARN("failed to switch_iterator", K(ret));
} else if (OB_FAIL(aggr_processor_.init_one_group())) {
LOG_WARN("failed to init one group", K(ret));
} else {
started_ = false;
}
return ret;
}
int ObScalarAggregateOp::rescan()
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObGroupByOp::rescan())) {
LOG_WARN("failed to rescan", K(ret));
} else if (OB_FAIL(aggr_processor_.init_one_group())) {
LOG_WARN("failed to init one group", K(ret));
} else {
started_ = false;
}
return ret;
}
int ObScalarAggregateOp::inner_get_next_row()
{
int ret = OB_SUCCESS;
if (started_) {
ret = OB_ITER_END;
} else {
LOG_DEBUG("before inner_get_next_row",
"aggr_hold_size",
aggr_processor_.get_aggr_hold_size(),
"aggr_used_size",
aggr_processor_.get_aggr_used_size());
started_ = true;
clear_evaluated_flag();
if (OB_FAIL(child_->get_next_row())) {
if (OB_ITER_END != ret) {
LOG_WARN("fail to get next row", K(ret));
} else if (OB_FAIL(aggr_processor_.collect_for_empty_set())) {
LOG_WARN("fail to prepare the aggr func", K(ret));
}
} else {
clear_evaluated_flag();
ObAggregateProcessor::GroupRow* group_row = NULL;
if (OB_FAIL(aggr_processor_.get_group_row(0, group_row))) {
LOG_WARN("failed to get_group_row", K(ret));
} else if (OB_ISNULL(group_row)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group_row is null", K(ret));
} else if (OB_FAIL(aggr_processor_.prepare(*group_row))) {
LOG_WARN("fail to prepare the aggr func", K(ret));
} else {
while (OB_SUCC(ret) && OB_SUCC(child_->get_next_row())) {
clear_evaluated_flag();
if (OB_FAIL(try_check_status())) {
LOG_WARN("check status failed", K(ret));
} else if (OB_FAIL(aggr_processor_.process(*group_row))) {
LOG_WARN("fail to process the aggr func", K(ret));
}
}
if (OB_ITER_END == ret) {
if (OB_FAIL(aggr_processor_.collect())) {
LOG_WARN("fail to collect result", K(ret));
}
}
}
}
LOG_DEBUG("after inner_get_next_row",
"hold_mem_size",
aggr_processor_.get_aggr_hold_size(),
"used_mem_size",
aggr_processor_.get_aggr_used_size());
}
return ret;
}
} // end namespace sql
} // end namespace oceanbase

View File

@ -0,0 +1,58 @@
/**
* 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_BASIC_OB_SCALAR_GROUPBY_OP_H_
#define OCEANBASE_BASIC_OB_SCALAR_GROUPBY_OP_H_
#include "common/row/ob_row_store.h"
#include "sql/engine/aggregate/ob_groupby_op.h"
namespace oceanbase {
namespace sql {
class ObScalarAggregateSpec : public ObGroupBySpec {
OB_UNIS_VERSION_V(1);
public:
ObScalarAggregateSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type) : ObGroupBySpec(alloc, type)
{}
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObScalarAggregateSpec);
};
class ObScalarAggregateOp : public ObGroupByOp {
public:
friend ObAggregateProcessor;
ObScalarAggregateOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
: ObGroupByOp(exec_ctx, spec, input), started_(false)
{}
virtual int inner_open() override;
virtual int inner_close() override;
virtual int rescan() override;
virtual int switch_iterator() override;
virtual int inner_get_next_row() override;
virtual void destroy() override;
// reset default value of %cur_rownum_ && %rownum_limit_
private:
// disallow copy
DISALLOW_COPY_AND_ASSIGN(ObScalarAggregateOp);
private:
bool started_;
};
} // end namespace sql
} // end namespace oceanbase
#endif // OCEANBASE_BASIC_OB_SCALAR_GROUPBY_OP_H_