Do not report ERROR when sample rate equals 100
This commit is contained in:
@ -1,410 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_CTX_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_CTX_H_
|
||||
|
||||
#include "sql/engine/aggregate/ob_aggregate_processor.h"
|
||||
#include "share/aggregate/util.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
using namespace sql;
|
||||
class AggBitVector;
|
||||
using AggrRowPtr = char *;
|
||||
using AggregateExtras = ObAggregateProcessor::ExtraResult **;
|
||||
// examples of aggregate row:
|
||||
// with extra idx:
|
||||
//
|
||||
// count(a), sum(b), count(distinct(a)), concat(...), implicit_first_row null bits
|
||||
// ---------------------------------------------------------------------------------------------------------------------------------
|
||||
// | int64 | <ob_number, int64_tmp_res>| int64 | <char *, in32_t> | <char *, int32> | int32 (idx) | bits |
|
||||
// ---------------------------------------------------------------------------------------------------------------------------------
|
||||
// |
|
||||
// |
|
||||
// | ... | |
|
||||
// |-------------------------------------| |
|
||||
// extra_info array | <distinct_extra *, concat_extra*> |<---------
|
||||
// |-------------------------------------|
|
||||
// | ... |
|
||||
//
|
||||
// no extra idx:
|
||||
// count(a), sum(b), implicit_first_row null bits
|
||||
// ---------------------------------------------------------------
|
||||
// | int64 | int128 | <char *, int32> | bits |
|
||||
// ---------------------------------------------------------------
|
||||
|
||||
struct AggrRowMeta
|
||||
{
|
||||
AggrRowMeta() :
|
||||
row_size_(0), col_cnt_(0), extra_cnt_(0), nullbits_offset_(0), extra_idx_offset_(0),
|
||||
col_offsets_(nullptr), tmp_res_sizes_(nullptr), extra_idxes_(nullptr), use_var_len_(nullptr)
|
||||
{}
|
||||
|
||||
void reset()
|
||||
{
|
||||
row_size_ = 0;
|
||||
col_cnt_ = 0;
|
||||
extra_cnt_ = 0;
|
||||
nullbits_offset_ = 0;
|
||||
col_offsets_ = nullptr;
|
||||
tmp_res_sizes_ = nullptr;
|
||||
extra_idx_offset_ = 0;
|
||||
extra_idxes_ = nullptr;
|
||||
use_var_len_ = nullptr;
|
||||
}
|
||||
|
||||
inline char *locate_cell_payload(const int32_t col_id, char *row) const
|
||||
{
|
||||
OB_ASSERT(col_id < col_cnt_);
|
||||
OB_ASSERT(col_offsets_ != nullptr);
|
||||
int32_t col_offset = col_offsets_[col_id];
|
||||
OB_ASSERT(col_offset < row_size_);
|
||||
return row + col_offset;
|
||||
}
|
||||
inline const char *locate_cell_payload(const int32_t col_id, const char *agg_row) const
|
||||
{
|
||||
return static_cast<const char *>(locate_cell_payload(col_id, const_cast<char *>(agg_row)));
|
||||
}
|
||||
inline int32_t get_cell_len(const int32_t col_id, const char *row) const
|
||||
{
|
||||
OB_ASSERT(col_id < col_cnt_);
|
||||
OB_ASSERT(col_offsets_ != nullptr);
|
||||
OB_ASSERT(tmp_res_sizes_ != nullptr);
|
||||
if (use_var_len_ != nullptr && use_var_len_->at(col_id)) {
|
||||
return *reinterpret_cast<const int32_t *>(row + col_offsets_[col_id] + sizeof(char *));
|
||||
} else {
|
||||
return col_offsets_[col_id + 1] - col_offsets_[col_id] - tmp_res_sizes_[col_id];
|
||||
}
|
||||
}
|
||||
|
||||
inline void locate_cell_payload(const int32_t col_id, char *agg_row, char *&cell,
|
||||
int32_t &cell_len) const
|
||||
{
|
||||
OB_ASSERT(col_id < col_cnt_);
|
||||
OB_ASSERT(col_offsets_ != nullptr);
|
||||
OB_ASSERT(tmp_res_sizes_ != nullptr);
|
||||
cell = agg_row + col_offsets_[col_id];
|
||||
if (use_var_len_ != nullptr && use_var_len_->at(col_id)) {
|
||||
cell_len = *reinterpret_cast<int32_t *>(cell + sizeof(char *));
|
||||
} else {
|
||||
cell_len = col_offsets_[col_id + 1] - col_offsets_[col_id] - tmp_res_sizes_[col_id];
|
||||
}
|
||||
}
|
||||
|
||||
inline NotNullBitVector &locate_notnulls_bitmap(char *agg_row) const
|
||||
{
|
||||
return *reinterpret_cast<NotNullBitVector *>(agg_row + nullbits_offset_);
|
||||
}
|
||||
|
||||
TO_STRING_KV(K_(row_size), K_(col_cnt), K_(extra_cnt), K_(nullbits_offset), K_(extra_idx_offset));
|
||||
int32_t row_size_;
|
||||
int32_t col_cnt_;
|
||||
int32_t extra_cnt_;
|
||||
// null bits offset
|
||||
int32_t nullbits_offset_;
|
||||
int32_t extra_idx_offset_;
|
||||
// agg_col memory offset
|
||||
int32_t *col_offsets_;
|
||||
int32_t *tmp_res_sizes_;
|
||||
// idx for extra infos
|
||||
int32_t *extra_idxes_;
|
||||
// if agg_col memory is described as <char *, int32>, it's marked as var_var_len
|
||||
AggBitVector *use_var_len_;
|
||||
};
|
||||
|
||||
struct RowSelector
|
||||
{
|
||||
RowSelector() : selector_array_(nullptr), size_(0)
|
||||
{}
|
||||
RowSelector(const uint16_t *selector_array, const int32_t size) :
|
||||
selector_array_(selector_array), size_(size)
|
||||
{}
|
||||
|
||||
bool is_empty() const
|
||||
{
|
||||
return selector_array_ == nullptr || size_ == 0;
|
||||
}
|
||||
int32_t index(const int64_t i) const
|
||||
{
|
||||
OB_ASSERT(i < size_);
|
||||
OB_ASSERT(selector_array_ != NULL);
|
||||
return selector_array_[i];
|
||||
}
|
||||
|
||||
const uint16_t *selector() const
|
||||
{
|
||||
return selector_array_;
|
||||
}
|
||||
|
||||
int32_t size() const
|
||||
{
|
||||
return size_;
|
||||
}
|
||||
|
||||
const uint16_t *selector_array_;
|
||||
const int32_t size_;
|
||||
};
|
||||
|
||||
struct RuntimeContext
|
||||
{
|
||||
RuntimeContext(sql::ObEvalCtx &eval_ctx, uint64_t tenant_id, ObIArray<ObAggrInfo> &aggr_infos,
|
||||
const lib::ObLabel &label, ObIAllocator &struct_allocator) :
|
||||
eval_ctx_(eval_ctx),
|
||||
aggr_infos_(aggr_infos),
|
||||
allocator_(label, OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id, ObCtxIds::WORK_AREA),
|
||||
op_monitor_info_(nullptr), io_event_observer_(nullptr), agg_row_meta_(),
|
||||
agg_rows_(ModulePageAllocator(label, tenant_id, ObCtxIds::WORK_AREA)),
|
||||
agg_extras_(ModulePageAllocator(label, tenant_id, ObCtxIds::WORK_AREA))
|
||||
{}
|
||||
|
||||
inline const AggrRowMeta &row_meta() const
|
||||
{
|
||||
return agg_row_meta_;
|
||||
}
|
||||
inline ObAggregateProcessor::ExtraResult *&get_extra(const int64_t agg_col_id, const char *agg_cell)
|
||||
{
|
||||
OB_ASSERT(agg_col_id < agg_row_meta_.col_cnt_);
|
||||
OB_ASSERT(agg_cell != nullptr);
|
||||
OB_ASSERT(row_meta().extra_idx_offset_ > 0);
|
||||
OB_ASSERT(row_meta().col_offsets_ != nullptr);
|
||||
OB_ASSERT(row_meta().extra_idxes_ != nullptr);
|
||||
const char *row = agg_cell - row_meta().col_offsets_[agg_col_id];
|
||||
int32_t extra_idx = *reinterpret_cast<const int32_t *>(row + row_meta().extra_idx_offset_);
|
||||
OB_ASSERT(row_meta().extra_idxes_[agg_col_id] >= 0);
|
||||
int32_t agg_extra_id = row_meta().extra_idxes_[agg_col_id];
|
||||
OB_ASSERT(agg_extras_.at(extra_idx) != nullptr
|
||||
&& row_meta().extra_cnt_> agg_extra_id);
|
||||
return agg_extras_.at(extra_idx)[agg_extra_id];
|
||||
}
|
||||
ObAggrInfo &locate_aggr_info(const int64_t agg_col_idx)
|
||||
{
|
||||
OB_ASSERT(agg_col_idx < aggr_infos_.count());
|
||||
return aggr_infos_.at(agg_col_idx);
|
||||
}
|
||||
|
||||
ObScale get_first_param_scale(int64_t agg_col_id) const
|
||||
{
|
||||
OB_ASSERT(!aggr_infos_.at(agg_col_id).param_exprs_.empty());
|
||||
OB_ASSERT(aggr_infos_.at(agg_col_id).param_exprs_.at(0) != NULL);
|
||||
if (ob_is_integer_type(aggr_infos_.at(agg_col_id).param_exprs_.at(0)->datum_meta_.type_)) {
|
||||
return DEFAULT_SCALE_FOR_INTEGER;
|
||||
} else {
|
||||
return aggr_infos_.at(agg_col_id).param_exprs_.at(0)->datum_meta_.scale_;
|
||||
}
|
||||
}
|
||||
|
||||
void get_agg_payload(const int32_t agg_col_id, const int32_t group_id, char *&payload,
|
||||
int32_t &len)
|
||||
{
|
||||
OB_ASSERT(group_id < agg_rows_.count());
|
||||
AggrRowPtr row = agg_rows_.at(group_id);
|
||||
row_meta().locate_cell_payload(agg_col_id, row, payload, len);
|
||||
return;
|
||||
}
|
||||
|
||||
void get_agg_payload(const int32_t agg_col_id, const int32_t group_id, const char *&payload,
|
||||
int32_t &len)
|
||||
{
|
||||
char *tmp_payload = nullptr;
|
||||
get_agg_payload(agg_col_id, group_id, tmp_payload, len);
|
||||
payload = tmp_payload;
|
||||
}
|
||||
|
||||
NotNullBitVector &locate_notnulls_bitmap(const int32_t agg_col_id, char *agg_cell) const
|
||||
{
|
||||
OB_ASSERT(row_meta().col_cnt_ > agg_col_id);
|
||||
OB_ASSERT(row_meta().col_offsets_ != nullptr);
|
||||
OB_ASSERT(row_meta().nullbits_offset_ >= 0);
|
||||
char *row = agg_cell - row_meta().col_offsets_[agg_col_id];
|
||||
return row_meta().locate_notnulls_bitmap(row);
|
||||
}
|
||||
|
||||
const NotNullBitVector &locate_notnulls_bitmap(const int32_t agg_col_id,
|
||||
const char *agg_cell) const
|
||||
{
|
||||
OB_ASSERT(row_meta().col_cnt_ > agg_col_id);
|
||||
OB_ASSERT(row_meta().col_offsets_ != nullptr);
|
||||
OB_ASSERT(row_meta().nullbits_offset_ >= 0);
|
||||
const char *row = agg_cell - row_meta().col_offsets_[agg_col_id];
|
||||
return row_meta().locate_notnulls_bitmap(const_cast<char *>(row));
|
||||
}
|
||||
|
||||
int32_t get_cell_len(const int32_t agg_col_id, const char *agg_cell) const
|
||||
{
|
||||
OB_ASSERT(agg_cell != nullptr);
|
||||
OB_ASSERT(row_meta().col_offsets_ != nullptr);
|
||||
OB_ASSERT(row_meta().col_cnt_ > agg_col_id);
|
||||
const char *row = agg_cell - row_meta().col_offsets_[agg_col_id];
|
||||
return row_meta().get_cell_len(agg_col_id, row);
|
||||
}
|
||||
|
||||
void set_agg_cell(const char *src, const int32_t data_len, const int32_t col_id,
|
||||
char *agg_cell) const
|
||||
{
|
||||
if (agg_row_meta_.use_var_len_ != nullptr && agg_row_meta_.use_var_len_->at(col_id)) {
|
||||
*reinterpret_cast<int64_t *>(agg_cell) = reinterpret_cast<int64_t>(src);
|
||||
*reinterpret_cast<int32_t *>(agg_cell + sizeof(char *)) = data_len;
|
||||
} else {
|
||||
MEMCPY(agg_cell, src, data_len);
|
||||
}
|
||||
}
|
||||
void reuse()
|
||||
{
|
||||
agg_rows_.reuse();
|
||||
for (int i = 0; i < agg_extras_.count(); i++) {
|
||||
if (OB_NOT_NULL(agg_extras_.at(i))) {
|
||||
for (int j = 0; j < row_meta().extra_cnt_; j++) {
|
||||
if (OB_NOT_NULL(agg_extras_.at(i)[j])) {
|
||||
agg_extras_.at(i)[j]->~ExtraResult();
|
||||
}
|
||||
} // end for
|
||||
}
|
||||
} // end for
|
||||
agg_extras_.reuse();
|
||||
allocator_.reset_remain_one_page();
|
||||
}
|
||||
void destroy()
|
||||
{
|
||||
for (int i = 0; i < agg_extras_.count(); i++) {
|
||||
if (OB_NOT_NULL(agg_extras_.at(i))) {
|
||||
for (int j = 0; j < row_meta().extra_cnt_; j++) {
|
||||
if (OB_NOT_NULL(agg_extras_.at(i)[j])) {
|
||||
agg_extras_.at(i)[j]->~ExtraResult();
|
||||
}
|
||||
} // end for
|
||||
}
|
||||
} // end for
|
||||
agg_rows_.reset();
|
||||
agg_extras_.reset();
|
||||
allocator_.reset();
|
||||
op_monitor_info_ = nullptr;
|
||||
io_event_observer_ = nullptr;
|
||||
agg_row_meta_.reset();
|
||||
}
|
||||
|
||||
int init_row_meta(ObIArray<ObAggrInfo> &aggr_infos, ObIAllocator &alloc);
|
||||
sql::ObEvalCtx &eval_ctx_;
|
||||
ObIArray<ObAggrInfo> &aggr_infos_;
|
||||
// used to allocate runtime data memory, such as rows for distinct extra.
|
||||
ObArenaAllocator allocator_;
|
||||
sql::ObMonitorNode *op_monitor_info_;
|
||||
sql::ObIOEventObserver *io_event_observer_;
|
||||
AggrRowMeta agg_row_meta_;
|
||||
ObSegmentArray<AggrRowPtr, OB_MALLOC_MIDDLE_BLOCK_SIZE, common::ModulePageAllocator>
|
||||
agg_rows_;
|
||||
ObSegmentArray<AggregateExtras, OB_MALLOC_MIDDLE_BLOCK_SIZE, common::ModulePageAllocator>
|
||||
agg_extras_;
|
||||
};
|
||||
|
||||
/*
|
||||
IAggregate is a abstract interface for aggregate functions, to implement a new aggregate function,
|
||||
do as following code template:
|
||||
```C++
|
||||
class AggFunc final: public BatchAggregateWrapper<AggFunc>
|
||||
{
|
||||
public:
|
||||
// need define in & out vector type class
|
||||
static const VecValueTypeClass IN_TC = {in_tc};
|
||||
static const VecValueTypeClass OUT_TC = {out_tc};
|
||||
public:
|
||||
template<typename ColumnFmt>
|
||||
int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *aggr_cell)
|
||||
{
|
||||
// do add row
|
||||
// `ColumnFmt` is type of input vector
|
||||
// ColumnFmt is specified format for input param
|
||||
}
|
||||
template<typename ColumnFmt>
|
||||
int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *aggr_cell)
|
||||
{
|
||||
// do add nullable row
|
||||
// if (columns.is_null(row_num)) { do add null }
|
||||
}
|
||||
template<typename ResultFmt>
|
||||
int collect_group_result(RuntimeContext &agg_ctx,
|
||||
const sql::ObExpr &agg_expr, const int32_t agg_col_id,
|
||||
const char *aggr_cell, const int32_t aggr_cell_len)
|
||||
{
|
||||
// do collect group result
|
||||
// `ResultFmt` is type of output vector
|
||||
// ResultFmt is specified result column format.
|
||||
}
|
||||
// optinal member function
|
||||
template<typename ColumnFmt>
|
||||
int add_param_batch(RuntimeContext &agg_ctx, ObBitVector &skip, const EvalBound &bound,
|
||||
const RowSelector &row_sel, const int32_t agg_col_id,
|
||||
const int32_t param_id, ColumnFmt ¶m_vec, char *aggr_cell)
|
||||
{
|
||||
// if aggregate function has multiple param exprs, e.g. `count`,
|
||||
// this member function should be defined to add batch for each param.
|
||||
}
|
||||
TO_STRING_KV("aggregate", {aggregate_name});
|
||||
};
|
||||
```
|
||||
*/
|
||||
class IAggregate
|
||||
{
|
||||
public:
|
||||
inline virtual int collect_batch_group_results(RuntimeContext &agg_ctx,
|
||||
const int32_t agg_col_id,
|
||||
const int32_t cur_group_id,
|
||||
const int32_t output_start_idx,
|
||||
const int32_t expect_batch_size,
|
||||
int32_t &output_size,
|
||||
const ObBitVector *skip = nullptr) = 0;
|
||||
|
||||
inline virtual int collect_batch_group_results(RuntimeContext &agg_ctx,
|
||||
const int32_t agg_col_id,
|
||||
const int32_t output_start_idx,
|
||||
const int32_t batch_size,
|
||||
const ObCompactRow **rows,
|
||||
const RowMeta &row_meta) = 0;
|
||||
inline virtual int add_batch_rows(RuntimeContext &agg_ctx,
|
||||
int32_t agg_col_idx,
|
||||
const sql::ObBitVector &skip, const sql::EvalBound &bound,
|
||||
char *agg_cell,
|
||||
const RowSelector row_sel = RowSelector{}) = 0;
|
||||
inline virtual int add_one_row(RuntimeContext &agg_ctx, const int64_t batch_idx,
|
||||
const int64_t batch_size, const bool is_null, const char *data,
|
||||
const int32_t data_len, int32_t agg_col_idx, char *agg_cell) = 0;
|
||||
// temp result for one batch,
|
||||
// used for sum
|
||||
inline virtual void *get_tmp_res(RuntimeContext &agg_ctx, int32_t agg_col_idx,
|
||||
char *agg_cell) = 0;
|
||||
|
||||
// calculation info such as obj_meta for one batch
|
||||
inline virtual int64_t get_batch_calc_info(RuntimeContext &agg_ctx, int32_t agg_col_idx,
|
||||
char *agg_cell) = 0;
|
||||
|
||||
// TODO: add rollup interface
|
||||
// for wrapper aggregates, like `ObDistinctWrapper`
|
||||
inline virtual void set_inner_aggregate(IAggregate *agg) = 0;
|
||||
// initializer
|
||||
virtual int init(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator) = 0;
|
||||
|
||||
virtual void reuse() = 0;
|
||||
|
||||
virtual void destroy() = 0;
|
||||
DECLARE_PURE_VIRTUAL_TO_STRING;
|
||||
};
|
||||
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_CTX_H_
|
||||
@ -1,146 +0,0 @@
|
||||
/**
|
||||
* 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 "count.h"
|
||||
#include "share/aggregate/single_row.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
namespace helper
|
||||
{
|
||||
int init_count_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg)
|
||||
{
|
||||
#define INIT_COUNT_CASE(vec_tc) \
|
||||
case (vec_tc): { \
|
||||
if (lib::is_oracle_mode()) { \
|
||||
ret = init_agg_func<CountAggregate<vec_tc, VEC_TC_NUMBER>>(agg_ctx, agg_col_id, \
|
||||
has_distinct, allocator, agg); \
|
||||
} else { \
|
||||
ret = init_agg_func<CountAggregate<vec_tc, VEC_TC_INTEGER>>(agg_ctx, agg_col_id, \
|
||||
has_distinct, allocator, agg); \
|
||||
} \
|
||||
} break
|
||||
|
||||
int ret = OB_SUCCESS;
|
||||
ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id);
|
||||
agg = nullptr;
|
||||
bool has_distinct = aggr_info.has_distinct_;
|
||||
// count(*) param_exprs is empty
|
||||
VecValueTypeClass vec_tc = VEC_TC_INTEGER;
|
||||
if (aggr_info.param_exprs_.count() > 0) {
|
||||
vec_tc =
|
||||
get_vec_value_tc(aggr_info.get_first_child_type(), aggr_info.get_first_child_datum_scale(),
|
||||
aggr_info.get_first_child_datum_precision());
|
||||
}
|
||||
switch (vec_tc) {
|
||||
LST_DO_CODE(INIT_COUNT_CASE, AGG_VEC_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid param format", K(ret), K(vec_tc));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
#undef INIT_COUNT_CASE
|
||||
}
|
||||
|
||||
} // end namespace helper
|
||||
|
||||
int quick_add_batch_rows_for_count(IAggregate *agg, RuntimeContext &agg_ctx,
|
||||
const bool is_single_row_agg, const sql::ObBitVector &skip,
|
||||
const sql::EvalBound &bound, const RowSelector &row_sel,
|
||||
const int32_t agg_col_id, char *agg_cell)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObUniformVector<true, VectorBasicOp<VEC_TC_NULL>> mock_cols(nullptr, nullptr);
|
||||
int64_t fake_calc_info = 0;
|
||||
if (OB_ISNULL(agg)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid null aggregate", K(ret));
|
||||
} else if (!is_single_row_agg) {
|
||||
auto &count_agg = *static_cast<CountAggregate<VEC_TC_INTEGER, VEC_TC_INTEGER> *>(agg);
|
||||
if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
ret =
|
||||
count_agg.add_row(agg_ctx, mock_cols, i, agg_col_id, agg_cell, nullptr, fake_calc_info);
|
||||
}
|
||||
} else if (!row_sel.is_empty()) {
|
||||
for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) {
|
||||
ret = count_agg.add_row(agg_ctx, mock_cols, row_sel.index(i), agg_col_id, agg_cell, nullptr,
|
||||
fake_calc_info);
|
||||
}
|
||||
} else {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (skip.at(i)) {
|
||||
} else {
|
||||
ret =
|
||||
count_agg.add_row(agg_ctx, mock_cols, i, agg_col_id, agg_cell, nullptr, fake_calc_info);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (lib::is_mysql_mode()) {
|
||||
auto &count_agg = *static_cast<SingleRowAggregate<T_FUN_COUNT, VEC_TC_INTEGER, VEC_TC_INTEGER> *>(agg);
|
||||
if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
ret =
|
||||
count_agg.add_row(agg_ctx, mock_cols, i, agg_col_id, agg_cell, nullptr, fake_calc_info);
|
||||
}
|
||||
} else if (!row_sel.is_empty()) {
|
||||
for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) {
|
||||
ret = count_agg.add_row(agg_ctx, mock_cols, row_sel.index(i), agg_col_id, agg_cell, nullptr,
|
||||
fake_calc_info);
|
||||
}
|
||||
} else {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (skip.at(i)) {
|
||||
} else {
|
||||
ret =
|
||||
count_agg.add_row(agg_ctx, mock_cols, i, agg_col_id, agg_cell, nullptr, fake_calc_info);
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
auto &count_agg = *static_cast<SingleRowAggregate<T_FUN_COUNT, VEC_TC_INTEGER, VEC_TC_NUMBER> *>(agg);
|
||||
if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
ret =
|
||||
count_agg.add_row(agg_ctx, mock_cols, i, agg_col_id, agg_cell, nullptr, fake_calc_info);
|
||||
}
|
||||
} else if (!row_sel.is_empty()) {
|
||||
for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) {
|
||||
ret = count_agg.add_row(agg_ctx, mock_cols, row_sel.index(i), agg_col_id, agg_cell, nullptr,
|
||||
fake_calc_info);
|
||||
}
|
||||
} else {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (skip.at(i)) {
|
||||
} else {
|
||||
ret =
|
||||
count_agg.add_row(agg_ctx, mock_cols, i, agg_col_id, agg_cell, nullptr, fake_calc_info);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
SQL_LOG(DEBUG, "count: quick add batch rows", K(ret), K(*reinterpret_cast<int64_t *>(agg_cell)),
|
||||
K(agg_col_id), K(is_single_row_agg));
|
||||
if (OB_FAIL(ret)) { SQL_LOG(WARN, "count: quick add batch rows failed", K(ret)); }
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace aggregate
|
||||
} // end namespace share
|
||||
} // end namespace oceanbase
|
||||
@ -1,141 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_COUNT_AGGREGATE_H_
|
||||
#define OCEANBASE_SHARE_COUNT_AGGREGATE_H_
|
||||
|
||||
#include "share/aggregate/iaggregate.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
|
||||
template<VecValueTypeClass in_tc, VecValueTypeClass out_tc>
|
||||
class CountAggregate final: public BatchAggregateWrapper<CountAggregate<in_tc, out_tc>>
|
||||
{
|
||||
static_assert(out_tc == VEC_TC_INTEGER || out_tc == VEC_TC_NUMBER, "must be integer or number");
|
||||
public:
|
||||
static const constexpr VecValueTypeClass IN_TC = in_tc;
|
||||
static const constexpr VecValueTypeClass OUT_TC = out_tc;
|
||||
public:
|
||||
CountAggregate() {}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
// TODO: impl add_batch_rows, using null mpas to optimize counting logic
|
||||
int add_param_batch(RuntimeContext &agg_ctx, const ObBitVector &skip, ObBitVector &pvt_skip,
|
||||
const EvalBound &bound, const RowSelector &row_sel, const int32_t agg_col_id,
|
||||
const int32_t param_id, ColumnFmt ¶m_vec, char *aggr_cell)
|
||||
{
|
||||
UNUSED(skip);
|
||||
int ret = OB_SUCCESS;
|
||||
if (row_sel.is_empty()) {
|
||||
for (int i = bound.start(); i < bound.end(); i++) {
|
||||
if (pvt_skip.at(i)) {
|
||||
} else if (param_vec.is_null(i)) {
|
||||
pvt_skip.set(i);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < row_sel.size(); i++) {
|
||||
if (param_vec.is_null(row_sel.index(i))) { pvt_skip.set(row_sel.index(i)); }
|
||||
}
|
||||
}
|
||||
if (param_id == agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.count() - 1) {
|
||||
// last param
|
||||
int64_t &count = *reinterpret_cast<int64_t *>(aggr_cell);
|
||||
if (row_sel.is_empty()) {
|
||||
for (int i = bound.start(); i < bound.end(); i++) { count += !pvt_skip.at(i); }
|
||||
} else {
|
||||
for (int i = 0; i < row_sel.size(); i++) { count += !pvt_skip.at(row_sel.index(i)); }
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <typename ColumnFormat>
|
||||
int collect_group_result(RuntimeContext &agg_ctx, const ObExpr &agg_expr,
|
||||
const int32_t agg_col_id, const char *data, const int32_t data_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(data_len);
|
||||
ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
int64_t res_data = *reinterpret_cast<const int64_t *>(data);
|
||||
int64_t output_idx = ctx.get_batch_idx();
|
||||
ColumnFormat &columns = *static_cast<ColumnFormat *>(agg_expr.get_vector(ctx));
|
||||
SQL_LOG(DEBUG, "count collect result", K(agg_col_id), K(res_data));
|
||||
if (lib::is_oracle_mode()) {
|
||||
char buf_alloc[number::ObNumber::MAX_CALC_BYTE_LEN] = {0};
|
||||
ObDataBuffer local_alloc(buf_alloc, number::ObNumber::MAX_CALC_BYTE_LEN);
|
||||
number::ObNumber tmp_nmb;
|
||||
if (OB_FAIL(tmp_nmb.from(res_data, local_alloc))) {
|
||||
SQL_LOG(WARN, "number::from failed", K(ret));
|
||||
} else {
|
||||
columns.set_number(output_idx, tmp_nmb);
|
||||
}
|
||||
} else {
|
||||
columns.set_int(output_idx, res_data);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int add_one_row(RuntimeContext &agg_ctx, int64_t row_num, int64_t batch_size,
|
||||
const bool is_null, const char *data, const int32_t data_len,
|
||||
int32_t agg_col_idx, char *agg_cell) override
|
||||
{
|
||||
UNUSEDx(agg_ctx, row_num, batch_size, data_len, agg_col_idx);
|
||||
int64_t &count = *reinterpret_cast<int64_t *>(agg_cell);
|
||||
if (OB_LIKELY(agg_ctx.aggr_infos_.at(agg_col_idx).param_exprs_.count() == 1)) {
|
||||
if (!is_null) { count++; }
|
||||
} else {
|
||||
// multiply params
|
||||
ObAggrInfo &info = agg_ctx.aggr_infos_.at(agg_col_idx);
|
||||
bool has_null = false;
|
||||
for (int i = 0; !has_null && i < info.param_exprs_.count(); i++) {
|
||||
has_null = info.param_exprs_.at(i)->get_vector(agg_ctx.eval_ctx_)->is_null(row_num);
|
||||
}
|
||||
if (!has_null) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
template <typename ColumnFmt>
|
||||
inline int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, columns, row_num, tmp_res, calc_info, agg_col_id);
|
||||
int64_t &data = *reinterpret_cast<int64_t *>(agg_cell);
|
||||
data++;
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
inline int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res,
|
||||
int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, tmp_res, calc_info, agg_col_id);
|
||||
if (!columns.is_null(row_num)) {
|
||||
int64_t &data = *reinterpret_cast<int64_t *>(agg_cell);
|
||||
data++;
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
TO_STRING_KV("aggregate", "count");
|
||||
};
|
||||
|
||||
} // end namespace aggregate
|
||||
} // end namespace share
|
||||
} // end namespace oceanbase
|
||||
#endif // OCEANBASE_SHARE_COUNT_AGGREGATE_H_
|
||||
@ -1,232 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_FIRST_ROW_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_FIRST_ROW_H_
|
||||
|
||||
#include "iaggregate.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
|
||||
/*
|
||||
first row are stored as <char *, len> in aggregate row
|
||||
count sum first_row_data
|
||||
------------------------------------------------------
|
||||
| int64_t | ob_number | <data_ptr, data_len> | ...
|
||||
------------------------------------------------------
|
||||
|
|
||||
| |----------------|
|
||||
|------------------->| copied payload |
|
||||
|----------------|
|
||||
*/
|
||||
template<VecValueTypeClass vec_tc>
|
||||
class FirstRowAggregate final: public BatchAggregateWrapper<FirstRowAggregate<vec_tc>>
|
||||
{
|
||||
public:
|
||||
static const VecValueTypeClass IN_TC = vec_tc;
|
||||
static const VecValueTypeClass OUT_TC = vec_tc;
|
||||
public:
|
||||
FirstRowAggregate() {}
|
||||
inline int add_batch_rows(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const sql::ObBitVector &skip, const sql::EvalBound &bound,
|
||||
char *agg_cell, const RowSelector row_sel = RowSelector{}) override
|
||||
{
|
||||
SQL_LOG(DEBUG, "implicit first row", K(agg_col_id),
|
||||
K(*agg_ctx.aggr_infos_.at(agg_col_id).expr_), K(bound));
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id);
|
||||
const char *payload = nullptr;
|
||||
bool null_cell = false;
|
||||
int32_t data_len = 0;
|
||||
#ifndef NDEBUG
|
||||
helper::print_input_rows(row_sel, skip, bound, agg_ctx.aggr_infos_.at(agg_col_id), true,
|
||||
agg_ctx.eval_ctx_, this, agg_col_id);
|
||||
#endif
|
||||
if (OB_LIKELY(not_nulls.at(agg_col_id))) {
|
||||
// already copied
|
||||
} else if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null expr", K(ret));
|
||||
} else if (row_sel.is_empty()) {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (!bound.get_all_rows_active() && skip.at(i)) { // do nothing
|
||||
} else {
|
||||
ObIVector *data_vec = aggr_info.expr_->get_vector(ctx);
|
||||
if (data_vec->is_null(i)) {
|
||||
null_cell = true;
|
||||
} else {
|
||||
data_vec->get_payload(i, payload, data_len);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) {
|
||||
ObIVector *data_vec = aggr_info.expr_->get_vector(ctx);
|
||||
if (data_vec->is_null(row_sel.index(i))) {
|
||||
null_cell = true;
|
||||
} else {
|
||||
aggr_info.expr_->get_vector(ctx)->get_payload(row_sel.index(i), payload, data_len);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && (payload != nullptr || null_cell)) {
|
||||
if (null_cell) {
|
||||
agg_ctx.set_agg_cell(nullptr, INT32_MAX, agg_col_id, agg_cell);
|
||||
} else if (data_len > 0) {
|
||||
void *tmp_buf = agg_ctx.allocator_.alloc(data_len);
|
||||
if (OB_ISNULL(tmp_buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMCPY(tmp_buf, payload, data_len);
|
||||
// store data ptr and len
|
||||
agg_ctx.set_agg_cell((char *)tmp_buf, data_len, agg_col_id, agg_cell);
|
||||
}
|
||||
} else {
|
||||
agg_ctx.set_agg_cell(nullptr, data_len, agg_col_id, agg_cell);
|
||||
}
|
||||
not_nulls.set(agg_col_id);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size,
|
||||
const bool is_null, const char *data, const int32_t data_len,
|
||||
int32_t agg_col_idx, char *agg_cell) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell);
|
||||
if (OB_LIKELY(not_nulls.at(agg_col_idx))) {
|
||||
// already copied
|
||||
} else if (!is_null) {
|
||||
if (data_len > 0) {
|
||||
if (OB_ISNULL(data)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
SQL_LOG(WARN, "invalid null payload", K(ret));
|
||||
} else {
|
||||
void *tmp_buf = agg_ctx.allocator_.alloc(data_len);
|
||||
if (OB_ISNULL(tmp_buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMCPY(tmp_buf, data, data_len);
|
||||
// store data ptr and len
|
||||
agg_ctx.set_agg_cell((char *)tmp_buf, data_len, agg_col_idx, agg_cell);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
agg_ctx.set_agg_cell(nullptr, data_len, agg_col_idx, agg_cell);
|
||||
}
|
||||
not_nulls.set(agg_col_idx);
|
||||
} else {
|
||||
agg_ctx.set_agg_cell(nullptr, INT32_MAX, agg_col_idx, agg_cell);
|
||||
not_nulls.set(agg_col_idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
inline int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr,
|
||||
const int32_t agg_col_id, const char *agg_cell,
|
||||
const int32_t agg_cell_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
int64_t output_idx = ctx.get_batch_idx();
|
||||
const NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
ColumnFmt *res_vec = static_cast<ColumnFmt *>(agg_expr.get_vector(ctx));
|
||||
if (OB_LIKELY(not_nulls.at(agg_col_id) && agg_cell_len != INT32_MAX)) {
|
||||
const char *payload = (const char *)(*reinterpret_cast<const int64_t *>(agg_cell));
|
||||
char *res_buf = nullptr;
|
||||
if (need_extend_buf<AggCalcType<vec_tc>>::value) {
|
||||
res_buf = agg_expr.get_str_res_mem(ctx, agg_cell_len);
|
||||
if (OB_ISNULL(res_buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
CellWriter<AggCalcType<vec_tc>>::set(payload, agg_cell_len, res_vec, output_idx, res_buf);
|
||||
}
|
||||
} else {
|
||||
CellWriter<AggCalcType<vec_tc>>::set(payload, agg_cell_len, res_vec, output_idx, res_buf);
|
||||
}
|
||||
} else {
|
||||
res_vec->set_null(output_idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<typename ColumnFmt>
|
||||
inline int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info);
|
||||
SQL_LOG(DEBUG, "add_row do nothing");
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
inline int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res,
|
||||
int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info);
|
||||
SQL_LOG(DEBUG, "add_nullable_row do nothing");
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
TO_STRING_KV("aggregate", "first_row");
|
||||
};
|
||||
|
||||
inline int init_first_row_aggregate(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg)
|
||||
{
|
||||
#define INIT_FIRST_ROW_AGG_CASE(vec_tc) \
|
||||
case (vec_tc): { \
|
||||
if (OB_ISNULL(buf = allocator.alloc(sizeof(FirstRowAggregate<vec_tc>)))) { \
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED; \
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret)); \
|
||||
} else { \
|
||||
agg = new (buf) FirstRowAggregate<vec_tc>(); \
|
||||
} \
|
||||
} break
|
||||
int ret = OB_SUCCESS;
|
||||
void *buf = nullptr;
|
||||
ObExpr *agg_expr = agg_ctx.aggr_infos_.at(agg_col_id).expr_;
|
||||
if (OB_ISNULL(agg_expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null aggregate expr", K(ret));
|
||||
} else {
|
||||
VecValueTypeClass res_vec = get_vec_value_tc(
|
||||
agg_expr->datum_meta_.type_, agg_expr->datum_meta_.scale_, agg_expr->datum_meta_.precision_);
|
||||
switch (res_vec) {
|
||||
LST_DO_CODE(INIT_FIRST_ROW_AGG_CASE, AGG_VEC_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid result tc", K(ret), K(res_vec));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_FIRST_ROW_H_
|
||||
@ -1,223 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_HASHMAP_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_HASHMAP_H_
|
||||
|
||||
#include <utility>
|
||||
|
||||
#include "lib/hash/ob_hashutils.h"
|
||||
#include "lib/container/ob_fixed_array.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
using namespace common;
|
||||
|
||||
template<typename Key, typename Value, typename hasher = hash::hash_func<Key>>
|
||||
struct HashMap
|
||||
{
|
||||
struct Bucket
|
||||
{
|
||||
static const uint64_t OCCUPIED_MASK = (1ULL << 63);
|
||||
Bucket(): hash_(0), key_(), value_() {}
|
||||
|
||||
bool is_occupied() { return (hash_ & OCCUPIED_MASK) != 0; }
|
||||
|
||||
void set_hash(uint64_t hash) { hash_ = (hash | OCCUPIED_MASK); }
|
||||
void reset() { hash_ = 0; }
|
||||
uint64_t hash_;
|
||||
Key key_;
|
||||
Value value_;
|
||||
TO_STRING_KV(K_(hash), K_(key), K_(value), K(is_occupied()));
|
||||
};
|
||||
HashMap(): buckets_(nullptr), cnt_(0), cap_(0), allocator_(nullptr) {}
|
||||
~HashMap() { destroy(); }
|
||||
inline int init(ObIAllocator &allocator, int64_t cap)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *buf = nullptr;
|
||||
int64_t buf_size = cap * sizeof(Bucket);
|
||||
if (OB_UNLIKELY(cap <= 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected cap size", K(ret), K(cap));
|
||||
} else if (OB_ISNULL(buf = allocator.alloc(buf_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMSET(buf, 0, buf_size);
|
||||
buckets_ = reinterpret_cast<Bucket *>(buf);
|
||||
cap_ = cap;
|
||||
cnt_ = 0;
|
||||
allocator_ = &allocator;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
inline int get(const Key &key, Value *&value)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t hash_val = 0;
|
||||
OB_ASSERT(buckets_ != NULL);
|
||||
if (OB_UNLIKELY(need_extend())) {
|
||||
SQL_LOG(INFO, "extending hashmap", K(cap_));
|
||||
if (OB_FAIL(extend())) {
|
||||
SQL_LOG(WARN, "extend hash table failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(hasher()(key, hash_val))) {
|
||||
SQL_LOG(WARN, "hahs calculation failed", K(ret));
|
||||
} else {
|
||||
int64_t probe_cnt = 0;
|
||||
Bucket *bucket = nullptr;
|
||||
|
||||
if (OB_NOT_NULL(bucket = inner_get(hash_val, key))) {
|
||||
if (bucket->is_occupied()) {
|
||||
value = &bucket->value_;
|
||||
} else {
|
||||
bucket->set_hash(hash_val);
|
||||
bucket->key_ = key;
|
||||
bucket->value_ = Value();
|
||||
value = &bucket->value_;
|
||||
cnt_++;
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null bucket", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int erase(const Key &key)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t hash_val = 0;
|
||||
OB_ASSERT(buckets_ != NULL);
|
||||
if (OB_FAIL(hasher()(key, hash_val))) {
|
||||
SQL_LOG(WARN, "hash calculation failed", K(ret));
|
||||
} else {
|
||||
Bucket *bucket = nullptr;
|
||||
if (OB_NOT_NULL(bucket = inner_get(hash_val, key))) {
|
||||
bucket->reset();
|
||||
cnt_--;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void reuse()
|
||||
{
|
||||
MEMSET(buckets_, 0, cap_ * sizeof(Bucket));
|
||||
cnt_ = 0;
|
||||
}
|
||||
|
||||
void destroy()
|
||||
{
|
||||
buckets_ = nullptr;
|
||||
cap_ = 0;
|
||||
cnt_ = 0;
|
||||
allocator_ = nullptr;
|
||||
}
|
||||
private:
|
||||
inline bool need_extend() const { return cnt_ * 2 >= cap_; }
|
||||
inline int64_t bucket_idx(const uint64_t hash_val) const
|
||||
{
|
||||
return (hash_val ^ Bucket::OCCUPIED_MASK) % cap_;
|
||||
}
|
||||
inline Bucket *inner_get(const uint64_t hash_val, const Key &key)
|
||||
{
|
||||
int64_t pos = bucket_idx(hash_val);
|
||||
int64_t probe_cnt = 0;
|
||||
Bucket *ret_bucket = nullptr;
|
||||
while (probe_cnt < cap_ && ret_bucket == NULL) {
|
||||
Bucket &bucket = buckets_[pos];
|
||||
if (bucket.is_occupied() && bucket.key_ == key) {
|
||||
ret_bucket = &bucket;
|
||||
break;
|
||||
} else if (bucket.is_occupied()) {
|
||||
pos = (pos + 1) % cap_;
|
||||
probe_cnt++;
|
||||
} else {
|
||||
// find empty slot
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (ret_bucket != nullptr) {
|
||||
// do nothing
|
||||
} else if (!buckets_[pos].is_occupied()) {
|
||||
ret_bucket = &buckets_[pos];
|
||||
}
|
||||
return ret_bucket;
|
||||
}
|
||||
|
||||
inline int extend()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t new_cap = cap_ * 2, old_cap = cap_;
|
||||
void *buf = nullptr;
|
||||
int64_t buf_size = new_cap * sizeof(Bucket);
|
||||
Bucket *old_buckets = buckets_;
|
||||
if (OB_UNLIKELY(new_cap <= 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid cap size", K(ret), K(new_cap));
|
||||
} else if (OB_ISNULL(allocator_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null allocator", K(ret));
|
||||
} else if (OB_ISNULL(buf = allocator_->alloc(buf_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMSET(buf, 0, buf_size);
|
||||
buckets_ = reinterpret_cast<Bucket *>(buf);
|
||||
cap_ = new_cap;
|
||||
Bucket *new_bucket = nullptr;
|
||||
for (int i = 0; OB_SUCC(ret) && i < old_cap; i++) {
|
||||
Bucket &old = old_buckets[i];
|
||||
if (!old.is_occupied()) {
|
||||
continue;
|
||||
} else if (OB_ISNULL(new_bucket = inner_get(old.hash_, old.key_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null new bucket", K(ret));
|
||||
} else if (OB_UNLIKELY(new_bucket->is_occupied())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "duplicated element", K(ret));
|
||||
} else {
|
||||
new_bucket->set_hash(old.hash_);
|
||||
new_bucket->key_ = old.key_;
|
||||
new_bucket->value_ = old.value_;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret) && buf != nullptr && allocator_ != nullptr) {
|
||||
allocator_->free(buf);
|
||||
buckets_ = old_buckets;
|
||||
cap_ = old_cap;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
private:
|
||||
// ObFixedArray<Bucket, ObIAllocator> *buckets_;
|
||||
Bucket *buckets_;
|
||||
int64_t cnt_;
|
||||
int64_t cap_;
|
||||
ObIAllocator *allocator_;
|
||||
|
||||
};
|
||||
} // aggregate
|
||||
} // share
|
||||
} // oceanbase
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_HASHMAP_H_
|
||||
@ -1,340 +0,0 @@
|
||||
/**
|
||||
* 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 "iaggregate.h"
|
||||
#include "share/datum/ob_datum_util.h"
|
||||
#include "share/aggregate/count.h"
|
||||
#include "share/aggregate/min_max.h"
|
||||
#include "share/aggregate/sum.h"
|
||||
#include "share/aggregate/first_row.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
namespace helper
|
||||
{
|
||||
using namespace sql;
|
||||
using namespace common;
|
||||
|
||||
extern int init_count_aggregate(RuntimeContext &agg_ctx,
|
||||
const int64_t agg_col_id, ObIAllocator &allocator,
|
||||
IAggregate *&agg);
|
||||
extern int init_min_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg);
|
||||
extern int init_max_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg);
|
||||
extern int init_sum_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg, int32 *tmp_res_size = NULL);
|
||||
extern int init_count_sum_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg);
|
||||
#define INIT_AGGREGATE_CASE(OP_TYPE, func_name) \
|
||||
case (OP_TYPE): { \
|
||||
ret = init_##func_name##_aggregate(agg_ctx, i, allocator, agg); \
|
||||
} break
|
||||
int init_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator,
|
||||
ObIArray<IAggregate *> &aggregates)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx &ctx= agg_ctx.eval_ctx_;
|
||||
if (OB_FAIL(agg_ctx.init_row_meta(agg_ctx.aggr_infos_, allocator))) {
|
||||
SQL_LOG(WARN, "init row meta failed", K(ret));
|
||||
}
|
||||
for (int i = 0; OB_SUCC(ret) && i < agg_ctx.aggr_infos_.count(); i++) {
|
||||
ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(i);
|
||||
IAggregate *agg = nullptr;
|
||||
if (aggr_info.is_implicit_first_aggr()) {
|
||||
if (OB_FAIL(init_first_row_aggregate(agg_ctx, i, allocator, agg))) {
|
||||
SQL_LOG(WARN, "init first row aggregate failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
switch (aggr_info.expr_->type_) {
|
||||
INIT_AGGREGATE_CASE(T_FUN_MIN, min);
|
||||
INIT_AGGREGATE_CASE(T_FUN_MAX, max);
|
||||
INIT_AGGREGATE_CASE(T_FUN_COUNT, count);
|
||||
INIT_AGGREGATE_CASE(T_FUN_SUM, sum);
|
||||
INIT_AGGREGATE_CASE(T_FUN_COUNT_SUM, count_sum);
|
||||
default: {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
SQL_LOG(WARN, "not supported aggregate function", K(ret), K(aggr_info.expr_->type_));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "init aggregate failed", K(ret));
|
||||
} else if (OB_ISNULL(agg)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null aggregate", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(aggregates.push_back(agg))) {
|
||||
SQL_LOG(WARN, "push back element failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
return ret;
|
||||
}
|
||||
#undef INIT_AGGREGATE_CASE
|
||||
|
||||
static int32_t agg_cell_tmp_res_size(RuntimeContext &agg_ctx, int64_t agg_col_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int ret_size = 0;
|
||||
int32_t tmp_res_size = 0;
|
||||
char buffer[1] = {0};
|
||||
IAggregate *agg = nullptr;
|
||||
ObDataBuffer local_allocator(buffer, 1);
|
||||
ObAggrInfo &info = agg_ctx.aggr_infos_.at(agg_col_id);
|
||||
if (info.is_implicit_first_aggr()) {
|
||||
// do nothing
|
||||
} else if (info.get_expr_type() == T_FUN_MIN || info.get_expr_type() == T_FUN_MAX) {
|
||||
VecValueTypeClass vec_tc = info.expr_->get_vec_value_tc();
|
||||
if (is_var_len_agg_cell(vec_tc)) {
|
||||
ret_size = sizeof(char *) + sizeof(int32_t); // <char *, int32_t>
|
||||
}
|
||||
} else if (info.get_expr_type() == T_FUN_SUM) {
|
||||
if (OB_FAIL( // ugly code, works just fine
|
||||
init_sum_aggregate(agg_ctx, agg_col_id, local_allocator, agg, &tmp_res_size))) {
|
||||
SQL_LOG(ERROR, "get sum tmp res size failed", K(ret));
|
||||
ob_abort();
|
||||
} else {
|
||||
ret_size = tmp_res_size;
|
||||
}
|
||||
}
|
||||
return ret_size;
|
||||
}
|
||||
|
||||
static int32_t reserved_agg_col_size(RuntimeContext &agg_ctx, int64_t agg_col_id)
|
||||
{
|
||||
int ret_size = 0;
|
||||
ObAggrInfo &aggr_info = agg_ctx.aggr_infos_.at(agg_col_id);
|
||||
const int64_t constexpr string_reserved_size = sizeof(char *) + sizeof(int32_t); // <char *, len>
|
||||
#define RTSIZE(vec_tc) sizeof(RTCType<vec_tc>)
|
||||
static const int32_t reserved_sizes[] = {
|
||||
0, // NULL
|
||||
RTSIZE(VEC_TC_INTEGER), // integer
|
||||
RTSIZE(VEC_TC_UINTEGER), // uinteger
|
||||
RTSIZE(VEC_TC_FLOAT), // float
|
||||
RTSIZE(VEC_TC_DOUBLE), // double
|
||||
RTSIZE(VEC_TC_FIXED_DOUBLE), // fixed_double
|
||||
number::ObNumber::MAX_CALC_BYTE_LEN, // number
|
||||
RTSIZE(VEC_TC_DATETIME), // datetime
|
||||
RTSIZE(VEC_TC_DATE), // date
|
||||
RTSIZE(VEC_TC_TIME), // time
|
||||
RTSIZE(VEC_TC_YEAR), // year
|
||||
0, // extend
|
||||
0, // unknown
|
||||
string_reserved_size, // string
|
||||
RTSIZE(VEC_TC_BIT), // bit
|
||||
RTSIZE(VEC_TC_ENUM_SET), // enum set
|
||||
0, // enum set inner
|
||||
RTSIZE(VEC_TC_TIMESTAMP_TZ), // timestamp tz
|
||||
RTSIZE(VEC_TC_TIMESTAMP_TINY), // timestamp tiny
|
||||
string_reserved_size, // raw
|
||||
RTSIZE(VEC_TC_INTERVAL_YM), // interval ym
|
||||
RTSIZE(VEC_TC_INTERVAL_DS), // interval ds
|
||||
string_reserved_size, // rowid
|
||||
string_reserved_size, // lob
|
||||
string_reserved_size, // json
|
||||
string_reserved_size, // geo
|
||||
string_reserved_size, // udt
|
||||
RTSIZE(VEC_TC_DEC_INT32), // dec_int32
|
||||
RTSIZE(VEC_TC_DEC_INT64), // dec_int64
|
||||
RTSIZE(VEC_TC_DEC_INT128), // dec_int128
|
||||
RTSIZE(VEC_TC_DEC_INT256), // dec_int256
|
||||
RTSIZE(VEC_TC_DEC_INT512), // dec_int512
|
||||
};
|
||||
static_assert(sizeof(reserved_sizes) / sizeof(reserved_sizes[0]) == MAX_VEC_TC, "");
|
||||
OB_ASSERT(aggr_info.expr_ != NULL);
|
||||
VecValueTypeClass res_tc = get_vec_value_tc(aggr_info.expr_->datum_meta_.type_,
|
||||
aggr_info.expr_->datum_meta_.scale_,
|
||||
aggr_info.expr_->datum_meta_.precision_);
|
||||
if (aggr_info.is_implicit_first_aggr()) {
|
||||
ret_size += string_reserved_size; // <char *, len>;
|
||||
} else if (aggr_info.get_expr_type() == T_FUN_COUNT) {
|
||||
// count returns ObNumberType in oracle mode,
|
||||
// we use int64_t as row counts recording type, and cast int64_t to ObNumberType in
|
||||
// `collect_group_result`
|
||||
ret_size += sizeof(int64_t);
|
||||
} else if (is_var_len_agg_cell(res_tc)) {
|
||||
ret_size += string_reserved_size;
|
||||
} else {
|
||||
ret_size += reserved_sizes[res_tc];
|
||||
}
|
||||
ret_size += agg_cell_tmp_res_size(agg_ctx, agg_col_id);
|
||||
return ret_size;
|
||||
}
|
||||
|
||||
inline bool has_extra_info(ObAggrInfo &info)
|
||||
{
|
||||
bool has = false;
|
||||
switch (info.get_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_MEDIAN:
|
||||
case T_FUN_GROUP_PERCENTILE_CONT:
|
||||
case T_FUN_GROUP_PERCENTILE_DISC:
|
||||
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:
|
||||
case T_FUN_PL_AGG_UDF:
|
||||
case T_FUN_JSON_ARRAYAGG:
|
||||
case T_FUN_ORA_JSON_ARRAYAGG:
|
||||
case T_FUN_JSON_OBJECTAGG:
|
||||
case T_FUN_ORA_JSON_OBJECTAGG:
|
||||
case T_FUN_ORA_XMLAGG:
|
||||
case T_FUN_HYBRID_HIST:
|
||||
case T_FUN_TOP_FRE_HIST:
|
||||
case T_FUN_AGG_UDF: {
|
||||
has = true;
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
break;
|
||||
}
|
||||
}
|
||||
has = has || info.has_distinct_;
|
||||
return has;
|
||||
}
|
||||
} // end namespace helper
|
||||
|
||||
int RuntimeContext::init_row_meta(ObIArray<ObAggrInfo> &aggr_infos, ObIAllocator &alloc)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
agg_row_meta_.row_size_ = 0;
|
||||
agg_row_meta_.col_cnt_ = aggr_infos.count();
|
||||
agg_row_meta_.extra_cnt_ = 0;
|
||||
int32_t offset = 0;
|
||||
bool has_extra = false;
|
||||
int64_t bit_vec_size =
|
||||
((aggr_infos.count() + AggBitVector::word_bits() - 1) / AggBitVector::word_bits())
|
||||
* AggBitVector::word_size();
|
||||
uint8_t *bit_payload = nullptr;
|
||||
if (OB_ISNULL(agg_row_meta_.col_offsets_ = (int32_t *)alloc.alloc(
|
||||
sizeof(int32_t)
|
||||
* (aggr_infos.count() + 1)))) { // one extra offset to calculate column size
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else if (OB_ISNULL(agg_row_meta_.extra_idxes_ =
|
||||
(int32_t *)alloc.alloc(sizeof(int32_t) * aggr_infos.count()))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret), K(aggr_infos.count()));
|
||||
} else if (OB_ISNULL(bit_payload = (uint8_t *)alloc.alloc(bit_vec_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else if (OB_ISNULL(agg_row_meta_.tmp_res_sizes_ =
|
||||
(int32_t *)alloc.alloc(sizeof(int32_t) * aggr_infos.count()))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMSET(agg_row_meta_.col_offsets_, -1, sizeof(int32_t) * (aggr_infos.count() + 1));
|
||||
MEMSET(agg_row_meta_.extra_idxes_, -1, sizeof(int32_t) * aggr_infos.count());
|
||||
MEMSET(bit_payload, 0, bit_vec_size);
|
||||
MEMSET(agg_row_meta_.tmp_res_sizes_, 0, sizeof(int32_t) * aggr_infos.count());
|
||||
agg_row_meta_.use_var_len_ = reinterpret_cast<AggBitVector *>(bit_payload);
|
||||
}
|
||||
int32_t agg_extra_id = 0;
|
||||
for (int i = 0; OB_SUCC(ret) && i < aggr_infos.count(); i++) {
|
||||
ObAggrInfo &info = aggr_infos.at(i);
|
||||
agg_row_meta_.row_size_ += helper::reserved_agg_col_size(*this, i);
|
||||
agg_row_meta_.tmp_res_sizes_[i] = helper::agg_cell_tmp_res_size(*this, i);
|
||||
agg_row_meta_.col_offsets_[i] = offset;
|
||||
offset = agg_row_meta_.row_size_;
|
||||
VecValueTypeClass vec_tc =
|
||||
get_vec_value_tc(info.expr_->datum_meta_.type_, info.expr_->datum_meta_.scale_,
|
||||
info.expr_->datum_meta_.precision_);
|
||||
if (info.is_implicit_first_aggr() || helper::is_var_len_agg_cell(vec_tc)) {
|
||||
agg_row_meta_.use_var_len_->set(i);
|
||||
}
|
||||
if (helper::has_extra_info(info)) {
|
||||
has_extra = true;
|
||||
agg_row_meta_.extra_idxes_[i] = agg_extra_id++;
|
||||
}
|
||||
}
|
||||
agg_row_meta_.extra_cnt_ = agg_extra_id;
|
||||
agg_row_meta_.col_offsets_[aggr_infos.count()] = agg_row_meta_.row_size_;
|
||||
if (has_extra) {
|
||||
agg_row_meta_.row_size_ += sizeof(int32_t);
|
||||
agg_row_meta_.extra_idx_offset_ = offset;
|
||||
offset = agg_row_meta_.row_size_;
|
||||
} else {
|
||||
agg_row_meta_.extra_idx_offset_ = -1;
|
||||
}
|
||||
agg_row_meta_.row_size_ += bit_vec_size;
|
||||
agg_row_meta_.nullbits_offset_ = offset;
|
||||
return ret;
|
||||
}
|
||||
|
||||
namespace helper
|
||||
{
|
||||
void print_input_rows(const RowSelector &row_sel, const sql::ObBitVector &skip,
|
||||
const sql::EvalBound &bound, const sql::ObAggrInfo &aggr_info,
|
||||
bool is_first_row, sql::ObEvalCtx &ctx, IAggregate *agg, int64_t col_id)
|
||||
{
|
||||
const char *payload = nullptr;
|
||||
int32_t len = 0;
|
||||
ObDatum d;
|
||||
sql::ObEvalCtx::BatchInfoScopeGuard batch_guard(ctx);
|
||||
if (row_sel.is_empty()) {
|
||||
for (int i = bound.start(); i < bound.end(); i++) {
|
||||
if (skip.at(i)) { continue; }
|
||||
batch_guard.set_batch_idx(i);
|
||||
if (is_first_row) {
|
||||
ObIVector *data_vec = aggr_info.expr_->get_vector(ctx);
|
||||
if (VEC_INVALID == aggr_info.expr_->get_format(ctx)) { // do nothing
|
||||
} else if (data_vec->is_null(i)) {
|
||||
d.set_null();
|
||||
} else {
|
||||
aggr_info.expr_->get_vector(ctx)->get_payload(i, payload, len);
|
||||
d.ptr_ = payload;
|
||||
d.len_ = len;
|
||||
}
|
||||
SQL_LOG(DEBUG, "add row", K(DATUM2STR(*aggr_info.expr_, d)), K(*agg), K(col_id));
|
||||
} else {
|
||||
SQL_LOG(DEBUG, "add row", K(VEC_ROWEXPR2STR(ctx, aggr_info.param_exprs_)), K(*agg),
|
||||
K(col_id));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < row_sel.size(); i++) {
|
||||
int idx = row_sel.index(i);
|
||||
batch_guard.set_batch_idx(idx);
|
||||
if (is_first_row) {
|
||||
ObIVector *data_vec = aggr_info.expr_->get_vector(ctx);
|
||||
if (VEC_INVALID == aggr_info.expr_->get_format(ctx)) { // do nothing
|
||||
} else if (data_vec->is_null(idx)) {
|
||||
d.set_null();
|
||||
} else {
|
||||
aggr_info.expr_->get_vector(ctx)->get_payload(idx, payload, len);
|
||||
d.ptr_ = payload;
|
||||
d.len_ = len;
|
||||
}
|
||||
SQL_LOG(DEBUG, "add row", K(DATUM2STR(*aggr_info.expr_, d)), K(*agg), K(col_id));
|
||||
} else {
|
||||
SQL_LOG(DEBUG, "add row", K(VEC_ROWEXPR2STR(ctx, aggr_info.param_exprs_)), K(*agg),
|
||||
K(col_id));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} // namespace helper
|
||||
|
||||
} // end namespace aggregate
|
||||
} // end namespace share
|
||||
} // end namespace oceanbase
|
||||
@ -1,802 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_IAGGREGATE_H_
|
||||
#define OCEANBASE_SHARE_IAGGREGATE_H_
|
||||
|
||||
#include "lib/container/ob_iarray.h"
|
||||
#include "lib/container/ob_fixed_array.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "sql/engine/expr/ob_expr.h"
|
||||
// TODO: split this big headers
|
||||
#include "sql/engine/aggregate/ob_aggregate_processor.h"
|
||||
#include "sql/engine/expr/ob_expr_util.h"
|
||||
#include "share/vector/vector_basic_op.h"
|
||||
#include "share/ob_define.h"
|
||||
#include "share/aggregate/util.h"
|
||||
#include "share/aggregate/agg_ctx.h"
|
||||
#include "sql/engine/basic/ob_compact_row.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
|
||||
namespace aggregate
|
||||
{
|
||||
using namespace sql;
|
||||
|
||||
int quick_add_batch_rows_for_count(IAggregate *agg, RuntimeContext &agg_ctx,
|
||||
const bool is_single_row_agg, const sql::ObBitVector &skip,
|
||||
const sql::EvalBound &bound, const RowSelector &row_sel,
|
||||
const int32_t agg_col_id, char *agg_cell);
|
||||
namespace helper
|
||||
{
|
||||
// printer helper
|
||||
void print_input_rows(const RowSelector &row_sel, const sql::ObBitVector &skip,
|
||||
const sql::EvalBound &bound, const sql::ObAggrInfo &aggr_info,
|
||||
bool is_first_row, sql::ObEvalCtx &ctx, IAggregate *agg, int64_t col_id);
|
||||
} // end helper
|
||||
|
||||
template<ObExprOperatorType agg_func, VecValueTypeClass in_tc, VecValueTypeClass out_tc>
|
||||
struct SingleRowAggregate;
|
||||
|
||||
template<typename Derived>
|
||||
class BatchAggregateWrapper: public IAggregate
|
||||
{
|
||||
public:
|
||||
inline void set_inner_aggregate(IAggregate *agg) override
|
||||
{
|
||||
UNUSEDx(agg);
|
||||
return;
|
||||
}
|
||||
|
||||
inline int init(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator) override
|
||||
{
|
||||
UNUSEDx(agg_ctx, agg_col_id, allocator);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
inline void destroy() override
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
inline void reuse() override
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
inline void* get_tmp_res(RuntimeContext &agg_ctx, int32_t agg_col_idx, char *agg_cell) override
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
inline int64_t get_batch_calc_info(RuntimeContext &agg_ctx, int32_t agg_col_idx,
|
||||
char *agg_cell) override
|
||||
{
|
||||
UNUSEDx(agg_ctx, agg_col_idx, agg_cell);
|
||||
return 0;
|
||||
}
|
||||
|
||||
int add_batch_rows(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const sql::ObBitVector &skip, const sql::EvalBound &bound, char *agg_cell,
|
||||
const RowSelector row_sel = RowSelector{}) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_ASSERT(agg_col_id < agg_ctx.aggr_infos_.count());
|
||||
Derived &derived = *static_cast<Derived *>(this);
|
||||
sql::ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
ObIArray<ObExpr *> ¶m_exprs = agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_;
|
||||
#ifndef NDEBUG
|
||||
helper::print_input_rows(row_sel, skip, bound, agg_ctx.aggr_infos_.at(agg_col_id), false,
|
||||
agg_ctx.eval_ctx_, this, agg_col_id);
|
||||
#endif
|
||||
if (param_exprs.count() == 1
|
||||
|| agg_ctx.aggr_infos_.at(agg_col_id).is_implicit_first_aggr()) { // by pass implicit first row
|
||||
VectorFormat fmt = VEC_INVALID;
|
||||
ObExpr *param_expr = nullptr;
|
||||
// by pass implicit first row
|
||||
// set param_expr to aggr_info.expr_
|
||||
if (agg_ctx.aggr_infos_.at(agg_col_id).is_implicit_first_aggr()) {
|
||||
fmt = agg_ctx.aggr_infos_.at(agg_col_id).expr_->get_format(ctx);
|
||||
param_expr = agg_ctx.aggr_infos_.at(agg_col_id).expr_;
|
||||
} else {
|
||||
fmt = param_exprs.at(0)->get_format(ctx);
|
||||
param_expr = param_exprs.at(0);
|
||||
}
|
||||
switch (fmt) {
|
||||
case common::VEC_UNIFORM: {
|
||||
ret = add_batch_rows<uniform_fmt<Derived::IN_TC, false>>(
|
||||
agg_ctx, skip, bound, *param_expr, agg_col_id, agg_cell, row_sel);
|
||||
break;
|
||||
}
|
||||
case common::VEC_UNIFORM_CONST: {
|
||||
ret = add_batch_rows<uniform_fmt<Derived::IN_TC, true>>(
|
||||
agg_ctx, skip, bound, *param_expr, agg_col_id, agg_cell, row_sel);
|
||||
break;
|
||||
}
|
||||
case common::VEC_FIXED: {
|
||||
ret = add_batch_rows<fixlen_fmt<Derived::IN_TC>>(
|
||||
agg_ctx, skip, bound, *param_expr, agg_col_id, agg_cell, row_sel);
|
||||
break;
|
||||
}
|
||||
case common::VEC_DISCRETE: {
|
||||
ret = add_batch_rows<discrete_fmt<Derived::IN_TC>>(
|
||||
agg_ctx, skip, bound, *param_expr, agg_col_id, agg_cell, row_sel);
|
||||
break;
|
||||
}
|
||||
case common::VEC_CONTINUOUS: {
|
||||
ret = add_batch_rows<continuous_fmt<Derived::IN_TC>>(
|
||||
agg_ctx, skip, bound, *param_expr, agg_col_id, agg_cell, row_sel);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected fmt", K(fmt), K(*param_exprs.at(0)));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "add batch rows failed", K(ret), K(fmt), K(param_exprs.at(0)->datum_meta_));
|
||||
}
|
||||
} else if (param_exprs.empty() && !agg_ctx.aggr_infos_.at(agg_col_id).is_implicit_first_aggr()) {
|
||||
ObItemType agg_fun_type = agg_ctx.aggr_infos_.at(agg_col_id).get_expr_type();
|
||||
if (T_FUN_COUNT == agg_fun_type) {
|
||||
if (OB_FAIL(quick_add_batch_rows_for_count(
|
||||
this, agg_ctx,
|
||||
(std::is_same<Derived, SingleRowAggregate<T_FUN_COUNT, VEC_TC_INTEGER, VEC_TC_INTEGER>>::value
|
||||
|| std::is_same<Derived, SingleRowAggregate<T_FUN_COUNT, VEC_TC_INTEGER, VEC_TC_NUMBER>>::value),
|
||||
skip, bound, row_sel, agg_col_id, agg_cell))) {
|
||||
SQL_LOG(WARN, "quick add batch rows failed", K(ret));
|
||||
}
|
||||
} else if (T_FUN_GROUP_ID == agg_fun_type) {
|
||||
// TODO:
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected aggregate function", K(ret), K(agg_fun_type),
|
||||
K(*agg_ctx.aggr_infos_.at(agg_col_id).expr_));
|
||||
}
|
||||
} else if (defined_add_param_batch<Derived>::value) {
|
||||
if (OB_FAIL(add_params_batch_row(agg_ctx, agg_col_id, skip, bound, row_sel, agg_cell))) {
|
||||
SQL_LOG(WARN, "add param batch rows failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "should not arrive here", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int collect_batch_group_results(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const int32_t cur_group_id, const int32_t output_start_idx,
|
||||
const int32_t expect_batch_size, int32_t &output_size,
|
||||
const ObBitVector *skip = nullptr) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_ASSERT(agg_col_id < agg_ctx.aggr_infos_.count());
|
||||
OB_ASSERT(agg_ctx.aggr_infos_.at(agg_col_id).expr_ != NULL);
|
||||
|
||||
ObExpr &agg_expr = *agg_ctx.aggr_infos_.at(agg_col_id).expr_;
|
||||
int32_t loop_cnt =
|
||||
min(expect_batch_size, static_cast<int32_t>(agg_ctx.agg_rows_.count()) - cur_group_id);
|
||||
if (loop_cnt <= 0) {
|
||||
SQL_LOG(DEBUG, "no need to collect", K(ret), K(agg_ctx.agg_rows_.count()), K(cur_group_id));
|
||||
} else {
|
||||
output_size = 0;
|
||||
if (OB_FAIL(agg_expr.init_vector_for_write(
|
||||
agg_ctx.eval_ctx_, agg_expr.get_default_res_format(), expect_batch_size))) {
|
||||
SQL_LOG(WARN, "init vector for write failed", K(ret));
|
||||
} else {
|
||||
VectorFormat res_vec = agg_expr.get_format(agg_ctx.eval_ctx_);
|
||||
VecValueTypeClass res_tc = agg_expr.get_vec_value_tc();
|
||||
switch (res_vec) {
|
||||
case common::VEC_FIXED: {
|
||||
ret = collect_group_results<fixlen_fmt<Derived::OUT_TC>>(
|
||||
agg_ctx, agg_col_id, cur_group_id, output_start_idx, loop_cnt, skip);
|
||||
break;
|
||||
}
|
||||
case common::VEC_DISCRETE: {
|
||||
ret = collect_group_results<discrete_fmt<Derived::OUT_TC>>(
|
||||
agg_ctx, agg_col_id, cur_group_id, output_start_idx, loop_cnt, skip);
|
||||
break;
|
||||
}
|
||||
case common::VEC_UNIFORM_CONST: {
|
||||
// must be null
|
||||
ret = ret = collect_group_results<uniform_fmt<VEC_TC_NULL, true>>(
|
||||
agg_ctx, agg_col_id, cur_group_id, output_start_idx, loop_cnt, skip);
|
||||
break;
|
||||
}
|
||||
case common::VEC_UNIFORM: {
|
||||
// must be null
|
||||
ret = collect_group_results<uniform_fmt<VEC_TC_NULL, false>>(
|
||||
agg_ctx, agg_col_id, cur_group_id, output_start_idx, loop_cnt, skip);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid result format", K(ret), K(res_vec), K(agg_col_id));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "collect batch group results failed", K(ret), K(res_vec));
|
||||
} else {
|
||||
output_size = loop_cnt;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int collect_batch_group_results(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const int32_t output_start_idx, const int32_t batch_size,
|
||||
const ObCompactRow **rows, const RowMeta &row_meta) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_ASSERT(agg_col_id < agg_ctx.aggr_infos_.count());
|
||||
OB_ASSERT(agg_ctx.aggr_infos_.at(agg_col_id).expr_ != NULL);
|
||||
|
||||
ObExpr &agg_expr = *agg_ctx.aggr_infos_.at(agg_col_id).expr_;
|
||||
if (OB_LIKELY(batch_size > 0)) {
|
||||
if (OB_FAIL(agg_expr.init_vector_for_write(
|
||||
agg_ctx.eval_ctx_, agg_expr.get_default_res_format(), batch_size))) {
|
||||
SQL_LOG(WARN, "init vector for write failed", K(ret));
|
||||
} else {
|
||||
VectorFormat res_fmt = agg_expr.get_format(agg_ctx.eval_ctx_);
|
||||
VecValueTypeClass res_tc = get_vec_value_tc(
|
||||
agg_expr.datum_meta_.type_, agg_expr.datum_meta_.scale_, agg_expr.datum_meta_.precision_);
|
||||
switch (res_fmt) {
|
||||
case VEC_FIXED: {
|
||||
ret = collect_group_results<fixlen_fmt<Derived::OUT_TC>>(
|
||||
agg_ctx, agg_col_id, output_start_idx, batch_size, rows, row_meta);
|
||||
break;
|
||||
}
|
||||
case VEC_DISCRETE: {
|
||||
ret = collect_group_results<discrete_fmt<Derived::OUT_TC>>(
|
||||
agg_ctx, agg_col_id, output_start_idx, batch_size, rows, row_meta);
|
||||
break;
|
||||
}
|
||||
case VEC_UNIFORM_CONST: {
|
||||
// must be null type
|
||||
ret = collect_group_results<uniform_fmt<VEC_TC_NULL, true>>(
|
||||
agg_ctx, agg_col_id, output_start_idx, batch_size, rows, row_meta);
|
||||
break;
|
||||
}
|
||||
case VEC_UNIFORM: {
|
||||
// must be null type
|
||||
ret = collect_group_results<uniform_fmt<VEC_TC_NULL, false>>(
|
||||
agg_ctx, agg_col_id, output_start_idx, batch_size, rows, row_meta);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid result format", K(ret), K(res_fmt), K(agg_col_id));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) { SQL_LOG(WARN, "collect batch group results failed", K(ret)); }
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
protected:
|
||||
template <typename ColumnFmt>
|
||||
int add_batch_rows(RuntimeContext &agg_ctx, const sql::ObBitVector &skip,
|
||||
const sql::EvalBound &bound, const ObExpr ¶m_expr,
|
||||
const int32_t agg_col_id, char *agg_cell, const RowSelector row_sel)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
auto &columns = *static_cast<ColumnFmt *>(param_expr.get_vector(ctx));
|
||||
bool all_not_null = !columns.has_null();
|
||||
Derived &derived = *static_cast<Derived *>(this);
|
||||
void *tmp_res = derived.get_tmp_res(agg_ctx, agg_col_id, agg_cell);
|
||||
int64_t calc_info = derived.get_batch_calc_info(agg_ctx, agg_col_id, agg_cell);
|
||||
if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) {
|
||||
if (all_not_null) {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (OB_FAIL(
|
||||
derived.add_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res, calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
if (agg_cell != nullptr) {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
not_nulls.set(agg_col_id);
|
||||
}
|
||||
} else {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res,
|
||||
calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
}
|
||||
} else if (!row_sel.is_empty()) {
|
||||
if (all_not_null) {
|
||||
for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) {
|
||||
if (OB_FAIL(derived.add_row(agg_ctx, columns, row_sel.index(i), agg_col_id, agg_cell,
|
||||
tmp_res, calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
if (agg_cell != nullptr) {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
not_nulls.set(agg_col_id);
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) {
|
||||
if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, row_sel.index(i), agg_col_id,
|
||||
agg_cell, tmp_res, calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
}
|
||||
} else {
|
||||
if (all_not_null) {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (skip.at(i)) {
|
||||
} else if (OB_FAIL(derived.add_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res,
|
||||
calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
if (agg_cell != nullptr) {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
not_nulls.set(agg_col_id);
|
||||
}
|
||||
} else {
|
||||
for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) {
|
||||
if (skip.at(i)) {
|
||||
} else if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, i, agg_col_id, agg_cell,
|
||||
tmp_res, calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(
|
||||
collect_tmp_result<Derived>::do_op(derived, agg_ctx, agg_col_id, agg_cell))) {
|
||||
SQL_LOG(WARN, "collect tmp result failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<>
|
||||
int add_batch_rows<ObVectorBase>(RuntimeContext &agg_ctx, const sql::ObBitVector &skip,
|
||||
const sql::EvalBound &bound, const ObExpr ¶m_expr,
|
||||
const int32_t agg_col_id, char *agg_cell,
|
||||
const RowSelector row_sel)
|
||||
{
|
||||
int ret = OB_NOT_IMPLEMENT;
|
||||
SQL_LOG(WARN, "not implemented", K(ret), K(*this));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ResultFmt>
|
||||
int collect_group_results(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const int32_t start_gid, const int32_t start_output_idx,
|
||||
const int32_t batch_size, const ObBitVector *skip)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
OB_ASSERT(agg_col_id < agg_ctx.aggr_infos_.count());
|
||||
OB_ASSERT(agg_ctx.aggr_infos_.at(agg_col_id).expr_ != NULL);
|
||||
ObExpr *agg_expr = agg_ctx.aggr_infos_.at(agg_col_id).expr_;
|
||||
ObEvalCtx::BatchInfoScopeGuard guard(agg_ctx.eval_ctx_);
|
||||
|
||||
const char *agg_cell = nullptr;
|
||||
int32_t agg_cell_len = 0;
|
||||
for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) {
|
||||
if (skip != nullptr && skip->at(start_output_idx + i)) { continue; }
|
||||
guard.set_batch_idx(start_output_idx + i);
|
||||
agg_cell = nullptr;
|
||||
agg_cell_len = 0;
|
||||
agg_ctx.get_agg_payload(agg_col_id, start_gid + i, agg_cell, agg_cell_len);
|
||||
if (OB_FAIL(static_cast<Derived *>(this)->template collect_group_result<ResultFmt>(
|
||||
agg_ctx, *agg_expr, agg_col_id, agg_cell, agg_cell_len))) {
|
||||
SQL_LOG(WARN, "collect group result failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <>
|
||||
int collect_group_results<ObVectorBase>(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const int32_t start_gid, const int32_t start_output_idx,
|
||||
const int32_t batch_size, const ObBitVector *skip)
|
||||
{
|
||||
int ret = OB_NOT_IMPLEMENT;
|
||||
SQL_LOG(WARN, "not implemented", K(ret), K(*this));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ResultFmt>
|
||||
int collect_group_results(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const int32_t output_start_idx, const int32_t batch_size,
|
||||
const ObCompactRow **rows, const RowMeta &row_meta)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(rows)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null rows", K(ret));
|
||||
} else {
|
||||
ObEvalCtx::BatchInfoScopeGuard batch_guard(agg_ctx.eval_ctx_);
|
||||
ObExpr *agg_expr = agg_ctx.aggr_infos_.at(agg_col_id).expr_;
|
||||
const char *agg_cell = nullptr;
|
||||
int32_t agg_cell_len = 0;
|
||||
if (OB_FAIL(agg_expr->init_vector_for_write(
|
||||
agg_ctx.eval_ctx_, agg_expr->get_default_res_format(), batch_size))) {
|
||||
SQL_LOG(WARN, "init vector for write failed", K(ret));
|
||||
}
|
||||
for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) {
|
||||
batch_guard.set_batch_idx(output_start_idx + i);
|
||||
if (OB_ISNULL(rows[i])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid compact row", K(ret), K(i));
|
||||
} else {
|
||||
const char *agg_row = static_cast<const char *>(rows[i]->get_extra_payload(row_meta));
|
||||
agg_cell = agg_ctx.row_meta().locate_cell_payload(agg_col_id, agg_row);
|
||||
agg_cell_len = agg_ctx.row_meta().get_cell_len(agg_col_id, agg_row);
|
||||
SQL_LOG(DEBUG, "collect group results", K(agg_col_id),
|
||||
K(agg_ctx.aggr_infos_.at(agg_col_id).get_expr_type()), KP(agg_cell),
|
||||
K(agg_cell_len), K(agg_cell), K(row_meta), KP(rows[i]));
|
||||
if (OB_FAIL(static_cast<Derived *>(this)->template collect_group_result<ResultFmt>(
|
||||
agg_ctx, *agg_expr, agg_col_id, agg_cell, agg_cell_len))) {
|
||||
SQL_LOG(WARN, "collect group result failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <>
|
||||
int collect_group_results<ObVectorBase>(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const int32_t output_start_idx, const int32_t batch_size,
|
||||
const ObCompactRow **rows, const RowMeta &row_meta)
|
||||
{
|
||||
int ret = OB_NOT_IMPLEMENT;
|
||||
SQL_LOG(WARN, "not implemented", K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
int add_params_batch_row(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const sql::ObBitVector &skip, const sql::EvalBound &bound,
|
||||
const RowSelector &row_sel, char *aggr_cell)
|
||||
{
|
||||
#define ADD_COLUMN_ROWS(vec_tc) \
|
||||
case (vec_tc): { \
|
||||
ret = add_param_batch<Derived>::do_op( \
|
||||
derived, agg_ctx, skip, *tmp_skip, bound, row_sel, agg_col_id, param_id, \
|
||||
*static_cast<ObFixedLengthFormat<RTCType<vec_tc>> *>(param_vec), aggr_cell); \
|
||||
} break
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx::TempAllocGuard alloc_guard(agg_ctx.eval_ctx_);
|
||||
ObIArray<ObExpr *> ¶m_exprs = agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_;
|
||||
int64_t skip_size = (bound.batch_size() / (sizeof(uint64_t) * CHAR_BIT) + 1) * sizeof(uint64_t);
|
||||
char *skip_buf = nullptr;
|
||||
if (OB_ISNULL(skip_buf = (char *)alloc_guard.get_allocator().alloc(skip_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMSET(skip_buf, 0, skip_size);
|
||||
ObBitVector *tmp_skip = to_bit_vector(skip_buf);
|
||||
tmp_skip->deep_copy(skip, skip_size);
|
||||
Derived &derived = *static_cast<Derived *>(this);
|
||||
for (int param_id = 0; OB_SUCC(ret) && param_id < param_exprs.count(); param_id++) {
|
||||
ObIVector *param_vec = param_exprs.at(param_id)->get_vector(agg_ctx.eval_ctx_);
|
||||
VectorFormat param_fmt = param_exprs.at(param_id)->get_format(agg_ctx.eval_ctx_);
|
||||
VecValueTypeClass param_tc = get_vec_value_tc(
|
||||
param_exprs.at(param_id)->datum_meta_.type_, param_exprs.at(param_id)->datum_meta_.scale_,
|
||||
param_exprs.at(param_id)->datum_meta_.precision_);
|
||||
switch (param_fmt) {
|
||||
case common::VEC_UNIFORM_CONST: {
|
||||
ret = add_param_batch<Derived>::do_op(
|
||||
derived, agg_ctx, skip, *tmp_skip, bound, row_sel, agg_col_id, param_id,
|
||||
*static_cast<ObUniformFormat<true> *>(param_vec), aggr_cell);
|
||||
break;
|
||||
}
|
||||
case common::VEC_UNIFORM: {
|
||||
ret = add_param_batch<Derived>::do_op(
|
||||
derived, agg_ctx, skip, *tmp_skip, bound, row_sel, agg_col_id, param_id,
|
||||
*static_cast<ObUniformFormat<false> *>(param_vec), aggr_cell);
|
||||
break;
|
||||
}
|
||||
case common::VEC_DISCRETE: {
|
||||
ret = add_param_batch<Derived>::do_op(
|
||||
derived, agg_ctx, skip, *tmp_skip, bound, row_sel, agg_col_id, param_id,
|
||||
*static_cast<ObDiscreteFormat *>(param_vec), aggr_cell);
|
||||
break;
|
||||
}
|
||||
case common::VEC_CONTINUOUS: {
|
||||
ret = add_param_batch<Derived>::do_op(
|
||||
derived, agg_ctx, skip, *tmp_skip, bound, row_sel, agg_col_id, param_id,
|
||||
*static_cast<ObContinuousFormat *>(param_vec), aggr_cell);
|
||||
break;
|
||||
}
|
||||
case common::VEC_FIXED: {
|
||||
switch (param_tc) {
|
||||
LST_DO_CODE(ADD_COLUMN_ROWS, AGG_FIXED_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNDEFINED;
|
||||
SQL_LOG(WARN, "invalid param type class", K(ret), K(param_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid param format", K(ret), K(param_fmt), K(param_tc));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
#undef ADD_COLUMN_ROWS
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
template<typename Aggregate>
|
||||
class DistinctWrapper final: public BatchAggregateWrapper<DistinctWrapper<Aggregate>>
|
||||
{
|
||||
using BaseClass = BatchAggregateWrapper<DistinctWrapper<Aggregate>>;
|
||||
public:
|
||||
static const VecValueTypeClass IN_TC = Aggregate::IN_TC;
|
||||
static const VecValueTypeClass OUT_TC = Aggregate::OUT_TC;
|
||||
public:
|
||||
DistinctWrapper(): agg_(nullptr) {}
|
||||
|
||||
inline void set_inner_aggregate(IAggregate *agg) override { agg_ = agg; }
|
||||
|
||||
int init(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(agg_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null aggregate", K(ret));
|
||||
} else if (OB_FAIL(agg_->init(agg_ctx, agg_col_id, allocator))) {
|
||||
SQL_LOG(WARN, "init aggregate failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int add_batch_rows(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const sql::ObBitVector &skip, const sql::EvalBound &bound, char *agg_cell,
|
||||
const RowSelector row_sel = RowSelector{}) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSEDx(agg_cell);
|
||||
OB_ASSERT(agg_ != NULL);
|
||||
OB_ASSERT(agg_col_id < agg_ctx.aggr_infos_.count());
|
||||
int64_t stored_row_cnt = 0;
|
||||
sql::ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
ObIArray<ObExpr *> ¶m_exprs = agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_;
|
||||
ObAggregateProcessor::ExtraResult* &extra = agg_ctx.get_extra(agg_col_id, agg_cell);
|
||||
ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id);
|
||||
if (extra == nullptr) {
|
||||
void *tmp_buf = nullptr;
|
||||
if (OB_ISNULL(tmp_buf =
|
||||
agg_ctx.allocator_.alloc(sizeof(ObAggregateProcessor::ExtraResult)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else if (OB_ISNULL(ctx.exec_ctx_.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null exec session", K(ret));
|
||||
} else if (OB_ISNULL(agg_ctx.op_monitor_info_) || OB_ISNULL(agg_ctx.io_event_observer_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null aggregatec ctx", K(ret), K(agg_ctx.op_monitor_info_),
|
||||
K(agg_ctx.io_event_observer_));
|
||||
} else {
|
||||
extra = new (tmp_buf)
|
||||
ObAggregateProcessor::ExtraResult(agg_ctx.allocator_, *agg_ctx.op_monitor_info_);
|
||||
const bool need_rewind = false; // TODO: rollup need rewind
|
||||
if (OB_FAIL(extra->init_distinct_set(
|
||||
ctx.exec_ctx_.get_my_session()->get_effective_tenant_id(), aggr_info, ctx,
|
||||
need_rewind, agg_ctx.io_event_observer_))) {
|
||||
SQL_LOG(WARN, "init distinct set failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(extra) || OB_ISNULL(extra->unique_sort_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null extra", K(ret));
|
||||
} else if (!row_sel.is_empty()) {
|
||||
if (OB_FAIL(extra->unique_sort_op_->add_batch(param_exprs, skip, bound.batch_size(),
|
||||
row_sel.selector(), row_sel.size()))) {
|
||||
SQL_LOG(WARN, "add batch failed", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(extra->unique_sort_op_->add_batch(param_exprs, skip, bound.batch_size(),
|
||||
bound.start(), &stored_row_cnt))) {
|
||||
SQL_LOG(WARN, "add batch rows failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <typename ResultFmt>
|
||||
int collect_group_result(RuntimeContext &agg_ctx, const ObExpr &agg_expr, int32_t agg_col_id,
|
||||
const char *agg_cell, const int32_t agg_cell_len)
|
||||
{
|
||||
// TODO: deal with rollup, sort need rewind
|
||||
// TODO: advance collect result
|
||||
UNUSED(agg_cell_len);
|
||||
int ret = OB_SUCCESS;
|
||||
const int64_t constexpr max_batch_size = 256;
|
||||
char skip_vector[max_batch_size] = {0};
|
||||
ObBitVector &mock_skip = *to_bit_vector(skip_vector);
|
||||
OB_ASSERT(agg_ != NULL);
|
||||
sql::ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
ObAggregateProcessor::ExtraResult *&extra = agg_ctx.get_extra(agg_col_id, agg_cell);
|
||||
ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id);
|
||||
ObEvalCtx::TempAllocGuard alloc_guard(ctx);
|
||||
int32_t tmp_data_len = 0;
|
||||
if (OB_ISNULL(extra) || OB_ISNULL(extra->unique_sort_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null extra", K(ret));
|
||||
} else if (OB_FAIL(extra->unique_sort_op_->sort())) {
|
||||
SQL_LOG(WARN, "sort failed", K(ret));
|
||||
} else {
|
||||
while(OB_SUCC(ret)) {
|
||||
int64_t read_rows = 0;
|
||||
if (OB_FAIL(extra->unique_sort_op_->get_next_batch(aggr_info.param_exprs_, max_batch_size,
|
||||
read_rows))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
} else {
|
||||
SQL_LOG(WARN, "read batch rows failed", K(ret));
|
||||
}
|
||||
break;
|
||||
} else if (read_rows <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "read unexpected zero rows", K(ret));
|
||||
} else {
|
||||
SQL_LOG(DEBUG, "read rows", K(read_rows), K(max_batch_size));
|
||||
sql::EvalBound bound(read_rows, true);
|
||||
if (OB_FAIL(static_cast<Aggregate *>(agg_)->add_batch_rows(
|
||||
agg_ctx, agg_col_id, mock_skip, bound, const_cast<char *>(agg_cell)))) {
|
||||
SQL_LOG(WARN, "add batch rows failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (FALSE_IT(tmp_data_len = agg_ctx.row_meta().get_cell_len(agg_col_id, agg_cell))) {
|
||||
} else if (OB_FAIL(static_cast<Aggregate *>(agg_)->template collect_group_result<ResultFmt>(
|
||||
agg_ctx, agg_expr, agg_col_id, agg_cell, tmp_data_len))) {
|
||||
SQL_LOG(WARN, "collect group result failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <typename ColumnFmt>
|
||||
inline int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *aggr_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, columns, row_num, agg_col_id, aggr_cell, tmp_res, calc_info);
|
||||
SQL_LOG(DEBUG, "add_row do nothing");
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
template <typename ColumnFmt>
|
||||
inline int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res,
|
||||
int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info);
|
||||
SQL_LOG(DEBUG, "add_nullable_row do nothing");
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
inline int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size,
|
||||
const bool is_null, const char *data, const int32_t data_len,
|
||||
int32_t agg_col_idx, char *agg_cell) override
|
||||
{
|
||||
// FIXME: opt performance
|
||||
sql::EvalBound bound(batch_size, batch_idx, batch_idx + 1, true);
|
||||
char mock_skip_data[1] = {0};
|
||||
ObBitVector &mock_skip = *to_bit_vector(mock_skip_data);
|
||||
return static_cast<Aggregate *>(agg_)->add_batch_rows(agg_ctx, agg_col_idx, mock_skip, bound,
|
||||
agg_cell);
|
||||
}
|
||||
void reuse() override
|
||||
{
|
||||
if (agg_ != NULL) {
|
||||
agg_->reuse();
|
||||
}
|
||||
}
|
||||
|
||||
void destroy() override
|
||||
{
|
||||
if (agg_ != NULL) {
|
||||
agg_->destroy();
|
||||
agg_ = nullptr;
|
||||
}
|
||||
}
|
||||
TO_STRING_KV("wrapper_type", "distinct", KP_(agg));
|
||||
private:
|
||||
IAggregate *agg_;
|
||||
};
|
||||
|
||||
//helper functions
|
||||
namespace helper
|
||||
{
|
||||
int init_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator,
|
||||
ObIArray<IAggregate *> &aggregates);
|
||||
|
||||
template <ObItemType op>
|
||||
int init_aggregate(sql::ObEvalCtx &ctx, RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg);
|
||||
|
||||
inline constexpr bool is_var_len_agg_cell(VecValueTypeClass vec_tc)
|
||||
{
|
||||
return vec_tc == VEC_TC_STRING
|
||||
|| vec_tc == VEC_TC_LOB
|
||||
|| vec_tc == VEC_TC_ROWID
|
||||
|| vec_tc == VEC_TC_RAW
|
||||
|| vec_tc == VEC_TC_JSON
|
||||
|| vec_tc == VEC_TC_GEO
|
||||
|| vec_tc == VEC_TC_UDT;
|
||||
}
|
||||
|
||||
template <typename AggType>
|
||||
int init_agg_func(RuntimeContext &agg_ctx, const int64_t agg_col_id, const bool has_distinct,
|
||||
ObIAllocator &allocator, IAggregate *&agg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *agg_buf = nullptr, *wrapper_buf = nullptr;
|
||||
sql::ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
IAggregate *wrapper = nullptr;
|
||||
if (OB_ISNULL(agg_buf = allocator.alloc(sizeof(AggType)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else if (FALSE_IT(agg = new (agg_buf) AggType())) {
|
||||
} else if (has_distinct) {
|
||||
if (OB_ISNULL(wrapper_buf = allocator.alloc(sizeof(DistinctWrapper<AggType>)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else if (FALSE_IT(wrapper = new (wrapper_buf) DistinctWrapper<AggType>())) {
|
||||
} else {
|
||||
wrapper->set_inner_aggregate(agg);
|
||||
agg = wrapper;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(agg->init(agg_ctx, agg_col_id, allocator))) {
|
||||
SQL_LOG(WARN, "init aggregate failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename AggType>
|
||||
int init_agg_func(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator,
|
||||
IAggregate *&agg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *agg_buf = nullptr, *wrapper_buf = nullptr;
|
||||
sql::ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
IAggregate *wrapper = nullptr;
|
||||
if (OB_ISNULL(agg_buf = allocator.alloc(sizeof(AggType)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else if (FALSE_IT(agg = new (agg_buf) AggType())) {
|
||||
} else if (OB_FAIL(agg->init(agg_ctx, agg_col_id, allocator))) {
|
||||
SQL_LOG(WARN, "init aggregate failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // end namespace helper
|
||||
} // end namespace aggregate
|
||||
} // end namespace share
|
||||
} // end namespace oceanbase
|
||||
#endif // OCEANBASE_SHARE_IAGGREGATE_H_
|
||||
@ -1,75 +0,0 @@
|
||||
/**
|
||||
* 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
|
||||
#include "min_max.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
namespace helper
|
||||
{
|
||||
template <ObItemType func_type>
|
||||
inline static int init_min_max_agg(RuntimeContext &agg_ctx,
|
||||
const int64_t agg_col_id, ObIAllocator &allocator,
|
||||
IAggregate *&agg)
|
||||
{
|
||||
#define INIT_AGGREGATE_CASE(vec_tc) \
|
||||
case (vec_tc): { \
|
||||
ret = init_agg_func<MinMaxAggregate<vec_tc, T_FUN_MIN == func_type>>( \
|
||||
agg_ctx, agg_col_id, aggr_info.has_distinct_, allocator, agg); \
|
||||
if (OB_FAIL(ret)) { SQL_LOG(WARN, "init aggregate failed", K(ret)); } \
|
||||
} break
|
||||
|
||||
int ret = OB_SUCCESS;
|
||||
agg = nullptr;
|
||||
|
||||
ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id);
|
||||
if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null expr", K(ret));
|
||||
} else {
|
||||
VecValueTypeClass res_vec =
|
||||
get_vec_value_tc(aggr_info.expr_->datum_meta_.type_, aggr_info.expr_->datum_meta_.scale_,
|
||||
aggr_info.expr_->datum_meta_.precision_);
|
||||
|
||||
switch (res_vec) {
|
||||
LST_DO_CODE(INIT_AGGREGATE_CASE, AGG_VEC_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected result type of min/max aggregate", K(ret), K(res_vec));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
|
||||
#undef INIT_AGGREGATE_CASE
|
||||
}
|
||||
|
||||
int init_min_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg)
|
||||
{
|
||||
return init_min_max_agg<T_FUN_MIN>(agg_ctx, agg_col_id, allocator, agg);
|
||||
}
|
||||
|
||||
int init_max_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg)
|
||||
{
|
||||
return init_min_max_agg<T_FUN_MAX>(agg_ctx, agg_col_id, allocator, agg);
|
||||
}
|
||||
} // end helper
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
@ -1,302 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_MIN_MAX_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_MIN_MAX_H_
|
||||
|
||||
#include "share/aggregate/iaggregate.h"
|
||||
|
||||
#include <utility>
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
using namespace sql;
|
||||
|
||||
struct CmpCalcInfo
|
||||
{
|
||||
CmpCalcInfo(ObObjMeta obj_meta, int16_t cell_len) :
|
||||
obj_meta_(obj_meta), agg_cell_len_(cell_len), calculated_(0) {}
|
||||
CmpCalcInfo() : obj_meta_(), agg_cell_len_(0), calculated_(0) {}
|
||||
operator int64_t() const { return flags_; }
|
||||
inline void set_calculated()
|
||||
{
|
||||
calculated_ = static_cast<int16_t>(1);
|
||||
}
|
||||
inline bool calculated() const { return calculated_ == 1; }
|
||||
union {
|
||||
struct {
|
||||
ObObjMeta obj_meta_;
|
||||
int16_t agg_cell_len_; // for fixed length type only
|
||||
int16_t calculated_;
|
||||
};
|
||||
int64_t flags_;
|
||||
};
|
||||
};
|
||||
|
||||
static_assert(sizeof(CmpCalcInfo) == sizeof(int64_t), "");
|
||||
// fixed length type
|
||||
// exampl of min/max cell in aggr_row:
|
||||
// min(int64)
|
||||
// --------------------
|
||||
// ... | int64 |...
|
||||
// --------------------
|
||||
//
|
||||
// variable length type
|
||||
// example of min/max cell in aggr_row
|
||||
// max(str)
|
||||
// -------------------------------------------
|
||||
// ...| <char *, int32>, <char *, int32> |...
|
||||
// -------------------------------------------
|
||||
// second pair of <char *, int32> is used to store tmp result of aggregate
|
||||
template<VecValueTypeClass vec_tc, bool is_min>
|
||||
class MinMaxAggregate final : public BatchAggregateWrapper<MinMaxAggregate<vec_tc, is_min>>
|
||||
{
|
||||
using buf_node = std::pair<char *, int32_t>;
|
||||
static const int32_t BUF_BLOCK_SIZE = 512;
|
||||
public:
|
||||
static const constexpr VecValueTypeClass IN_TC = vec_tc;
|
||||
static const constexpr VecValueTypeClass OUT_TC = vec_tc;
|
||||
public:
|
||||
|
||||
// TODO: remove info for window function optimization
|
||||
public:
|
||||
MinMaxAggregate() {}
|
||||
|
||||
int init(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator) override
|
||||
{
|
||||
UNUSEDx(agg_col_id, allocator);
|
||||
int ret = OB_SUCCESS;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int add_one_row(RuntimeContext &agg_ctx, int64_t row_num, int64_t batch_size, const bool is_null,
|
||||
const char *data, const int32_t data_len, int32_t agg_col_idx,
|
||||
char *agg_cell) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell);
|
||||
if (!is_null) {
|
||||
int cmp_ret = 0;
|
||||
ObObjMeta obj_meta;
|
||||
if (not_nulls.at(agg_col_idx)) {
|
||||
if (VecTCCmpCalc<vec_tc, vec_tc>::cmp == BasicCmpCalc<VEC_TC_INTEGER, VEC_TC_INTEGER>::cmp
|
||||
|| vec_tc == VEC_TC_FLOAT
|
||||
|| vec_tc == VEC_TC_DOUBLE
|
||||
|| vec_tc == VEC_TC_NUMBER
|
||||
|| vec_tc == VEC_TC_TIMESTAMP_TINY
|
||||
|| vec_tc == VEC_TC_TIMESTAMP_TZ
|
||||
|| vec_tc == VEC_TC_INTERVAL_DS) {
|
||||
// no need for obj meta
|
||||
} else {
|
||||
obj_meta = agg_ctx.aggr_infos_.at(agg_col_idx).param_exprs_.at(0)->obj_meta_;
|
||||
}
|
||||
if (!helper::is_var_len_agg_cell(vec_tc)) {
|
||||
ret = VecTCCmpCalc<vec_tc, vec_tc>::cmp(
|
||||
obj_meta, obj_meta, agg_cell,
|
||||
VEC_TC_NUMBER == vec_tc ? number::ObNumber::MAX_CALC_BYTE_LEN : sizeof(RTCType<vec_tc>),
|
||||
data, data_len, cmp_ret);
|
||||
} else {
|
||||
int32_t agg_cell_len = *reinterpret_cast<int32_t *>(agg_cell + sizeof(char *));
|
||||
const char *agg_data = reinterpret_cast<const char *>(*reinterpret_cast<int64_t *>(agg_cell));
|
||||
ret = VecTCCmpCalc<vec_tc, vec_tc>::cmp(obj_meta, obj_meta, agg_data, agg_cell_len, data, data_len, cmp_ret);
|
||||
}
|
||||
if ((is_min && cmp_ret > 0) || (!is_min && cmp_ret < 0)) {
|
||||
if (!helper::is_var_len_agg_cell(vec_tc)) {
|
||||
MEMCPY(agg_cell, data, data_len);
|
||||
} else {
|
||||
*reinterpret_cast<int64_t *>(agg_cell) = reinterpret_cast<int64_t>(data);
|
||||
*reinterpret_cast<int32_t *>(agg_cell + sizeof(char *)) = data_len;
|
||||
if (OB_FAIL(set_tmp_var_agg_data(agg_ctx, agg_col_idx, agg_cell))) {
|
||||
SQL_LOG(WARN, "set var aggregate data failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (helper::is_var_len_agg_cell(vec_tc)) {
|
||||
*reinterpret_cast<int64_t *>(agg_cell) = reinterpret_cast<int64_t>(data);
|
||||
*reinterpret_cast<int32_t *>(agg_cell + sizeof(char *)) = data_len;
|
||||
set_tmp_var_agg_data(agg_ctx, agg_col_idx, agg_cell);
|
||||
} else {
|
||||
MEMCPY(agg_cell, data, data_len);
|
||||
}
|
||||
not_nulls.set(agg_col_idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <typename ColumnFmt>
|
||||
int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int64_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
UNUSED(tmp_res);
|
||||
int ret = OB_SUCCESS;
|
||||
const char *row_data = nullptr;
|
||||
int32_t row_len = 0;
|
||||
int cmp_ret = 0;
|
||||
columns.get_payload(row_num, row_data, row_len);
|
||||
CmpCalcInfo &cmp_info = reinterpret_cast<CmpCalcInfo &>(calc_info);
|
||||
if (!helper::is_var_len_agg_cell(vec_tc)) {
|
||||
if (cmp_info.calculated()) {
|
||||
// do not need to read not_nulls bitmap for fixed length (include ObNUmber) types
|
||||
ret = VecTCCmpCalc<vec_tc, vec_tc>::cmp(cmp_info.obj_meta_, cmp_info.obj_meta_, agg_cell,
|
||||
cmp_info.agg_cell_len_, row_data, row_len, cmp_ret);
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "compare failed", K(ret));
|
||||
} else if ((is_min && cmp_ret > 0) || (!is_min && cmp_ret < 0)) {
|
||||
MEMCPY(agg_cell, row_data, row_len);
|
||||
}
|
||||
} else {
|
||||
MEMCPY(agg_cell, row_data, row_len);
|
||||
cmp_info.set_calculated();
|
||||
}
|
||||
} else {
|
||||
int32_t agg_cell_len = *reinterpret_cast<int32_t *>(agg_cell + sizeof(char *));
|
||||
const char *agg_data = reinterpret_cast<const char *>(*reinterpret_cast<int64_t *>(agg_cell));
|
||||
if (cmp_info.calculated()) {
|
||||
ret = VecTCCmpCalc<vec_tc, vec_tc>::cmp(cmp_info.obj_meta_, cmp_info.obj_meta_, agg_data,
|
||||
agg_cell_len, row_data, row_len, cmp_ret);
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "compare failed", K(ret));
|
||||
} else if ((is_min && cmp_ret > 0) || (!is_min && cmp_ret < 0)) {
|
||||
*reinterpret_cast<int64_t *>(agg_cell) = reinterpret_cast<int64_t>(row_data);
|
||||
*reinterpret_cast<int32_t *>(agg_cell + sizeof(char *)) = row_len;
|
||||
}
|
||||
} else {
|
||||
*reinterpret_cast<int64_t *>(agg_cell) = reinterpret_cast<int64_t>(row_data);
|
||||
*reinterpret_cast<int32_t *>(agg_cell + sizeof(char *)) = row_len;
|
||||
cmp_info.set_calculated();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (columns.is_null(row_num)) {
|
||||
// do nothing
|
||||
SQL_LOG(DEBUG, "add null row", K(is_min), K(agg_col_id), K(row_num));
|
||||
} else if (OB_FAIL(
|
||||
add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
} else {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
not_nulls.set(agg_col_id);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr,
|
||||
const int32_t agg_col_id, const char *agg_cell,
|
||||
const int32_t agg_cell_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
int64_t output_idx = ctx.get_batch_idx();
|
||||
ColumnFmt *res_vec = static_cast<ColumnFmt *>(agg_expr.get_vector(ctx));
|
||||
const NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
if (OB_LIKELY(not_nulls.at(agg_col_id))) {
|
||||
if (helper::is_var_len_agg_cell(vec_tc)) {
|
||||
char *res_buf = agg_expr.get_str_res_mem(ctx, agg_cell_len);
|
||||
if (OB_ISNULL(res_buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
const char *data =
|
||||
reinterpret_cast<const char *>(*reinterpret_cast<const int64_t *>(agg_cell));
|
||||
CellWriter<AggCalcType<vec_tc>>::set(data, agg_cell_len, res_vec, output_idx, res_buf);
|
||||
}
|
||||
} else {
|
||||
CellWriter<AggCalcType<vec_tc>>::set(agg_cell, agg_cell_len, res_vec, output_idx, nullptr);
|
||||
}
|
||||
} else {
|
||||
res_vec->set_null(output_idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int collect_tmp_result(RuntimeContext &agg_ctx, const int32_t agg_col_id, char *agg_cell)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
// if result value has variable result length, e.g. string value,
|
||||
// we need copy value to tmp buffer in case value ptr stored was changed after next batch loop.
|
||||
if (not_nulls.at(agg_col_id) && helper::is_var_len_agg_cell(vec_tc)) {
|
||||
if (OB_FAIL(set_tmp_var_agg_data(agg_ctx, agg_col_id, agg_cell))) {
|
||||
SQL_LOG(WARN, "set variable aggregate data failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void reuse() override
|
||||
{
|
||||
}
|
||||
|
||||
void destroy() override
|
||||
{
|
||||
}
|
||||
|
||||
inline int64_t get_batch_calc_info(RuntimeContext &agg_ctx, int32_t agg_col_idx,
|
||||
char *agg_cell) override
|
||||
{
|
||||
ObObjMeta &obj_meta = agg_ctx.locate_aggr_info(agg_col_idx).param_exprs_.at(0)->obj_meta_;
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell);
|
||||
int32_t agg_cell_len = 0;
|
||||
if (!helper::is_var_len_agg_cell(vec_tc)) {
|
||||
agg_cell_len = agg_ctx.row_meta().get_cell_len(agg_col_idx, nullptr/*not used*/);
|
||||
}
|
||||
CmpCalcInfo info = CmpCalcInfo(obj_meta, static_cast<int16_t>(agg_cell_len));
|
||||
if (not_nulls.at(agg_col_idx)) { info.set_calculated(); }
|
||||
return info;
|
||||
}
|
||||
|
||||
TO_STRING_KV("aggregate", (is_min ? "min" : "max"), K(vec_tc));
|
||||
private:
|
||||
int set_tmp_var_agg_data(RuntimeContext &agg_ctx, const int32_t agg_col_id, char *agg_cell)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (helper::is_var_len_agg_cell(vec_tc)) {
|
||||
char *agg_data = reinterpret_cast<char *>(*reinterpret_cast<int64_t *>(agg_cell));
|
||||
int32_t agg_data_len = *reinterpret_cast<int32_t *>(agg_cell + sizeof(char *));
|
||||
char *tmp_buf = reinterpret_cast<char *>(*reinterpret_cast<int64_t *>(agg_cell + sizeof(char *) + sizeof(int32_t)));
|
||||
int32_t &cap = *reinterpret_cast<int32_t *>(agg_cell + sizeof(int32_t) + sizeof(char *) * 2);
|
||||
if (cap < agg_data_len) {
|
||||
int32_t new_cap = 2 *((agg_data_len + BUF_BLOCK_SIZE - 1) / BUF_BLOCK_SIZE) * BUF_BLOCK_SIZE;
|
||||
void *new_buf = nullptr;
|
||||
if (OB_ISNULL(new_buf = agg_ctx.allocator_.alloc(new_cap))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
tmp_buf = (char *)new_buf;
|
||||
cap = new_cap;
|
||||
*reinterpret_cast<int64_t *>(agg_cell + sizeof(char *) + sizeof(int32_t)) = reinterpret_cast<int64_t>(tmp_buf);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
MEMCPY(tmp_buf, agg_data, agg_data_len);
|
||||
*reinterpret_cast<int64_t *>(agg_cell) = reinterpret_cast<int64_t>(tmp_buf);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
};
|
||||
|
||||
} // end namespace aggregate
|
||||
} // end namespace share
|
||||
} // end namespace oceanbase
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_MIN_MAX_H_
|
||||
@ -1,745 +0,0 @@
|
||||
/**
|
||||
* 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
|
||||
|
||||
#include "processor.h"
|
||||
#include "share/aggregate/iaggregate.h"
|
||||
#include "share/aggregate/single_row.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
|
||||
int Processor::init()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (inited_) {
|
||||
LOG_DEBUG("already inited, do nothing");
|
||||
} else if (agg_ctx_.aggr_infos_.count() <= 0) {
|
||||
// do nothing
|
||||
} else if (OB_UNLIKELY(agg_ctx_.aggr_infos_.count() >= MAX_SUPPORTED_AGG_CNT)) {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
SQL_LOG(WARN, "too many aggregations, not supported", K(ret));
|
||||
} else if (OB_FAIL(aggregates_.reserve(agg_ctx_.aggr_infos_.count()))) {
|
||||
SQL_LOG(WARN, "reserved allocator failed", K(ret));
|
||||
} else if (OB_FAIL(helper::init_aggregates(agg_ctx_, allocator_, aggregates_))) {
|
||||
SQL_LOG(WARN, "init aggregates failed", K(ret));
|
||||
} else if (OB_FAIL(add_one_row_fns_.prepare_allocate(agg_ctx_.aggr_infos_.count()))) {
|
||||
SQL_LOG(WARN, "prepare allocate elements failed", K(ret));
|
||||
} else {
|
||||
clear_add_one_row_fns();
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
inited_ = true;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::init_fast_single_row_aggs()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (support_fast_single_row_agg_) {
|
||||
if (fast_single_row_aggregates_.count() > 0) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(fast_single_row_aggregates_.reserve(agg_ctx_.aggr_infos_.count()))) {
|
||||
SQL_LOG(WARN, "reserve elements failed", K(ret));
|
||||
} else if (OB_FAIL(helper::init_single_row_aggregates(agg_ctx_, allocator_,
|
||||
fast_single_row_aggregates_))) {
|
||||
SQL_LOG(WARN, "init single row aggregate failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void Processor::destroy()
|
||||
{
|
||||
agg_ctx_.destroy();
|
||||
for (int i = 0; i < aggregates_.count(); i++) {
|
||||
if (OB_NOT_NULL(aggregates_.at(i))) {
|
||||
aggregates_.at(i)->destroy();
|
||||
}
|
||||
}
|
||||
aggregates_.reset();
|
||||
for (int i = 0; i < fast_single_row_aggregates_.count(); i++) {
|
||||
if (OB_NOT_NULL(fast_single_row_aggregates_.at(i))) {
|
||||
fast_single_row_aggregates_.at(i)->destroy();
|
||||
}
|
||||
}
|
||||
fast_single_row_aggregates_.reset();
|
||||
allocator_.reset();
|
||||
inited_ = false;
|
||||
}
|
||||
|
||||
void Processor::reuse()
|
||||
{
|
||||
agg_ctx_.reuse();
|
||||
for (int i = 0; i < aggregates_.count(); i++) {
|
||||
if (OB_NOT_NULL(aggregates_.at(i))) {
|
||||
aggregates_.at(i)->reuse();
|
||||
}
|
||||
}
|
||||
for (int i = 0; i < fast_single_row_aggregates_.count(); i++) {
|
||||
if (OB_NOT_NULL(fast_single_row_aggregates_.at(i))) {
|
||||
fast_single_row_aggregates_.at(i)->reuse();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int Processor::add_batch_rows(const int32_t start_agg_id, const int32_t end_agg_id,
|
||||
AggrRowPtr row, const ObBatchRows &brs, uint16_t begin,
|
||||
uint16_t end)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
SQL_LOG(WARN, "not inited", K(ret));
|
||||
} else if (OB_LIKELY(brs.size_ > 0)) {
|
||||
OB_ASSERT(aggregates_.count() >= end_agg_id && start_agg_id >= 0);
|
||||
sql::EvalBound bound(brs.size_, begin, end, brs.all_rows_active_);
|
||||
CK(OB_NOT_NULL(brs.skip_));
|
||||
for (int col_id = start_agg_id; OB_SUCC(ret) && col_id < end_agg_id;
|
||||
col_id++) {
|
||||
char *aggr_cell = agg_ctx_.row_meta().locate_cell_payload(col_id, row);
|
||||
if (OB_FAIL(aggregates_.at(col_id)->add_batch_rows(agg_ctx_, col_id, *brs.skip_, bound,
|
||||
aggr_cell))) {
|
||||
SQL_LOG(WARN, "add batch rows failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::add_batch_rows(const int32_t start_agg_id, const int32_t end_agg_id,
|
||||
AggrRowPtr row, const ObBatchRows &brs,
|
||||
const uint16_t *selector_array, uint16_t count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
SQL_LOG(WARN, "not inited", K(ret));
|
||||
} else if (OB_LIKELY(brs.size_ > 0)) {
|
||||
CK(OB_NOT_NULL(brs.skip_));
|
||||
OB_ASSERT(aggregates_.count() >= end_agg_id && start_agg_id >= 0);
|
||||
sql::EvalBound bound(brs.size_, 0, brs.size_, brs.all_rows_active_);
|
||||
for (int32_t col_id = start_agg_id; OB_SUCC(ret) && col_id < end_agg_id;
|
||||
col_id++) {
|
||||
char *aggr_cell = agg_ctx_.row_meta().locate_cell_payload(col_id, row);
|
||||
if (OB_FAIL(aggregates_.at(col_id)->add_batch_rows(
|
||||
agg_ctx_, col_id, *brs.skip_, bound, aggr_cell, RowSelector(selector_array, count)))) {
|
||||
SQL_LOG(WARN, "add batch rows failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::collect_group_results(const RowMeta &row_meta,
|
||||
const ObIArray<ObExpr *> &groupby_exprs,
|
||||
const int32_t output_batch_size, ObBatchRows &output_brs,
|
||||
int64_t &cur_group_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t batch_size =
|
||||
min(output_batch_size, static_cast<int32_t>(agg_ctx_.agg_rows_.count() - cur_group_id));
|
||||
if (OB_UNLIKELY(!inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
SQL_LOG(WARN, "not inited", K(ret));
|
||||
} else if (OB_UNLIKELY(batch_size <= 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected batch size", K(ret), K(batch_size));
|
||||
} else {
|
||||
int32_t aggr_cnt = static_cast<int32_t>(agg_ctx_.aggr_infos_.count());
|
||||
// default to batch_size
|
||||
// if no aggregate functions, there is no need to collect group results, e.g.
|
||||
// `select /*+parallel(2)*/ count(distinct b), a from t group by a`, stage 1.
|
||||
// However, HashGroupBy operator still adds aggregate rows into processor and
|
||||
// calls `collect_group_results` to get output row size to stop iteration.
|
||||
int32_t output_size = batch_size;
|
||||
bool got_result = false;
|
||||
ObEvalCtx::BatchInfoScopeGuard guard(agg_ctx_.eval_ctx_);
|
||||
guard.set_batch_size(batch_size);
|
||||
for (int i = 0; OB_SUCC(ret) && i < agg_ctx_.aggr_infos_.count(); i++) {
|
||||
int32_t agg_col_idx = i;
|
||||
int32_t output_start_idx = static_cast<int32_t>(output_brs.size_);
|
||||
int32_t cur_batch_size = 0;
|
||||
int32_t start_gid = static_cast<int32_t>(cur_group_id);
|
||||
ObAggrInfo &aggr_info = agg_ctx_.aggr_infos_.at(agg_col_idx);
|
||||
if (OB_UNLIKELY(agg_col_idx >= aggregates_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected agg_col_idx", K(agg_col_idx), K(aggregates_));
|
||||
} else if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null aggregate expr", K(ret));
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(aggregates_.at(agg_col_idx)
|
||||
->collect_batch_group_results(agg_ctx_, agg_col_idx, start_gid,
|
||||
output_start_idx, batch_size,
|
||||
cur_batch_size))) {
|
||||
SQL_LOG(WARN, "collect group results failed", K(ret), K(batch_size));
|
||||
} else if (!got_result) {
|
||||
output_size = cur_batch_size;
|
||||
got_result = true;
|
||||
} else if (OB_UNLIKELY(output_size != cur_batch_size)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexepcted output batch", K(output_size), K(cur_batch_size), K(ret));
|
||||
} else {
|
||||
agg_ctx_.aggr_infos_.at(i).expr_->get_eval_info(agg_ctx_.eval_ctx_).projected_ = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) { clear_op_evaluated_flag(); }
|
||||
if (OB_SUCC(ret)) {
|
||||
ObEvalCtx::TempAllocGuard alloc_guard(agg_ctx_.eval_ctx_);
|
||||
ObFixedArray<const ObCompactRow *, ObIAllocator> stored_rows(alloc_guard.get_allocator(),
|
||||
output_size);
|
||||
for (int i = 0; OB_SUCC(ret) && i < output_size; i++) {
|
||||
int64_t group_id = cur_group_id + i;
|
||||
AggrRowPtr agg_row = agg_ctx_.agg_rows_.at(group_id);
|
||||
const ObCompactRow &row = get_groupby_stored_row(row_meta, agg_row);
|
||||
if (OB_FAIL(stored_rows.push_back(&row))) {
|
||||
SQL_LOG(WARN, "push back element failed", K(ret));
|
||||
}
|
||||
}
|
||||
for (int col_id = 0; OB_SUCC(ret) && col_id < groupby_exprs.count(); col_id++) {
|
||||
ObExpr *out_expr = groupby_exprs.at(col_id);
|
||||
if (OB_ISNULL(out_expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null expr", K(ret));
|
||||
} else if (OB_UNLIKELY(out_expr->is_const_expr())) {
|
||||
// do not project const exprs
|
||||
// do nothing
|
||||
} else if (OB_FAIL(out_expr->init_vector_default(agg_ctx_.eval_ctx_, output_batch_size))) {
|
||||
SQL_LOG(WARN, "init vector failed", K(ret));
|
||||
} else if (OB_FAIL(out_expr->get_vector(agg_ctx_.eval_ctx_)
|
||||
->from_rows(row_meta, stored_rows.get_data(), output_size, col_id))) {
|
||||
SQL_LOG(WARN, "from rows failed", K(ret));
|
||||
}
|
||||
}
|
||||
for (int i = 0; OB_SUCC(ret) && i < groupby_exprs.count(); i++) {
|
||||
if (!groupby_exprs.at(i)->is_const_expr()) {
|
||||
groupby_exprs.at(i)->set_evaluated_projected(agg_ctx_.eval_ctx_);
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG_DEBUG("collect group results", K(ret), K(output_size), K(cur_group_id), K(output_brs),
|
||||
K(output_batch_size));
|
||||
if (OB_SUCC(ret)) {
|
||||
output_brs.size_ += output_size;
|
||||
cur_group_id += output_size;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::collect_group_results(const RowMeta &row_meta,
|
||||
const ObIArray<ObExpr *> &groupby_exprs,
|
||||
const int32_t batch_size, const ObCompactRow **rows,
|
||||
ObBatchRows &output_brs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_LIKELY(inited_ && batch_size > 0)) {
|
||||
ObEvalCtx::BatchInfoScopeGuard batch_guard(agg_ctx_.eval_ctx_);
|
||||
batch_guard.set_batch_size(batch_size);
|
||||
for (int col_id = 0; OB_SUCC(ret) && col_id < agg_ctx_.aggr_infos_.count(); col_id++) {
|
||||
ObAggrInfo &aggr_info = agg_ctx_.aggr_infos_.at(col_id);
|
||||
if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null aggregate expr", K(ret));
|
||||
} else if (OB_FAIL(aggregates_.at(col_id)->collect_batch_group_results(
|
||||
agg_ctx_, col_id, 0, batch_size, rows, row_meta))) {
|
||||
SQL_LOG(WARN, "collect batch group results", K(ret));
|
||||
} else {
|
||||
agg_ctx_.aggr_infos_.at(col_id).expr_->get_eval_info(agg_ctx_.eval_ctx_).projected_ = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) { clear_op_evaluated_flag(); }
|
||||
|
||||
for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) {
|
||||
LOG_DEBUG("stored group rows", "input", CompactRow2STR(row_meta, *rows[i], &groupby_exprs));
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
ObEvalCtx::TempAllocGuard alloc_guard(agg_ctx_.eval_ctx_);
|
||||
for (int col_id = 0; OB_SUCC(ret) && col_id < groupby_exprs.count(); col_id++) {
|
||||
ObExpr *out_expr = groupby_exprs.at(col_id);
|
||||
if (OB_ISNULL(out_expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null output expr", K(ret));
|
||||
} else if (OB_UNLIKELY(out_expr->is_const_expr())) {
|
||||
// do not project const exprs
|
||||
// do nothing
|
||||
} else if (OB_FAIL(out_expr->init_vector_default(agg_ctx_.eval_ctx_, batch_size))) {
|
||||
SQL_LOG(WARN, "init vector failed", K(ret));
|
||||
} else if (OB_FAIL(out_expr->get_vector(agg_ctx_.eval_ctx_)
|
||||
->from_rows(row_meta, rows, batch_size, col_id))) {
|
||||
SQL_LOG(WARN, "from rows failed", K(ret));
|
||||
}
|
||||
}
|
||||
for (int col_id = 0; OB_SUCC(ret) && col_id < groupby_exprs.count(); col_id++) {
|
||||
if (!groupby_exprs.at(col_id)->is_const_expr()) {
|
||||
groupby_exprs.at(col_id)->set_evaluated_projected(agg_ctx_.eval_ctx_);
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG_DEBUG("collect group results", K(ret), K(batch_size), K(output_brs));
|
||||
if (OB_SUCC(ret)) {
|
||||
output_brs.size_ += batch_size;
|
||||
}
|
||||
} else if (OB_UNLIKELY(!inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
SQL_LOG(WARN, "not inited", K(ret));
|
||||
} else if (batch_size <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected batch size", K(ret), K(batch_size));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::setup_rt_info(AggrRowPtr row,
|
||||
const int32_t row_size)
|
||||
{
|
||||
static const int constexpr extra_arr_buf_size = 16;
|
||||
int ret = OB_SUCCESS;
|
||||
OB_ASSERT(row_size == agg_ctx_.row_meta().row_size_);
|
||||
char *extra_array_buf = nullptr;
|
||||
MEMSET(row, 0, row_size);
|
||||
for (int col_id = 0; col_id < agg_ctx_.aggr_infos_.count(); col_id++) {
|
||||
ObDatumMeta &res_meta = agg_ctx_.aggr_infos_.at(col_id).expr_->datum_meta_;
|
||||
VecValueTypeClass res_tc = get_vec_value_tc(res_meta.type_, res_meta.scale_, res_meta.precision_);
|
||||
char *cell = nullptr;
|
||||
int32_t cell_len = 0;
|
||||
agg_ctx_.row_meta().locate_cell_payload(col_id, row, cell, cell_len);
|
||||
// oracle mode use ObNumber as result type for count aggregation
|
||||
// we use int64_t as result type for count aggregation in aggregate row
|
||||
// and cast int64_t to ObNumber during `collect_group_result`
|
||||
if (res_tc == VEC_TC_NUMBER && agg_ctx_.aggr_infos_.at(col_id).get_expr_type() != T_FUN_COUNT) {
|
||||
ObNumberDesc &d = *reinterpret_cast<ObNumberDesc *>(cell);
|
||||
// set zero number
|
||||
d.len_ = 0;
|
||||
d.sign_ = number::ObNumber::POSITIVE;
|
||||
d.exp_ = 0;
|
||||
} else if (res_tc == VEC_TC_FLOAT) {
|
||||
*reinterpret_cast<float *>(cell) = float();
|
||||
} else if (res_tc == VEC_TC_DOUBLE || res_tc == VEC_TC_FIXED_DOUBLE) {
|
||||
*reinterpret_cast<double *>(cell) = double();
|
||||
}
|
||||
}
|
||||
int extra_size = agg_ctx_.row_meta().extra_cnt_ * sizeof(char *);
|
||||
if (agg_ctx_.row_meta().extra_cnt_ > 0) {
|
||||
if (cur_extra_rt_info_idx_ % extra_arr_buf_size == 0) {
|
||||
cur_extra_rt_info_idx_ = 0;
|
||||
extra_rt_info_buf_ = nullptr;
|
||||
extra_rt_info_buf_ = (char *)agg_ctx_.allocator_.alloc(extra_size * extra_arr_buf_size);
|
||||
if (OB_ISNULL(extra_rt_info_buf_)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else {
|
||||
extra_array_buf = extra_rt_info_buf_ + (cur_extra_rt_info_idx_ * extra_size);
|
||||
cur_extra_rt_info_idx_++;
|
||||
MEMSET(extra_array_buf, 0, extra_size);
|
||||
if (OB_FAIL(agg_ctx_.agg_extras_.push_back((AggregateExtras)extra_array_buf))) {
|
||||
SQL_LOG(WARN, "push back element failed", K(ret));
|
||||
} else {
|
||||
*reinterpret_cast<int32_t *>(row + agg_ctx_.row_meta().extra_idx_offset_) =
|
||||
static_cast<int32_t>(agg_ctx_.agg_extras_.count()) - 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::single_row_agg_batch(AggrRowPtr *agg_rows, const int64_t batch_size,
|
||||
ObEvalCtx &eval_ctx, const ObBitVector &skip)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx);
|
||||
batch_info_guard.set_batch_size(batch_size);
|
||||
LOG_DEBUG("by pass single row aggregate", K(batch_size), K(support_fast_single_row_agg_));
|
||||
if (OB_UNLIKELY(!inited_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
SQL_LOG(WARN, "not inited", K(ret));
|
||||
} else if (OB_ISNULL(agg_rows)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null aggregate rows", K(ret));
|
||||
} else if (FALSE_IT(MEMSET(agg_rows[0], 0, get_aggregate_row_size() * batch_size))) {
|
||||
} else if (!support_fast_single_row_agg_) {
|
||||
sql::EvalBound bound(1, true);
|
||||
for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) {
|
||||
if (skip.at(i)) {
|
||||
} else if (OB_FAIL(setup_rt_info(agg_rows[i], get_aggregate_row_size()))) {
|
||||
SQL_LOG(WARN, "setup runtime info failed", K(ret));
|
||||
} else {
|
||||
AggrRowPtr row = agg_rows[i];
|
||||
int32_t aggr_cell_len = 0;
|
||||
int32_t output_size = 0;
|
||||
for (int agg_col_id = 0; OB_SUCC(ret) && agg_col_id < aggregates_.count(); agg_col_id++) {
|
||||
char *aggr_cell = agg_ctx_.row_meta().locate_cell_payload(agg_col_id, row);
|
||||
int32_t aggr_cell_len = agg_ctx_.row_meta().get_cell_len(agg_col_id, row);
|
||||
if (OB_FAIL(aggregates_.at(agg_col_id)->add_batch_rows(agg_ctx_,
|
||||
agg_col_id,
|
||||
skip,
|
||||
bound,
|
||||
aggr_cell))) {
|
||||
SQL_LOG(WARN, "add batch rows failed", K(ret));
|
||||
} else if (OB_FAIL(aggregates_.at(agg_col_id)->collect_batch_group_results(
|
||||
agg_ctx_, agg_col_id, i, i, 1,
|
||||
output_size))) {
|
||||
SQL_LOG(WARN, "collect result batch faile", K(ret));
|
||||
} else if (OB_UNLIKELY(output_size != 1)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid output size", K(output_size));
|
||||
}
|
||||
} // end for
|
||||
}
|
||||
} // end for
|
||||
} else {
|
||||
EvalBound bound(batch_size, skip.accumulate_bit_cnt(batch_size) == 0);
|
||||
int32_t output_size = 0;
|
||||
char *aggr_cell = nullptr; // fake aggr_cell
|
||||
int32_t aggr_cell_len = 0;
|
||||
for (int i = 0; OB_SUCC(ret) && i < fast_single_row_aggregates_.count(); i++) {
|
||||
ObAggrInfo &aggr_info = agg_ctx_.aggr_infos_.at(i);
|
||||
if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null expr", K(ret));
|
||||
} else if (aggr_info.is_implicit_first_aggr()) {
|
||||
// do nothing
|
||||
} else {
|
||||
int32_t output_size = 0;
|
||||
if (OB_FAIL(fast_single_row_aggregates_.at(i)->add_batch_rows(agg_ctx_, i, skip, bound,
|
||||
aggr_cell))) {
|
||||
SQL_LOG(WARN, "add batch rows faile", K(ret));
|
||||
} else if (OB_FAIL(fast_single_row_aggregates_.at(i)->collect_batch_group_results(
|
||||
agg_ctx_, i, bound.start(), bound.start(),
|
||||
static_cast<int32_t>(bound.batch_size()), output_size,
|
||||
(bound.get_all_rows_active() ? nullptr : &skip)))) {
|
||||
SQL_LOG(WARN, "collect batch group results faile", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::eval_aggr_param_batch(const ObBatchRows &brs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool need_calc_param = (brs.size_ != 0);
|
||||
for (int i = 0; OB_SUCC(ret) && need_calc_param && i < agg_ctx_.aggr_infos_.count(); i++) {
|
||||
ObAggrInfo &aggr_info = agg_ctx_.aggr_infos_.at(i);
|
||||
for (int j = 0; OB_SUCC(ret) && j < aggr_info.param_exprs_.count(); j++) {
|
||||
if (OB_FAIL(aggr_info.param_exprs_.at(j)->eval_vector(agg_ctx_.eval_ctx_, brs))) {
|
||||
SQL_LOG(WARN, "eval params batch failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::collect_scalar_results(const RowMeta &row_meta, const ObCompactRow **rows,
|
||||
const int32_t batch_size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(rows)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null aggregate row", K(ret));
|
||||
} else if (OB_LIKELY(batch_size > 0)) {
|
||||
for (int col_id = 0; OB_SUCC(ret) && col_id < agg_ctx_.aggr_infos_.count(); col_id++) {
|
||||
const ObAggrInfo &aggr_info = agg_ctx_.aggr_infos_.at(col_id);
|
||||
if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null aggregate expr", K(ret));
|
||||
} else if (OB_FAIL(aggregates_.at(col_id)->collect_batch_group_results(
|
||||
agg_ctx_, col_id, 0, batch_size, rows, row_meta))) {
|
||||
LOG_WARN("collect batch group results failed", K(ret));
|
||||
} else {
|
||||
aggr_info.expr_->set_evaluated_projected(agg_ctx_.eval_ctx_);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected batch_size", K(ret), K(batch_size));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::collect_empty_set(bool collect_for_third_stage) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int i = 0; OB_SUCC(ret) && i < agg_ctx_.aggr_infos_.count(); i++) {
|
||||
ObAggrInfo &aggr_info = agg_ctx_.aggr_infos_.at(i);
|
||||
if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid null aggregate info", K(ret));
|
||||
} else {
|
||||
// init vector before collect group results
|
||||
VectorFormat vec_fmt = aggr_info.expr_->get_default_res_format();
|
||||
if (OB_FAIL(aggr_info.expr_->init_vector_for_write(agg_ctx_.eval_ctx_, vec_fmt,
|
||||
agg_ctx_.eval_ctx_.get_batch_size()))) {
|
||||
LOG_WARN("init vector failed", K(ret));
|
||||
}
|
||||
}
|
||||
ObIVector *res_vec = nullptr;
|
||||
int64_t output_idx = agg_ctx_.eval_ctx_.get_batch_idx();
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(res_vec = aggr_info.expr_->get_vector(agg_ctx_.eval_ctx_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid null vector", K(ret));
|
||||
} else if (aggr_info.is_implicit_first_aggr() && !collect_for_third_stage) {
|
||||
res_vec->set_null(output_idx);
|
||||
} else {
|
||||
switch(aggr_info.get_expr_type()) {
|
||||
case T_FUN_COUNT:
|
||||
case T_FUN_COUNT_SUM:
|
||||
case T_FUN_APPROX_COUNT_DISTINCT:
|
||||
case T_FUN_KEEP_COUNT:
|
||||
case T_FUN_GROUP_PERCENT_RANK: {
|
||||
if (lib::is_oracle_mode()) {
|
||||
number::ObNumber zero_nmb;
|
||||
zero_nmb.set_zero();
|
||||
res_vec->set_number(output_idx, zero_nmb);
|
||||
} else {
|
||||
res_vec->set_int(output_idx, 0);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case T_FUN_GROUP_RANK:
|
||||
case T_FUN_GROUP_DENSE_RANK:
|
||||
case T_FUN_GROUP_CUME_DIST: {
|
||||
number::ObNumber result_num;
|
||||
int64_t num = 1;
|
||||
ObNumStackOnceAlloc tmp_alloc;
|
||||
if (OB_FAIL(result_num.from(num, tmp_alloc))) {
|
||||
LOG_WARN("failed to create number", K(ret));
|
||||
} else {
|
||||
res_vec->set_number(output_idx, result_num);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS:
|
||||
case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE: {
|
||||
ret = llc_init_empty(*aggr_info.expr_, agg_ctx_.eval_ctx_);
|
||||
break;
|
||||
}
|
||||
case T_FUN_SYS_BIT_AND:
|
||||
case T_FUN_SYS_BIT_OR:
|
||||
case T_FUN_SYS_BIT_XOR: {
|
||||
uint64_t init_val =
|
||||
(aggr_info.get_expr_type() == T_FUN_SYS_BIT_AND ? UINT_MAX_VAL[ObUInt64Type] : 0);
|
||||
res_vec->set_uint(output_idx, init_val);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
res_vec->set_null(output_idx);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
aggr_info.expr_->set_evaluated_projected(agg_ctx_.eval_ctx_);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::init_scalar_aggregate_row(ObCompactRow *&row, RowMeta &row_meta,
|
||||
ObIAllocator &allocator)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObExpr *, 1> mock_exprs;
|
||||
int64_t row_size = 0;
|
||||
void *row_buf = nullptr;
|
||||
row_meta.reset();
|
||||
if (OB_FAIL(row_meta.init(mock_exprs, agg_ctx_.row_meta().row_size_))) {
|
||||
LOG_WARN("init row meta failed", K(ret));
|
||||
} else {
|
||||
row_size = row_meta.get_row_fixed_size() + agg_ctx_.row_meta().row_size_;
|
||||
if (OB_ISNULL(row_buf = allocator.alloc(row_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("allocate memory failed", K(ret));
|
||||
} else {
|
||||
row = new(row_buf)ObCompactRow();
|
||||
row->init(row_meta);
|
||||
AggrRowPtr agg_row = (char *)row->get_extra_payload(row_meta);
|
||||
if (OB_FAIL(setup_rt_info(agg_row, agg_ctx_.row_meta().row_size_))) {
|
||||
LOG_WARN("setup rt info failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::generate_group_rows(AggrRowPtr *row_arr, const int32_t batch_size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *rows_buf = nullptr;
|
||||
if (agg_ctx_.aggr_infos_.count() == 0) {
|
||||
// first stage with no non-distinc-agg
|
||||
// do nothing
|
||||
} else if (OB_ISNULL(row_arr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected null row array", K(ret));
|
||||
} else if (OB_ISNULL(rows_buf =
|
||||
agg_ctx_.allocator_.alloc(agg_ctx_.row_meta().row_size_ * batch_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret), K(batch_size));
|
||||
} else {
|
||||
MEMSET(rows_buf, 0, agg_ctx_.row_meta().row_size_ * batch_size);
|
||||
int32_t offset = 0;
|
||||
for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) {
|
||||
row_arr[i] = static_cast<char *>(rows_buf) + offset;
|
||||
offset += agg_ctx_.row_meta().row_size_;
|
||||
if (OB_FAIL(add_one_aggregate_row(row_arr[i], agg_ctx_.row_meta().row_size_, true))) {
|
||||
SQL_LOG(WARN, "setup rt info failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret) && rows_buf != nullptr) { agg_ctx_.allocator_.free(rows_buf); }
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::add_one_aggregate_row(AggrRowPtr data, const int32_t row_size,
|
||||
bool push_agg_row /*true*/)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(setup_rt_info(data, row_size))) {
|
||||
SQL_LOG(WARN, "setup runtime info failed", K(ret));
|
||||
} else if (push_agg_row && OB_FAIL(agg_ctx_.agg_rows_.push_back(data))) {
|
||||
SQL_LOG(WARN, "push back element failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
inline static int add_one_row(IAggregate *aggr, RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
AggrRowPtr row, ObIVector *data_vec, const int64_t batch_idx,
|
||||
const int64_t batch_size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const char *data = nullptr;
|
||||
int32_t data_len = 0;
|
||||
ColumnFmt *columns = reinterpret_cast<ColumnFmt *>(data_vec);
|
||||
char *agg_cell = agg_ctx.row_meta().locate_cell_payload(agg_col_id, row);
|
||||
bool is_null = false;
|
||||
if (!std::is_same<ObVectorBase, ColumnFmt>::value) {
|
||||
reinterpret_cast<ColumnFmt *>(data_vec)->get_payload(batch_idx, data, data_len);
|
||||
is_null = reinterpret_cast<ColumnFmt *>(data_vec)->is_null(batch_idx);
|
||||
}
|
||||
if (OB_FAIL(aggr->add_one_row(agg_ctx, batch_idx, batch_size, is_null, data, data_len, agg_col_id,
|
||||
agg_cell))) {
|
||||
SQL_LOG(WARN, "add one row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int Processor::prepare_adding_one_row()
|
||||
{
|
||||
#define GET_FIXED_FN(tc) \
|
||||
case (tc): { \
|
||||
fn_ptr = aggregate::add_one_row<ObFixedLengthFormat<RTCType<tc>>>; \
|
||||
} break
|
||||
int ret = OB_SUCCESS;
|
||||
if (FALSE_IT(clear_add_one_row_fns())) {
|
||||
SQL_LOG(WARN, "reserve failed", K(ret));
|
||||
} else {
|
||||
for (int i = 0; OB_SUCC(ret) && i < agg_ctx_.aggr_infos_.count(); i++) {
|
||||
ObAggrInfo &info = agg_ctx_.aggr_infos_.at(i);
|
||||
add_one_row_fn fn_ptr = nullptr;
|
||||
if ((info.param_exprs_.count() <= 0 && !info.is_implicit_first_aggr())
|
||||
|| info.param_exprs_.count() > 1) {
|
||||
fn_ptr = aggregate::add_one_row<ObVectorBase>;
|
||||
} else {
|
||||
ObDatumMeta meta;
|
||||
VectorFormat fmt;
|
||||
if (info.is_implicit_first_aggr()) {
|
||||
meta = info.expr_->datum_meta_;
|
||||
fmt = info.expr_->get_format(agg_ctx_.eval_ctx_);
|
||||
} else {
|
||||
meta = info.param_exprs_.at(0)->datum_meta_;
|
||||
fmt = info.param_exprs_.at(0)->get_format(agg_ctx_.eval_ctx_);
|
||||
}
|
||||
VecValueTypeClass vec_tc = get_vec_value_tc(meta.type_, meta.scale_, meta.precision_);
|
||||
switch(fmt) {
|
||||
case common::VEC_UNIFORM: {
|
||||
fn_ptr = aggregate::add_one_row<ObUniformFormat<false>>;
|
||||
break;
|
||||
}
|
||||
case common::VEC_UNIFORM_CONST: {
|
||||
fn_ptr = aggregate::add_one_row<ObUniformFormat<true>>;
|
||||
break;
|
||||
}
|
||||
case common::VEC_DISCRETE: {
|
||||
fn_ptr = aggregate::add_one_row<ObDiscreteFormat>;
|
||||
break;
|
||||
}
|
||||
case common::VEC_CONTINUOUS: {
|
||||
fn_ptr = aggregate::add_one_row<ObContinuousFormat>;
|
||||
break;
|
||||
}
|
||||
case common::VEC_FIXED: {
|
||||
switch(vec_tc) {
|
||||
LST_DO_CODE(GET_FIXED_FN, AGG_FIXED_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
add_one_row_fns_.at(i) = fn_ptr;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::finish_adding_one_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_add_one_row_fns();
|
||||
return ret;
|
||||
}
|
||||
|
||||
int Processor::llc_init_empty(ObExpr &expr, ObEvalCtx &eval_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char *llc_bitmap_buf = nullptr;
|
||||
const int64_t llc_bitmap_size = ObAggregateProcessor::get_llc_size();
|
||||
if (OB_ISNULL(llc_bitmap_buf = expr.get_str_res_mem(eval_ctx, llc_bitmap_size))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
MEMSET(llc_bitmap_buf, 0, llc_bitmap_size);
|
||||
expr.get_vector(eval_ctx)->set_payload_shallow(eval_ctx.get_batch_idx(), llc_bitmap_buf,
|
||||
llc_bitmap_size);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
@ -1,228 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_PROCESSOR_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_PROCESSOR_H_
|
||||
|
||||
#include "share/aggregate/agg_ctx.h"
|
||||
#include "sql/resolver/expr/ob_raw_expr.h"
|
||||
#include "sql/engine/basic/ob_compact_row.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
class IAggregate;
|
||||
class Processor
|
||||
{
|
||||
public:
|
||||
Processor(sql::ObEvalCtx &eval_ctx, ObIArray<ObAggrInfo> &aggr_infos, const lib::ObLabel &label,
|
||||
sql::ObMonitorNode &monitor_info, const int64_t tenant_id) :
|
||||
inited_(false),
|
||||
support_fast_single_row_agg_(false), has_distinct_(false), has_order_by_(false), dir_id_(-1),
|
||||
op_eval_infos_(nullptr),
|
||||
allocator_(label, OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id, ObCtxIds::WORK_AREA),
|
||||
agg_ctx_(eval_ctx, tenant_id, aggr_infos, label, allocator_),
|
||||
aggregates_(allocator_, aggr_infos.count()),
|
||||
fast_single_row_aggregates_(allocator_, aggr_infos.count()), extra_rt_info_buf_(nullptr),
|
||||
cur_extra_rt_info_idx_(0), add_one_row_fns_(allocator_, aggr_infos.count())
|
||||
{}
|
||||
~Processor() { destroy(); }
|
||||
int init();
|
||||
void destroy();
|
||||
void reuse();
|
||||
|
||||
inline int32_t get_aggregate_row_size() const { return agg_ctx_.row_meta().row_size_; }
|
||||
|
||||
int add_batch_rows(const int32_t start_agg_id, const int32_t end_agg_id, AggrRowPtr row,
|
||||
const ObBatchRows &brs, uint16_t begin, uint16_t end);
|
||||
int add_batch_rows(const int32_t start_agg_id, const int32_t end_agg_id, AggrRowPtr row,
|
||||
const ObBatchRows &brs, const uint16_t *selector_array, uint16_t count);
|
||||
|
||||
int prepare_adding_one_row();
|
||||
|
||||
int finish_adding_one_row();
|
||||
|
||||
inline int add_one_row(const int32_t start_agg_id, const int32_t end_agg_id, AggrRowPtr row,
|
||||
const int64_t batch_idx, const int64_t batch_size, ObIVector **aggr_vectors)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIVector *data_vec = nullptr;
|
||||
ObEvalCtx &ctx = agg_ctx_.eval_ctx_;
|
||||
for (int col_id = start_agg_id; OB_SUCC(ret) && col_id < end_agg_id; col_id++) {
|
||||
add_one_row_fn fn = add_one_row_fns_.at(col_id);
|
||||
if (OB_FAIL(
|
||||
fn(aggregates_.at(col_id), agg_ctx_, col_id, row, aggr_vectors[col_id], batch_idx, batch_size))) {
|
||||
SQL_LOG(WARN, "add one row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int collect_group_results(const RowMeta &row_meta, const ObIArray<ObExpr *> &group_exprs,
|
||||
const int32_t output_batch_size, ObBatchRows &output_brs,
|
||||
int64_t &cur_group_id);
|
||||
|
||||
int collect_group_results(const RowMeta &row_meta, const ObIArray<ObExpr *> &group_exprs,
|
||||
const int32_t batch_size, const ObCompactRow **row,
|
||||
ObBatchRows &output_brs);
|
||||
|
||||
int collect_scalar_results(const RowMeta &row_meta, const ObCompactRow **rows,
|
||||
const int32_t batch_size);
|
||||
|
||||
int collect_empty_set(bool collect_for_third_stage) const;
|
||||
int single_row_agg_batch(AggrRowPtr *agg_rows, const int64_t batch_size,
|
||||
ObEvalCtx &eval_ctx, const ObBitVector &skip);
|
||||
|
||||
// TODO: remove aggr_cell_len for add_batch_rows
|
||||
// TODO: add eval_param_batch
|
||||
|
||||
int eval_aggr_param_batch(const ObBatchRows &brs);
|
||||
|
||||
int generate_group_rows(AggrRowPtr *row_arr, const int32_t batch_size);
|
||||
int add_one_aggregate_row(AggrRowPtr data, const int32_t row_size,
|
||||
bool push_agg_row = true);
|
||||
|
||||
inline int64_t get_aggr_used_size() const
|
||||
{
|
||||
return agg_ctx_.allocator_.used();
|
||||
}
|
||||
|
||||
inline int64_t get_aggr_hold_size() const
|
||||
{
|
||||
return agg_ctx_.allocator_.total();
|
||||
}
|
||||
inline void prefetch_aggregate_row(AggrRowPtr row) const
|
||||
{
|
||||
__builtin_prefetch(row, 0/* read */, 2 /*high temp locality*/);
|
||||
}
|
||||
inline void set_support_fast_single_row_agg(const bool support_fast_single_row_agg)
|
||||
{
|
||||
support_fast_single_row_agg_ = support_fast_single_row_agg;
|
||||
}
|
||||
|
||||
inline bool support_fast_single_row_agg() const
|
||||
{
|
||||
return support_fast_single_row_agg_;
|
||||
}
|
||||
|
||||
inline void set_dir_id(const int64_t dir_id)
|
||||
{
|
||||
dir_id_ = dir_id;
|
||||
}
|
||||
inline int64_t dir_id() const
|
||||
{
|
||||
return dir_id_;
|
||||
}
|
||||
|
||||
inline void set_io_event_observer(sql::ObIOEventObserver *io_observer)
|
||||
{
|
||||
agg_ctx_.io_event_observer_ = io_observer;
|
||||
}
|
||||
|
||||
inline void set_op_monitor_info(sql::ObMonitorNode *op_monitor_info)
|
||||
{
|
||||
agg_ctx_.op_monitor_info_ = op_monitor_info;
|
||||
}
|
||||
inline bool has_distinct() const
|
||||
{
|
||||
return has_distinct_;
|
||||
}
|
||||
|
||||
inline bool has_order_by() const
|
||||
{
|
||||
return has_order_by_;
|
||||
}
|
||||
|
||||
// FIXME: support all aggregate functions
|
||||
inline static bool all_supported_aggregate_functions(const ObIArray<sql::ObRawExpr *> &aggr_exprs,
|
||||
bool is_scalar_gby = false)
|
||||
{
|
||||
bool supported = true;
|
||||
for (int i = 0; supported && i < aggr_exprs.count(); i++) {
|
||||
ObAggFunRawExpr *agg_expr = static_cast<ObAggFunRawExpr *>(aggr_exprs.at(i));
|
||||
OB_ASSERT(agg_expr != NULL);
|
||||
// TODO: remove distinct constraint @zongmei.zzm
|
||||
supported = aggregate::supported_aggregate_function(agg_expr->get_expr_type())
|
||||
&& !agg_expr->is_param_distinct();
|
||||
}
|
||||
return supported;
|
||||
}
|
||||
|
||||
inline int64_t aggregates_cnt() const
|
||||
{
|
||||
return agg_ctx_.aggr_infos_.count();
|
||||
}
|
||||
|
||||
inline void set_op_eval_infos(ObIArray<ObEvalInfo *> *eval_infos)
|
||||
{
|
||||
op_eval_infos_ = eval_infos;
|
||||
}
|
||||
|
||||
inline void clear_op_evaluated_flag()
|
||||
{
|
||||
if (OB_NOT_NULL(op_eval_infos_)) {
|
||||
for (int i = 0; i < op_eval_infos_->count(); i++) {
|
||||
op_eval_infos_->at(i)->clear_evaluated_flag();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static const ObCompactRow &get_groupby_stored_row(const RowMeta &row_meta,
|
||||
const AggrRowPtr agg_row)
|
||||
{
|
||||
const char *payload = (const char *)(agg_row) - row_meta.extra_off_;
|
||||
return *reinterpret_cast<const ObCompactRow *>(payload - sizeof(RowHeader));
|
||||
}
|
||||
|
||||
int init_scalar_aggregate_row(ObCompactRow *&row, RowMeta &row_meta, ObIAllocator &allocator);
|
||||
|
||||
int init_fast_single_row_aggs();
|
||||
private:
|
||||
int setup_rt_info(AggrRowPtr data, const int32_t row_size);
|
||||
|
||||
int setup_bypass_rt_infos(const int64_t batch_size);
|
||||
|
||||
int llc_init_empty(ObExpr &expr, ObEvalCtx &eval_ctx) const;
|
||||
inline void clear_add_one_row_fns()
|
||||
{
|
||||
for (int i = 0; i < agg_ctx_.aggr_infos_.count(); i++) {
|
||||
add_one_row_fns_.at(i) = nullptr;
|
||||
}
|
||||
}
|
||||
private:
|
||||
static const int32_t MAX_SUPPORTED_AGG_CNT = 1000000;
|
||||
friend class ObScalarAggregateVecOp;
|
||||
using add_one_row_fn = int (*)(IAggregate *, RuntimeContext &, const int32_t, AggrRowPtr,
|
||||
ObIVector *, const int64_t, const int64_t);
|
||||
bool inited_;
|
||||
bool support_fast_single_row_agg_;
|
||||
bool has_distinct_;
|
||||
bool has_order_by_;
|
||||
int64_t dir_id_;
|
||||
ObIArray<ObEvalInfo *> *op_eval_infos_;
|
||||
// used to allocate structure memory, such as aggregates_
|
||||
ObArenaAllocator allocator_;
|
||||
RuntimeContext agg_ctx_;
|
||||
ObFixedArray<IAggregate *, ObIAllocator> aggregates_;
|
||||
ObFixedArray<IAggregate *, ObIAllocator> fast_single_row_aggregates_;
|
||||
char *extra_rt_info_buf_;
|
||||
int32_t cur_extra_rt_info_idx_;
|
||||
ObFixedArray<add_one_row_fn, ObIAllocator> add_one_row_fns_;
|
||||
// ObFixedArray<typename T>
|
||||
};
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_PROCESSOR_H_
|
||||
@ -1,364 +0,0 @@
|
||||
/**
|
||||
* 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
|
||||
|
||||
#include "single_row.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
#define INIT_GENERAL_CASE(func_type, vec_tc) \
|
||||
case (vec_tc): { \
|
||||
ret = \
|
||||
init_agg_func<SingleRowAggregate<func_type, vec_tc, vec_tc>>(agg_ctx, i, allocator, agg); \
|
||||
if (OB_FAIL(ret)) { SQL_LOG(WARN, "init aggregate failed", K(vec_tc), K(*aggr_info.expr_)); } \
|
||||
} break
|
||||
|
||||
#define INIT_GENERAL_MIN_CASE(vec_tc) INIT_GENERAL_CASE(T_FUN_MIN, vec_tc)
|
||||
|
||||
#define INIT_GENERAL_MAX_CASE(vec_tc) INIT_GENERAL_CASE(T_FUN_MAX, vec_tc)
|
||||
|
||||
#define INIT_GENERAL_COUNT_SUM_CASE(vec_tc) INIT_GENERAL_CASE(T_FUN_COUNT_SUM, vec_tc)
|
||||
|
||||
#define INIT_COUNT_CASE(vec_tc) \
|
||||
case (vec_tc): { \
|
||||
if (lib::is_oracle_mode()) { \
|
||||
ret = init_agg_func<SingleRowAggregate<T_FUN_COUNT, vec_tc, VEC_TC_NUMBER>>(agg_ctx, i, \
|
||||
allocator, agg); \
|
||||
} else { \
|
||||
ret = init_agg_func<SingleRowAggregate<T_FUN_COUNT, vec_tc, VEC_TC_INTEGER>>( \
|
||||
agg_ctx, i, allocator, agg); \
|
||||
} \
|
||||
if (OB_FAIL(ret)) { SQL_LOG(WARN, "init aggregate failed", K(vec_tc), K(*aggr_info.expr_)); } \
|
||||
} break
|
||||
|
||||
#define INIT_SUM_TO_NMB_CASE(vec_tc) \
|
||||
ret = init_agg_func<SingleRowAggregate<T_FUN_SUM, vec_tc, VEC_TC_NUMBER>>(agg_ctx, i, allocator, \
|
||||
agg); \
|
||||
if (OB_FAIL(ret)) { SQL_LOG(WARN, "init aggregate failed", K(vec_tc), K(*aggr_info.expr_)); }
|
||||
|
||||
#define INIT_SUM_TO_DEC_CASE(vec_tc, dec_tc) \
|
||||
ret = init_agg_func<SingleRowAggregate<T_FUN_SUM, vec_tc, dec_tc>>(agg_ctx, i, allocator, agg); \
|
||||
if (OB_FAIL(ret)) { \
|
||||
SQL_LOG(WARN, "init aggregate failed", K(vec_tc), K(dec_tc), K(*aggr_info.expr_)); \
|
||||
}
|
||||
|
||||
namespace helper
|
||||
{
|
||||
static int init_single_row_sum_agg(VecValueTypeClass in_tc, VecValueTypeClass out_tc,
|
||||
RuntimeContext &agg_ctx, int64_t i, ObIAllocator &allocator,
|
||||
IAggregate *&agg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObAggrInfo &aggr_info = agg_ctx.aggr_infos_.at(i);
|
||||
switch (in_tc) {
|
||||
case VEC_TC_INTEGER: {
|
||||
switch (out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_INTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_INTEGER, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_INTEGER, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_UINTEGER: {
|
||||
switch (out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_UINTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_UINTEGER, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_UINTEGER, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT32: {
|
||||
switch(out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_DEC_INT32);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT32, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT32, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT64: {
|
||||
switch (out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_DEC_INT64);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT64, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT64, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
switch (out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT128, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT128, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
switch (out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT256, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT256, VEC_TC_DEC_INT512);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: {
|
||||
switch(out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_DEC_INT512);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: {
|
||||
INIT_SUM_TO_DEC_CASE(VEC_TC_DEC_INT512, VEC_TC_DEC_INT512);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_NUMBER: {
|
||||
if (out_tc != VEC_TC_NUMBER) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
} else {
|
||||
INIT_SUM_TO_NMB_CASE(VEC_TC_NUMBER);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_FLOAT: {
|
||||
if (out_tc != VEC_TC_FLOAT) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
} else {
|
||||
ret =
|
||||
init_agg_func<SingleRowAggregate<T_FUN_SUM, VEC_TC_FLOAT, VEC_TC_FLOAT>>(agg_ctx, i, allocator, agg);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DOUBLE: {
|
||||
if (out_tc != VEC_TC_DOUBLE) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
} else {
|
||||
ret =
|
||||
init_agg_func<SingleRowAggregate<T_FUN_SUM, VEC_TC_DOUBLE, VEC_TC_DOUBLE>>(agg_ctx, i, allocator, agg);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_FIXED_DOUBLE: {
|
||||
if (out_tc != VEC_TC_FIXED_DOUBLE) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
} else {
|
||||
ret = init_agg_func<SingleRowAggregate<T_FUN_SUM, VEC_TC_FIXED_DOUBLE, VEC_TC_FIXED_DOUBLE>>(
|
||||
agg_ctx, i, allocator, agg);
|
||||
}
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "init fast single aggregate failed", K(in_tc), K(out_tc));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int init_single_row_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator,
|
||||
ObIArray<IAggregate *> &aggregates)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
IAggregate *agg= nullptr;
|
||||
for (int i = 0; OB_SUCC(ret) && i < agg_ctx.aggr_infos_.count(); i++) {
|
||||
ObAggrInfo &aggr_info = agg_ctx.aggr_infos_.at(i);
|
||||
agg = nullptr;
|
||||
if (OB_ISNULL(aggr_info.expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid null aggregate expr", K(ret));
|
||||
} else if (aggr_info.get_expr_type() == T_FUN_COUNT && aggr_info.param_exprs_.empty()) {
|
||||
if (lib::is_mysql_mode()) {
|
||||
ret = init_agg_func<SingleRowAggregate<T_FUN_COUNT, VEC_TC_INTEGER, VEC_TC_INTEGER>>(
|
||||
agg_ctx, i, allocator, agg);
|
||||
} else {
|
||||
ret = init_agg_func<SingleRowAggregate<T_FUN_COUNT, VEC_TC_INTEGER, VEC_TC_NUMBER>>(
|
||||
agg_ctx, i, allocator, agg);
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("init single row aggregate failed", K(ret));
|
||||
}
|
||||
} else if (aggr_info.is_implicit_first_aggr()) {
|
||||
// ObDatumMeta &res_meta = aggr_info.expr_->datum_meta_;
|
||||
// VecValueTypeClass res_vec =
|
||||
// get_vec_value_tc(res_meta.type_, res_meta.scale_, res_meta.precision_);
|
||||
// switch (res_vec) {
|
||||
// LST_DO_CODE(INIT_GENERAL_CASE, AGG_VEC_TC_LIST);
|
||||
// default: {
|
||||
// ret = OB_ERR_UNEXPECTED;
|
||||
// SQL_LOG(WARN, "unexpected vector type class", K(res_vec), K(*aggr_info.expr_));
|
||||
// break;
|
||||
// }
|
||||
// }
|
||||
// do nothing
|
||||
} else {
|
||||
ObDatumMeta &res_meta = aggr_info.expr_->datum_meta_;
|
||||
ObDatumMeta &child_meta = aggr_info.param_exprs_.at(0)->datum_meta_;
|
||||
VecValueTypeClass res_vec =
|
||||
get_vec_value_tc(res_meta.type_, res_meta.scale_, res_meta.precision_);
|
||||
VecValueTypeClass param_vec =
|
||||
get_vec_value_tc(child_meta.type_, child_meta.scale_, child_meta.precision_);
|
||||
switch(aggr_info.get_expr_type()) {
|
||||
case T_FUN_COUNT: {
|
||||
switch (param_vec) {
|
||||
LST_DO_CODE(INIT_COUNT_CASE, AGG_VEC_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "init count aggregate failed", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case T_FUN_SUM: {
|
||||
if (OB_FAIL(init_single_row_sum_agg(param_vec, res_vec, agg_ctx, i, allocator, agg))) {
|
||||
SQL_LOG(WARN, "init sum agg failed", K(param_vec), K(res_vec), K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case T_FUN_MIN: {
|
||||
switch (param_vec) {
|
||||
LST_DO_CODE(INIT_GENERAL_MIN_CASE, AGG_VEC_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected vector type class", K(param_vec), K(*aggr_info.expr_));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case T_FUN_MAX: {
|
||||
switch (param_vec) {
|
||||
LST_DO_CODE(INIT_GENERAL_MAX_CASE, AGG_VEC_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected vector type class", K(param_vec), K(*aggr_info.expr_));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case T_FUN_COUNT_SUM: {
|
||||
switch (param_vec) {
|
||||
LST_DO_CODE(INIT_GENERAL_COUNT_SUM_CASE, AGG_VEC_TC_LIST);
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected vector type class", K(param_vec), K(*aggr_info.expr_));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
SQL_LOG(WARN, "not supported aggregate operation", K(ret), K(*aggr_info.expr_));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(aggregates.push_back(agg))) {
|
||||
SQL_LOG(WARN, "push back element failed", K(ret));
|
||||
}
|
||||
} // end for
|
||||
return ret;
|
||||
}
|
||||
} // end helper
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
@ -1,211 +0,0 @@
|
||||
/**
|
||||
* 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 OCEANBSE_SHARE_AGGREGATE_SINGLE_ROW_H_
|
||||
#define OCEANBSE_SHARE_AGGREGATE_SINGLE_ROW_H_
|
||||
#include "iaggregate.h"
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
template <ObExprOperatorType agg_func, VecValueTypeClass in_tc, VecValueTypeClass out_tc>
|
||||
class SingleRowAggregate final : public BatchAggregateWrapper<SingleRowAggregate<agg_func, in_tc, out_tc>>
|
||||
{
|
||||
using ParamType = typename std::conditional<agg_func == T_FUN_COUNT, int64_t, AggCalcType<in_tc>>::type;
|
||||
using ResultType = AggCalcType<out_tc>;
|
||||
public:
|
||||
static const constexpr VecValueTypeClass IN_TC = in_tc;
|
||||
static const constexpr VecValueTypeClass OUT_TC = out_tc;
|
||||
public:
|
||||
SingleRowAggregate(): is_first_row_(false) {}
|
||||
template <typename ResultFmt>
|
||||
inline int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr,
|
||||
const int32_t agg_col_id, const char *agg_cell,
|
||||
const int32_t agg_cell_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const NotNullBitVector ¬nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
int64_t output_idx = agg_ctx.eval_ctx_.get_batch_idx();
|
||||
ResultFmt *res_vec = static_cast<ResultFmt *>(agg_expr.get_vector(agg_ctx.eval_ctx_));
|
||||
if (notnulls.at(agg_col_id) && T_FUN_COUNT == agg_func) {
|
||||
if (lib::is_oracle_mode()) {
|
||||
static const uint32_t constexpr one_val[2] = {3221225473, 1};
|
||||
res_vec->set_number(output_idx, *reinterpret_cast<const number::ObCompactNumber *>(one_val));
|
||||
} else {
|
||||
res_vec->set_int(output_idx, 1);
|
||||
}
|
||||
} else if (notnulls.at(agg_col_id)) {
|
||||
if (agg_func != T_FUN_SUM || in_tc == out_tc) {
|
||||
const char *agg_data = agg_cell;
|
||||
if (helper::is_var_len_agg_cell(in_tc)) {
|
||||
agg_data = reinterpret_cast<const char *>(*reinterpret_cast<const int64_t *>(agg_cell));
|
||||
char *res_buf = nullptr;
|
||||
if (OB_ISNULL(res_buf = agg_expr.get_str_res_mem(agg_ctx.eval_ctx_, agg_cell_len))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
CellWriter<ResultType>::set(agg_data, agg_cell_len, res_vec, output_idx, res_buf);
|
||||
}
|
||||
} else {
|
||||
CellWriter<ResultType>::set(agg_data, agg_cell_len, res_vec, output_idx, nullptr);
|
||||
}
|
||||
} else if (out_tc == VEC_TC_NUMBER) {
|
||||
ResultType *out_val = nullptr;
|
||||
int32_t out_len = 0;
|
||||
ObScale in_scale = agg_ctx.get_first_param_scale(agg_col_id);
|
||||
char local_buf[64] = {0};
|
||||
ObDataBuffer tmp_alloc(local_buf, 64);
|
||||
ret = Caster<ParamType, ResultType>::to_type(agg_cell, agg_cell_len, in_scale,
|
||||
tmp_alloc, out_val, out_len);
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "cast value failed", K(ret));
|
||||
} else {
|
||||
CellWriter<ResultType>::set(reinterpret_cast<const char *>(out_val), out_len, res_vec,
|
||||
output_idx, nullptr);
|
||||
}
|
||||
} else {
|
||||
CellWriter<ResultType>::cp_and_set(*reinterpret_cast<const ParamType *>(agg_cell), res_vec,
|
||||
output_idx, nullptr);
|
||||
}
|
||||
} else if (agg_func == T_FUN_COUNT) {
|
||||
static const uint32_t constexpr zero_val[1] = {2147483648};
|
||||
if (lib::is_oracle_mode()) {
|
||||
// number::ObNumber tmp_nmb;
|
||||
// tmp_nmb.set_zero();
|
||||
res_vec->set_number(output_idx, *reinterpret_cast<const number::ObCompactNumber *>(zero_val));
|
||||
} else {
|
||||
res_vec->set_int(output_idx, 0);
|
||||
}
|
||||
} else {
|
||||
res_vec->set_null(output_idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
inline int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size,
|
||||
const bool is_null, const char *data, const int32_t data_len,
|
||||
int32_t agg_col_idx, char *agg_cell) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
AggrRowPtr agg_row = agg_ctx.agg_rows_.at(batch_idx);
|
||||
NotNullBitVector ¬nulls = agg_ctx.row_meta().locate_notnulls_bitmap(agg_row);
|
||||
if (agg_func != T_FUN_COUNT) {
|
||||
if (OB_LIKELY(!is_null)) {
|
||||
char *cell = agg_ctx.row_meta().locate_cell_payload(agg_col_idx, agg_row);
|
||||
if (helper::is_var_len_agg_cell(in_tc)) {
|
||||
*reinterpret_cast<int64_t *>(cell) = reinterpret_cast<int64_t>(data);
|
||||
*reinterpret_cast<int32_t *>(cell + sizeof(char *)) = data_len;
|
||||
} else {
|
||||
MEMCPY(cell, data, data_len);
|
||||
}
|
||||
notnulls.set(agg_col_idx);
|
||||
}
|
||||
} else if (!is_null) { // COUNT function, only need to set not null
|
||||
notnulls.set(agg_col_idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <typename ColumnFmt>
|
||||
inline int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_cell, tmp_res, calc_info);
|
||||
AggrRowPtr agg_row = agg_ctx.agg_rows_.at(row_num);
|
||||
NotNullBitVector ¬nulls = agg_ctx.row_meta().locate_notnulls_bitmap(agg_row);
|
||||
if (agg_func != T_FUN_COUNT) {
|
||||
char *cell = agg_ctx.row_meta().locate_cell_payload(agg_col_id, agg_row);
|
||||
const char *payload = nullptr;
|
||||
int32_t len = 0;
|
||||
columns.get_payload(row_num, payload, len);
|
||||
if (helper::is_var_len_agg_cell(in_tc)) {
|
||||
*reinterpret_cast<int64_t *>(cell) = reinterpret_cast<int64_t>(payload);
|
||||
*reinterpret_cast<int32_t *>(cell + sizeof(char *)) = len;
|
||||
} else {
|
||||
MEMCPY(cell, payload, len);
|
||||
}
|
||||
}
|
||||
notnulls.set(agg_col_id);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
template <typename ColumnFmt>
|
||||
inline int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res,
|
||||
int64_t &calc_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (columns.is_null(row_num)) {
|
||||
// do nothing
|
||||
} else {
|
||||
ret = add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int add_param_batch(RuntimeContext &agg_ctx, const ObBitVector &skip, ObBitVector &pvt_skip,
|
||||
const EvalBound &bound, const RowSelector &row_sel, const int32_t agg_col_id,
|
||||
const int32_t param_id, ColumnFmt ¶m_vec, char *aggr_cell)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
AggrRowPtr agg_row = nullptr;
|
||||
if (row_sel.is_empty()) {
|
||||
for (int i = bound.start(); i < bound.end(); i++) {
|
||||
if (pvt_skip.at(i)) { continue; }
|
||||
if (param_vec.is_null(i)) { pvt_skip.set(i); }
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < row_sel.size(); i++) {
|
||||
if (param_vec.is_null(row_sel.index(i))) { pvt_skip.set(row_sel.index(i)); }
|
||||
}
|
||||
}
|
||||
if (param_id == agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.count() - 1) {
|
||||
// last param expr
|
||||
if (row_sel.is_empty()) {
|
||||
for (int i = bound.start(); i < bound.end(); i++) {
|
||||
if (skip.at(i)) { continue; }
|
||||
agg_row = agg_ctx.agg_rows_.at(i);
|
||||
NotNullBitVector ¬nulls = agg_ctx.row_meta().locate_notnulls_bitmap(agg_row);
|
||||
if (!pvt_skip.at(i)) {
|
||||
notnulls.set(i);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < row_sel.size(); i++) {
|
||||
agg_row = agg_ctx.agg_rows_.at(row_sel.index(i));
|
||||
NotNullBitVector ¬nulls = agg_ctx.row_meta().locate_notnulls_bitmap(agg_row);
|
||||
if (!pvt_skip.at(row_sel.index(i))) {
|
||||
notnulls.set(row_sel.index(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
void set_first_row(bool is_first_row) { is_first_row_ = is_first_row; }
|
||||
TO_STRING_KV("aggregate", "single_row", K(in_tc), K(out_tc), K(is_first_row_));
|
||||
private:
|
||||
bool is_first_row_;
|
||||
};
|
||||
|
||||
namespace helper
|
||||
{
|
||||
int init_single_row_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator,
|
||||
ObIArray<IAggregate *> &aggregates);
|
||||
} // end helper
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
|
||||
#endif // OCEANBSE_SHARE_AGGREGATE_SINGLE_ROW_H_
|
||||
@ -1,277 +0,0 @@
|
||||
/**
|
||||
* 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
|
||||
|
||||
#include "sum.h"
|
||||
#include "sum_nmb.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
|
||||
namespace helper
|
||||
{
|
||||
|
||||
int init_sum_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg, int32_t *tmp_res_size)
|
||||
{
|
||||
#define INIT_SUM_AGGREGATE_TMP_STORE(in_tc, out_tc, tmp_tc) \
|
||||
if (tmp_res_size != nullptr) { \
|
||||
*tmp_res_size = sizeof(RTCType<tmp_tc>); \
|
||||
} else { \
|
||||
ret = init_agg_func<SumAggregateWithTempStore<in_tc, out_tc, RTCType<tmp_tc>>>( \
|
||||
agg_ctx, agg_col_id, aggr_info.has_distinct_, allocator, agg); \
|
||||
}
|
||||
|
||||
#define INIT_SUM_AGGREGATE(in_tc, out_tc) \
|
||||
if (tmp_res_size != nullptr) { \
|
||||
*tmp_res_size = 0; \
|
||||
} else { \
|
||||
ret = init_agg_func<SumAggregate<in_tc, out_tc>>(agg_ctx, agg_col_id, aggr_info.has_distinct_, \
|
||||
allocator, agg); \
|
||||
}
|
||||
|
||||
int ret = OB_SUCCESS;
|
||||
ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id);
|
||||
OB_ASSERT(aggr_info.expr_ != NULL);
|
||||
OB_ASSERT(aggr_info.expr_->args_ != NULL && aggr_info.expr_->arg_cnt_ > 0);
|
||||
ObDatumMeta &in_meta = aggr_info.expr_->args_[0]->datum_meta_;
|
||||
ObDatumMeta &out_meta = aggr_info.expr_->datum_meta_;
|
||||
VecValueTypeClass in_tc = get_vec_value_tc(in_meta.type_, in_meta.scale_, in_meta.precision_);
|
||||
VecValueTypeClass out_tc = get_vec_value_tc(out_meta.type_, out_meta.scale_, out_meta.precision_);
|
||||
void *buf = nullptr;
|
||||
switch (in_tc) {
|
||||
case (VEC_TC_INTEGER): { // integer
|
||||
switch (out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_INTEGER, VEC_TC_NUMBER, VEC_TC_INTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT64: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_INTEGER, VEC_TC_DEC_INT64);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_INTEGER, VEC_TC_DEC_INT128, VEC_TC_INTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_INTEGER, VEC_TC_DEC_INT256, VEC_TC_INTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_INTEGER, VEC_TC_DEC_INT512, VEC_TC_INTEGER);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case (VEC_TC_UINTEGER): { // uinteger
|
||||
switch (out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_UINTEGER, VEC_TC_NUMBER, VEC_TC_UINTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT64: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_UINTEGER, VEC_TC_DEC_INT64);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_UINTEGER, VEC_TC_DEC_INT128, VEC_TC_UINTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_UINTEGER, VEC_TC_DEC_INT256, VEC_TC_UINTEGER);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_UINTEGER, VEC_TC_DEC_INT512, VEC_TC_UINTEGER);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case (VEC_TC_FLOAT): { // float
|
||||
if (out_tc != VEC_TC_FLOAT) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
} else {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_FLOAT, VEC_TC_FLOAT);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case (VEC_TC_FIXED_DOUBLE): { // fixed double
|
||||
if (out_tc != VEC_TC_FIXED_DOUBLE) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
} else {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_FIXED_DOUBLE, VEC_TC_FIXED_DOUBLE);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case (VEC_TC_DOUBLE): { // double
|
||||
if (out_tc != VEC_TC_DOUBLE) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
} else {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DOUBLE, VEC_TC_DOUBLE);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case (VEC_TC_NUMBER): { // number
|
||||
if (out_tc != VEC_TC_NUMBER) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
} else {
|
||||
if (tmp_res_size != nullptr) {
|
||||
*tmp_res_size = 0;
|
||||
} else {
|
||||
ret = init_agg_func<SumNumberAggregate>(agg_ctx, agg_col_id, aggr_info.has_distinct_,
|
||||
allocator, agg);
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT32: { // int32
|
||||
switch(out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_DEC_INT32, VEC_TC_NUMBER, VEC_TC_DEC_INT64);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT32, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT64: { // int64
|
||||
switch(out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_DEC_INT64, VEC_TC_NUMBER, VEC_TC_DEC_INT64);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT64, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT64, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: { // int128
|
||||
switch(out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_DEC_INT128, VEC_TC_NUMBER, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT128: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT128, VEC_TC_DEC_INT128);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT128, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: { // int256
|
||||
switch(out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_DEC_INT256, VEC_TC_NUMBER, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT256: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT256, VEC_TC_DEC_INT256);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT256, VEC_TC_DEC_INT512);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: { // int512
|
||||
switch(out_tc) {
|
||||
case VEC_TC_NUMBER: {
|
||||
INIT_SUM_AGGREGATE_TMP_STORE(VEC_TC_DEC_INT512 , VEC_TC_NUMBER, VEC_TC_DEC_INT512);
|
||||
break;
|
||||
}
|
||||
case VEC_TC_DEC_INT512: {
|
||||
INIT_SUM_AGGREGATE(VEC_TC_DEC_INT512, VEC_TC_DEC_INT512);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected in & out type class", K(in_tc), K(out_tc));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
#undef INIT_SUM_AGGREGATE
|
||||
}
|
||||
|
||||
int init_count_sum_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id,
|
||||
ObIAllocator &allocator, IAggregate *&agg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (lib::is_oracle_mode()) {
|
||||
return init_agg_func<SumNumberAggregate>(
|
||||
agg_ctx, agg_col_id, agg_ctx.aggr_infos_.at(agg_col_id).has_distinct_, allocator, agg);
|
||||
} else {
|
||||
return init_agg_func<SumAggregate<VEC_TC_INTEGER, VEC_TC_INTEGER>>(
|
||||
agg_ctx, agg_col_id, agg_ctx.aggr_infos_.at(agg_col_id).has_distinct_, allocator, agg);
|
||||
}
|
||||
}
|
||||
} // end helper
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
@ -1,525 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_SUM_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_SUM_H_
|
||||
|
||||
#include "share/aggregate/iaggregate.h"
|
||||
|
||||
#include <type_traits>
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
|
||||
struct SumCalcInfo
|
||||
{
|
||||
SumCalcInfo(): flags_(0) {}
|
||||
SumCalcInfo(int32_t agg_cell_len, int16_t scale) :
|
||||
agg_cell_len_(agg_cell_len), scale_(scale), reserved_(0)
|
||||
{}
|
||||
SumCalcInfo(int64_t flags): flags_(flags) {}
|
||||
operator int64_t() const { return flags_; }
|
||||
union {
|
||||
struct {
|
||||
int32_t agg_cell_len_;
|
||||
int16_t scale_;
|
||||
int16_t reserved_;
|
||||
};
|
||||
int64_t flags_;
|
||||
};
|
||||
};
|
||||
|
||||
static_assert(sizeof(SumCalcInfo) <= sizeof(int64_t), "");
|
||||
// sum aggregate rules:
|
||||
// int/uint -> number
|
||||
// decint->decint'
|
||||
// number->number
|
||||
// float->float
|
||||
// double->double
|
||||
|
||||
// param & result are same types
|
||||
template<VecValueTypeClass in_tc, VecValueTypeClass out_tc>
|
||||
class SumAggregate final: public BatchAggregateWrapper<SumAggregate<in_tc, out_tc>>
|
||||
{
|
||||
using ResultType = AggCalcType<out_tc>;
|
||||
using ParamType = AggCalcType<in_tc>;
|
||||
public:
|
||||
static const constexpr VecValueTypeClass IN_TC = in_tc;
|
||||
static const constexpr VecValueTypeClass OUT_TC = out_tc;
|
||||
public:
|
||||
SumAggregate() {}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
UNUSED(tmp_res);
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
const char* param_payload = nullptr;
|
||||
int32_t param_len = 0;
|
||||
columns.get_payload(row_num, param_payload, param_len);
|
||||
const ParamType *lparam = reinterpret_cast<const ParamType *>(param_payload);
|
||||
const ResultType &rparam = *reinterpret_cast<const ResultType *>(agg_cell);
|
||||
SQL_LOG(DEBUG, "sum add row", K(agg_col_id));
|
||||
if ((is_decint_vec(in_tc) && is_decint_vec(out_tc)) // sum(int64/int32) -> int128
|
||||
|| (in_tc == VEC_TC_INTEGER && out_tc == VEC_TC_INTEGER)) { // count_sum
|
||||
ret = add_values(*lparam, rparam, agg_cell, sizeof(ResultType));
|
||||
} else {
|
||||
SumCalcInfo &sum_calc_info = reinterpret_cast<SumCalcInfo &>(calc_info);
|
||||
if (OB_FAIL(add_to_result(*lparam, rparam, sum_calc_info.scale_, agg_cell,
|
||||
sum_calc_info.agg_cell_len_))) {
|
||||
SQL_LOG(WARN, "add_to_result failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (columns.is_null(row_num)) {
|
||||
SQL_LOG(DEBUG, "add null row", K(ret), K(row_num));
|
||||
} else if (OB_FAIL(
|
||||
add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
} else {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
not_nulls.set(agg_col_id);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr,
|
||||
const int32_t agg_col_id, const char *agg_cell,
|
||||
const int32_t agg_cell_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
int64_t output_idx = ctx.get_batch_idx();
|
||||
const NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
ColumnFmt *res_vec = static_cast<ColumnFmt *>(agg_expr.get_vector(agg_ctx.eval_ctx_));
|
||||
if (OB_LIKELY(not_nulls.at(agg_col_id))) {
|
||||
CellWriter<ResultType>::set(agg_cell, agg_cell_len, res_vec, output_idx, nullptr);
|
||||
} else if (agg_expr.type_ == T_FUN_COUNT_SUM) {
|
||||
if (VEC_TC_INTEGER == out_tc) {
|
||||
int64_t res = 0;
|
||||
res_vec->set_payload(output_idx, &res, sizeof(int64_t));
|
||||
} else if (VEC_TC_NUMBER == out_tc) {
|
||||
number::ObNumber zero;
|
||||
zero.set_zero();
|
||||
res_vec->set_payload(output_idx, &zero, sizeof(ObNumberDesc));
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected output result type", K(ret), K(out_tc));
|
||||
}
|
||||
} else {
|
||||
res_vec->set_null(output_idx);
|
||||
}
|
||||
if (out_tc == VEC_TC_DEC_INT512 && OB_SUCC(ret) && lib::is_mysql_mode()) {
|
||||
// check mysql decimal int overflow
|
||||
const int512_t &res = *reinterpret_cast<const int512_t *>(agg_cell);
|
||||
if (res <= wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MIN
|
||||
|| res >= wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MAX) {
|
||||
int ret = OB_ERR_TRUNCATED_WRONG_VALUE;
|
||||
ObString decimal_type_str("DECIMAL");
|
||||
char buf[MAX_PRECISION_DECIMAL_INT_512];
|
||||
int64_t pos = 0;
|
||||
wide::to_string(wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MAX_AVAILABLE, buf,
|
||||
MAX_PRECISION_DECIMAL_INT_512, pos);
|
||||
LOG_USER_WARN(OB_ERR_TRUNCATED_WRONG_VALUE, decimal_type_str.length(),
|
||||
decimal_type_str.ptr(), static_cast<int32_t>(pos), buf);
|
||||
SQL_LOG(WARN, "decimal int out of range", K(ret));
|
||||
// overflow, set datum to max available decimal int
|
||||
const ObDecimalInt *max_available_val = reinterpret_cast<const ObDecimalInt *>(
|
||||
&(wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MAX_AVAILABLE));
|
||||
res_vec->set_payload_shallow(output_idx, max_available_val, sizeof(int512_t));
|
||||
ret = OB_SUCCESS; // reset ret to SUCCESS, just log user warnings
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
inline int64_t get_batch_calc_info(RuntimeContext &agg_ctx, int32_t agg_col_idx,
|
||||
char *agg_cell) override
|
||||
{
|
||||
UNUSED(agg_cell);
|
||||
ObScale scale = agg_ctx.get_first_param_scale(agg_col_idx);
|
||||
int32_t agg_cell_len = agg_ctx.row_meta().get_cell_len(agg_col_idx, nullptr /*not used*/);
|
||||
return static_cast<int64_t>(SumCalcInfo(agg_cell_len, scale));
|
||||
}
|
||||
|
||||
int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size,
|
||||
const bool is_null, const char *data, const int32_t data_len, int32_t agg_col_idx,
|
||||
char *agg_cell) override
|
||||
{
|
||||
UNUSEDx(data_len, batch_idx, batch_size);
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!is_null)) {
|
||||
const ParamType *lparam = reinterpret_cast<const ParamType *>(data);
|
||||
const ResultType &rparam = *reinterpret_cast<const ResultType *>(agg_cell);
|
||||
if ((is_decint_vec(in_tc) && is_decint_vec(out_tc)) // sum(int64/int32) -> int128
|
||||
|| (in_tc == VEC_TC_INTEGER && out_tc == VEC_TC_INTEGER)) { // count_sum
|
||||
ret = add_values(*lparam, rparam, agg_cell, sizeof(ResultType));
|
||||
} else {
|
||||
SumCalcInfo calc_info = get_batch_calc_info(agg_ctx, agg_col_idx, agg_cell);
|
||||
|
||||
if (OB_FAIL(add_to_result(*lparam, rparam, calc_info.scale_, agg_cell,
|
||||
calc_info.agg_cell_len_))) {
|
||||
SQL_LOG(WARN, "add_to_result failed", K(ret), K(*this), K(*lparam), K(rparam), K(batch_idx), KP(data));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell);
|
||||
not_nulls.set(agg_col_idx);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
TO_STRING_KV("aggregate", "sum", K(in_tc), K(out_tc));
|
||||
private:
|
||||
int add_to_result(const ParamType &lparam, const ResultType &rparam, const ObScale scale,
|
||||
char *res_buf, const int32_t res_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret = add_overflow(lparam, rparam, res_buf, res_len);
|
||||
if (OB_FAIL(ret)) {
|
||||
if (ret == OB_OPERATE_OVERFLOW && out_tc == VEC_TC_FLOAT) {
|
||||
if (!lib::is_oracle_mode()) {
|
||||
char buf[OB_MAX_TWO_OPERATOR_EXPR_LENGTH] = {0};
|
||||
int64_t buf_len = OB_MAX_TWO_OPERATOR_EXPR_LENGTH;
|
||||
int64_t pos = 0;
|
||||
BUF_PRINTF("'(");
|
||||
BUF_PRINTO(lparam);
|
||||
BUF_PRINTF(" + ");
|
||||
BUF_PRINTO(rparam);
|
||||
BUF_PRINTF(")'");
|
||||
LOG_USER_ERROR(OB_OPERATE_OVERFLOW, (in_tc == VEC_TC_FLOAT ? "BINARY_FLOAT" : "DOUBLE"),
|
||||
buf);
|
||||
SQL_LOG(WARN, "do_overflow failed", K(lparam), K(rparam), K(ret));
|
||||
} else {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
};
|
||||
|
||||
template <VecValueTypeClass in_tc, VecValueTypeClass out_tc, typename TmpStore>
|
||||
class SumAggregateWithTempStore final
|
||||
: public BatchAggregateWrapper<SumAggregateWithTempStore<in_tc, out_tc, TmpStore>>
|
||||
{
|
||||
using ParamType = AggCalcType<in_tc>;
|
||||
using ResultType = AggCalcType<out_tc>;
|
||||
public:
|
||||
static const constexpr VecValueTypeClass IN_TC = in_tc;
|
||||
static const constexpr VecValueTypeClass OUT_TC = out_tc;
|
||||
public:
|
||||
SumAggregateWithTempStore() {}
|
||||
|
||||
int init(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int collect_tmp_result(RuntimeContext &agg_ctx, const int32_t agg_col_id, char *agg_cell)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
TmpStore &tmp_res = *reinterpret_cast<TmpStore *>(get_tmp_res(agg_ctx, agg_col_id, agg_cell));
|
||||
if (OB_LIKELY(tmp_res != 0)) {
|
||||
ResultType &res = *reinterpret_cast<ResultType *>(agg_cell);
|
||||
const int32_t agg_cell_len = agg_ctx.row_meta().get_cell_len(agg_col_id, agg_cell);
|
||||
ObScale scale = agg_ctx.get_first_param_scale(agg_col_id);
|
||||
if (out_tc == VEC_TC_NUMBER) {
|
||||
if (OB_FAIL(add_value_to_nmb(tmp_res, scale, agg_cell))) {
|
||||
SQL_LOG(WARN, "add value to nmb failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
ResultType *res_val = reinterpret_cast<ResultType *>(agg_cell);
|
||||
ret = add_values(tmp_res, *res_val, agg_cell, agg_ctx.get_cell_len(agg_col_id, agg_cell));
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "do op failed", K(ret));
|
||||
} else {
|
||||
tmp_res = 0;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size,
|
||||
const bool is_null, const char *data, const int32_t data_len, int32_t agg_col_idx,
|
||||
char *agg_cell) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!is_null) {
|
||||
const ParamType *row_param = reinterpret_cast<const ParamType *>(data);
|
||||
TmpStore *tmp_res = reinterpret_cast<TmpStore *>(get_tmp_res(agg_ctx, agg_col_idx, agg_cell));
|
||||
SQL_LOG(DEBUG, "sum with tmp::add_one_row ", K(*row_param), K(*tmp_res), K(batch_idx));
|
||||
ret = add_overflow(*row_param, *tmp_res, reinterpret_cast<char *>(tmp_res), sizeof(TmpStore));
|
||||
if (OB_FAIL(ret)) {
|
||||
if (OB_LIKELY(OB_OPERATE_OVERFLOW == ret)) {
|
||||
ret = OB_SUCCESS;
|
||||
if (out_tc == VEC_TC_NUMBER) {
|
||||
ObScale scale = agg_ctx.get_first_param_scale(agg_col_idx);
|
||||
if (OB_FAIL(add_value_to_nmb(*tmp_res, scale, agg_cell))) {
|
||||
SQL_LOG(WARN, "add value to nmb failed", K(ret));
|
||||
} else if (OB_FAIL(add_value_to_nmb(*row_param, scale, agg_cell))) {
|
||||
SQL_LOG(WARN, "add value to nmb failed", K(ret));
|
||||
} else {
|
||||
*tmp_res = 0;
|
||||
}
|
||||
} else {
|
||||
ResultType &res_val = *reinterpret_cast<ResultType *>(agg_cell);
|
||||
if(OB_FAIL(add_values(*row_param, res_val, agg_cell, sizeof(ResultType)))) {
|
||||
} else if (OB_FAIL(add_values(*tmp_res, res_val, agg_cell, sizeof(ResultType)))) {
|
||||
} else {
|
||||
*tmp_res = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "do addition failed", K(ret));
|
||||
} else {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell);
|
||||
not_nulls.set(agg_col_idx);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int64_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res_ptr, int64_t &calc_info)
|
||||
{
|
||||
OB_ASSERT(tmp_res_ptr != NULL);
|
||||
int ret = OB_SUCCESS;
|
||||
const char *param_payload = nullptr;
|
||||
int32_t param_len = 0;
|
||||
columns.get_payload(row_num, param_payload, param_len);
|
||||
const ParamType *row_param = reinterpret_cast<const ParamType *>(param_payload);
|
||||
TmpStore &tmp_res = *reinterpret_cast<TmpStore *>(tmp_res_ptr);
|
||||
|
||||
SQL_LOG(DEBUG, "sum add row", K(row_param), K(row_num), K(tmp_res));
|
||||
ret = add_overflow(*row_param, tmp_res, reinterpret_cast<char *>(&tmp_res), sizeof(TmpStore));
|
||||
if (OB_FAIL(ret)) {
|
||||
if (OB_LIKELY(ret == OB_OPERATE_OVERFLOW)) {
|
||||
ret = OB_SUCCESS;
|
||||
if (out_tc == VEC_TC_NUMBER) {
|
||||
ObScale in_scale = agg_ctx.get_first_param_scale(agg_col_id);
|
||||
if (OB_FAIL(add_value_to_nmb(tmp_res, in_scale, agg_cell))) {
|
||||
SQL_LOG(WARN, "add_value_to_nmb failed", K(ret));
|
||||
} else if (OB_FAIL(add_value_to_nmb(*row_param, in_scale, agg_cell))) {
|
||||
SQL_LOG(WARN, "add_value_to_nmb failed", K(ret));
|
||||
} else {
|
||||
tmp_res = 0;
|
||||
}
|
||||
} else {
|
||||
ResultType &res_val = *reinterpret_cast<ResultType *>(agg_cell);
|
||||
if (OB_FAIL(add_values(tmp_res, res_val, agg_cell, sizeof(ResultType)))) {
|
||||
SQL_LOG(WARN, "add values failed", K(ret));
|
||||
} else if (OB_FAIL(add_values(*row_param, res_val, agg_cell, sizeof(ResultType)))) {
|
||||
SQL_LOG(WARN, "add values failed", K(ret));
|
||||
} else {
|
||||
tmp_res = 0;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
SQL_LOG(WARN, "add overflow failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int64_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res_ptr,
|
||||
int64_t &calc_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (columns.is_null(row_num)) {
|
||||
SQL_LOG(DEBUG, "sum add null", K(ret));
|
||||
// do nothing
|
||||
} else if (OB_FAIL(add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res_ptr,
|
||||
calc_info))) {
|
||||
SQL_LOG(WARN, "add row failed", K(ret));
|
||||
} else {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
not_nulls.set(agg_col_id);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr,
|
||||
const int32_t agg_col_id, const char *agg_cell, const int32_t agg_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx &ctx = agg_ctx.eval_ctx_;
|
||||
const NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
int64_t output_idx = ctx.get_batch_idx();
|
||||
ColumnFmt *res_vec = static_cast<ColumnFmt *>(agg_expr.get_vector(ctx));
|
||||
if (OB_LIKELY(not_nulls.at(agg_col_id))) {
|
||||
TmpStore *tmp_res = reinterpret_cast<TmpStore *>(
|
||||
get_tmp_res(agg_ctx, agg_col_id, const_cast<char *>(agg_cell)));
|
||||
if (VEC_TC_NUMBER == out_tc) {
|
||||
ObScale scale = agg_ctx.get_first_param_scale(agg_col_id);
|
||||
if (OB_FAIL(add_value_to_nmb(*tmp_res, scale, const_cast<char *>(agg_cell)))) {
|
||||
SQL_LOG(WARN, "add value to nmb failed", K(ret));
|
||||
} else {
|
||||
CellWriter<ResultType>::set(agg_cell, agg_len, res_vec, output_idx, nullptr);
|
||||
}
|
||||
} else {
|
||||
ResultType *res_val = reinterpret_cast<ResultType *>(const_cast<char *>(agg_cell));
|
||||
ret = add_values(*tmp_res, *res_val, const_cast<char *>(agg_cell), agg_len);
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "add values failed", K(ret));
|
||||
} else {
|
||||
CellWriter<ResultType>::set(agg_cell, agg_len, res_vec, output_idx, nullptr);
|
||||
}
|
||||
}
|
||||
} else if (agg_expr.type_ == T_FUN_COUNT_SUM) {
|
||||
if (lib::is_oracle_mode() || out_tc == VEC_TC_NUMBER) {
|
||||
number::ObNumber res_nmb;
|
||||
res_nmb.set_zero();
|
||||
res_vec->set_number(output_idx, res_nmb);
|
||||
} else if (wide::IsIntegral<ResultType>::value) { // decimal int is used
|
||||
set_decint_zero<ColumnFmt, sizeof(ResultType)>(res_vec, output_idx);
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "invalid result type of count sum", K(ret), K(out_tc));
|
||||
}
|
||||
} else {
|
||||
static_cast<ColumnFmt *>(agg_expr.get_vector(ctx))->set_null(output_idx);
|
||||
}
|
||||
if (out_tc == VEC_TC_DEC_INT512 && OB_SUCC(ret) && lib::is_mysql_mode()) {
|
||||
// check mysql decimal int overflow
|
||||
const int512_t &res = *reinterpret_cast<const int512_t *>(agg_cell);
|
||||
if (res <= wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MIN
|
||||
|| res >= wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MAX) {
|
||||
int ret = OB_ERR_TRUNCATED_WRONG_VALUE;
|
||||
ObString decimal_type_str("DECIMAL");
|
||||
char buf[MAX_PRECISION_DECIMAL_INT_512];
|
||||
int64_t pos = 0;
|
||||
wide::to_string(wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MAX_AVAILABLE, buf,
|
||||
MAX_PRECISION_DECIMAL_INT_512, pos);
|
||||
LOG_USER_WARN(OB_ERR_TRUNCATED_WRONG_VALUE, decimal_type_str.length(),
|
||||
decimal_type_str.ptr(), static_cast<int32_t>(pos), buf);
|
||||
SQL_LOG(WARN, "decimal int out of range", K(ret));
|
||||
// overflow, set datum to max available decimal int
|
||||
const ObDecimalInt *max_available_val = reinterpret_cast<const ObDecimalInt *>(
|
||||
&(wide::ObDecimalIntConstValue::MYSQL_DEC_INT_MAX_AVAILABLE));
|
||||
res_vec->set_payload_shallow(output_idx, max_available_val, sizeof(int512_t));
|
||||
ret = OB_SUCCESS; // reset ret to SUCCESS, just log user warnings
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline void *get_tmp_res(RuntimeContext &agg_ctx, int32_t agg_col_id, char *agg_cell) override
|
||||
{
|
||||
int32_t cell_len = agg_ctx.get_cell_len(agg_col_id, agg_cell);
|
||||
return (void *)(agg_cell + cell_len);
|
||||
}
|
||||
|
||||
inline int64_t get_batch_calc_info(RuntimeContext &agg_ctx, int32_t agg_col_id,
|
||||
char *agg_cell) override
|
||||
{
|
||||
UNUSED(agg_cell);
|
||||
ObScale scale = agg_ctx.get_first_param_scale(agg_col_id);
|
||||
int32_t agg_cell_len = agg_ctx.row_meta().get_cell_len(agg_col_id, nullptr /*not used*/);
|
||||
return static_cast<int64_t>(SumCalcInfo(agg_cell_len, scale));
|
||||
}
|
||||
TO_STRING_KV("aggregate", "sum_with_tmp_store", K(in_tc), K(out_tc));
|
||||
private:
|
||||
template<typename ColumnFmt, int int_len>
|
||||
void set_decint_zero(ColumnFmt *res_vec, const int32_t output_idx)
|
||||
{
|
||||
switch (int_len) {
|
||||
case sizeof(int32_t): {
|
||||
int32_t v = 0;
|
||||
res_vec->set_payload(output_idx, &v, sizeof(int32_t));
|
||||
break;
|
||||
}
|
||||
case sizeof(int64_t): {
|
||||
int64_t v = 0;
|
||||
res_vec->set_payload(output_idx, &v, sizeof(int64_t));
|
||||
break;
|
||||
}
|
||||
case sizeof(int128_t): {
|
||||
int128_t v = 0;
|
||||
res_vec->set_payload(output_idx, &v, sizeof(int128_t));
|
||||
break;
|
||||
}
|
||||
case sizeof(int256_t): {
|
||||
int256_t v = 0;
|
||||
res_vec->set_payload(output_idx, &v, sizeof(int256_t));
|
||||
break;
|
||||
}
|
||||
case sizeof(int512_t): {
|
||||
int512_t v = 0;
|
||||
res_vec->set_payload(output_idx, &v, sizeof(int512_t));
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ob_assert(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
template<typename T>
|
||||
int add_value_to_nmb(const T &v, const ObScale in_scale, char *agg_cell)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObNumStackAllocator<2> tmp_alloc;
|
||||
number::ObNumber res_nmb, param_nmb;
|
||||
number::ObNumber tmp(*reinterpret_cast<number::ObCompactNumber *>(agg_cell));
|
||||
if (sizeof(T) > sizeof(int64_t)) {
|
||||
if (OB_FAIL(wide::to_number(v, in_scale, tmp_alloc, param_nmb))) {
|
||||
SQL_LOG(WARN, "to_number failed", K(ret));
|
||||
}
|
||||
} else if (sizeof(T) <= sizeof(int32_t)) {
|
||||
int64_t tmp_v = v;
|
||||
if (OB_FAIL(wide::to_number(tmp_v, in_scale, tmp_alloc, param_nmb))) {
|
||||
SQL_LOG(WARN, "wide::to_number failed", K(ret));
|
||||
}
|
||||
} else if (std::is_same<T, uint64_t>::value
|
||||
&& OB_FAIL(
|
||||
wide::to_number(static_cast<uint64_t>(v), in_scale, tmp_alloc, param_nmb))) {
|
||||
SQL_LOG(WARN, "wide::to_number failed", K(ret));
|
||||
} else if (OB_FAIL(wide::to_number(static_cast<int64_t>(v), in_scale, tmp_alloc, param_nmb))) {
|
||||
SQL_LOG(WARN, "wide::to_number failed", K(ret));
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(tmp.add_v3(param_nmb, res_nmb, tmp_alloc))) {
|
||||
SQL_LOG(WARN, "ObNumber::add_v3 failed", K(ret));
|
||||
} else {
|
||||
number::ObCompactNumber *cnum = reinterpret_cast<number::ObCompactNumber *>(agg_cell);
|
||||
cnum->desc_ = res_nmb.d_;
|
||||
MEMCPY(&cnum->digits_[0], res_nmb.get_digits(), res_nmb.d_.len_ * sizeof(uint32_t));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
};
|
||||
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_SUM_H_
|
||||
@ -1,278 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_SUM_NMB_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_SUM_NMB_H_
|
||||
|
||||
#include "share/aggregate/iaggregate.h"
|
||||
#include "sql/engine/aggregate/ob_aggregate_util.h"
|
||||
|
||||
#define ACCUMULATE_NMB(fmt) \
|
||||
if (!input_vec->has_null()) { \
|
||||
if (row_sel.is_empty()) { \
|
||||
ret = sql::number_accumulator(input_wrapper<fmt, false>(input_vec), allocator1, allocator2, \
|
||||
result, sum_digits, all_skip, bound_wrapper(bound, skip)); \
|
||||
} else { \
|
||||
ret = sql::number_accumulator(input_wrapper<fmt, false>(input_vec), allocator1, allocator2, \
|
||||
result, sum_digits, all_skip, rowsel_wrapper(row_sel)); \
|
||||
} \
|
||||
} else { \
|
||||
if (row_sel.is_empty()) { \
|
||||
ret = sql::number_accumulator(input_wrapper<fmt, true>(input_vec), allocator1, allocator2, \
|
||||
result, sum_digits, all_skip, bound_wrapper(bound, skip)); \
|
||||
} else { \
|
||||
ret = sql::number_accumulator(input_wrapper<fmt, true>(input_vec), allocator1, allocator2, \
|
||||
result, sum_digits, all_skip, rowsel_wrapper(row_sel)); \
|
||||
} \
|
||||
}
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
// fast number sum aggregation, wrapper of `sql::number_accumulator`
|
||||
class SumNumberAggregate final: public BatchAggregateWrapper<SumNumberAggregate>
|
||||
{
|
||||
public:
|
||||
static const constexpr VecValueTypeClass IN_TC = VEC_TC_NUMBER;
|
||||
static const constexpr VecValueTypeClass OUT_TC = VEC_TC_NUMBER;
|
||||
|
||||
public:
|
||||
SumNumberAggregate(): is_ora_count_sum_(false) {}
|
||||
int init(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
is_ora_count_sum_ = agg_ctx.aggr_infos_.at(agg_col_id).get_expr_type() == T_FUN_COUNT_SUM;
|
||||
return ret;
|
||||
}
|
||||
inline int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size,
|
||||
const bool is_null, const char *data, const int32_t data_len,
|
||||
int32_t agg_col_idx, char *agg_cell)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_LIKELY(!is_null)) {
|
||||
ObNumber l_num(*reinterpret_cast<const ObCompactNumber *>(data));
|
||||
ObNumber r_num(*reinterpret_cast<const ObCompactNumber *>(agg_cell));
|
||||
char res_buf[ObNumber::MAX_CALC_BYTE_LEN] = {0};
|
||||
ObCompactNumber *res_cnum = reinterpret_cast<ObCompactNumber *>(res_buf);
|
||||
ObNumber::Desc &res_desc = res_cnum->desc_;
|
||||
uint32_t *res_digits = res_cnum->digits_;
|
||||
if (ObNumber::try_fast_add(l_num, r_num, res_digits, res_desc)) {
|
||||
int32_t cp_len = sizeof(ObNumberDesc) + res_desc.len_ * sizeof(uint32_t);
|
||||
MEMCPY(agg_cell, res_buf, cp_len);
|
||||
} else {
|
||||
const ObCompactNumber *l_cnum = reinterpret_cast<const ObCompactNumber *>(data);
|
||||
const ObCompactNumber *r_cnum = reinterpret_cast<const ObCompactNumber *>(agg_cell);
|
||||
ret = add_values(*l_cnum, *r_cnum, agg_cell, ObNumber::MAX_CALC_BYTE_LEN);
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "adder number failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell);
|
||||
not_nulls.set(agg_col_idx);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
inline int add_batch_rows(RuntimeContext &agg_ctx, const int32_t agg_col_id,
|
||||
const sql::ObBitVector &skip, const sql::EvalBound &bound, char *agg_cell,
|
||||
const RowSelector row_sel = RowSelector{}) override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
#ifndef NDEBUG
|
||||
helper::print_input_rows(row_sel, skip, bound, agg_ctx.aggr_infos_.at(agg_col_id), false,
|
||||
agg_ctx.eval_ctx_, this, agg_col_id);
|
||||
#endif // NDEBUG
|
||||
OB_ASSERT(agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.count() == 1);
|
||||
ObExpr *param_expr = agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.at(0);
|
||||
OB_ASSERT(param_expr != NULL);
|
||||
ObIVector *input_vec = param_expr->get_vector(agg_ctx.eval_ctx_);
|
||||
VectorFormat in_fmt = param_expr->get_format(agg_ctx.eval_ctx_);
|
||||
uint32_t sum_digits[number::ObNumber::OB_CALC_BUFFER_SIZE] = {0};
|
||||
char buf_alloc1[number::ObNumber::MAX_CALC_BYTE_LEN] = {0};
|
||||
char buf_alloc2[number::ObNumber::MAX_CALC_BYTE_LEN] = {0};
|
||||
ObDataBuffer allocator1(buf_alloc1, number::ObNumber::MAX_CALC_BYTE_LEN);
|
||||
ObDataBuffer allocator2(buf_alloc2, number::ObNumber::MAX_CALC_BYTE_LEN);
|
||||
number::ObNumber result(*reinterpret_cast<number::ObCompactNumber *>(agg_cell));
|
||||
bool all_skip = true;
|
||||
switch (in_fmt) {
|
||||
case common::VEC_UNIFORM: {
|
||||
ACCUMULATE_NMB(ObUniformFormat<false>);
|
||||
break;
|
||||
}
|
||||
case common::VEC_UNIFORM_CONST: {
|
||||
ACCUMULATE_NMB(ObUniformFormat<true>);
|
||||
break;
|
||||
}
|
||||
case common::VEC_DISCRETE: {
|
||||
ACCUMULATE_NMB(ObDiscreteFormat);
|
||||
break;
|
||||
}
|
||||
case common::VEC_CONTINUOUS: {
|
||||
ACCUMULATE_NMB(ObContinuousFormat);
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_LOG(WARN, "unexpected format for sum aggregate", K(ret), K(*this), K(in_fmt));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
SQL_LOG(WARN, "accumulate number failed", K(ret));
|
||||
} else if (OB_LIKELY(!all_skip)) {
|
||||
NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
not_nulls.set(agg_col_id);
|
||||
number::ObCompactNumber *cnum = reinterpret_cast<number::ObCompactNumber *>(agg_cell);
|
||||
cnum->desc_ = result.d_;
|
||||
MEMCPY(&(cnum->digits_[0]), result.get_digits(), result.d_.len_ * sizeof(uint32_t));
|
||||
}
|
||||
SQL_LOG(DEBUG, "number result", K(result), K(all_skip), K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
inline int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *aggr_cell, void *tmp_res, int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, columns, row_num, agg_col_id, aggr_cell, tmp_res, calc_info);
|
||||
SQL_LOG(DEBUG, "add_row do nothing");
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
inline int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num,
|
||||
const int32_t agg_col_id, char *agg_cell, void *tmp_res,
|
||||
int64_t &calc_info)
|
||||
{
|
||||
UNUSEDx(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info);
|
||||
SQL_LOG(DEBUG, "add_nullable_row do nothing");
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
template <typename ColumnFmt>
|
||||
int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr,
|
||||
const int32_t agg_col_id, const char *agg_cell,
|
||||
const int32_t agg_cell_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t output_idx = agg_ctx.eval_ctx_.get_batch_idx();
|
||||
const NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell);
|
||||
ObIVector *output_vec = agg_expr.get_vector(agg_ctx.eval_ctx_);
|
||||
if (OB_LIKELY(not_nulls.at(agg_col_id))) {
|
||||
static_cast<ColumnFmt *>(output_vec)->set_number(output_idx, *reinterpret_cast<const number::ObCompactNumber *>(agg_cell));
|
||||
} else if (is_ora_count_sum_){
|
||||
number::ObNumber zero;
|
||||
zero.set_zero();
|
||||
static_cast<ColumnFmt *>(output_vec)->set_payload(output_idx, &zero, sizeof(ObNumberDesc));
|
||||
} else {
|
||||
static_cast<ColumnFmt *>(output_vec)->set_null(output_idx);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
TO_STRING_KV("aggregate", "sum_nmb", K_(is_ora_count_sum));
|
||||
private:
|
||||
template<typename ColumnFmt, bool has_null>
|
||||
struct input_wrapper
|
||||
{
|
||||
static const constexpr bool _param_maybe_null = has_null;
|
||||
input_wrapper(): input_vec_(nullptr), d_() {}
|
||||
input_wrapper(ObIVector *input_vec): input_vec_(input_vec), d_() {}
|
||||
OB_INLINE ObDatum *at(const int64_t idx) const
|
||||
{
|
||||
const char *payload = nullptr;
|
||||
d_.reset();
|
||||
if (has_null && static_cast<ColumnFmt *>(input_vec_)->is_null(idx)) {
|
||||
d_.set_null();
|
||||
} else {
|
||||
payload = static_cast<ColumnFmt *>(input_vec_)->get_payload(idx);
|
||||
d_.set_number_shallow(*reinterpret_cast<const number::ObCompactNumber *>(payload));
|
||||
}
|
||||
return &d_;
|
||||
}
|
||||
ObIVector *input_vec_;
|
||||
mutable ObDatum d_;
|
||||
};
|
||||
|
||||
struct rowsel_wrapper
|
||||
{
|
||||
rowsel_wrapper(const RowSelector &row_sel): row_sel_(row_sel) {}
|
||||
OB_INLINE uint16_t begin() const
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
OB_INLINE uint16_t end() const
|
||||
{
|
||||
return row_sel_.size();
|
||||
}
|
||||
|
||||
OB_INLINE uint16_t get_batch_index(const uint16_t &idx) const
|
||||
{
|
||||
return row_sel_.index(idx);
|
||||
}
|
||||
OB_INLINE void next(uint16_t &idx) const
|
||||
{
|
||||
idx += 1;
|
||||
}
|
||||
const RowSelector &row_sel_;
|
||||
};
|
||||
|
||||
struct bound_wrapper
|
||||
{
|
||||
bound_wrapper(const sql::EvalBound &bound, const ObBitVector &skip) : bound_(bound), skip_(skip)
|
||||
{}
|
||||
|
||||
OB_INLINE uint16_t begin() const
|
||||
{
|
||||
uint16_t idx = bound_.start();
|
||||
if (OB_LIKELY(bound_.get_all_rows_active())) {
|
||||
return idx;
|
||||
}
|
||||
while (idx < bound_.end() && skip_.at(idx)) {
|
||||
idx++;
|
||||
}
|
||||
return idx;
|
||||
}
|
||||
OB_INLINE uint16_t end() const
|
||||
{
|
||||
return bound_.end();
|
||||
}
|
||||
|
||||
OB_INLINE uint16_t get_batch_index(const uint16_t &idx) const
|
||||
{
|
||||
return idx;
|
||||
}
|
||||
OB_INLINE void next(uint16_t &idx) const
|
||||
{
|
||||
if (OB_LIKELY(bound_.get_all_rows_active())) {
|
||||
idx += 1;
|
||||
} else {
|
||||
do {
|
||||
idx++;
|
||||
} while(idx < bound_.end() && skip_.at(idx));
|
||||
}
|
||||
}
|
||||
const sql::EvalBound &bound_;
|
||||
const ObBitVector &skip_;
|
||||
};
|
||||
|
||||
private:
|
||||
bool is_ora_count_sum_;
|
||||
};
|
||||
} // end aggregate
|
||||
} // end share
|
||||
} // end oceanbase
|
||||
#undef ACCUMULATE_NMB
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_SUM_NMB_H_
|
||||
@ -1,623 +0,0 @@
|
||||
/**
|
||||
* 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_SHARE_AGGREGATE_UTIL_H_
|
||||
#define OCEANBASE_SHARE_AGGREGATE_UTIL_H_
|
||||
|
||||
#include <type_traits>
|
||||
#include <cmath>
|
||||
|
||||
#include "lib/number/ob_number_v2.h"
|
||||
#include "common/object/ob_obj_type.h"
|
||||
#include "sql/engine/ob_bit_vector.h"
|
||||
#include "share/vector/ob_vector_define.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
class ObEvalCtx;
|
||||
class ObExpr;
|
||||
}
|
||||
|
||||
namespace common
|
||||
{
|
||||
class ObVectorBase;
|
||||
}
|
||||
namespace share
|
||||
{
|
||||
namespace aggregate
|
||||
{
|
||||
using namespace oceanbase::common;
|
||||
|
||||
// ================
|
||||
// format helper
|
||||
template <VecValueTypeClass vec_tc>
|
||||
using fixlen_fmt =
|
||||
typename std::conditional<is_fixed_length_vec(vec_tc), ObFixedLengthFormat<RTCType<vec_tc>>,
|
||||
ObVectorBase>::type;
|
||||
|
||||
template <VecValueTypeClass vec_tc>
|
||||
using discrete_fmt =
|
||||
typename std::conditional<is_discrete_vec(vec_tc), ObDiscreteFormat, ObVectorBase>::type;
|
||||
|
||||
template <VecValueTypeClass vec_tc>
|
||||
using continuous_fmt =
|
||||
typename std::conditional<is_continuous_vec(vec_tc), ObContinuousFormat, ObVectorBase>::type;
|
||||
|
||||
template <VecValueTypeClass vec_tc, bool is_const>
|
||||
using uniform_fmt =
|
||||
typename std::conditional<is_uniform_vec(vec_tc), ObUniformFormat<is_const>, ObVectorBase>::type;
|
||||
|
||||
// ================
|
||||
// member function helpers
|
||||
template<typename T, typename = std::void_t<>>
|
||||
struct defined_collect_tmp_result: std::false_type {};
|
||||
|
||||
template <typename T>
|
||||
struct defined_collect_tmp_result<T, std::void_t<decltype(&T::collect_tmp_result)>>
|
||||
: std::true_type {};
|
||||
|
||||
template <typename T, typename = std::void_t<>>
|
||||
struct defined_add_param_batch: std::false_type {};
|
||||
|
||||
template <typename T>
|
||||
struct defined_add_param_batch<T, std::void_t<decltype(&T::template add_param_batch<ObVectorBase>)>>
|
||||
: std::true_type
|
||||
{};
|
||||
|
||||
template<typename T, bool defined = defined_collect_tmp_result<T>::value>
|
||||
struct collect_tmp_result
|
||||
{
|
||||
template<typename... Args>
|
||||
inline static int do_op(T &v, Args&&... args)
|
||||
{
|
||||
return v.collect_tmp_result(std::forward<Args>(args)...);
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct collect_tmp_result<T, false>
|
||||
{
|
||||
template<typename... Args>
|
||||
inline static int do_op(T &v, Args&&... args)
|
||||
{
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T, bool defined = defined_add_param_batch<T>::value>
|
||||
struct add_param_batch
|
||||
{
|
||||
template<typename... Args>
|
||||
inline static int do_op(T &v, Args&&... args)
|
||||
{
|
||||
return v.add_param_batch(std::forward<Args>(args)...);
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct add_param_batch<T, false>
|
||||
{
|
||||
template<typename... Args>
|
||||
inline static int do_op(T &v, Args&&... args)
|
||||
{
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
};
|
||||
|
||||
// ================
|
||||
// read/writer helper
|
||||
|
||||
template<typename T>
|
||||
struct need_extend_buf
|
||||
{
|
||||
static const constexpr bool value =
|
||||
(std::is_same<T, ObString>::value || std::is_same<T, ObObj>::value);
|
||||
};
|
||||
template<typename T>
|
||||
struct CellWriter
|
||||
{
|
||||
template <typename Vector>
|
||||
inline static void set(const char *data, const int32_t data_len, Vector *vec,
|
||||
const int64_t output_idx, char *res_buf)
|
||||
{
|
||||
OB_ASSERT(vec != NULL);
|
||||
if (need_extend_buf<T>::value) {
|
||||
MEMCPY(res_buf, data, data_len);
|
||||
vec->set_payload_shallow(output_idx, res_buf , data_len);
|
||||
} else {
|
||||
vec->set_payload(output_idx, data, data_len);
|
||||
}
|
||||
}
|
||||
|
||||
template<typename Vector, typename in_type>
|
||||
inline static void cp_and_set(const in_type &in_val, Vector *vec,
|
||||
const int64_t output_idx, char *res_buf)
|
||||
{
|
||||
UNUSEDx(res_buf);
|
||||
OB_ASSERT(vec != NULL);
|
||||
T out_val = in_val;
|
||||
vec->set_payload(output_idx, &out_val, sizeof(T));
|
||||
}
|
||||
};
|
||||
|
||||
// ugly code, for compilation's sake
|
||||
template <>
|
||||
struct CellWriter<char[0]>
|
||||
{
|
||||
template <typename Vector>
|
||||
inline static void set(const char *data, const int32_t data_len, Vector *vec,
|
||||
const int64_t output_idx, char *res_buf)
|
||||
{
|
||||
UNUSEDx(data, data_len, res_buf);
|
||||
OB_ASSERT(vec != NULL);
|
||||
vec->set_null(output_idx);
|
||||
}
|
||||
|
||||
template <typename Vector, typename in_type>
|
||||
inline static void cp_and_set(const in_type &in_val, Vector *vec, const int64_t output_idx,
|
||||
char *res_buf)
|
||||
{
|
||||
UNUSEDx(in_val, res_buf);
|
||||
vec->set_null(output_idx);
|
||||
}
|
||||
};
|
||||
|
||||
// for compact number
|
||||
// using `set_number` to store number value so that cell's length can be set properly
|
||||
// otherwise `hash_v2` value of cell will be inconsistent.
|
||||
template<>
|
||||
struct CellWriter<number::ObCompactNumber>
|
||||
{
|
||||
template <typename Vector>
|
||||
inline static void set(const char *data, const int32_t data_len, Vector *vec,
|
||||
const int64_t output_idx, char *res_buf)
|
||||
{
|
||||
UNUSED(res_buf);
|
||||
OB_ASSERT(vec != NULL);
|
||||
OB_ASSERT(data != NULL);
|
||||
OB_ASSERT(data_len > 0);
|
||||
const number::ObCompactNumber *cnum = reinterpret_cast<const number::ObCompactNumber *>(data);
|
||||
vec->set_number(output_idx, *cnum);
|
||||
}
|
||||
|
||||
template <typename Vector, typename in_type>
|
||||
inline static void cp_and_set(const in_type &in_val, Vector *vec,
|
||||
const int64_t output_idx, char *res_buf)
|
||||
{
|
||||
UNUSEDx(in_val, vec, output_idx, res_buf);
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
// ================
|
||||
// calculation helper
|
||||
struct OverflowChecker
|
||||
{
|
||||
template <typename LType, typename RType,
|
||||
class = typename std::enable_if<std::is_integral<LType>::value
|
||||
&& std::is_integral<RType>::value>::type>
|
||||
static inline bool check_overflow(const LType &l, const RType &r, RType &res)
|
||||
{
|
||||
return __builtin_add_overflow(l, r, &res);
|
||||
}
|
||||
|
||||
static inline bool check_overflow(const float l, const float r, float &res)
|
||||
{
|
||||
res = l + r;
|
||||
return (std::isinf(res) != 0);
|
||||
}
|
||||
|
||||
static inline bool check_overflow(const double l, const double r, double &res)
|
||||
{
|
||||
res = l + r;
|
||||
return (std::isinf(res) != 0);
|
||||
}
|
||||
template <unsigned Bits, typename Signed>
|
||||
static inline bool check_overflow(const int64_t l, const wide::ObWideInteger<Bits, Signed> &r,
|
||||
wide::ObWideInteger<Bits, Signed> &res)
|
||||
{
|
||||
return OB_OPERATE_OVERFLOW == r.template add<wide::CheckOverFlow>(l, res);
|
||||
}
|
||||
|
||||
template <unsigned Bits, typename Signed>
|
||||
static inline bool check_overflow(const uint64_t l, const wide::ObWideInteger<Bits, Signed> &r,
|
||||
wide::ObWideInteger<Bits, Signed> &res)
|
||||
{
|
||||
return OB_OPERATE_OVERFLOW == r.template add<wide::CheckOverFlow>(l, res);
|
||||
}
|
||||
|
||||
template <unsigned Bits, typename Signed, unsigned Bits2, typename Signed2>
|
||||
static inline bool check_overflow(const wide::ObWideInteger<Bits, Signed> &l,
|
||||
const wide::ObWideInteger<Bits2, Signed2> &r,
|
||||
wide::ObWideInteger<Bits2, Signed2> &res)
|
||||
{
|
||||
return OB_OPERATE_OVERFLOW == r.template add<wide::CheckOverFlow>(l, res);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename L, typename R>
|
||||
inline int add_overflow(const L &l, const R &r, char *res_buf, const int32_t res_len)
|
||||
{
|
||||
int ret = OB_NOT_IMPLEMENT;
|
||||
SQL_LOG(WARN, "not implement", K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline int add_overflow<float, float>(const float &l, const float &r, char *res_buf,
|
||||
const int32_t res_len)
|
||||
{
|
||||
float &res = *reinterpret_cast<float *>(res_buf);
|
||||
if (OverflowChecker::check_overflow(l, r, res)) {
|
||||
return OB_OPERATE_OVERFLOW;
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline int add_overflow<double, double>(const double &l, const double &r, char *res_buf,
|
||||
const int32_t res_len)
|
||||
{
|
||||
double &res = *reinterpret_cast<double *>(res_buf);
|
||||
// overflow checking is not needed for double
|
||||
res = l + r;
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
template<>
|
||||
inline int add_overflow<int64_t, int64_t>(const int64_t &l, const int64_t &r, char *res_buf, const int32_t res_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t tmp_r = 0;
|
||||
int64_t &res = *reinterpret_cast<int64_t *>(res_buf);
|
||||
if (OverflowChecker::check_overflow(l, r, tmp_r)) {
|
||||
ret = OB_OPERATE_OVERFLOW;
|
||||
} else {
|
||||
res = tmp_r;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<>
|
||||
inline int add_overflow<uint64_t, uint64_t>(const uint64_t &l, const uint64_t &r, char *res_buf, const int32_t res_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t tmp_r = 0;
|
||||
uint64_t &res = *reinterpret_cast<uint64_t *>(res_buf);
|
||||
if (OverflowChecker::check_overflow(l, r, tmp_r)) {
|
||||
ret = OB_OPERATE_OVERFLOW;
|
||||
} else {
|
||||
res = tmp_r;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<>
|
||||
inline int add_overflow<int32_t, int64_t>(const int32_t &l, const int64_t &r, char *res_buf, const int32_t res_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t tmp_l = l, tmp_res = 0;
|
||||
int64_t &res = *reinterpret_cast<int64_t *>(res_buf);
|
||||
if (OverflowChecker::check_overflow(tmp_l, r, tmp_res)) {
|
||||
ret = OB_OPERATE_OVERFLOW;
|
||||
} else {
|
||||
res = tmp_res;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <unsigned Bits, typename Signed>
|
||||
inline int add_overflow(const wide::ObWideInteger<Bits, Signed> &l,
|
||||
const wide::ObWideInteger<Bits, Signed> &r,
|
||||
char *res_buf,
|
||||
const int32_t res_len)
|
||||
{
|
||||
using integer = wide::ObWideInteger<Bits, Signed>;
|
||||
int ret = OB_SUCCESS;
|
||||
char tmp_res_buf[sizeof(integer)] = {0};
|
||||
integer &tmp_res = *reinterpret_cast<integer *>(tmp_res_buf);
|
||||
ret = l.template add<wide::CheckOverFlow>(r, tmp_res);
|
||||
if (OB_SUCC(ret)) {
|
||||
MEMCPY(res_buf, tmp_res_buf, sizeof(integer));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
template <>
|
||||
inline int
|
||||
add_overflow<number::ObCompactNumber, number::ObCompactNumber>(const number::ObCompactNumber &l,
|
||||
const number::ObCompactNumber &r,
|
||||
char *res_buf, const int32_t res_len)
|
||||
{
|
||||
UNUSEDx(res_len);
|
||||
int ret = OB_SUCCESS;
|
||||
char buf_alloc[number::ObNumber::MAX_CALC_BYTE_LEN] = {0};
|
||||
ObDataBuffer local_allocator(buf_alloc, number::ObNumber::MAX_CALC_BYTE_LEN);
|
||||
number::ObNumber param1(l);
|
||||
number::ObNumber param2(r);
|
||||
number::ObNumber res_nmb;
|
||||
if (OB_FAIL(param1.add_v3(param2, res_nmb, local_allocator))) {
|
||||
SQL_LOG(WARN, "add_v3 failed", K(ret));
|
||||
} else {
|
||||
*reinterpret_cast<uint32_t *>(res_buf) = res_nmb.d_.desc_;
|
||||
MEMCPY(res_buf + sizeof(uint32_t), res_nmb.get_digits(), res_nmb.d_.len_ * sizeof(uint32_t));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<typename L, typename R>
|
||||
inline int add_values(const L &l, const R &r, char *res_buf, const int32_t res_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
R &res = *reinterpret_cast<R *>(res_buf) ;
|
||||
res = r + l;
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline int
|
||||
add_values<number::ObCompactNumber, number::ObCompactNumber>(const number::ObCompactNumber &l,
|
||||
const number::ObCompactNumber &r,
|
||||
char *res_buf, const int32_t res_len)
|
||||
{
|
||||
return add_overflow(l, r, res_buf, res_len);
|
||||
}
|
||||
|
||||
template <unsigned Bits, typename Signed>
|
||||
inline int add_values(const wide::ObWideInteger<Bits, Signed> &, const number::ObCompactNumber &,
|
||||
char *res_buf, const int32_t)
|
||||
{
|
||||
int ret = OB_NOT_SUPPORTED;
|
||||
SQL_LOG(WARN, "can't add", K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline int add_values(const int64_t &l, const number::ObCompactNumber &r, char *res_buf,
|
||||
const int32_t res_len)
|
||||
{
|
||||
int ret = OB_NOT_SUPPORTED;
|
||||
SQL_LOG(WARN, "can't add", K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline int add_values(const uint64_t &l, const number::ObCompactNumber &r, char *res_buf,
|
||||
const int32_t res_len)
|
||||
{
|
||||
int ret = OB_NOT_SUPPORTED;
|
||||
SQL_LOG(WARN, "can't add", K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline int add_values(const int32_t &l, const number::ObCompactNumber &r, char *res_buf,
|
||||
const int32_t res_len)
|
||||
{
|
||||
int ret = OB_NOT_SUPPORTED;
|
||||
SQL_LOG(WARN, "can't add", K(ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
template<typename Input, typename Output>
|
||||
struct Caster
|
||||
{
|
||||
inline static int to_type(const char *src, const int32_t src_len, const ObScale scale,
|
||||
ObIAllocator &alloc, Output *&out_val, int32_t &out_len)
|
||||
{
|
||||
UNUSEDx(src_len, scale);
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(out_val = (Output *)alloc.alloc(sizeof(Output)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
*out_val = *reinterpret_cast<const Input *>(src);
|
||||
out_len = sizeof(Output);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
};
|
||||
|
||||
template<typename Input>
|
||||
struct Caster<Input, char[0]>
|
||||
{
|
||||
using Output = char[0];
|
||||
inline static int to_type(const char *src, const int32_t src_len, const ObScale scale,
|
||||
ObIAllocator &alloc, Output *&out_val, int32_t &out_len)
|
||||
{
|
||||
UNUSEDx(src, src_len, scale, alloc, out_val, out_len);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
};
|
||||
|
||||
template<typename Integer>
|
||||
struct Caster<Integer, number::ObCompactNumber>
|
||||
{
|
||||
static_assert(std::is_integral<Integer>::value, "must be native integer");
|
||||
inline static int to_type(const char *src, const int32_t src_len, const ObScale scale,
|
||||
ObIAllocator &alloc, number::ObCompactNumber *&out, int32_t &out_len)
|
||||
{
|
||||
UNUSEDx(src_len, scale);
|
||||
int ret = OB_SUCCESS;
|
||||
char local_buf[number::ObNumber::MAX_CALC_BYTE_LEN] = {0};
|
||||
ObDataBuffer tmp_alloc(local_buf, number::ObNumber::MAX_CALC_BYTE_LEN);
|
||||
number::ObNumber result_nmb;
|
||||
uint32_t *out_buf = nullptr;
|
||||
const Integer &in_val = *reinterpret_cast<const Integer *>(src);
|
||||
if (std::is_signed<Integer>::value) { // int
|
||||
int64_t tmp_v = in_val;
|
||||
if (OB_FAIL(wide::to_number(in_val, scale, tmp_alloc, result_nmb))) {
|
||||
SQL_LOG(WARN, "to_number failed", K(ret));
|
||||
}
|
||||
} else { // uint
|
||||
uint64_t tmp_v = in_val;
|
||||
if (OB_FAIL(result_nmb.from(tmp_v, tmp_alloc))) {
|
||||
SQL_LOG(WARN, "cast to number failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(out_buf = (uint32_t *)alloc.alloc(
|
||||
sizeof(ObNumberDesc) + result_nmb.d_.len_ * sizeof(uint32_t)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
out = reinterpret_cast<number::ObCompactNumber *>(out_buf);
|
||||
out->desc_ = result_nmb.d_;
|
||||
MEMCPY(out->digits_, result_nmb.get_digits(), result_nmb.d_.len_ * sizeof(uint32_t));
|
||||
out_len = sizeof(ObNumberDesc) + result_nmb.d_.len_ * sizeof(uint32_t);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
};
|
||||
|
||||
template<>
|
||||
struct Caster<number::ObCompactNumber, number::ObCompactNumber>
|
||||
{
|
||||
inline static int to_type(const char *src, const int32_t src_len, const ObScale scale,
|
||||
ObIAllocator &alloc, number::ObCompactNumber *&out, int32_t &out_len)
|
||||
{
|
||||
UNUSEDx(scale, alloc);
|
||||
// shadow copy
|
||||
out_len = src_len;
|
||||
out =
|
||||
const_cast<number::ObCompactNumber *>(reinterpret_cast<const number::ObCompactNumber *>(src));
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
};
|
||||
|
||||
template<typename Integer>
|
||||
struct DecintToNmb
|
||||
{
|
||||
inline static int to_type(const char *src, const int32_t src_len, const ObScale scale,
|
||||
ObIAllocator &alloc, number::ObCompactNumber *&out, int32_t &out_len)
|
||||
{
|
||||
UNUSED(src_len);
|
||||
int ret = OB_SUCCESS;
|
||||
char local_buf[number::ObNumber::MAX_CALC_BYTE_LEN] = {0};
|
||||
ObDataBuffer tmp_alloc(local_buf, number::ObNumber::MAX_CALC_BYTE_LEN);
|
||||
number::ObNumber res_nmb;
|
||||
char *tmp_buf = nullptr;
|
||||
if (OB_FAIL(wide::to_number(*reinterpret_cast<const Integer *>(src), scale, tmp_alloc,
|
||||
res_nmb))) {
|
||||
SQL_LOG(WARN, "to_number failed", K(ret));
|
||||
} else if (OB_ISNULL(tmp_buf = (char *)alloc.alloc(sizeof(ObNumberDesc)
|
||||
+ res_nmb.d_.len_ * sizeof(uint32_t)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
SQL_LOG(WARN, "allocate memory failed", K(ret));
|
||||
} else {
|
||||
out = reinterpret_cast<number::ObCompactNumber *>(tmp_buf);
|
||||
out->desc_ = res_nmb.d_;
|
||||
MEMCPY(out->digits_, res_nmb.get_digits(), res_nmb.d_.len_ * sizeof(uint32_t));
|
||||
out_len = sizeof(ObNumberDesc) + res_nmb.d_.len_ * sizeof(uint32_t);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
};
|
||||
|
||||
template<>
|
||||
struct Caster<int128_t, number::ObCompactNumber>: public DecintToNmb<int128_t> {};
|
||||
|
||||
template<>
|
||||
struct Caster<int256_t, number::ObCompactNumber>: public DecintToNmb<int256_t> {};
|
||||
|
||||
template<>
|
||||
struct Caster<int512_t, number::ObCompactNumber>: public DecintToNmb<int512_t> {};
|
||||
|
||||
// ================
|
||||
// calculation types
|
||||
template <VecValueTypeClass vec_tc>
|
||||
using AggCalcType = typename std::conditional<vec_tc == VEC_TC_NUMBER, number::ObCompactNumber,
|
||||
RTCType<vec_tc>>::type;
|
||||
|
||||
// TODO: use small word such as uint8 for NotNullBitVecotr
|
||||
struct AggBitVector: sql::ObTinyBitVector
|
||||
{
|
||||
inline static int64_t word_bits() { return sql::ObTinyBitVector::WORD_BITS; }
|
||||
inline static int64_t word_size() { return sql::ObTinyBitVector::BYTES_PER_WORD; }
|
||||
};
|
||||
|
||||
using NotNullBitVector = AggBitVector;
|
||||
|
||||
inline bool supported_aggregate_function(const ObItemType agg_op)
|
||||
{
|
||||
switch (agg_op) {
|
||||
case T_FUN_COUNT:
|
||||
case T_FUN_MIN:
|
||||
case T_FUN_MAX:
|
||||
case T_FUN_COUNT_SUM:
|
||||
case T_FUN_SUM: {
|
||||
return true;
|
||||
}
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
#define AGG_FIXED_TC_LIST \
|
||||
VEC_TC_INTEGER, \
|
||||
VEC_TC_UINTEGER, \
|
||||
VEC_TC_FLOAT, \
|
||||
VEC_TC_DOUBLE, \
|
||||
VEC_TC_FIXED_DOUBLE, \
|
||||
VEC_TC_DATETIME, \
|
||||
VEC_TC_DATE, \
|
||||
VEC_TC_TIME, \
|
||||
VEC_TC_YEAR, \
|
||||
VEC_TC_BIT, \
|
||||
VEC_TC_ENUM_SET, \
|
||||
VEC_TC_TIMESTAMP_TZ, \
|
||||
VEC_TC_TIMESTAMP_TINY, \
|
||||
VEC_TC_INTERVAL_YM, \
|
||||
VEC_TC_INTERVAL_DS, \
|
||||
VEC_TC_DEC_INT32, \
|
||||
VEC_TC_DEC_INT64, \
|
||||
VEC_TC_DEC_INT128, \
|
||||
VEC_TC_DEC_INT256, \
|
||||
VEC_TC_DEC_INT512
|
||||
|
||||
#define AGG_VEC_TC_LIST \
|
||||
VEC_TC_NULL, \
|
||||
VEC_TC_INTEGER, \
|
||||
VEC_TC_UINTEGER, \
|
||||
VEC_TC_FLOAT, \
|
||||
VEC_TC_DOUBLE, \
|
||||
VEC_TC_FIXED_DOUBLE, \
|
||||
VEC_TC_NUMBER, \
|
||||
VEC_TC_DATETIME, \
|
||||
VEC_TC_DATE, \
|
||||
VEC_TC_TIME, \
|
||||
VEC_TC_YEAR, \
|
||||
VEC_TC_STRING, \
|
||||
VEC_TC_BIT, \
|
||||
VEC_TC_ENUM_SET, \
|
||||
VEC_TC_ENUM_SET_INNER, \
|
||||
VEC_TC_TIMESTAMP_TZ, \
|
||||
VEC_TC_TIMESTAMP_TINY, \
|
||||
VEC_TC_RAW, \
|
||||
VEC_TC_INTERVAL_YM, \
|
||||
VEC_TC_INTERVAL_DS, \
|
||||
VEC_TC_ROWID, \
|
||||
VEC_TC_LOB, \
|
||||
VEC_TC_JSON, \
|
||||
VEC_TC_GEO, \
|
||||
VEC_TC_UDT, \
|
||||
VEC_TC_DEC_INT32, \
|
||||
VEC_TC_DEC_INT64, \
|
||||
VEC_TC_DEC_INT128, \
|
||||
VEC_TC_DEC_INT256, \
|
||||
VEC_TC_DEC_INT512
|
||||
|
||||
} // end namespace aggregate
|
||||
} // end namespace share
|
||||
} // end namespace oceanbase
|
||||
#endif // OCEANBASE_SHARE_AGGREGATE_UTIL_H_
|
||||
Reference in New Issue
Block a user