[FEAT MERGE] ap perf optimization

Co-authored-by: wangt1xiuyi <13547954130@163.com>
Co-authored-by: XIAO-HOU <372060054@qq.com>
Co-authored-by: z404289981 <z404289981@163.com>
This commit is contained in:
DengzhiLiu 2024-03-26 07:51:42 +00:00 committed by ob-robot
parent cd6b56ce4a
commit 5aa0503125
85 changed files with 3566 additions and 832 deletions

View File

@ -35,6 +35,8 @@ public:
inline int init(ObIAllocator *alloc, int64_t n_bit);
inline void set(uint64_t hash_val);
inline void sets(uint64_t *hash_vals, int64_t count);
inline char *get_buckets() { return buckets_; }
inline int64_t get_bucket_num() { return n_bucket_; }
inline uint64_t estimate();
void reuse()
{
@ -54,6 +56,8 @@ public:
n_count_ = 0;
}
TO_STRING_KV(K_(n_bucket), K_(n_bit), K_(n_count), KP_(alloc), KP_(buckets));
private:
inline int32_t calc_leading_zero(uint64_t hash_value)
{

View File

@ -380,7 +380,7 @@ TEST_F(TestCGGroupByScanner, test_init)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -402,7 +402,7 @@ TEST_F(TestCGGroupByScanner, test_decide_group_size)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -427,7 +427,7 @@ TEST_F(TestCGGroupByScanner, test_decide_can_group_by)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -458,7 +458,7 @@ TEST_F(TestCGGroupByScanner, test_read_distinct)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -499,7 +499,7 @@ TEST_F(TestCGGroupByScanner, test_read_reference)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -549,7 +549,7 @@ TEST_F(TestCGGroupByScanner, test_calc_aggregate_group_by)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -628,7 +628,7 @@ TEST_F(TestCGGroupByScanner, test_calc_aggregate_group_by_with_bitmap)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;

View File

@ -553,7 +553,6 @@ TEST_F(TestCGScanner, test_filter)
filter.datum_params_.push_back(arg_datum);
filter.filter_.expr_->args_[1] = reinterpret_cast<sql::ObExpr *>(expr_buf3) + 1;
filter.filter_.expr_->args_[1]->type_ = T_REF_COLUMN;
ASSERT_EQ(OB_SUCCESS, filter.init_obj_set());
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
exprs.init(1);

View File

@ -381,7 +381,7 @@ TEST_F(TestCSCGGroupByScanner, test_init)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -403,7 +403,7 @@ TEST_F(TestCSCGGroupByScanner, test_decide_group_size)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -428,7 +428,7 @@ TEST_F(TestCSCGGroupByScanner, test_decide_can_group_by)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -459,7 +459,7 @@ TEST_F(TestCSCGGroupByScanner, test_read_distinct)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -500,7 +500,7 @@ TEST_F(TestCSCGGroupByScanner, test_read_reference)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -546,7 +546,7 @@ TEST_F(TestCSCGGroupByScanner, test_calc_aggregate_group_by)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;
@ -630,7 +630,7 @@ TEST_F(TestCSCGGroupByScanner, test_calc_aggregate_group_by_with_bitmap)
prepare_test_case(is_reverse_scan);
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObVectorStore vector_store(eval_ctx_.batch_size_, eval_ctx_, context_);
vector_store.group_by_cell_ = &group_by_cell;

View File

@ -349,7 +349,7 @@ TEST_F(TestPushdownAggregate, test_init_group_by_cell)
access_param_.iter_param_.group_by_cols_project_ = &group_by_cols_project_;
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ObIArray<ObAggCell*> &agg_cell = group_by_cell.get_agg_cells();
ASSERT_EQ(4, group_by_cell.get_agg_cells().count());
@ -423,7 +423,7 @@ TEST_F(TestPushdownAggregate, test_decide_use_group_by1)
access_param_.iter_param_.group_by_cols_project_ = &group_by_cols_project_;
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
int64_t row_count = 100;
int64_t distinct_count = 10;
@ -508,7 +508,7 @@ TEST_F(TestPushdownAggregate, test_decide_use_group_by2)
access_param_.iter_param_.group_by_cols_project_ = &group_by_cols_project_;
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
int64_t row_count = 1000;
int64_t distinct_count = 10;
@ -597,7 +597,7 @@ TEST_F(TestPushdownAggregate, test_eval_batch)
access_param_.iter_param_.group_by_cols_project_ = &group_by_cols_project_;
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
const int64_t distinct_cnt = 2;
@ -703,7 +703,7 @@ TEST_F(TestPushdownAggregate, test_eval_batch_with_null)
access_param_.iter_param_.group_by_cols_project_ = &group_by_cols_project_;
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
const int64_t distinct_cnt = 2;
@ -813,7 +813,7 @@ TEST_F(TestPushdownAggregate, test_copy_output_rows)
access_param_.iter_param_.group_by_cols_project_ = &group_by_cols_project_;
ObGroupByCell group_by_cell(eval_ctx_.batch_size_, allocator_);
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, eval_ctx_));
ASSERT_EQ(OB_SUCCESS, group_by_cell.init(access_param_, context_, eval_ctx_));
ASSERT_EQ(eval_ctx_.batch_size_, group_by_cell.get_batch_size());
ObDatum *col_datums = output_exprs_.at(1)->locate_batch_datums(eval_ctx_);

View File

@ -2591,6 +2591,7 @@ extern const char *get_type_name(int type);
(op) == T_FUN_GROUP_ID || \
(op) == T_FUN_ORA_XMLAGG || \
(op) == T_FUN_SYS_ST_ASMVT || \
(op) == T_FUN_SUM_OPNSIZE ||\
((op) >= T_FUN_SYS_BIT_AND && (op) <= T_FUN_SYS_BIT_XOR))
#define MAYBE_ROW_OP(op) ((op) >= T_OP_EQ && (op) <= T_OP_NE)
#define IS_PSEUDO_COLUMN_TYPE(op) \

View File

@ -6224,20 +6224,21 @@ int ObDbmsStats::init_gather_task_info(ObExecContext &ctx,
return ret;
}
//} else if (OB_FAIL(schema->has_all_column_group(has_all_column_group))) {
// LOG_WARN("failed to check has row store", K(ret));
// } else if (OB_FALSE_IT(has_normal_column_group = schema->is_normal_column_store_table())) {
int ObDbmsStats::init_column_group_stat_param(const share::schema::ObTableSchema &table_schema,
ObIArray<ObColumnGroupStatParam> &column_group_params)
{
int ret = OB_SUCCESS;
ObSEArray<const ObColumnGroupSchema *, 8> column_group_metas;
uint64_t data_version = 0;
bool is_column_store = false;
if (OB_FAIL(GET_MIN_DATA_VERSION(table_schema.get_tenant_id(), data_version))) {
LOG_WARN("fail to get tenant data version", KR(ret));
} else if (data_version < DATA_VERSION_4_3_0_0) {
//do nothing
} else if (OB_FAIL(table_schema.get_is_column_store(is_column_store))) {
LOG_WARN("failed to get is column store", K(ret));
} else if (!is_column_store) {
//do nothing
} else if (OB_FAIL(table_schema.get_store_column_groups(column_group_metas))) { // get cg metas without empty default cg
LOG_WARN("failed to get column group metas", K(ret));
} else {

View File

@ -68,7 +68,8 @@ int ObBasicStatsEstimator::estimate(const ObOptStatGatherParam &param,
column_params.count(),
src_col_stats))) {
LOG_WARN("failed init col stats", K(ret));
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_))) {
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_,
param.use_column_store_ && dst_opt_stats.count() == 1))) {
LOG_WARN("failed to fill hints", K(ret));
} else if (OB_FAIL(add_from_table(param.db_name_, param.tab_name_))) {
LOG_WARN("failed to add from table", K(ret));
@ -152,7 +153,9 @@ int ObBasicStatsEstimator::estimate(const ObOptStatGatherParam &param,
int ObBasicStatsEstimator::estimate_block_count(ObExecContext &ctx,
const ObTableStatParam &param,
PartitionIdBlockMap &id_block_map)
PartitionIdBlockMap &id_block_map,
bool &use_column_store,
bool &use_split_part)
{
int ret = OB_SUCCESS;
ObGlobalTableStat global_tab_stat;
@ -164,6 +167,8 @@ int ObBasicStatsEstimator::estimate_block_count(ObExecContext &ctx,
ObArray<uint64_t> column_group_ids;
uint64_t table_id = share::is_oracle_mapping_real_virtual_table(param.table_id_) ?
share::get_real_table_mappings_tid(param.table_id_) : param.table_id_;
use_column_store = false;
use_split_part = false;
if (is_virtual_table(table_id)) {//virtual table no need estimate block count
//do nothing
} else if (OB_FAIL(get_all_tablet_id_and_object_id(param, tablet_ids, partition_ids))) {
@ -180,6 +185,8 @@ int ObBasicStatsEstimator::estimate_block_count(ObExecContext &ctx,
partition_ids, column_group_ids, estimate_result))) {
LOG_WARN("failed to do estimate block count", K(ret));
} else {
int64_t total_sstable_row_cnt = 0;
int64_t total_memtable_row_cnt = 0;
for (int64_t i = 0; OB_SUCC(ret) && i < estimate_result.count(); ++i) {
BlockNumStat *block_num_stat = NULL;
void *buf = NULL;
@ -193,6 +200,8 @@ int ObBasicStatsEstimator::estimate_block_count(ObExecContext &ctx,
block_num_stat = new (buf) BlockNumStat();
block_num_stat->tab_macro_cnt_ = estimate_result.at(i).macro_block_count_;
block_num_stat->tab_micro_cnt_ = estimate_result.at(i).micro_block_count_;
total_sstable_row_cnt += estimate_result.at(i).sstable_row_count_;
total_memtable_row_cnt += estimate_result.at(i).memtable_row_count_;
int64_t partition_id = static_cast<int64_t>(estimate_result.at(i).part_id_);
if (OB_FAIL(block_num_stat->cg_macro_cnt_arr_.assign(estimate_result.at(i).cg_macro_cnt_arr_)) ||
OB_FAIL(block_num_stat->cg_micro_cnt_arr_.assign(estimate_result.at(i).cg_micro_cnt_arr_))) {
@ -240,8 +249,16 @@ int ObBasicStatsEstimator::estimate_block_count(ObExecContext &ctx,
}
}
if (OB_SUCC(ret)) {
if (param.part_level_ == share::schema::PARTITION_LEVEL_ONE ||
param.part_level_ == share::schema::PARTITION_LEVEL_TWO) {
if (OB_FAIL(check_can_use_column_store_and_split_part_gather(total_sstable_row_cnt,
total_memtable_row_cnt,
column_group_ids.count(),
partition_ids.count(),
param.degree_,
use_column_store,
use_split_part))) {
LOG_WARN("failed to check can use column table and split part gather", K(ret));
} else if (param.part_level_ == share::schema::PARTITION_LEVEL_ONE ||
param.part_level_ == share::schema::PARTITION_LEVEL_TWO) {
BlockNumStat *block_num_stat = NULL;
void *buf = NULL;
if (OB_ISNULL(param.allocator_)) {
@ -1302,34 +1319,31 @@ int ObBasicStatsEstimator::check_stat_need_re_estimate(const ObOptStatGatherPara
int ObBasicStatsEstimator::fill_hints(common::ObIAllocator &alloc,
const ObString &table_name,
int64_t gather_vectorize)
int64_t gather_vectorize,
bool use_column_store)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(table_name.empty())) {
ObSqlString default_hints;
const char *use_col_tab_hint = lib::is_oracle_mode() ? " USE_COLUMN_TABLE(\"%.*s\")" : " USE_COLUMN_TABLE(`%.*s`) ";
const char *use_full_table_hint = lib::is_oracle_mode() ? " FULL(\"%.*s\") " : " FULL(`%.*s`) ";
if (OB_UNLIKELY(table_name.empty() || gather_vectorize < 0)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(table_name));
LOG_WARN("get unexpected null", K(table_name), K(gather_vectorize));
} else if (OB_FAIL(default_hints.append_fmt("NO_REWRITE USE_PLAN_CACHE(NONE) DBMS_STATS OPT_PARAM('ROWSETS_MAX_ROWS', %ld)",
gather_vectorize))) {
LOG_WARN("failed to append fmt", K(ret));
} else if (OB_FAIL(default_hints.append_fmt(use_full_table_hint,
table_name.length(),
table_name.ptr()))) {
LOG_WARN("failed to append fmt", K(ret));
} else if (use_column_store && OB_FAIL(default_hints.append_fmt(use_col_tab_hint,
table_name.length(),
table_name.ptr()))) {
LOG_WARN("failed to append fmt", K(ret));
} else if (OB_FAIL(add_hint(default_hints.string(), alloc))) {
LOG_WARN("failed to add hint", K(ret));
} else {
const char *fmt_str = "NO_REWRITE USE_PLAN_CACHE(NONE) DBMS_STATS FULL(%.*s) OPT_PARAM('ROWSETS_MAX_ROWS', %ld)";
int64_t buf_len = table_name.length() + strlen(fmt_str);
char *buf = NULL;
if (OB_ISNULL(buf = static_cast<char *>(alloc.alloc(buf_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to alloc memory", K(buf), K(buf_len));
} else {
int64_t real_len = sprintf(buf, fmt_str, table_name.length(), table_name.ptr(), gather_vectorize);
if (OB_UNLIKELY(real_len < 0 || real_len > buf_len)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected error", K(real_len));
} else {
ObString hint_str;
hint_str.assign_ptr(buf, real_len);
if (OB_FAIL(add_hint(hint_str, alloc))) {
LOG_WARN("failed to add hint", K(ret));
} else {
LOG_TRACE("succeed to fill index info", K(hint_str));
}
}
}
LOG_TRACE("succeed to fill index info", K(default_hints));
}
return ret;
}
@ -1346,5 +1360,36 @@ int ObBasicStatsEstimator::generate_column_group_ids(const ObTableStatParam &par
return ret;
}
int ObBasicStatsEstimator::check_can_use_column_store_and_split_part_gather(const int64_t sstable_row_cnt,
const int64_t memtable_row_cnt,
const int64_t cg_cnt,
const int64_t part_cnt,
const int64_t degree,
bool &use_column_store,
bool &use_split_part)
{
int ret = OB_SUCCESS;
use_split_part = false;
use_column_store = false;
int64_t total_rowcnt = sstable_row_cnt + memtable_row_cnt;
const int64_t SPLIT_PART_MINIMUM_ROW_COUNT = 1000000;
const int64_t COST_SCHEDULER_GATHER_PER_PART = 10000;//10ms
const int64_t COST_GATHER_MINIMUM_ROW_COUNT_PER_COLUMN = 100000;//100ms
const double RATIO_OF_IMPROVEMENT = 0.3;
if (cg_cnt > 0 && sstable_row_cnt > memtable_row_cnt * 10) {
use_column_store = true;
if (use_column_store && part_cnt > 1 && degree > 0 && total_rowcnt >= SPLIT_PART_MINIMUM_ROW_COUNT) {
double cost_scheduler_split_part = 1.0 * COST_SCHEDULER_GATHER_PER_PART * part_cnt;
double cost_of_no_split_gather_part = 1.0 * total_rowcnt / SPLIT_PART_MINIMUM_ROW_COUNT * (cg_cnt / 2 + 1) * COST_GATHER_MINIMUM_ROW_COUNT_PER_COLUMN / degree;
double cost_of_improvement = cost_of_no_split_gather_part * RATIO_OF_IMPROVEMENT;
use_split_part = cost_of_improvement > cost_scheduler_split_part;
}
}
LOG_TRACE("check_can_use_column_store_and_split_part_gather", K(use_split_part), K(use_column_store),
K(cg_cnt), K(part_cnt), K(degree),
K(sstable_row_cnt), K(memtable_row_cnt));
return ret;
}
} // end of common
} // end of oceanbase

View File

@ -56,7 +56,9 @@ public:
static int estimate_block_count(ObExecContext &ctx,
const ObTableStatParam &param,
PartitionIdBlockMap &id_block_map);
PartitionIdBlockMap &id_block_map,
bool &use_column_store,
bool &use_split_part);
static int estimate_modified_count(ObExecContext &ctx,
const uint64_t tenant_id,
@ -148,7 +150,10 @@ public:
template <class T>
int add_stat_item(const T &item);
int fill_hints(common::ObIAllocator &alloc, const ObString &table_name, int64_t gather_vectorize);
int fill_hints(common::ObIAllocator &alloc,
const ObString &table_name,
int64_t gather_vectorize,
bool use_column_store);
private:
@ -167,6 +172,14 @@ private:
static int generate_column_group_ids(const ObTableStatParam &param,
ObIArray<uint64_t> &column_group_ids);
static int check_can_use_column_store_and_split_part_gather(const int64_t sstable_row_cnt,
const int64_t memtable_row_cnt,
const int64_t cg_cnt,
const int64_t part_cnt,
const int64_t degree,
bool &use_column_store,
bool &use_split_part);
};
}

View File

@ -170,7 +170,9 @@ int ObDbmsStatsExecutor::no_split_gather_stats(ObExecContext &ctx,
} else if (OB_FAIL(gather_helper.running_monitor_.add_monitor_info(ObOptStatRunningPhase::GATHER_SUBPART_STATS))) {
LOG_WARN("failed to add add monitor info", K(ret));
} else if (OB_FAIL(ObDbmsStatsUtils::prepare_gather_stat_param(param, SUBPARTITION_LEVEL, partition_id_block_map,
false, gather_helper.gather_vectorize_, gather_param))) {
false, gather_helper.gather_vectorize_,
gather_helper.use_column_store_,
gather_param))) {
LOG_WARN("failed to prepare gather stat param", K(ret));
} else if (OB_FAIL(do_gather_stats(ctx, trans, gather_param,
param.subpart_infos_,
@ -195,7 +197,9 @@ int ObDbmsStatsExecutor::no_split_gather_stats(ObExecContext &ctx,
if (OB_FAIL(gather_helper.running_monitor_.add_monitor_info(ObOptStatRunningPhase::GATHER_PART_STATS))) {
LOG_WARN("failed to add add monitor info", K(ret));
} else if (OB_FAIL(ObDbmsStatsUtils::prepare_gather_stat_param(param, PARTITION_LEVEL, partition_id_block_map,
false, gather_helper.gather_vectorize_, gather_param))) {
false, gather_helper.gather_vectorize_,
gather_helper.use_column_store_,
gather_param))) {
LOG_WARN("failed to prepare gather stat param", K(ret));
} else if (OB_FAIL(do_gather_stats(ctx, trans, gather_param,
param.part_infos_,
@ -245,7 +249,9 @@ int ObDbmsStatsExecutor::no_split_gather_stats(ObExecContext &ctx,
} else if (OB_FAIL(gather_helper.running_monitor_.add_monitor_info(ObOptStatRunningPhase::GATHER_GLOBAL_STATS))) {
LOG_WARN("failed to add add monitor info", K(ret));
} else if (OB_FAIL(ObDbmsStatsUtils::prepare_gather_stat_param(param, TABLE_LEVEL, partition_id_block_map,
false, gather_helper.gather_vectorize_, gather_param))) {
false, gather_helper.gather_vectorize_,
gather_helper.use_column_store_,
gather_param))) {
LOG_WARN("failed to prepare gather stat param", K(ret));
} else if (OB_FAIL(do_gather_stats(ctx, trans, gather_param,
dummy_part_infos,
@ -284,7 +290,9 @@ int ObDbmsStatsExecutor::prepare_gather_stats(ObExecContext &ctx,
} else if (param.need_estimate_block_ &&
share::schema::ObTableType::EXTERNAL_TABLE != param.ref_table_type_ &&
OB_FAIL(ObBasicStatsEstimator::estimate_block_count(ctx, param,
partition_id_block_map))) {
partition_id_block_map,
gather_helper.use_column_store_,
gather_helper.use_split_part_))) {
LOG_WARN("failed to estimate block count", K(ret));
} else if (OB_FAIL(check_need_split_gather(param, gather_helper))) {
LOG_WARN("failed to check need split gather", K(ret));
@ -313,7 +321,9 @@ int ObDbmsStatsExecutor::split_gather_partition_stats(ObExecContext &ctx,
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected error", K(ret), K(param), K(stat_level), K(gather_helper));
} else if (OB_FAIL(ObDbmsStatsUtils::prepare_gather_stat_param(param, stat_level, partition_id_block_map, true,
gather_helper.gather_vectorize_, gather_param))) {
gather_helper.gather_vectorize_,
gather_helper.use_column_store_,
gather_param))) {
LOG_WARN("failed to prepare gather stat param", K(ret));
} else {//need split gather
int64_t idx_part = 0;
@ -526,7 +536,9 @@ int ObDbmsStatsExecutor::split_gather_global_stats(ObExecContext &ctx,
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected error", K(ret), K(param), K(gather_helper));
} else if (OB_FAIL(ObDbmsStatsUtils::prepare_gather_stat_param(param, TABLE_LEVEL, partition_id_block_map, true,
gather_helper.gather_vectorize_, gather_param))) {
gather_helper.gather_vectorize_,
gather_helper.use_column_store_,
gather_param))) {
LOG_WARN("failed to prepare gather stat param", K(ret));
} else {//need split gather
int64_t idx_col = 0;
@ -588,7 +600,7 @@ int ObDbmsStatsExecutor::split_gather_global_stats(ObExecContext &ctx,
all_tstats,
all_cstats))) {
if (gather_param.sepcify_scn_ > 0 &&
(ret = OB_TABLE_DEFINITION_CHANGED || OB_SNAPSHOT_DISCARDED == ret)) {
(ret == OB_TABLE_DEFINITION_CHANGED || OB_SNAPSHOT_DISCARDED == ret)) {
LOG_WARN("failed to specify snapshot to gather stats, try no specify snapshot to gather stats", K(ret));
gather_param.sepcify_scn_ = 0;
allocator.reuse();
@ -671,6 +683,7 @@ int ObDbmsStatsExecutor::check_need_split_gather(const ObTableStatParam &param,
bool need_histgoram = param.subpart_stat_param_.need_modify_ ? param.subpart_stat_param_.gather_histogram_ :
(param.part_stat_param_.need_modify_ ? param.part_stat_param_.gather_histogram_ : param.global_stat_param_.gather_histogram_);
partition_cnt = partition_cnt == 0 ? 1 : partition_cnt;
int64_t origin_partition_cnt = partition_cnt;
int64_t gather_vectorize = DEFAULT_STAT_GATHER_VECTOR_BATCH_SIZE;
//cache table stat size
int64_t tab_stat_size = sizeof(ObOptTableStat) * partition_cnt;
@ -689,13 +702,20 @@ int ObDbmsStatsExecutor::check_need_split_gather(const ObTableStatParam &param,
} else if (OB_UNLIKELY(max_wa_memory_size < MIN_GATHER_WORK_ARANA_SIZE)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected error", K(ret), K(max_wa_memory_size), K(MIN_GATHER_WORK_ARANA_SIZE));
} else if (OB_LIKELY(max_memory_used <= max_wa_memory_size)) {
} else if (max_memory_used <= max_wa_memory_size && (!gather_helper.use_split_part_ || partition_cnt<=1)) {
gather_helper.maximum_gather_col_cnt_ = column_cnt;
gather_helper.maximum_gather_part_cnt_ = partition_cnt;
gather_helper.is_split_gather_ = false;
gather_helper.gather_vectorize_ = gather_vectorize;
} else {
//firstly, split according the partition
//firstly, split according the partition
if (gather_helper.use_split_part_) {
partition_cnt = 1;
tab_stat_size = sizeof(ObOptTableStat) * partition_cnt;
col_stat_size = (sizeof(ObOptColumnStat) + ObOptColumnStat::NUM_LLC_BUCKET) * partition_cnt * column_cnt +
col_histogram_size * partition_cnt;
max_memory_used = tab_stat_size + col_stat_size + calc_stat_size;
}
while (partition_cnt > 1 && max_memory_used > max_wa_memory_size) {
partition_cnt = partition_cnt / 2;
tab_stat_size = sizeof(ObOptTableStat) * partition_cnt;
@ -706,7 +726,7 @@ int ObDbmsStatsExecutor::check_need_split_gather(const ObTableStatParam &param,
if (max_memory_used <= max_wa_memory_size) {
gather_helper.maximum_gather_col_cnt_ = column_cnt;
gather_helper.maximum_gather_part_cnt_ = partition_cnt;
gather_helper.is_split_gather_ = true;
gather_helper.is_split_gather_ = origin_partition_cnt != partition_cnt;
gather_helper.gather_vectorize_ = gather_vectorize;
} else {
const int64_t MINIMUM_OF_VECTOR_SIZE = 8;
@ -719,7 +739,7 @@ int ObDbmsStatsExecutor::check_need_split_gather(const ObTableStatParam &param,
if (max_memory_used <= max_wa_memory_size) {
gather_helper.maximum_gather_col_cnt_ = column_cnt;
gather_helper.maximum_gather_part_cnt_ = partition_cnt;
gather_helper.is_split_gather_ = true;
gather_helper.is_split_gather_ = origin_partition_cnt != partition_cnt;
gather_helper.gather_vectorize_ = gather_vectorize;
} else {
//lastly, split according the column
@ -741,7 +761,7 @@ int ObDbmsStatsExecutor::check_need_split_gather(const ObTableStatParam &param,
K(max_wa_memory_size), K(tab_stat_size), K(col_histogram_size),
K(col_stat_size), K(calc_stat_size), K(gather_helper));
if (gather_helper.is_split_gather_) {
LOG_INFO("stat gather will use split gather", K(param.degree_), K(max_memory_used),
LOG_TRACE("stat gather will use split gather", K(param.degree_), K(max_memory_used),
K(max_wa_memory_size), K(tab_stat_size),
K(col_histogram_size), K(col_stat_size),
K(calc_stat_size), K(gather_helper));
@ -1194,6 +1214,8 @@ int ObDbmsStatsExecutor::gather_index_stats(ObExecContext &ctx,
ObArray<ObOptColumnStat *> empty_cstats;
ObOptStatGatherParam gather_param;
PartitionIdBlockMap partition_id_block_map;
bool use_column_store = false;
bool use_split_part = false;
LOG_TRACE("begin gather index stats", K(param));
if (OB_FAIL(partition_id_block_map.create(10000,
ObModIds::OB_HASH_BUCKET_TABLE_STATISTICS,
@ -1202,10 +1224,13 @@ int ObDbmsStatsExecutor::gather_index_stats(ObExecContext &ctx,
LOG_WARN("failed to create hash map", K(ret));
} else if (param.need_estimate_block_ &&
OB_FAIL(ObBasicStatsEstimator::estimate_block_count(ctx, param,
partition_id_block_map))) {
partition_id_block_map,
use_column_store,
use_split_part))) {
LOG_WARN("failed to estimate block count", K(ret));
} else if (OB_FAIL(ObDbmsStatsUtils::prepare_gather_stat_param(param, INVALID_LEVEL, &partition_id_block_map,
false, DEFAULT_STAT_GATHER_VECTOR_BATCH_SIZE, gather_param))) {
false, DEFAULT_STAT_GATHER_VECTOR_BATCH_SIZE,
use_column_store, gather_param))) {
LOG_WARN("failed to prepare gather stat param", K(ret));
} else if (OB_FAIL(gather_param.column_params_.assign(param.column_params_))) {
LOG_WARN("failed to assign", K(ret));

View File

@ -34,7 +34,9 @@ struct GatherHelper
maximum_gather_col_cnt_(1),
is_approx_gather_(false),
gather_vectorize_(DEFAULT_STAT_GATHER_VECTOR_BATCH_SIZE),
running_monitor_(running_monitor)
running_monitor_(running_monitor),
use_column_store_(false),
use_split_part_(false)
{}
bool is_split_gather_;
int64_t maximum_gather_part_cnt_;
@ -42,12 +44,16 @@ struct GatherHelper
bool is_approx_gather_;
int64_t gather_vectorize_;
ObOptStatRunningMonitor &running_monitor_;
bool use_column_store_;
bool use_split_part_;
TO_STRING_KV(K(is_split_gather_),
K(maximum_gather_part_cnt_),
K(maximum_gather_col_cnt_),
K(is_approx_gather_),
K(gather_vectorize_),
K(running_monitor_));
K(running_monitor_),
K(use_column_store_),
K(use_split_part_));
};
class ObDbmsStatsExecutor

View File

@ -1061,6 +1061,7 @@ int ObDbmsStatsUtils::prepare_gather_stat_param(const ObTableStatParam &param,
const PartitionIdBlockMap *partition_id_block_map,
bool is_split_gather,
int64_t gather_vectorize,
bool use_column_store,
ObOptStatGatherParam &gather_param)
{
int ret = OB_SUCCESS;
@ -1091,6 +1092,7 @@ int ObDbmsStatsUtils::prepare_gather_stat_param(const ObTableStatParam &param,
gather_param.data_table_name_ = param.data_table_name_;
gather_param.global_part_id_ = param.global_part_id_;
gather_param.gather_vectorize_ = gather_vectorize;
gather_param.use_column_store_ = use_column_store;
return gather_param.column_group_params_.assign(param.column_group_params_);
}

View File

@ -172,6 +172,7 @@ public:
const PartitionIdBlockMap *partition_id_block_map,
bool is_split_gather,
int64_t gather_vectorize,
bool use_column_store,
ObOptStatGatherParam &gather_param);
static int merge_split_gather_tab_stats(ObIArray<ObOptTableStat *> &all_tstats,

View File

@ -70,7 +70,7 @@ int ObHybridHistEstimator::estimate(const ObOptStatGatherParam &param,
est_percent,
no_sample_idx))) {
LOG_WARN("failed to add hybrid hist stat items", K(ret));
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_))) {
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_, false))) {
LOG_WARN("failed to fill hints", K(ret));
} else if (OB_FAIL(add_from_table(param.db_name_, param.tab_name_))) {
LOG_WARN("failed to add from table", K(ret));
@ -206,7 +206,7 @@ int ObHybridHistEstimator::estimate_no_sample_col_hydrid_hist(ObIAllocator &allo
hybrid_col_stats,
no_sample_idx))) {
LOG_WARN("failed to add no sample hybrid hist stat items", K(ret));
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_))) {
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_, false))) {
LOG_WARN("failed to fill hints", K(ret));
} else if (OB_FAIL(ObDbmsStatsUtils::get_valid_duration_time(param.gather_start_time_,
param.max_duration_time_,

View File

@ -703,7 +703,7 @@ int ObIncrementalStatEstimator::derive_global_col_stat(ObExecContext &ctx,
ObSEArray<ObOptStat, 1> tmp_opt_stats;
if (OB_FAIL(ObDbmsStatsUtils::prepare_gather_stat_param(param, approx_level, NULL, false,
DEFAULT_STAT_GATHER_VECTOR_BATCH_SIZE,
gather_param))) {
false, gather_param))) {
LOG_WARN("failed to assign", K(ret));
} else if (OB_FAIL(gather_param.column_params_.assign(param.column_params_))) {
LOG_WARN("failed to assign", K(ret));

View File

@ -242,6 +242,8 @@ int ObIndexStatsEstimator::fast_gather_index_stats(ObExecContext &ctx,
ObSEArray<ObOptTableStat *, 4> index_table_stats;
ObArenaAllocator allocator(ObModIds::OB_SQL_PARSER);
PartitionIdBlockMap partition_id_block_map;
bool use_column_store = false;
bool use_split_part = false;
is_fast_gather = false;
LOG_TRACE("begin to fast gather index stats", K(data_param), K(index_param));
if (OB_FAIL(get_all_need_gather_partition_ids(data_param, index_param, gather_part_ids))) {
@ -260,7 +262,10 @@ int ObIndexStatsEstimator::fast_gather_index_stats(ObExecContext &ctx,
index_param.tenant_id_))) {
LOG_WARN("failed to create hash map", K(ret));
} else if (index_param.need_estimate_block_ &&
OB_FAIL(ObBasicStatsEstimator::estimate_block_count(ctx, index_param, partition_id_block_map))) {
OB_FAIL(ObBasicStatsEstimator::estimate_block_count(ctx, index_param,
partition_id_block_map,
use_column_store,
use_split_part))) {
LOG_WARN("failed to estimate block count", K(ret));
} else {
bool is_continued = true;

View File

@ -598,7 +598,8 @@ struct ObOptStatGatherParam {
data_table_name_(),
global_part_id_(-1),
gather_vectorize_(DEFAULT_STAT_GATHER_VECTOR_BATCH_SIZE),
sepcify_scn_(0)
sepcify_scn_(0),
use_column_store_(false)
{}
int assign(const ObOptStatGatherParam &other);
uint64_t tenant_id_;
@ -623,6 +624,7 @@ struct ObOptStatGatherParam {
int64_t global_part_id_;
int64_t gather_vectorize_;
uint64_t sepcify_scn_;
bool use_column_store_;
TO_STRING_KV(K(tenant_id_),
K(db_name_),
@ -643,7 +645,8 @@ struct ObOptStatGatherParam {
K(data_table_name_),
K(global_part_id_),
K(gather_vectorize_),
K(sepcify_scn_));
K(sepcify_scn_),
K(use_column_store_));
};
struct ObOptStat

View File

@ -28,7 +28,7 @@ namespace common
int ObStatRowCount::gen_expr(char *buf, const int64_t buf_len, int64_t &pos)
{
int ret = OB_SUCCESS;
if (OB_FAIL(databuff_printf( buf, buf_len, pos, " COUNT(1)"))) {
if (OB_FAIL(databuff_printf( buf, buf_len, pos, " COUNT(*)"))) {
LOG_WARN("failed to print buf row count expr", K(ret));
}
return ret;

View File

@ -199,8 +199,7 @@ public:
virtual bool is_needed() const { return col_param_ != NULL && col_param_->need_avg_len(); }
const char *get_fmt() const
{
return lib::is_oracle_mode() ? " AVG(SYS_OP_OPNSIZE(\"%.*s\"))"
: " AVG(SYS_OP_OPNSIZE(`%.*s`))";
return lib::is_oracle_mode() ? " SUM_OPNSIZE(\"%.*s\")/decode(COUNT(*),0,1,COUNT(*))" : " SUM_OPNSIZE(`%.*s`)/(case when COUNT(*) = 0 then 1 else COUNT(*) end)";
}
virtual int decode(ObObj &obj) override;
};

View File

@ -645,7 +645,7 @@ int ObTopkHistEstimator::estimate(const ObOptStatGatherParam &param,
LOG_WARN("failed to add topk hist stat items", K(ret));
} else if (get_item_size() <= 0) {
//no need topk histogram item.
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_))) {
} else if (OB_FAIL(fill_hints(allocator, param.tab_name_, param.gather_vectorize_, false))) {
LOG_WARN("failed to fill hints", K(ret));
} else if (OB_FAIL(add_from_table(param.db_name_, param.tab_name_))) {
LOG_WARN("failed to add from table", K(ret));

View File

@ -31,6 +31,7 @@
#include "sql/engine/basic/ob_material_op_impl.h"
#include "share/stat/ob_hybrid_hist_estimator.h"
#include "share/stat/ob_dbms_stats_utils.h"
#include "sql/engine/expr/ob_expr_sys_op_opnsize.h"
#ifdef OB_BUILD_ORACLE_XML
#include "lib/xml/ob_xml_util.h"
#include "lib/xml/ob_xml_tree.h"
@ -1792,7 +1793,8 @@ int ObAggregateProcessor::collect_for_empty_set()
case T_FUN_COUNT_SUM:
case T_FUN_APPROX_COUNT_DISTINCT:
case T_FUN_KEEP_COUNT:
case T_FUN_GROUP_PERCENT_RANK: {
case T_FUN_GROUP_PERCENT_RANK:
case T_FUN_SUM_OPNSIZE: {
ObDatum &result = aggr_info.expr_->locate_datum_for_write(eval_ctx_);
aggr_info.expr_->set_evaluated_projected(eval_ctx_);
if (lib::is_mysql_mode()) {
@ -2744,6 +2746,12 @@ int ObAggregateProcessor::rollup_aggregation(AggrCell &aggr_cell, AggrCell &roll
LOG_USER_ERROR(OB_NOT_SUPPORTED, "rollup contain bit_xor");
break;
}
case T_FUN_SUM_OPNSIZE: {
ret = OB_NOT_SUPPORTED;
LOG_WARN("rollup contain sum_opnsize still not supported", K(ret));
LOG_USER_ERROR(OB_NOT_SUPPORTED, "rollup contain sum_opnsize");
break;
}
default:
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unknown aggr function type", K(aggr_fun));
@ -3054,6 +3062,23 @@ int ObAggregateProcessor::prepare_aggr_result(const ObChunkDatumStore::StoredRow
aggr_cell.set_tiny_num_used();
break;
}
case T_FUN_SUM_OPNSIZE: {
int64_t size = 0;
if (OB_UNLIKELY(stored_row.cnt_ != 1) ||
OB_ISNULL(param_exprs) ||
OB_UNLIKELY(param_exprs->count() != 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("curr_row_results count is not 1", K(stored_row), K(ret));
} else if (OB_FAIL(ObExprSysOpOpnsize::calc_sys_op_opnsize(param_exprs->at(0),
&stored_row.cells()[0],
size))) {
LOG_WARN("failed to calc sys op opnsize", K(ret));
} else {
aggr_cell.set_tiny_num_int(size);
aggr_cell.set_tiny_num_used();
}
break;
}
default:
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unknown aggr function type", K(aggr_fun), K(ret));
@ -3271,6 +3296,30 @@ int ObAggregateProcessor::process_aggr_batch_result(
ret = bitwise_calc_batch(aggr_input_datums, aggr_cell, aggr_fun, selector);
break;
}
case T_FUN_SUM_OPNSIZE: {
if (OB_ISNULL(param_exprs) ||
OB_UNLIKELY(param_exprs->count() != 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("curr_row_results count is not 1", K(ret), KPC(param_exprs));
} else {
ObDatumVector aggr_input_datums = param_exprs->at(0)->locate_expr_datumvector(eval_ctx_);
for (uint16_t it = selector.begin(); OB_SUCC(ret) && it < selector.end(); selector.next(it)) {
uint64_t nth_row = selector.get_batch_index(it);
int64_t size = 0;
if (OB_FAIL(ObExprSysOpOpnsize::calc_sys_op_opnsize(param_exprs->at(0),
aggr_input_datums.at(nth_row),
size))) {
LOG_WARN("failed to calc sys op opnsize", K(ret));
} else {
int64_t origin_size = aggr_cell.get_tiny_num_int();
int64_t new_size = origin_size + size;
aggr_cell.set_tiny_num_int(new_size);
aggr_cell.set_tiny_num_used();
}
}
}
break;
}
default:
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unknown aggr function type", K(aggr_fun), K(ret));
@ -3531,6 +3580,25 @@ int ObAggregateProcessor::process_aggr_result(const ObChunkDatumStore::StoredRow
}
break;
}
case T_FUN_SUM_OPNSIZE: {
int64_t size = 0;
if (OB_UNLIKELY(stored_row.cnt_ != 1) ||
OB_ISNULL(param_exprs) ||
OB_UNLIKELY(param_exprs->count() != 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("curr_row_results count is not 1", K(stored_row), K(ret));
} else if (OB_FAIL(ObExprSysOpOpnsize::calc_sys_op_opnsize(param_exprs->at(0),
&stored_row.cells()[0],
size))) {
LOG_WARN("failed to calc sys op opnsize", K(ret));
} else {
int64_t origin_size = aggr_cell.get_tiny_num_int();
int64_t new_size = origin_size + size;
aggr_cell.set_tiny_num_int(new_size);
aggr_cell.set_tiny_num_used();
}
break;
}
default:
LOG_WARN("unknown aggr function type", K(aggr_fun), K(ret));
break;
@ -4386,6 +4454,24 @@ int ObAggregateProcessor::collect_aggr_result(
result.set_uint(aggr_cell.get_tiny_num_uint());
break;
}
case T_FUN_SUM_OPNSIZE: {
int64_t size = aggr_cell.get_tiny_num_int();
if (size > 0) {
if (lib::is_mysql_mode()) {
result.set_int(size);
} else {
char local_buff[ObNumber::MAX_BYTE_LEN];
ObDataBuffer local_alloc(local_buff, ObNumber::MAX_BYTE_LEN);
ObNumber result_num;
if (OB_FAIL(result_num.from(size, local_alloc))) {
LOG_WARN("failed to convert to number", K(ret));
} else {
result.set_number(result_num);
}
}
}
break;
}
default:
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unknown aggr function type", K(aggr_fun));

View File

@ -296,6 +296,9 @@ int ObPushdownFilterConstructor::is_white_mode(const ObRawExpr* raw_expr, bool &
} else {
if (static_cg_.get_cur_cluster_version() < CLUSTER_VERSION_4_3_0_0) {
const ObObjMeta &col_meta = child->get_result_meta();
if (T_OP_IN == item_type) {
need_check = false;
}
for (int64_t i = 1; OB_SUCC(ret) && need_check && i < raw_expr->get_param_count(); i++) {
if (OB_ISNULL(child = raw_expr->get_param_expr(i))) {
ret = OB_ERR_UNEXPECTED;
@ -311,9 +314,11 @@ int ObPushdownFilterConstructor::is_white_mode(const ObRawExpr* raw_expr, bool &
}
}
} else {
const ObRawExpr *param_exprs = T_OP_IN == item_type ? raw_expr->get_param_expr(1) : raw_expr;
int64_t i = T_OP_IN == item_type ? 0 : 1;
const ObExprResType &col_type = child->get_result_type();
for (int64_t i = 1; OB_SUCC(ret) && need_check && i < raw_expr->get_param_count(); i++) {
if (OB_ISNULL(child = raw_expr->get_param_expr(i))) {
for (; OB_SUCC(ret) && need_check && i < param_exprs->get_param_count(); i++) {
if (OB_ISNULL(child = param_exprs->get_param_expr(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null child expr", K(ret), K(i));
} else {
@ -335,6 +340,7 @@ int ObPushdownFilterConstructor::is_white_mode(const ObRawExpr* raw_expr, bool &
case T_OP_GE:
case T_OP_GT:
case T_OP_NE:
case T_OP_IN:
is_white = true;
break;
default:
@ -1391,6 +1397,18 @@ int ObPushdownFilterExecutor::execute_skipping_filter(ObBoolMask &bm)
return ret;
}
void ObPushdownFilterExecutor::clear()
{
if (is_filter_white_node()) {
static_cast<ObWhiteFilterExecutor*>(this)->clear_in_datums();
} else if (is_logic_op_node()) {
sql::ObPushdownFilterExecutor **children = get_childs();
for (uint32_t i = 0; i < get_child_count(); ++i) {
children[i]->clear();
}
}
}
bool ObPushdownFilterExecutor::check_sstable_index_filter()
{
bool is_needed_to_do_filter = true;
@ -1760,13 +1778,30 @@ void ObPhysicalFilterExecutor::clear_evaluated_infos()
}
int ObWhiteFilterExecutor::init_evaluated_datums()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(filter_.expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), KPC(filter_.expr_));
} else if (WHITE_OP_IN == filter_.get_op_type()) {
if (OB_FAIL(init_in_eval_datums())) {
LOG_WARN("Failed to init eval datums for WHITE_OP_IN filter", K(ret));
}
} else if (OB_FAIL(init_compare_eval_datums())) {
LOG_WARN("Failed to init eval datums for compare white filter", K(ret));
}
LOG_DEBUG("[PUSHDOWN], white pushdown filter inited datum params", K(datum_params_));
return ret;
}
int ObWhiteFilterExecutor::init_compare_eval_datums()
{
int ret = OB_SUCCESS;
ObEvalCtx &eval_ctx = op_.get_eval_ctx();
ObObjMeta col_obj_type;
ObObjMeta param_obj_type;
ObObjMeta col_obj_meta;
ObObjMeta param_obj_meta;
bool is_ref_column_found = false;
if (OB_UNLIKELY(nullptr == filter_.expr_ || 2 > filter_.expr_->arg_cnt_)) {
if (OB_UNLIKELY(filter_.expr_->arg_cnt_ < 2)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), KPC(filter_.expr_));
} else if (OB_FAIL(ObPhysicalFilterExecutor::init_evaluated_datums())) {
@ -1774,13 +1809,14 @@ int ObWhiteFilterExecutor::init_evaluated_datums()
} else if (OB_FAIL(init_array_param(datum_params_, filter_.expr_->arg_cnt_))) {
LOG_WARN("Failed to alloc params", K(ret));
} else {
null_param_contained_ = false;
for (int64_t i = 0; OB_SUCC(ret) && i < filter_.expr_->arg_cnt_; i++) {
if (OB_ISNULL(filter_.expr_->args_[i])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null expr arguments", K(ret), K(i));
} else if (filter_.expr_->args_[i]->type_ == T_REF_COLUMN) {
is_ref_column_found = true;
col_obj_type = filter_.expr_->args_[i]->obj_meta_;
col_obj_meta = filter_.expr_->args_[i]->obj_meta_;
// skip column reference expr
continue;
} else {
@ -1789,12 +1825,13 @@ int ObWhiteFilterExecutor::init_evaluated_datums()
LOG_WARN("evaluate filter arg expr failed", K(ret), K(i));
} else if (OB_FAIL(datum_params_.push_back(*datum))) {
LOG_WARN("Failed to push back datum", K(ret));
} else if (is_null_param(*datum, param_obj_meta)) {
null_param_contained_ = true;
} else {
param_obj_type = filter_.expr_->args_[i]->obj_meta_;
param_obj_meta = filter_.expr_->args_[i]->obj_meta_;
}
}
}
LOG_DEBUG("[PUSHDOWN], white pushdown filter inited datum params", K(datum_params_));
}
if (OB_SUCC(ret)) {
@ -1802,64 +1839,132 @@ int ObWhiteFilterExecutor::init_evaluated_datums()
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected no ref column found", K(ret));
} else {
check_null_params();
cmp_func_ = get_datum_cmp_func(col_obj_type, param_obj_type);
if (WHITE_OP_IN == filter_.get_op_type() && OB_FAIL(init_obj_set())) {
LOG_WARN("Failed to init Object hash set in filter node", K(ret));
}
cmp_func_ = get_datum_cmp_func(col_obj_meta, param_obj_meta);
}
}
return ret;
}
void ObWhiteFilterExecutor::check_null_params()
int ObWhiteFilterExecutor::init_in_eval_datums()
{
null_param_contained_ = false;
for (int64_t i = 0; !null_param_contained_ && i < datum_params_.count(); i++) {
if ((lib::is_mysql_mode() && datum_params_.at(i).is_null())
|| (lib::is_oracle_mode() && (datum_params_.at(i).is_null()
|| (filter_.expr_->args_[i]->obj_meta_.is_character_type() && (0 == datum_params_.at(i).len_))))) {
int ret = OB_SUCCESS;
ObEvalCtx &eval_ctx = op_.get_eval_ctx();
ObObjMeta col_obj_meta;
ObObjMeta param_obj_meta;
if (OB_UNLIKELY(filter_.expr_->arg_cnt_ != 2)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), KPC(filter_.expr_));
} else if (OB_UNLIKELY(nullptr == filter_.expr_->args_[0] ||
T_REF_COLUMN != filter_.expr_->args_[0]->type_ ||
nullptr == filter_.expr_->args_[1] ||
0 >= filter_.expr_->inner_func_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), KPC(filter_.expr_), KP(filter_.expr_->args_[0]), KP(filter_.expr_->args_[1]));
} else if (OB_FAIL(ObPhysicalFilterExecutor::init_evaluated_datums())) {
LOG_WARN("Failed to init evaluated datums", K(ret));
} else if (OB_FAIL(init_array_param(datum_params_, filter_.expr_->inner_func_cnt_))) {
LOG_WARN("Failed to alloc params", K(ret));
} else if (OB_FAIL(init_param_set(filter_.expr_->inner_func_cnt_, filter_.expr_->args_[1]->args_[0]))) {
LOG_WARN("Failed to init datum set", K(ret));
} else {
col_obj_meta = filter_.expr_->args_[0]->obj_meta_;
null_param_contained_ = false;
ObDatum *datum = nullptr;
for (int i = 0; OB_SUCC(ret) && i < filter_.expr_->inner_func_cnt_; ++i) {
const ObExpr *cur_arg = filter_.expr_->args_[1]->args_[i];
if (OB_ISNULL(cur_arg)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null arg", K(ret), K(cur_arg));
} else if (i == 0) {
param_obj_meta = cur_arg->obj_meta_;
} else if (param_obj_meta.get_type() != cur_arg->obj_meta_.get_type()) { // make sure all type is identical
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected param obj meta", K(ret), K(param_obj_meta), K(cur_arg->obj_meta_));
}
if (OB_SUCC(ret)) {
if (OB_FAIL(cur_arg->eval(eval_ctx, datum))) {
LOG_WARN("Evaluate filter arg expr failed", K(ret), K(i));
} else if (is_null_param(*datum, param_obj_meta)) {
// skip null in filter IN
} else if (OB_FAIL(add_to_param_set_and_array(*datum, cur_arg))) {
LOG_WARN("Failed to add param to set", K(ret), KPC(datum), K(cur_arg));
}
}
}
if (datum_params_.count() == 0) {
null_param_contained_ = true;
}
}
return;
if (OB_SUCC(ret)) {
bool mock_equal = false;
ObDatumComparator cmp(filter_.expr_->args_[1]->args_[0]->basic_funcs_->null_first_cmp_, ret, mock_equal);
std::sort(datum_params_.begin(), datum_params_.end(), cmp);
if (OB_FAIL(ret)) {
LOG_WARN("Failed to sort datums", K(ret));
} else {
cmp_func_ = get_datum_cmp_func(col_obj_meta, param_obj_meta);
cmp_func_rev_ = get_datum_cmp_func(param_obj_meta, col_obj_meta);
// When initializing a parameter set, the corresponding hash and comparison functions of the parameter type are used.
// However, during subsequent exist checks, comparison is done between the parameter and the column.
// Therefore, it is necessary to convert the corresponding function types.
param_set_.set_hash_and_cmp_func(filter_.expr_->args_[0]->basic_funcs_->murmur_hash_v2_, cmp_func_rev_);
}
}
return ret;
}
int ObWhiteFilterExecutor::init_obj_set()
int ObWhiteFilterExecutor::init_param_set(const int64_t count, const ObExpr *param_arg)
{
int ret = OB_SUCCESS;
if (param_set_.created()) {
param_set_.destroy();
}
if (OB_FAIL(param_set_.create(datum_params_.count() * 2))) {
LOG_WARN("Failed to create hash set", K(ret));
if (OB_FAIL(param_set_.create(count * 2))) {
LOG_WARN("Failed to create hash set", K(ret), K(count));
} else {
param_set_.set_hash_and_cmp_func(param_arg->basic_funcs_->murmur_hash_v2_, param_arg->basic_funcs_->null_first_cmp_);
}
for (int i = 0; OB_SUCC(ret) && i < datum_params_.count(); ++i) {
ObObj obj;
if (OB_FAIL(datum_params_.at(i).to_obj(obj, filter_.expr_->args_[i]->obj_meta_))) {
LOG_WARN("convert datum to obj failed", K(ret));
} else if (OB_FAIL(param_set_.set_refactored(obj))) {
if (OB_UNLIKELY(ret != OB_HASH_EXIST)) {
LOG_WARN("Failed to insert object into hashset", K(ret));
} else {
ret = OB_SUCCESS;
}
return ret;
}
int ObWhiteFilterExecutor::add_to_param_set_and_array(const ObDatum &datum, const ObExpr *cur_arg)
{
int ret = OB_SUCCESS;
if (OB_FAIL(param_set_.set_refactored(datum))) {
if (OB_UNLIKELY(ret != OB_HASH_EXIST)) {
LOG_WARN("Failed to insert object into hashset", K(ret), K(datum));
} else {
ret = OB_SUCCESS;
}
} else if (OB_FAIL(datum_params_.push_back(datum))) {
LOG_WARN("Failed to add datum to datum array", K(ret), K(datum));
}
return ret;
}
int ObWhiteFilterExecutor::exist_in_datum_set(const ObDatum &datum, bool &is_exist) const
{
int ret = OB_SUCCESS;
is_exist = false;
if (param_set_.count() > 0) {
if (OB_FAIL(param_set_.exist_refactored(datum, is_exist))) {
LOG_WARN("Failed to search datum in param set", K(ret), K(datum));
}
}
return ret;
}
int ObWhiteFilterExecutor::exist_in_obj_set(const ObObj &obj, bool &is_exist) const
int ObWhiteFilterExecutor::exist_in_datum_array(const ObDatum &datum, bool &is_exist, const int64_t offset) const
{
int ret = param_set_.exist_refactored(obj);
if (OB_HASH_EXIST == ret) {
ret = OB_SUCCESS;
is_exist = true;
} else if (OB_HASH_NOT_EXIST == ret) {
ret = OB_SUCCESS;
is_exist = false;
} else {
LOG_WARN("Failed to search in obj_set in pushed down filter node", K(ret), K(obj));
int ret = OB_SUCCESS;
is_exist = false;
if (datum_params_.count() > 0) {
ObDatumComparator cmp(cmp_func_rev_, ret, is_exist);
std::lower_bound(datum_params_.begin() + offset, datum_params_.end(), datum, cmp);
if (OB_FAIL(ret)) {
LOG_WARN("Failed to search datum in param array", K(ret), K(datum));
}
}
return ret;
}
@ -2605,7 +2710,10 @@ void PushdownFilterInfo::reset()
}
allocator_ = nullptr;
}
filter_ = nullptr;
if (OB_NOT_NULL(filter_)) {
filter_->clear();
filter_ = nullptr;
}
param_ = nullptr;
context_ = nullptr;
is_inited_ = false;

View File

@ -21,6 +21,7 @@
#include "share/datum/ob_datum.h"
#include "share/datum/ob_datum_funcs.h"
#include "sql/engine/expr/ob_expr.h"
#include "sql/engine/expr/ob_expr_in.h"
#include "sql/engine/ob_operator.h"
namespace oceanbase
@ -335,7 +336,8 @@ public:
virtual int get_filter_val_meta(common::ObObjMeta &obj_meta) const;
inline virtual ObObjType get_filter_arg_obj_type(int64_t arg_idx) const
{
return expr_->args_[arg_idx]->obj_meta_.get_type();
const ObExpr *expr = WHITE_OP_IN == op_type_ ? expr_->args_[1] : expr_;
return expr->args_[arg_idx]->obj_meta_.get_type();
}
// mapping array from white filter's operation type to common::ObCmpOp
@ -654,6 +656,7 @@ public:
blocksstable::ObIMicroBlockRowScanner *micro_scanner,
const bool use_vectorize);
int execute_skipping_filter(ObBoolMask &bm);
virtual void clear(); // release array and set memory used by WHITE_OP_IN filter.
DECLARE_VIRTUAL_TO_STRING;
protected:
int find_evaluated_datums(
@ -710,7 +713,7 @@ public:
n_datum_eval_flags_(0),
datum_eval_flags_(nullptr)
{}
~ObPhysicalFilterExecutor();
virtual ~ObPhysicalFilterExecutor();
int filter(blocksstable::ObStorageDatum *datums, int64_t col_cnt, const sql::ObBitVector &skip_bit, bool &ret_val);
virtual int init_evaluated_datums() override;
virtual int filter(ObEvalCtx &eval_ctx, const sql::ObBitVector &skip_bit, bool &filtered) = 0;
@ -759,6 +762,107 @@ private:
ObBitVector *skip_bit_;
};
class ObWhiteFilterParam
{
public:
ObWhiteFilterParam() : datum_(nullptr), hash_func_(nullptr), cmp_func_(nullptr) {}
ObWhiteFilterParam(const ObDatum *datum,
const ObExprHashFuncType *hash_func,
const ObDatumCmpFuncType *cmp_func)
: datum_(datum),
hash_func_(hash_func),
cmp_func_(cmp_func)
{}
~ObWhiteFilterParam() {}
inline bool operator==(const ObWhiteFilterParam &other) const
{
bool equal_ret = true;
if (datum_->is_null() && other.datum_->is_null()) {
} else if (datum_->is_null() || other.datum_->is_null()) {
equal_ret = false;
} else {
int cmp_ret = 0;
(*cmp_func_)(*datum_, *other.datum_, cmp_ret);
equal_ret = cmp_ret == 0;
}
return equal_ret;
}
inline int hash(uint64_t &hash_val, uint64_t seed = 0) const
{
int ret = OB_SUCCESS;
if (OB_FAIL((*hash_func_)(*datum_, seed, seed))) {
STORAGE_LOG(WARN, "Failed to do hash for datum", K_(datum));
} else {
hash_val = seed;
}
return ret;
}
TO_STRING_KV(KPC_(datum), KP_(hash_func), KP_(cmp_func));
private:
const ObDatum *datum_;
const ObExprHashFuncType *hash_func_;
const ObDatumCmpFuncType *cmp_func_;
};
class ObWhiteFilterParamHashSet
{
public:
ObWhiteFilterParamHashSet() : set_(), hash_func_(nullptr), cmp_func_(nullptr) {}
~ObWhiteFilterParamHashSet()
{
destroy();
}
void destroy()
{
if (set_.created()) {
(void)set_.destroy();
}
}
inline bool created() const
{
return set_.created();
}
inline int create(int param_num)
{
ObMemAttr attr(MTL_ID(), common::ObModIds::OB_HASH_BUCKET);
return set_.create(param_num, attr);
}
inline int64_t count() const
{
return set_.size();
}
inline int set_refactored(const ObDatum &datum, int flag = 0 /*deduplicated*/)
{
ObWhiteFilterParam param(&datum, &hash_func_, &cmp_func_);
return set_.set_refactored(param, flag);
}
inline int exist_refactored(const ObDatum &datum, bool &is_exist) const
{
ObWhiteFilterParam param(&datum, &hash_func_, &cmp_func_);
int ret = set_.exist_refactored(param);
if (OB_HASH_EXIST == ret) {
is_exist = true;
ret = OB_SUCCESS;
} else if (OB_HASH_NOT_EXIST == ret) {
is_exist = false;
ret = OB_SUCCESS;
} else {
STORAGE_LOG(WARN, "Failed to search in hashset", K(ret), K(param));
}
return ret;
}
inline void set_hash_and_cmp_func(const ObExprHashFuncType hash_func, const ObDatumCmpFuncType cmp_func)
{
hash_func_ = hash_func;
cmp_func_ = cmp_func;
}
TO_STRING_KV(K_(set), K_(hash_func), K_(cmp_func));
private:
common::hash::ObHashSet<ObWhiteFilterParam> set_;
ObExprHashFuncType hash_func_;
ObDatumCmpFuncType cmp_func_;
};
class ObWhiteFilterExecutor : public ObPhysicalFilterExecutor
{
public:
@ -766,13 +870,12 @@ public:
ObPushdownWhiteFilterNode &filter,
ObPushdownOperator &op)
: ObPhysicalFilterExecutor(alloc, op, PushdownExecutorType::WHITE_FILTER_EXECUTOR),
cmp_func_(nullptr), null_param_contained_(false), datum_params_(alloc), filter_(filter) {}
cmp_func_(nullptr), cmp_func_rev_(nullptr), null_param_contained_(false), datum_params_(alloc), filter_(filter)
{}
~ObWhiteFilterExecutor()
{
datum_params_.reset();
if (param_set_.created()) {
(void)param_set_.destroy();
}
param_set_.destroy();
}
OB_INLINE ObPushdownWhiteFilterNode &get_filter_node() { return filter_; }
@ -783,25 +886,43 @@ public:
virtual int init_evaluated_datums() override;
OB_INLINE const common::ObIArray<common::ObDatum> &get_datums() const
{ return datum_params_; }
OB_INLINE const common::ObDatum &get_min_param() const
{ return datum_params_.at(0); }
OB_INLINE const common::ObDatum &get_max_param() const
{ return datum_params_.at(datum_params_.count() - 1); }
OB_INLINE bool null_param_contained() const { return null_param_contained_; }
int exist_in_obj_set(const common::ObObj &obj, bool &is_exist) const;
bool is_obj_set_created() const { return param_set_.created(); };
int exist_in_datum_set(const common::ObDatum &datum, bool &is_exist) const;
int exist_in_datum_array(const common::ObDatum &datum, bool &is_exist, const int64_t offset = 0) const;
OB_INLINE ObWhiteFilterOperatorType get_op_type() const
{ return filter_.get_op_type(); }
bool is_cmp_op_with_null_ref_value() const;
INHERIT_TO_STRING_KV("ObPushdownWhiteFilterExecutor", ObPushdownFilterExecutor,
K_(null_param_contained), K_(datum_params), K(param_set_.created()),
K_(null_param_contained), K_(datum_params), K_(param_set),
K_(filter));
virtual int filter(ObEvalCtx &eval_ctx, const sql::ObBitVector &skip_bit, bool &filtered) override;
virtual void clear_in_datums()
{
if (WHITE_OP_IN == filter_.get_op_type()) {
datum_params_.reset();
param_set_.destroy();
}
}
protected:
void check_null_params();
int init_obj_set();
OB_INLINE bool is_null_param(const ObDatum &datum, const ObObjMeta obj_meta)
{
return datum.is_null() || (lib::is_oracle_mode() && obj_meta.is_character_type() && (0 == datum.len_));
}
int init_compare_eval_datums();
int init_in_eval_datums();
int init_param_set(const int64_t count, const ObExpr *cur_arg);
int add_to_param_set_and_array(const ObDatum &datum, const ObExpr *cur_arg);
public:
common::ObDatumCmpFuncType cmp_func_;
common::ObDatumCmpFuncType cmp_func_rev_;
protected:
bool null_param_contained_;
common::ObFixedArray<common::ObDatum, common::ObIAllocator> datum_params_;
common::hash::ObHashSet<common::ObObj> param_set_;
ObWhiteFilterParamHashSet param_set_;
ObPushdownWhiteFilterNode &filter_;
};

View File

@ -1288,7 +1288,8 @@ public:
case T_OP_EQ: // fall through
case T_OP_NSEQ: // fall through
case T_OP_SQ_EQ: // fall through
case T_OP_SQ_NSEQ: {
case T_OP_SQ_NSEQ:
case T_OP_IN: {
cmp_op = common::CO_EQ;
break;
}

View File

@ -63,7 +63,7 @@ int ObExprSysOpOpnsize::cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_e
return ret;
}
int ObExprSysOpOpnsize::calc_sys_op_opnsize(ObExpr *expr, ObDatum *arg, int64_t &size) {
int ObExprSysOpOpnsize::calc_sys_op_opnsize(const ObExpr *expr,const ObDatum *arg, int64_t &size) {
int ret = OB_SUCCESS;
if (OB_ISNULL(arg) || OB_ISNULL(expr)) {
ret = OB_ERR_UNEXPECTED;

View File

@ -33,7 +33,7 @@ public:
ObExpr &rt_expr) const override;
static int calc_sys_op_opnsize_expr(const ObExpr &expr, ObEvalCtx &ctx,
ObDatum &res);
static int calc_sys_op_opnsize(ObExpr *expr, ObDatum *arg, int64_t &size);
static int calc_sys_op_opnsize(const ObExpr *expr, const ObDatum *arg, int64_t &size);
private:
DISALLOW_COPY_AND_ASSIGN(ObExprSysOpOpnsize);
};

View File

@ -7115,7 +7115,8 @@ int ObLogPlan::check_basic_groupby_pushdown(const ObIArray<ObAggFunRawExpr*> &ag
T_FUN_TOP_FRE_HIST != aggr_expr->get_expr_type() &&
T_FUN_SYS_BIT_AND != aggr_expr->get_expr_type() &&
T_FUN_SYS_BIT_OR != aggr_expr->get_expr_type() &&
T_FUN_SYS_BIT_XOR != aggr_expr->get_expr_type()) {
T_FUN_SYS_BIT_XOR != aggr_expr->get_expr_type() &&
T_FUN_SUM_OPNSIZE != aggr_expr->get_expr_type()) {
can_push = false;
} else if (aggr_expr->is_param_distinct()) {
can_push = false;
@ -14682,7 +14683,9 @@ int ObLogPlan::check_scalar_aggr_can_storage_pushdown(const uint64_t table_id,
} else if (T_FUN_COUNT != cur_aggr->get_expr_type()
&& T_FUN_MIN != cur_aggr->get_expr_type()
&& T_FUN_MAX != cur_aggr->get_expr_type()
&& T_FUN_SUM != cur_aggr->get_expr_type()) {
&& T_FUN_SUM != cur_aggr->get_expr_type()
&& T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS != cur_aggr->get_expr_type()
&& T_FUN_SUM_OPNSIZE != cur_aggr->get_expr_type()) {
can_push = false;
} else if (1 < cur_aggr->get_real_param_count()) {
can_push = false;

View File

@ -8189,10 +8189,11 @@ int ObOptimizerUtil::generate_pullup_aggr_expr(ObRawExprFactory &expr_factory,
T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE == origin_expr->get_expr_type() ||
T_FUN_SYS_BIT_AND == origin_expr->get_expr_type() ||
T_FUN_SYS_BIT_OR == origin_expr->get_expr_type() ||
T_FUN_SYS_BIT_XOR == origin_expr->get_expr_type()) {
T_FUN_SYS_BIT_XOR == origin_expr->get_expr_type() ||
T_FUN_SUM_OPNSIZE == origin_expr->get_expr_type()) {
/* MAX(a) -> MAX(MAX(a)), MIN(a) -> MIN(MIN(a)) SUM(a) -> SUM(SUM(a)) */
ObItemType pullup_aggr_type = origin_expr->get_expr_type();
if (T_FUN_COUNT == pullup_aggr_type) {
if (T_FUN_COUNT == pullup_aggr_type || T_FUN_SUM_OPNSIZE == pullup_aggr_type) {
pullup_aggr_type = T_FUN_COUNT_SUM;
} else if (T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS == pullup_aggr_type) {
pullup_aggr_type = T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE;

View File

@ -853,6 +853,7 @@ static const NonReservedKeyword Mysql_none_reserved_keywords[] =
{"substr", SUBSTR},
{"substring", SUBSTRING},
{"sum", SUM},
{"sum_opnsize", SUM_OPNSIZE},
{"super", SUPER},
{"suspend", SUSPEND},
{"successful", SUCCESSFUL},

View File

@ -348,7 +348,7 @@ END_P SET_VAR DELIMITER
SYNCHRONIZATION SYNCHRONOUS STOP STORAGE STORAGE_FORMAT_VERSION STORE STORING STRING
SUBCLASS_ORIGIN SUBDATE SUBJECT SUBPARTITION SUBPARTITIONS SUBSTR SUBSTRING SUCCESSFUL SUM
SUPER SUSPEND SWAPS SWITCH SWITCHES SWITCHOVER SYSTEM SYSTEM_USER SYSDATE SESSION_ALIAS
SIZE SKEWONLY SEQUENCE SLOG STATEMENT_ID SKIP_HEADER SKIP_BLANK_LINES STATEMENT
SIZE SKEWONLY SEQUENCE SLOG STATEMENT_ID SKIP_HEADER SKIP_BLANK_LINES STATEMENT SUM_OPNSIZE
TABLE_CHECKSUM TABLE_MODE TABLE_ID TABLE_NAME TABLEGROUPS TABLES TABLESPACE TABLET TABLET_ID TABLET_MAX_SIZE
TEMPLATE TEMPORARY TEMPTABLE TENANT TEXT THAN TIME TIMESTAMP TIMESTAMPADD TIMESTAMPDIFF TP_NO
@ -3048,6 +3048,10 @@ MOD '(' expr ',' expr ')'
{
malloc_non_terminal_node($$, result->malloc_pool_, T_FUN_SYS_GEOMCOLLECTION, 1, NULL);
}
| SUM_OPNSIZE '(' expr ')'
{
malloc_non_terminal_node($$, result->malloc_pool_, T_FUN_SUM_OPNSIZE, 2, NULL, $3);
}
;
sys_interval_func:
@ -11446,6 +11450,30 @@ relation_factor %prec LOWER_PARENS
merge_nodes($$, result, T_INDEX_HINT_LIST, $5);
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 4, $1, $$, $2, $3);
}
| relation_factor use_partition sample_clause use_flashback %prec LOWER_PARENS
{
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, NULL, $2, $3, $4);
}
| relation_factor use_partition sample_clause seed use_flashback %prec LOWER_PARENS
{
if ($3 != NULL) {
$3->children_[2] = $4;
}
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, NULL, $2, $3, $5);
}
| relation_factor use_partition sample_clause use_flashback index_hint_list %prec LOWER_PARENS
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $5);
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, $$, $2, $3, $4);
}
| relation_factor use_partition sample_clause seed use_flashback index_hint_list %prec LOWER_PARENS
{
if ($3 != NULL) {
$3->children_[2] = $4;
}
merge_nodes($$, result, T_INDEX_HINT_LIST, $6);
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, $$, $2, $3, $5);
}
| relation_factor sample_clause %prec LOWER_PARENS
{
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 4, $1, NULL, NULL, $2);
@ -11470,6 +11498,30 @@ relation_factor %prec LOWER_PARENS
merge_nodes($$, result, T_INDEX_HINT_LIST, $4);
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 4, $1, $$, NULL, $2);
}
| relation_factor sample_clause use_flashback %prec LOWER_PARENS
{
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, NULL, NULL, $2, $3);
}
| relation_factor sample_clause seed use_flashback %prec LOWER_PARENS
{
if ($2 != NULL) {
$2->children_[2] = $3;
}
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, NULL, NULL, $2, $4);
}
| relation_factor sample_clause use_flashback index_hint_list %prec LOWER_PARENS
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $4);
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, $$, NULL, $2, $3);
}
| relation_factor sample_clause seed use_flashback index_hint_list %prec LOWER_PARENS
{
if ($2 != NULL) {
$2->children_[2] = $3;
}
merge_nodes($$, result, T_INDEX_HINT_LIST, $5);
malloc_non_terminal_node($$, result->malloc_pool_, T_ORG, 5, $1, $$, NULL, $2, $4);
}
| relation_factor index_hint_list %prec LOWER_PARENS
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $2);
@ -11521,6 +11573,54 @@ relation_factor %prec LOWER_PARENS
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 5, $1, $6, NULL, $2, $3);
$$->sql_str_off_ = @1.first_column;
}
| relation_factor sample_clause use_flashback AS relation_name
{
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $5, NULL, NULL, $2, $3);
}
| relation_factor sample_clause seed use_flashback AS relation_name
{
if ($2 != NULL) {
$2->children_[2] = $3;
}
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $6, NULL, NULL, $2, $4);
}
| relation_factor use_partition sample_clause use_flashback AS relation_name
{
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $6, NULL, $2, $3, $4);
}
| relation_factor use_partition sample_clause seed use_flashback AS relation_name
{
if ($3 != NULL) {
$3->children_[2] = $4;
}
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $7, NULL, $2, $3, $5);
}
| relation_factor sample_clause use_flashback AS relation_name index_hint_list
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $6);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $5, $$, NULL, $2, $3);
}
| relation_factor sample_clause seed use_flashback AS relation_name index_hint_list
{
if ($2 != NULL) {
$2->children_[2] = $3;
}
merge_nodes($$, result, T_INDEX_HINT_LIST, $7);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $6, $$, NULL, $2, $4);
}
| relation_factor use_partition sample_clause use_flashback AS relation_name index_hint_list
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $7);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $6, $$, $2, $3, $4);
}
| relation_factor use_partition sample_clause seed use_flashback AS relation_name index_hint_list
{
if ($3 != NULL) {
$3->children_[2] = $4;
}
merge_nodes($$, result, T_INDEX_HINT_LIST, $8);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $7, $$, $2, $3, $5);
}
| relation_factor AS relation_name index_hint_list
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $4);
@ -11652,6 +11752,54 @@ relation_factor %prec LOWER_PARENS
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 5, $1, $4, $$, $2, $3);
$$->sql_str_off_ = @1.first_column;
}
| relation_factor sample_clause use_flashback relation_name
{
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $4, NULL, NULL, $2, $3);
}
| relation_factor sample_clause seed use_flashback relation_name
{
if ($2 != NULL) {
$2->children_[2] = $3;
}
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $5, NULL, NULL, $2, $4);
}
| relation_factor use_partition sample_clause use_flashback relation_name
{
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $5, NULL, $2, $3, $4);
}
| relation_factor use_partition sample_clause seed use_flashback relation_name
{
if ($3 != NULL) {
$3->children_[2] = $4;
}
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $6, NULL, $2, $3, $5);
}
| relation_factor sample_clause use_flashback relation_name index_hint_list
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $5);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $4, $$, NULL, $2, $3);
}
| relation_factor sample_clause seed use_flashback relation_name index_hint_list
{
if ($2 != NULL) {
$2->children_[2] = $3;
}
merge_nodes($$, result, T_INDEX_HINT_LIST, $6);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $5, $$, NULL, $2, $4);
}
| relation_factor use_partition sample_clause use_flashback relation_name index_hint_list
{
merge_nodes($$, result, T_INDEX_HINT_LIST, $6);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $5, $$, $2, $3, $4);
}
| relation_factor use_partition sample_clause seed use_flashback relation_name index_hint_list
{
if ($3 != NULL) {
$3->children_[2] = $4;
}
merge_nodes($$, result, T_INDEX_HINT_LIST, $7);
malloc_non_terminal_node($$, result->malloc_pool_, T_ALIAS, 6, $1, $6, $$, $2, $3, $5);
}
| TABLE '(' simple_expr ')' %prec LOWER_PARENS
{
malloc_non_terminal_node($$, result->malloc_pool_, T_TABLE_COLLECTION_EXPRESSION, 2, $3, NULL);
@ -20454,6 +20602,7 @@ ACCOUNT
| KV_ATTRIBUTES
| OBJECT_ID
| TRANSFER
| SUM_OPNSIZE
;
unreserved_keyword_special:

View File

@ -1091,6 +1091,8 @@ int ObRawExprPrinter::print(ObAggFunRawExpr *expr)
SET_SYMBOL_IF_EMPTY("approx_count_distinct_synopsis");
case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE:
SET_SYMBOL_IF_EMPTY("approx_count_distinct_synopsis_merge");
case T_FUN_SUM_OPNSIZE:
SET_SYMBOL_IF_EMPTY("sum_opnsize");
case T_FUN_PL_AGG_UDF:{
if (type == T_FUN_PL_AGG_UDF) {
if (OB_ISNULL(expr->get_pl_agg_udf_expr()) ||

View File

@ -1431,7 +1431,8 @@ int ObRawExprDeduceType::visit(ObAggFunRawExpr &expr)
case T_FUN_REGR_COUNT:
case T_FUN_COUNT_SUM:
case T_FUN_APPROX_COUNT_DISTINCT:
case T_FUN_KEEP_COUNT: {
case T_FUN_KEEP_COUNT:
case T_FUN_SUM_OPNSIZE: {
if (lib::is_oracle_mode()) {
result_type.set_number();
result_type.set_scale(0);
@ -2089,6 +2090,7 @@ int ObRawExprDeduceType::check_group_aggr_param(ObAggFunRawExpr &expr)
&& T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS != expr.get_expr_type()
&& T_FUN_TOP_FRE_HIST != expr.get_expr_type()
&& T_FUN_HYBRID_HIST != expr.get_expr_type()
&& T_FUN_SUM_OPNSIZE != expr.get_expr_type()
&& 1 != get_expr_output_column(*param_expr)) {
ret = OB_ERR_INVALID_COLUMN_NUM;
LOG_USER_ERROR(OB_ERR_INVALID_COLUMN_NUM, (int64_t)1);
@ -2115,6 +2117,7 @@ int ObRawExprDeduceType::check_group_aggr_param(ObAggFunRawExpr &expr)
&& T_FUN_MEDIAN != expr.get_expr_type()
&& T_FUN_GROUP_PERCENTILE_CONT != expr.get_expr_type()
&& T_FUN_GROUP_PERCENTILE_DISC != expr.get_expr_type()
&& T_FUN_SUM_OPNSIZE != expr.get_expr_type()
&& !expr.is_need_deserialize_row()
&& !(T_FUN_PL_AGG_UDF == expr.get_expr_type() && !expr.is_param_distinct())
&& !(T_FUN_WM_CONCAT == expr.get_expr_type() && !expr.is_param_distinct()))) {
@ -2134,7 +2137,7 @@ int ObRawExprDeduceType::check_group_aggr_param(ObAggFunRawExpr &expr)
K(ret), K(param_expr->get_subschema_id()));
} else {
ret = OB_ERR_INVALID_TYPE_FOR_OP;
LOG_WARN("lob or json type parameter not expected", K(ret));
LOG_WARN("lob or json type parameter not expected", K(ret), K(expr));
}
}
}

View File

@ -762,7 +762,8 @@ int ObRawExprResolverImpl::do_recursive_resolve(const ParseNode *node, ObRawExpr
case T_FUN_JSON_OBJECTAGG:
case T_FUN_ORA_JSON_ARRAYAGG:
case T_FUN_ORA_JSON_OBJECTAGG:
case T_FUN_ORA_XMLAGG: {
case T_FUN_ORA_XMLAGG:
case T_FUN_SUM_OPNSIZE:{
if (OB_FAIL(process_agg_node(node, expr))) {
LOG_WARN("fail to process agg node", K(ret), K(node));

View File

@ -574,7 +574,8 @@ int ObTransformDBlink::check_link_expr_valid(ObRawExpr *expr, bool &is_valid)
} else if (T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS == expr->get_expr_type() ||
T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE == expr->get_expr_type() ||
T_FUN_SYS_ESTIMATE_NDV == expr->get_expr_type() ||
T_OP_GET_USER_VAR == expr->get_expr_type()) {
T_OP_GET_USER_VAR == expr->get_expr_type() ||
T_FUN_SUM_OPNSIZE == expr->get_expr_type()) {
// special function is invalid
} else if (expr->get_result_type().is_ext()) {
// special type is invalid

View File

@ -141,7 +141,7 @@ void ObAggRow::reuse()
}
}
int ObAggRow::init(const ObTableAccessParam &param, const int64_t batch_size)
int ObAggRow::init(const ObTableAccessParam &param, const ObTableAccessContext &context, const int64_t batch_size)
{
int ret = OB_SUCCESS;
const common::ObIArray<share::schema::ObColumnParam *> *out_cols_param = param.iter_param_.get_col_params();
@ -166,7 +166,8 @@ int ObAggRow::init(const ObTableAccessParam &param, const int64_t batch_size)
int32_t col_index = param.iter_param_.read_info_->get_columns_index().at(col_offset);
const share::schema::ObColumnParam *col_param = out_cols_param->at(col_offset);
sql::ObExpr *expr = param.output_exprs_->at(i);
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, expr, batch_size);
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, expr,
batch_size, is_pad_char_to_full_length(context.sql_mode_));
if (OB_FAIL(agg_cell_factory_.alloc_cell(basic_info, dummy_agg_cells_))) {
LOG_WARN("Failed to alloc agg cell", K(ret), K(i));
} else if (FALSE_IT(cell = dummy_agg_cells_.at(dummy_agg_cells_.count() - 1))) {
@ -197,7 +198,8 @@ int ObAggRow::init(const ObTableAccessParam &param, const int64_t batch_size)
} else {
need_access_data_ = true;
}
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, agg_expr, batch_size);
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, agg_expr,
batch_size, is_pad_char_to_full_length(context.sql_mode_));
if (OB_FAIL(agg_cell_factory_.alloc_cell(basic_info, agg_cells_, exclude_null))) {
LOG_WARN("Failed to alloc agg cell", K(ret), K(i));
}
@ -278,7 +280,7 @@ int ObAggregatedStore::init(const ObTableAccessParam &param)
K(param.aggregate_exprs_->count()), K(param.iter_param_.agg_cols_project_->count()));
} else if (OB_FAIL(ObBlockBatchedRowStore::init(param))) {
LOG_WARN("Failed to init ObBlockBatchedRowStore", K(ret));
} else if (OB_FAIL(agg_row_.init(param, batch_size_))) {
} else if (OB_FAIL(agg_row_.init(param, context_, batch_size_))) {
LOG_WARN("Failed to init agg cells", K(ret));
} else if (OB_FAIL(check_agg_in_row_mode(param.iter_param_))) {
LOG_WARN("Failed to check agg in row mode", K(ret));
@ -443,13 +445,13 @@ int ObAggregatedStore::collect_aggregated_row(blocksstable::ObDatumRow *&row)
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < agg_row_.get_agg_count(); ++i) {
ObAggCell *cell = agg_row_.at(i);
if (OB_FAIL(cell->collect_result(eval_ctx_, is_pad_char_to_full_length(context_.sql_mode_)))) {
if (OB_FAIL(cell->collect_result(eval_ctx_))) {
LOG_WARN("Failed to fill agg result", K(ret), K(i), K(*cell));
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < agg_row_.get_dummy_agg_count(); ++i) {
ObAggCell *cell = agg_row_.at_dummy(i);
if (OB_FAIL(cell->collect_result(eval_ctx_, false))) {
if (OB_FAIL(cell->collect_result(eval_ctx_))) {
LOG_WARN("Failed to fill agg result", K(ret), K(i), K(*cell));
}
}

View File

@ -62,7 +62,7 @@ public:
~ObAggRow();
void reset();
void reuse();
int init(const ObTableAccessParam &param, const int64_t batch_size);
int init(const ObTableAccessParam &param, const ObTableAccessContext &context, const int64_t batch_size);
OB_INLINE int64_t get_agg_count() const { return agg_cells_.count(); }
OB_INLINE int64_t get_dummy_agg_count() const { return dummy_agg_cells_.count(); }
OB_INLINE bool has_lob_column_out() const { return has_lob_column_out_; }

View File

@ -1549,7 +1549,7 @@ int ObIndexTreeMultiPassPrefetcher<DATA_PREFETCH_DEPTH, INDEX_PREFETCH_DEPTH>::O
&& OB_FAIL(sstable_index_filter->check_range(prefetcher.iter_param_->read_info_, index_info,
*(prefetcher.access_ctx_->allocator_)))) {
LOG_WARN("Fail to check if can skip prefetch", K(ret), K(index_info));
} else if (index_info.is_filter_always_false()) {
} else if (index_info.is_filter_always_false() || prefetcher.skip_index_block(index_info)) {
} else if (nullptr != prefetcher.agg_row_store_ && prefetcher.agg_row_store_->can_agg_index_info(index_info)) {
if (OB_FAIL(prefetcher.agg_row_store_->fill_index_info(index_info))) {
LOG_WARN("Fail to agg index info", K(ret), KPC(this));

View File

@ -173,6 +173,10 @@ protected:
ObSSTable &sstable,
const ObTableIterParam &iter_param,
ObTableAccessContext &access_ctx);
OB_INLINE bool skip_index_block(const ObMicroIndexInfo &index_block_info)
{
return !sstable_->is_major_sstable() && !index_block_info.contain_uncommitted_row() && index_block_info.get_max_merged_trans_version() <= access_ctx_->trans_version_range_.base_version_;
}
private:
ObMicroBlockDataHandle &get_read_handle(const int64_t level)
{

View File

@ -229,6 +229,8 @@ int ObMultipleMerge::reset_tables()
} else {
ret = OB_SUCCESS;
}
} else if (OB_FAIL(set_base_version())) {
STORAGE_LOG(WARN, "fail to set base version", K(ret));
} else if (OB_FAIL(construct_iters())) {
STORAGE_LOG(WARN, "fail to construct iters", K(ret));
} else {
@ -891,6 +893,8 @@ int ObMultipleMerge::open()
STORAGE_LOG(WARN, "Failed to init datum row", K(ret));
} else if (OB_FAIL(nop_pos_.init(*access_ctx_->stmt_allocator_, access_param_->get_max_out_col_cnt()))) {
STORAGE_LOG(WARN, "Fail to init nop pos, ", K(ret));
} else if (OB_FAIL(set_base_version())) {
STORAGE_LOG(WARN, "Fail to set base version", K(ret));
} else if (access_param_->iter_param_.is_use_iter_pool()) {
if (OB_FAIL(access_ctx_->alloc_iter_pool(access_param_->iter_param_.is_use_column_store()))) {
LOG_WARN("Failed to init iter pool", K(ret));
@ -962,7 +966,7 @@ int ObMultipleMerge::alloc_row_store(ObTableAccessContext &context, const ObTabl
ret = common::OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to alloc aggregated store", K(ret));
} else if (FALSE_IT(group_by_cell_ = new (buf) ObGroupByCell(0, *context.stmt_allocator_))) {
} else if (OB_FAIL(group_by_cell_->init_for_single_row(param, param.get_op()->get_eval_ctx()))) {
} else if (OB_FAIL(group_by_cell_->init_for_single_row(param, context, param.get_op()->get_eval_ctx()))) {
LOG_WARN("Failed to init group by cell for single row", K(ret));
}
}
@ -1154,10 +1158,6 @@ int ObMultipleMerge::prepare_read_tables(bool refresh)
ret = OB_NOT_INIT;
LOG_WARN("ObMultipleMerge has not been inited", K(ret), K_(get_table_param), KP_(access_param),
KP_(access_ctx));
} else if (OB_UNLIKELY(!access_ctx_->query_flag_.is_whole_macro_scan() &&
0 != access_ctx_->trans_version_range_.base_version_)) {
ret = OB_ERR_SYS;
LOG_WARN("base version should be 0", K(ret), K(access_ctx_->trans_version_range_.base_version_));
} else if (!refresh && get_table_param_->tablet_iter_.table_iter()->is_valid()) {
if (OB_FAIL(prepare_tables_from_iterator(*get_table_param_->tablet_iter_.table_iter()))) {
LOG_WARN("prepare tables fail", K(ret), K(get_table_param_->tablet_iter_.table_iter()));
@ -1500,5 +1500,20 @@ void ObMultipleMerge::dump_table_statistic_for_4377()
LOG_ERROR("==================== End table info ====================");
}
int ObMultipleMerge::set_base_version() const {
int ret = OB_SUCCESS;
// When the major table is currently being processed, the snapshot version is taken and placed
// in the current context for base version to filter unnecessary rows in the mini or minor sstable
if (OB_LIKELY(tables_.count() > 0)) {
ObITable *table = nullptr;
if (OB_FAIL(tables_.at(0, table))) {
STORAGE_LOG(WARN, "Fail to get the first store", K(ret));
} else if (table->is_major_sstable()) {
access_ctx_->trans_version_range_.base_version_ = table->get_snapshot_version();
}
}
return ret;
}
}
}

View File

@ -84,6 +84,7 @@ protected:
void reuse_iter_array();
void dump_tx_statistic_for_4377(ObStoreCtx *store_ctx);
void dump_table_statistic_for_4377();
int set_base_version() const;
private:
int get_next_normal_row(blocksstable::ObDatumRow *&row);

View File

@ -393,6 +393,7 @@ ObAggCell::ObAggCell(const ObAggCellBasicInfo &basic_info, common::ObIAllocator
agg_row_reader_(nullptr),
col_datums_(nullptr),
group_by_result_datum_buf_(nullptr),
bitmap_(nullptr),
group_by_result_cnt_(0),
is_assigned_to_group_by_processor_(false)
{
@ -419,6 +420,10 @@ void ObAggCell::reset()
allocator_.free(agg_row_reader_);
agg_row_reader_ = nullptr;
}
if (nullptr != bitmap_) {
allocator_.free(bitmap_);
bitmap_ = nullptr;
}
col_datums_ = nullptr;
free_group_by_buf(allocator_, group_by_result_datum_buf_);
group_by_result_cnt_ = 0;
@ -433,6 +438,9 @@ void ObAggCell::reuse()
skip_index_datum_.reuse();
skip_index_datum_.set_null();
group_by_result_cnt_ = 0;
if (nullptr != bitmap_) {
bitmap_->reuse();
}
if (nullptr != col_datums_) {
for (int64_t i = 0; i < basic_info_.batch_size_; ++i) {
col_datums_[i].set_null();
@ -569,14 +577,12 @@ int ObAggCell::copy_single_output_row(sql::ObEvalCtx &ctx)
return ret;
}
int ObAggCell::collect_result(sql::ObEvalCtx &ctx, bool need_padding)
int ObAggCell::collect_result(sql::ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
ObDatum &result = basic_info_.agg_expr_->locate_datum_for_write(ctx);
if (OB_FAIL(fill_default_if_need(result_datum_))) {
LOG_WARN("Failed to fill default", K(ret), KPC(this));
} else if (need_padding && OB_FAIL(pad_column_if_need(result_datum_))) {
LOG_WARN("Failed to pad column", K(ret), KPC(this));
} else if (OB_FAIL(result.from_storage_datum(result_datum_, basic_info_.agg_expr_->obj_datum_map_))) {
LOG_WARN("Failed to from storage datum", K(ret), K(result_datum_), K(result), KPC(this));
} else {
@ -629,6 +635,28 @@ int ObAggCell::output_extra_group_by_result(const int64_t start, const int64_t c
return ret;
}
int ObAggCell::reserve_bitmap(const int64_t count)
{
int ret = OB_SUCCESS;
void *buf = nullptr;
if (OB_NOT_NULL(bitmap_)) {
if (OB_FAIL(bitmap_->reserve(count))) {
LOG_WARN("Failed to reserve bitmap", K(ret));
} else {
bitmap_->reuse(); // all false
}
} else {
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObBitmap)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to alloc memory for bitmap", K(ret));
} else if (FALSE_IT(bitmap_ = new (buf) ObBitmap(allocator_))) {
} else if (OB_FAIL(bitmap_->init(count))) { // all false
LOG_WARN("Failed to init bitmap", K(ret));
}
}
return ret;
}
int ObAggCell::prepare_def_datum()
{
int ret = OB_SUCCESS;
@ -638,6 +666,8 @@ int ObAggCell::prepare_def_datum()
if (!def_cell.is_nop_value()) {
if (OB_FAIL(def_datum_.from_obj_enhance(def_cell))) {
STORAGE_LOG(WARN, "Failed to transfer obj to datum", K(ret));
} else if (OB_FAIL(pad_column_if_need(def_datum_))) {
LOG_WARN("Failed to pad default datum", K(ret), K_(basic_info), K_(def_datum));
} else if (def_cell.is_lob_storage() && !def_cell.is_null()) {
// lob def value must have no lob header when not null, should add lob header for default value
ObString data = def_datum_.get_string();
@ -682,6 +712,7 @@ int ObAggCell::pad_column_if_need(blocksstable::ObStorageDatum &datum)
if (OB_ISNULL(basic_info_.col_param_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected, col param is null", K(ret), K(basic_info_.col_offset_));
} else if (!basic_info_.need_padding()) {
} else if (OB_FAIL(pad_column(basic_info_.col_param_->get_meta_type(), basic_info_.col_param_->get_accuracy(), allocator_, datum))) {
LOG_WARN("Fail to pad column", K(ret), K(basic_info_.col_offset_), KPC(this));
}
@ -741,6 +772,7 @@ int ObAggCell::read_agg_datum(
// TODO: @luhaopeng.lhp fix col_index in cg, use 0 temporarily
meta.col_idx_ = is_cg ? 0 : static_cast<uint32_t>(basic_info_.col_index_);
switch (agg_type_) {
case ObPDAggType::PD_SUM_OP_SIZE:
case ObPDAggType::PD_COUNT: {
meta.col_type_ = blocksstable::SK_IDX_NULL_COUNT;
break;
@ -767,7 +799,8 @@ int ObAggCell::read_agg_datum(
LOG_WARN("Fail to init aggregate row reader", K(ret));
} else if (OB_FAIL(agg_row_reader_->read(meta, skip_index_datum_))) {
LOG_WARN("Failed read aggregate row", K(ret), K(meta));
} else if (ObPDAggType::PD_COUNT == agg_type_ && skip_index_datum_.is_null()) {
} else if ((ObPDAggType::PD_COUNT == agg_type_ || ObPDAggType::PD_SUM_OP_SIZE == agg_type_) &&
skip_index_datum_.is_null()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected, skip index datum is null", K(ret), K(meta), K(index_info));
}
@ -855,28 +888,36 @@ int ObCountAggCell::init(const bool is_group_by, sql::ObEvalCtx *eval_ctx)
int ObCountAggCell::eval(blocksstable::ObStorageDatum &datum, const int64_t row_count)
{
int ret = OB_SUCCESS;
if (!exclude_null_) {
if (OB_UNLIKELY(row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid row count", K(ret), K(row_count));
} else if (!exclude_null_) {
row_count_ += row_count;
} else if (OB_FAIL(fill_default_if_need(datum))) {
LOG_WARN("Failed to fill default", K(ret), KPC(this));
} else if (!datum.is_null()) {
row_count_ += row_count;
}
aggregated_ = true;
LOG_DEBUG("after count row", K(ret), K(row_count), K(row_count_));
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("after count row", K(ret), K(row_count_));
return ret;
}
int ObCountAggCell::eval_batch(const common::ObDatum *datums, const int64_t row_count)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(exclude_null_ || nullptr != datums)) {
if (OB_UNLIKELY(nullptr == datums || row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid row count", K(ret), KP(datums), K(row_count));
} else if (OB_UNLIKELY(exclude_null_ || nullptr != datums)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("should not read data", K(ret), K(exclude_null_), KP(datums));
} else {
row_count_ += row_count;
aggregated_ = true;
}
aggregated_ = true;
LOG_DEBUG("after count batch", K(ret), K(row_count), K(row_count_));
return ret;
}
@ -903,7 +944,9 @@ int ObCountAggCell::eval_micro_block(
row_count_ += valid_row_count;
}
}
aggregated_ = true;
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("eval_micro_block", K(ret), K(row_count), K(row_count_));
return ret;
}
@ -922,7 +965,9 @@ int ObCountAggCell::eval_index_info(const blocksstable::ObMicroIndexInfo &index_
} else {
row_count_ += index_info.get_row_count() - skip_index_datum_.get_int();
}
aggregated_ = true;
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("eval_index_info", K(ret), K(index_info.get_row_count()), K(row_count_));
return ret;
}
@ -1117,9 +1162,8 @@ int ObCountAggCell::collect_batch_result_in_group_by(const int64_t distinct_cnt)
return ret;
}
int ObCountAggCell::collect_result(sql::ObEvalCtx &ctx, bool need_padding)
int ObCountAggCell::collect_result(sql::ObEvalCtx &ctx)
{
UNUSED(need_padding);
int ret = OB_SUCCESS;
ObDatum &result = basic_info_.agg_expr_->locate_datum_for_write(ctx);
sql::ObEvalInfo &eval_info = basic_info_.agg_expr_->get_eval_info(ctx);
@ -1147,7 +1191,6 @@ ObMinAggCell::ObMinAggCell(const ObAggCellBasicInfo &basic_info, common::ObIAllo
datum_allocator_("ObStorageAgg", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID())
{
agg_type_ =ObPDAggType::PD_MIN;
cmp_fun_ = basic_info_.agg_expr_->basic_funcs_->null_first_cmp_;
result_datum_.set_null();
}
@ -1167,14 +1210,34 @@ void ObMinAggCell::reuse()
datum_allocator_.reuse();
}
int ObMinAggCell::init(const bool is_group_by, sql::ObEvalCtx *eval_ctx)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(basic_info_.agg_expr_->basic_funcs_) ||
OB_ISNULL(basic_info_.agg_expr_->basic_funcs_->null_first_cmp_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected agg expr", K(ret), K(basic_info_.agg_expr_));
} else if (OB_FAIL(ObAggCell::init(is_group_by, eval_ctx))) {
LOG_WARN("Failed to init agg cell", K(ret));
} else {
cmp_fun_ = basic_info_.agg_expr_->basic_funcs_->null_first_cmp_;
}
return ret;
}
int ObMinAggCell::eval(blocksstable::ObStorageDatum &storage_datum, const int64_t row_count)
{
UNUSED(row_count);
int ret = OB_SUCCESS;
int cmp_ret = 0;
if (OB_FAIL(fill_default_if_need(storage_datum))) {
if (OB_UNLIKELY(row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), K(row_count));
} else if (OB_FAIL(fill_default_if_need(storage_datum))) {
LOG_WARN("Failed to fill default", K(ret), K(storage_datum), K(*this));
} else if (storage_datum.is_null()) {
} else if (OB_FAIL(pad_column_if_need(storage_datum))) {
LOG_WARN("Failed to pad column", K(ret), K_(basic_info), K(storage_datum));
} else if (result_datum_.is_null()) {
if (OB_FAIL(result_datum_.deep_copy(storage_datum, datum_allocator_))) {
LOG_WARN("Failed to deep copy datum", K(ret), K(storage_datum), K(basic_info_.col_offset_));
@ -1184,16 +1247,18 @@ int ObMinAggCell::eval(blocksstable::ObStorageDatum &storage_datum, const int64_
} else if (cmp_ret > 0 && OB_FAIL(deep_copy_datum(storage_datum, datum_allocator_))) {
LOG_WARN("Failed to deep copy datum", K(ret), K(storage_datum), K(result_datum_), K(basic_info_.col_offset_));
}
aggregated_ = true;
if (OB_SUCC(ret)) {
aggregated_ = true;
}
return ret;
}
int ObMinAggCell::eval_batch(const common::ObDatum *datums, const int64_t count)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(cmp_fun_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null cmp fun", K(ret));
if (OB_UNLIKELY(nullptr == datums || count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), KP(datums), K(count));
} else {
int cmp_ret = 0;
blocksstable::ObStorageDatum tmp_min_datum;
@ -1221,8 +1286,10 @@ int ObMinAggCell::eval_batch(const common::ObDatum *datums, const int64_t count)
LOG_WARN("Failed to deep copy datum", K(ret), K(tmp_min_datum), K(result_datum_), K(basic_info_.col_offset_));
}
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
}
aggregated_ = true;
return ret;
}
@ -1307,7 +1374,6 @@ ObMaxAggCell::ObMaxAggCell(const ObAggCellBasicInfo &basic_info, common::ObIAllo
datum_allocator_("ObStorageAgg", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID())
{
agg_type_ = ObPDAggType::PD_MAX;
cmp_fun_ = basic_info_.agg_expr_->basic_funcs_->null_first_cmp_;
result_datum_.set_null();
}
@ -1327,14 +1393,34 @@ void ObMaxAggCell::reuse()
datum_allocator_.reuse();
}
int ObMaxAggCell::init(const bool is_group_by, sql::ObEvalCtx *eval_ctx)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(basic_info_.agg_expr_->basic_funcs_) ||
OB_ISNULL(basic_info_.agg_expr_->basic_funcs_->null_first_cmp_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected agg expr", K(ret), K(basic_info_.agg_expr_));
} else if (OB_FAIL(ObAggCell::init(is_group_by, eval_ctx))) {
LOG_WARN("Failed to init agg cell", K(ret));
} else {
cmp_fun_ = basic_info_.agg_expr_->basic_funcs_->null_first_cmp_;
}
return ret;
}
int ObMaxAggCell::eval(blocksstable::ObStorageDatum &storage_datum, const int64_t row_count)
{
UNUSED(row_count);
int ret = OB_SUCCESS;
int cmp_ret = 0;
if (OB_FAIL(fill_default_if_need(storage_datum))) {
if (OB_UNLIKELY(row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), K(row_count));
} else if (OB_FAIL(fill_default_if_need(storage_datum))) {
LOG_WARN("Failed to fill default", K(ret), K(storage_datum), K(*this));
} else if (storage_datum.is_null()) {
} else if (OB_FAIL(pad_column_if_need(storage_datum))) {
LOG_WARN("Failed to pad column", K(ret), K_(basic_info), K(storage_datum));
} else if (result_datum_.is_null()) {
if (OB_FAIL(result_datum_.deep_copy(storage_datum, datum_allocator_))) {
LOG_WARN("Failed to deep copy datum", K(ret), K(storage_datum), K(basic_info_.col_offset_));
@ -1344,16 +1430,18 @@ int ObMaxAggCell::eval(blocksstable::ObStorageDatum &storage_datum, const int64_
} else if (cmp_ret < 0 && OB_FAIL(deep_copy_datum(storage_datum, datum_allocator_))) {
LOG_WARN("Failed to deep copy datum", K(ret), K(storage_datum), K(result_datum_), K(basic_info_.col_offset_));
}
aggregated_ = true;
if (OB_SUCC(ret)) {
aggregated_ = true;
}
return ret;
}
int ObMaxAggCell::eval_batch(const common::ObDatum *datums, const int64_t count)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(cmp_fun_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null cmp fun", K(ret));
if (OB_UNLIKELY(nullptr == datums || count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), KP(datums), K(count));
} else {
int cmp_ret = 0;
blocksstable::ObStorageDatum tmp_min_datum;
@ -1381,8 +1469,10 @@ int ObMaxAggCell::eval_batch(const common::ObDatum *datums, const int64_t count)
LOG_WARN("Failed to deep copy datum", K(ret), K(tmp_min_datum), K(result_datum_), K(basic_info_.col_offset_));
}
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
}
aggregated_ = true;
return ret;
}
@ -1457,6 +1547,383 @@ int ObMaxAggCell::eval_batch_in_group_by(
return ret;
}
ObHyperLogLogAggCell::ObHyperLogLogAggCell(const ObAggCellBasicInfo &basic_info, common::ObIAllocator &allocator)
: ObAggCell(basic_info, allocator),
hash_func_(nullptr),
ndv_calculator_(),
def_hash_value_(0)
{
agg_type_ = ObPDAggType::PD_HLL;
}
void ObHyperLogLogAggCell::reset()
{
hash_func_ = nullptr;
if (OB_ISNULL(ndv_calculator_)) {
ndv_calculator_->destroy();
allocator_.free(ndv_calculator_);
ndv_calculator_ = nullptr;
}
def_hash_value_ = 0;
ObAggCell::reset();
}
void ObHyperLogLogAggCell::reuse()
{
ObAggCell::reuse();
if (OB_ISNULL(ndv_calculator_)) {
ndv_calculator_->reuse();
}
}
int ObHyperLogLogAggCell::init(const bool is_group_by, sql::ObEvalCtx *eval_ctx)
{
int ret = OB_SUCCESS;
void *buf = nullptr;
if (OB_ISNULL(basic_info_.agg_expr_->args_) ||
OB_ISNULL(basic_info_.agg_expr_->args_[0]) ||
OB_ISNULL(basic_info_.agg_expr_->args_[0]->basic_funcs_) ||
OB_ISNULL(basic_info_.agg_expr_->args_[0]->basic_funcs_->murmur_hash_) ||
OB_UNLIKELY(T_REF_COLUMN != basic_info_.agg_expr_->args_[0]->type_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected agg expr", K(ret), K(basic_info_.agg_expr_));
} else if (OB_FAIL(ObAggCell::init(is_group_by, eval_ctx))) {
LOG_WARN("Failed to init agg cell", K(ret));
} else if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObHyperLogLogCalculator)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to alloc memory for hyperloglog calculator", K(ret));
} else {
ndv_calculator_ = new (buf) ObHyperLogLogCalculator();
if (OB_FAIL(ndv_calculator_->init(&allocator_, LLC_BUCKET_BITS))) {
LOG_WARN("Failed to init ndv calculator", K(ret));
} else {
hash_func_ = basic_info_.agg_expr_->args_[0]->basic_funcs_->murmur_hash_;
}
}
if (OB_SUCC(ret) && nullptr != basic_info_.col_param_ &&
!(basic_info_.col_param_->get_orig_default_value().is_nop_value())) {
uint64_t hash_value = 0;
if (OB_FAIL(prepare_def_datum())) {
LOG_WARN("Failed to prepare default datum", K(ret));
} else if (def_datum_.is_null()) {
} else if (OB_FAIL(hash_func_(def_datum_, hash_value, hash_value))) {
LOG_WARN("Failed to do hash", K(ret));
} else {
def_hash_value_ = hash_value;
}
}
return ret;
}
// fuse padding datums in expr, storage_datum is not padding
int ObHyperLogLogAggCell::eval(blocksstable::ObStorageDatum &storage_datum, const int64_t row_count)
{
int ret = OB_SUCCESS;
uint64_t hash_value = 0; // same as ObAggregateProcessor llc hash_value
if (OB_UNLIKELY(row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), K(row_count));
} else if (storage_datum.is_nop()) {
if (!def_datum_.is_null()) {
ndv_calculator_->set(def_hash_value_);
}
} else if (storage_datum.is_null()) {
// ndv does not consider null
} else if (OB_FAIL(pad_column_if_need(storage_datum))) {
LOG_WARN("Failed to pad column", K(ret), K_(basic_info), K(storage_datum));
} else if (OB_FAIL(hash_func_(storage_datum, hash_value, hash_value))) {
LOG_WARN("Failed to do hash", K(ret));
} else {
ndv_calculator_->set(hash_value);
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("after set ndv hash", K(hash_value), K(storage_datum), K_(ndv_calculator));
return ret;
}
// eval_batch() is invoked by ObAggCell::eval_micro_block()
// Like Min/Max aggregate, does not have the capability for batch processing now.
int ObHyperLogLogAggCell::eval_batch(const common::ObDatum *datums, const int64_t count)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(nullptr == datums || count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid row count", K(ret), KP(datums), K(count));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < count; ++i) {
uint64_t hash_value = 0; // same as ObAggregateProcessor llc hash_value
const common::ObDatum &datum = datums[i];
if (datum.is_null()) {
// ndv does not consider null
} else if (OB_FAIL(hash_func_(datum, hash_value, hash_value))) {
LOG_WARN("Failed to do hash", K(ret));
} else {
ndv_calculator_->set(hash_value);
}
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
}
LOG_DEBUG("after set ndv hash batch", K(count), K_(ndv_calculator));
return ret;
}
int ObHyperLogLogAggCell::collect_result(sql::ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
ObDatum &result = basic_info_.agg_expr_->locate_datum_for_write(ctx);
sql::ObEvalInfo &eval_info = basic_info_.agg_expr_->get_eval_info(ctx);
result.set_string(ndv_calculator_->get_buckets(), ndv_calculator_->get_bucket_num());
eval_info.evaluated_ = true;
LOG_DEBUG("collect result", K_(ndv_calculator), K(result), KPC(this));
return ret;
}
ObSumOpSizeAggCell::ObSumOpSizeAggCell(
const ObAggCellBasicInfo &basic_info,
common::ObIAllocator &allocator,
const bool exclude_null)
: ObAggCell(basic_info, allocator),
op_size_(0),
def_op_size_(0),
total_size_(0),
exclude_null_(exclude_null)
{
agg_type_ = ObPDAggType::PD_SUM_OP_SIZE;
}
void ObSumOpSizeAggCell::reset()
{
op_size_ = 0;
def_op_size_ = 0;
total_size_ = 0;
exclude_null_ = false;
ObAggCell::reset();
}
void ObSumOpSizeAggCell::reuse()
{
ObAggCell::reuse();
total_size_ = 0;
}
int ObSumOpSizeAggCell::init(const bool is_group_by, sql::ObEvalCtx *eval_ctx)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObAggCell::init(is_group_by, eval_ctx))) {
LOG_WARN("Failed to init agg cell", K(ret));
} else if (OB_FAIL(set_op_size())) {
LOG_WARN("Failed to get op size", K(ret));
} else if (nullptr != basic_info_.col_param_ &&
!(basic_info_.col_param_->get_orig_default_value().is_nop_value())) {
if (OB_FAIL(prepare_def_datum())) {
LOG_WARN("Failed to prepare default datum", K(ret));
} else if (OB_FAIL(get_datum_op_size(def_datum_, def_op_size_))) {
LOG_WARN("Failed to get default datum length", K(ret), K_(def_datum));
}
}
return ret;
}
int ObSumOpSizeAggCell::set_op_size()
{
int ret = OB_SUCCESS;
ObObjDatumMapType type = OBJ_DATUM_MAPPING_MAX;
if (OB_ISNULL(basic_info_.agg_expr_->args_) ||
OB_ISNULL(basic_info_.agg_expr_->args_[0]) ||
OB_UNLIKELY(T_REF_COLUMN != basic_info_.agg_expr_->args_[0]->type_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("arg is null", K(ret), KPC(basic_info_.agg_expr_));
} else if (FALSE_IT(type = basic_info_.agg_expr_->args_[0]->obj_datum_map_)) {
} else if (OB_UNLIKELY(type >= common::OBJ_DATUM_MAPPING_MAX || type <= common::OBJ_DATUM_NULL)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected type", K(ret), K(type));
} else if (is_fixed_length_type()) {
switch (type) {
case common::OBJ_DATUM_8BYTE_DATA : {
op_size_ = sizeof(ObDatum) + 8;
break;
}
case common::OBJ_DATUM_4BYTE_DATA : {
op_size_ = sizeof(ObDatum) + 4;
break;
}
case common::OBJ_DATUM_1BYTE_DATA : {
op_size_ = sizeof(ObDatum) + 1;
break;
}
case common::OBJ_DATUM_4BYTE_LEN_DATA : {
op_size_ = sizeof(ObDatum) + 12;
break;
}
case common::OBJ_DATUM_2BYTE_LEN_DATA : {
op_size_ = sizeof(ObDatum) + 10;
break;
}
case common::OBJ_DATUM_FULL : {
op_size_ = sizeof(ObDatum) + 16;
break;
}
default: {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected fixed length type", K(ret), K(type));
}
}
}
return ret;
}
int ObSumOpSizeAggCell::get_datum_op_size(const ObDatum &datum, int64_t &length)
{
int ret = OB_SUCCESS;
if (!is_lob_col() || datum.is_null()) {
length = sizeof(ObDatum) + datum.len_;
} else {
ObLobLocatorV2 locator(datum.get_string(), basic_info_.agg_expr_->args_[0]->obj_meta_.has_lob_header());
int64_t lob_data_byte_len = 0;
if (OB_FAIL(locator.get_lob_data_byte_len(lob_data_byte_len))) {
LOG_WARN("Failed to get lob data byte len", K(ret), K(locator));
} else {
length = sizeof(ObDatum) + lob_data_byte_len;
}
}
return ret;
}
int ObSumOpSizeAggCell::eval(blocksstable::ObStorageDatum &storage_datum, const int64_t row_count)
{
int ret = OB_SUCCESS;
int64_t length = 0;
if (OB_UNLIKELY(row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), K(row_count));
} else if (storage_datum.is_nop()) {
total_size_ += def_op_size_ * row_count;
} else if (OB_FAIL(pad_column_if_need(storage_datum))) {
LOG_WARN("Failed to pad column", K(ret), K_(basic_info), K(storage_datum));
} else if (OB_FAIL(get_datum_op_size(storage_datum, length))) {
LOG_WARN("Failed to get datum length", K(ret), K(storage_datum));
} else {
total_size_ += length * row_count; // row_count is not always 1
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("after sum op size", K(ret), K(length), K_(total_size), K_(op_size), K_(def_op_size));
return ret;
}
// eval_batch() is invoked by ObAggCell::eval_micro_block()
// Like Min/Max aggregate, does not have the capability for batch processing now.
int ObSumOpSizeAggCell::eval_batch(const common::ObDatum *datums, const int64_t row_count)
{
int ret = OB_SUCCESS;
int64_t length = 0;
if (OB_UNLIKELY(nullptr == datums || row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), KP(datums), K(row_count));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < row_count; ++i) {
// consider lob
const ObDatum datum = datums[i];
if (OB_FAIL(get_datum_op_size(datum, length))) {
LOG_WARN("Failed to get datum length", K(ret), K(datum));
} else {
total_size_ += length;
}
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
}
LOG_DEBUG("after sum op size batch", K(ret), K(row_count), K_(op_size), K_(total_size));
return ret;
}
int ObSumOpSizeAggCell::eval_micro_block(
const ObTableIterParam &iter_param,
const ObTableAccessContext &context,
const int32_t col_offset,
blocksstable::ObIMicroBlockReader *reader,
const int64_t *row_ids,
const int64_t row_count)
{
int ret = OB_SUCCESS;
int64_t size = 0;
if (need_access_data()) {
if (is_fixed_length_type()) {
int64_t valid_row_count = 0;
if (OB_ISNULL(row_ids)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected, row_ids is null", K(ret), KPC(this), K(row_count));
} else if (OB_FAIL(reader->get_row_count(col_offset, row_ids, row_count, false, valid_row_count))) {
LOG_WARN("Failed to get row count from micro block decoder", K(ret), KPC(this), K(row_count));
} else {
total_size_ += (row_count - valid_row_count) * sizeof(ObDatum) + valid_row_count * op_size_;
}
} else if (OB_FAIL(ObAggCell::eval_micro_block(iter_param, context, col_offset, reader, row_ids, row_count))) {
LOG_WARN("Failed to eval micro block", K(ret));
}
} else {
total_size_ += row_count * op_size_;
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("eval micro block in sum op size", K(ret), K(row_count), K_(exclude_null), K_(op_size), K_(total_size));
return ret;
}
int ObSumOpSizeAggCell::eval_index_info(const blocksstable::ObMicroIndexInfo &index_info, const bool is_cg)
{
int ret = OB_SUCCESS;
// consider the judge condition
if (!is_cg && (!index_info.can_blockscan(is_lob_col()) || index_info.is_left_border() || index_info.is_right_border())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected, the micro index info must can blockscan and not border", K(ret));
} else if (!exclude_null_) {
total_size_ += index_info.get_row_count() * op_size_;
} else if (OB_UNLIKELY(skip_index_datum_.is_null())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected skip index datum, is null", K(ret), K(index_info));
} else {
int64_t null_count = skip_index_datum_.get_int();
total_size_ += (index_info.get_row_count() - null_count) * op_size_ + null_count * sizeof(ObDatum);
}
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("eval_index_info", K(ret), K(index_info.get_row_count()), K_(op_size), K_(total_size));
return ret;
}
int ObSumOpSizeAggCell::collect_result(sql::ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
ObDatum &result = basic_info_.agg_expr_->locate_datum_for_write(ctx);
sql::ObEvalInfo &eval_info = basic_info_.agg_expr_->get_eval_info(ctx);
if (lib::is_oracle_mode()) {
common::number::ObNumber result_num;
char local_buff[common::number::ObNumber::MAX_BYTE_LEN];
common::ObDataBuffer local_alloc(local_buff, common::number::ObNumber::MAX_BYTE_LEN);
if (OB_FAIL(result_num.from(total_size_, local_alloc))) {
LOG_WARN("Failed to cons number from uint", K(ret), K_(total_size));
} else {
result.set_number(result_num);
eval_info.evaluated_ = true;
}
} else {
result.set_uint(total_size_);
eval_info.evaluated_ = true;
}
LOG_DEBUG("collect result", K(result), KPC(this));
return ret;
}
ObSumAggCell::ObSumAggCell(const ObAggCellBasicInfo &basic_info, common::ObIAllocator &allocator)
: ObAggCell(basic_info, allocator),
obj_tc_(ObNullTC),
@ -1850,9 +2317,11 @@ int ObSumAggCell::init_decimal_int_func()
int ObSumAggCell::eval(blocksstable::ObStorageDatum &datum, const int64_t row_count)
{
UNUSED(row_count);
int ret = OB_SUCCESS;
if (OB_FAIL(fill_default_if_need(datum))) {
if (OB_UNLIKELY(row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), K(row_count));
} else if (OB_FAIL(fill_default_if_need(datum))) {
LOG_WARN("Failed to fill default", K(ret), K(datum), KPC(this));
} else if (datum.is_null()) {
} else {
@ -1862,7 +2331,9 @@ int ObSumAggCell::eval(blocksstable::ObStorageDatum &datum, const int64_t row_co
}
}
}
aggregated_ = true;
if (OB_SUCC(ret)) {
aggregated_ = true;
}
LOG_DEBUG("after process rows", KPC(this));
return ret;
}
@ -1870,10 +2341,14 @@ int ObSumAggCell::eval(blocksstable::ObStorageDatum &datum, const int64_t row_co
int ObSumAggCell::eval_batch(const common::ObDatum *datums, const int64_t count)
{
int ret = OB_SUCCESS;
if (OB_FAIL((this->*eval_batch_func_)(datums, count))) {
if (OB_UNLIKELY(nullptr == datums || count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid row count", K(ret), K(count));
} else if (OB_FAIL((this->*eval_batch_func_)(datums, count))) {
LOG_WARN("Failed to eval batch", K(ret));
} else {
aggregated_ = true;
}
aggregated_ = true;
return ret;
}
@ -2099,13 +2574,13 @@ int ObSumAggCell::collect_batch_result_in_group_by(const int64_t distinct_cnt)
return ret;
}
int ObSumAggCell::collect_result(sql::ObEvalCtx &ctx, bool need_padding)
int ObSumAggCell::collect_result(sql::ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
ObDatum &result = basic_info_.agg_expr_->locate_datum_for_write(ctx);
sql::ObEvalInfo &eval_info = basic_info_.agg_expr_->get_eval_info(ctx);
if (!sum_use_int_flag_) {
if (OB_FAIL(ObAggCell::collect_result(ctx, need_padding))) {
if (OB_FAIL(ObAggCell::collect_result(ctx))) {
LOG_WARN("Failed to collect_result", K(ret), KPC(this));
}
} else if (ObIntTC != obj_tc_ && ObUIntTC != obj_tc_) {
@ -2638,9 +3113,14 @@ int ObFirstRowAggCell::init(const bool is_group_by, sql::ObEvalCtx *eval_ctx)
int ObFirstRowAggCell::eval(blocksstable::ObStorageDatum &datum, const int64_t row_count)
{
int ret = OB_SUCCESS;
if (!aggregated_) {
if (OB_UNLIKELY(row_count < 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid row count", K(ret), K(row_count));
} else if (!aggregated_) {
if (OB_FAIL(fill_default_if_need(datum))) {
LOG_WARN("Failed to fill default", K(ret), KPC(this));
} else if (OB_FAIL(pad_column_if_need(datum))) {
LOG_WARN("Failed to pad column", K(ret), K_(basic_info), K(datum));
} else if (OB_FAIL(result_datum_.deep_copy(datum, datum_allocator_))) {
LOG_WARN("Failed to deep copy datum", K(ret), K(datum));
} else {
@ -2736,13 +3216,13 @@ int ObFirstRowAggCell::eval_batch_in_group_by(
return ret;
}
int ObFirstRowAggCell::collect_result(sql::ObEvalCtx &ctx, bool need_padding)
int ObFirstRowAggCell::collect_result(sql::ObEvalCtx &ctx)
{
int ret = OB_SUCCESS;
if (!aggregated_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected, must be aggregated in single/batch row", K(ret));
} else if (OB_FAIL(ObAggCell::collect_result(ctx, need_padding))) {
} else if (OB_FAIL(ObAggCell::collect_result(ctx))) {
LOG_WARN("Failed to collect_result", K(ret), KPC(this));
}
return ret;
@ -2855,6 +3335,22 @@ int ObPDAggFactory::alloc_cell(
}
break;
}
case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS: {
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObHyperLogLogAggCell))) ||
OB_ISNULL(cell = new(buf) ObHyperLogLogAggCell(basic_info, allocator_))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to alloc memory for agg cell", K(ret));
}
break;
}
case T_FUN_SUM_OPNSIZE: {
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObSumOpSizeAggCell))) ||
OB_ISNULL(cell = new(buf) ObSumOpSizeAggCell(basic_info, allocator_, exclude_null))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to alloc memory for agg cell", K(ret));
}
break;
}
case T_FUN_SUM: {
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObSumAggCell))) ||
OB_ISNULL(cell = new(buf) ObSumAggCell(basic_info, allocator_))) {
@ -2951,7 +3447,7 @@ void ObGroupByCell::reuse()
projected_cnt_ = 0;
}
int ObGroupByCell::init(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ctx)
int ObGroupByCell::init(const ObTableAccessParam &param, const ObTableAccessContext &context, sql::ObEvalCtx &eval_ctx)
{
int ret = OB_SUCCESS;
const common::ObIArray<share::schema::ObColumnParam *> *out_cols_param = param.iter_param_.get_col_params();
@ -2972,7 +3468,8 @@ int ObGroupByCell::init(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ct
int32_t col_index = param.iter_param_.read_info_->get_columns_index().at(col_offset);
const share::schema::ObColumnParam *col_param = out_cols_param->at(col_offset);
sql::ObExpr *expr = param.output_exprs_->at(i);
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, expr, batch_size_);
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, expr,
batch_size_, is_pad_char_to_full_length(context.sql_mode_));
if (group_by_col_offset_ == col_offset) {
group_by_col_datums = expr->locate_batch_datums(eval_ctx);
if (OB_ISNULL(group_by_col_datums)) {
@ -2993,7 +3490,7 @@ int ObGroupByCell::init(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ct
} else if (OB_ISNULL(group_by_col_datums)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected group by col datums", K(ret), K(param));
} else if (OB_FAIL(init_agg_cells(param, eval_ctx, false))) {
} else if (OB_FAIL(init_agg_cells(param, context, eval_ctx, false))) {
LOG_WARN("Failed to init agg_cells", K(ret));
} else {
if (agg_cells_.count() > 2) {
@ -3013,7 +3510,7 @@ int ObGroupByCell::init(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ct
return ret;
}
int ObGroupByCell::init_for_single_row(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ctx)
int ObGroupByCell::init_for_single_row(const ObTableAccessParam &param, const ObTableAccessContext &context, sql::ObEvalCtx &eval_ctx)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(nullptr == param.iter_param_.group_by_cols_project_ ||
@ -3021,7 +3518,7 @@ int ObGroupByCell::init_for_single_row(const ObTableAccessParam &param, sql::ObE
nullptr == param.iter_param_.get_col_params())) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid argument", K(ret), K(param.iter_param_));
} else if (OB_FAIL(init_agg_cells(param, eval_ctx, true))) {
} else if (OB_FAIL(init_agg_cells(param, context, eval_ctx, true))) {
LOG_WARN("Failed to init agg_cells", K(ret));
}
return ret;
@ -3294,7 +3791,7 @@ int ObGroupByCell::init_uniform_header(
return ret;
}
int ObGroupByCell::init_agg_cells(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ctx, const bool is_for_single_row)
int ObGroupByCell::init_agg_cells(const ObTableAccessParam &param, const ObTableAccessContext &context, sql::ObEvalCtx &eval_ctx, const bool is_for_single_row)
{
int ret = OB_SUCCESS;
const common::ObIArray<share::schema::ObColumnParam *> *out_cols_param = param.iter_param_.get_col_params();
@ -3309,7 +3806,8 @@ int ObGroupByCell::init_agg_cells(const ObTableAccessParam &param, sql::ObEvalCt
exclude_null = col_param->is_nullable_for_write();
}
}
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, agg_expr, batch_size_);
ObAggCellBasicInfo basic_info(col_offset, col_index, col_param, agg_expr,
batch_size_, is_pad_char_to_full_length(context.sql_mode_));
if (OB_FAIL(agg_cell_factory_.alloc_cell(basic_info, agg_cells_, exclude_null, !is_for_single_row, &eval_ctx))) {
LOG_WARN("Failed to alloc agg cell", K(ret), K(i));
}

View File

@ -15,6 +15,7 @@
#include "lib/allocator/ob_allocator.h"
#include "sql/engine/expr/ob_expr.h"
#include "lib/utility/ob_hyperloglog.h"
#include "storage/ob_i_store.h"
#include "storage/blocksstable/ob_datum_row.h"
#include "storage/blocksstable/index_block/ob_index_block_row_struct.h"
@ -188,6 +189,8 @@ enum ObPDAggType
PD_COUNT = 0,
PD_MIN,
PD_MAX,
PD_HLL,
PD_SUM_OP_SIZE,
PD_SUM,
PD_FIRST_ROW,
PD_MAX_TYPE
@ -200,12 +203,14 @@ struct ObAggCellBasicInfo
const int32_t col_index,
const share::schema::ObColumnParam *col_param,
sql::ObExpr *agg_expr,
const int64_t batch_size)
const int64_t batch_size,
const bool is_padding_mode)
: col_offset_(col_offset),
col_index_(col_index),
col_param_(col_param),
agg_expr_(agg_expr),
batch_size_(batch_size)
batch_size_(batch_size),
is_padding_mode_(is_padding_mode)
{}
~ObAggCellBasicInfo() { reset(); }
void reset()
@ -215,17 +220,23 @@ struct ObAggCellBasicInfo
col_param_ = nullptr;
agg_expr_ = nullptr;
batch_size_ = 0;
is_padding_mode_ = false;
}
OB_INLINE bool is_valid() const
{
return col_offset_ >= 0 && nullptr != agg_expr_ && batch_size_ >= 0;
}
TO_STRING_KV(K_(col_offset), K_(col_index), KPC_(col_param), K_(agg_expr), K_(batch_size));
OB_INLINE bool need_padding() const
{
return is_padding_mode_ && nullptr != col_param_ && col_param_->get_meta_type().is_fixed_len_char_type();
}
TO_STRING_KV(K_(col_offset), K_(col_index), KPC_(col_param), K_(agg_expr), K_(batch_size), K_(is_padding_mode));
int32_t col_offset_; // offset in projector
int32_t col_index_; // column index
const share::schema::ObColumnParam *col_param_;
sql::ObExpr *agg_expr_;
int64_t batch_size_;
bool is_padding_mode_;
};
class ObAggCell
@ -236,7 +247,9 @@ public:
virtual void reset();
virtual void reuse();
virtual int init(const bool is_group_by, sql::ObEvalCtx *eval_ctx);
// need to fill default value
virtual int eval(blocksstable::ObStorageDatum &datum, const int64_t row_count = 1) = 0;
// no need to fill default value
virtual int eval_batch(const common::ObDatum *datums, const int64_t count) = 0;
virtual int eval_micro_block(
const ObTableIterParam &iter_param,
@ -257,19 +270,25 @@ public:
virtual int copy_output_row(const int32_t datum_offset);
virtual int copy_output_rows(const int32_t datum_offset);
virtual int copy_single_output_row(sql::ObEvalCtx &ctx);
virtual int collect_result(sql::ObEvalCtx &ctx, bool need_padding);
virtual int collect_result(sql::ObEvalCtx &ctx);
virtual int collect_batch_result_in_group_by(const int64_t distinct_cnt);
virtual int can_use_index_info(const blocksstable::ObMicroIndexInfo &index_info, const bool is_cg, bool &can_agg);
virtual bool need_access_data() const { return true; }
virtual bool finished() const { return false; }
virtual int reserve_group_by_buf(const int64_t size);
virtual int output_extra_group_by_result(const int64_t start, const int64_t count);
virtual int reserve_bitmap(const int64_t count);
OB_INLINE ObPDAggType get_type() const { return agg_type_; }
OB_INLINE bool is_min_agg() const { return agg_type_ == PD_MIN; }
OB_INLINE bool is_max_agg() const { return agg_type_ == PD_MAX; }
OB_INLINE bool is_aggregated() const { return aggregated_; }
OB_INLINE ObBitmap &get_bitmap() { return *bitmap_; }
OB_INLINE int32_t get_col_offset() const { return basic_info_.col_offset_; }
OB_INLINE common::ObDatum *get_col_datums() const { return col_datums_; }
OB_INLINE const sql::ObExpr *get_agg_expr() const { return basic_info_.agg_expr_; }
OB_INLINE bool is_lob_col() const { return is_lob_col_; }
OB_INLINE const ObDatum &get_result_datum() const { return result_datum_; }
OB_INLINE ObObjType get_obj_type() const { return basic_info_.agg_expr_->obj_meta_.get_type(); }
OB_INLINE common::ObObjDatumMapType get_datum_map_type() const { return basic_info_.agg_expr_->obj_datum_map_; }
OB_INLINE void set_group_by_result_cnt(const int64_t group_by_result_cnt) { group_by_result_cnt_ = group_by_result_cnt; }
OB_INLINE bool is_assigned_to_group_by_processor() const { return is_assigned_to_group_by_processor_; }
@ -303,6 +322,7 @@ protected:
common::ObDatum *col_datums_;
// store the aggregated result
ObGroupByExtendableBuf<ObDatum> *group_by_result_datum_buf_;
ObBitmap *bitmap_;
int64_t group_by_result_cnt_;
bool is_assigned_to_group_by_processor_;
private:
@ -341,7 +361,7 @@ public:
virtual int copy_output_row(const int32_t datum_offset) override;
virtual int copy_output_rows(const int32_t datum_offset) override;
virtual int copy_single_output_row(sql::ObEvalCtx &ctx) override;
virtual int collect_result(sql::ObEvalCtx &ctx, bool need_padding) override;
virtual int collect_result(sql::ObEvalCtx &ctx) override;
virtual int collect_batch_result_in_group_by(const int64_t distinct_cnt) override;
virtual bool need_access_data() const override { return exclude_null_; }
INHERIT_TO_STRING_KV("ObAggCell", ObAggCell, K_(exclude_null), K_(row_count));
@ -357,6 +377,7 @@ public:
virtual ~ObMinAggCell() { reset(); };
virtual void reset() override;
virtual void reuse() override;
virtual int init(const bool is_group_by, sql::ObEvalCtx *eval_ctx) override;
virtual int eval(blocksstable::ObStorageDatum &datum, const int64_t row_count = 1) override;
virtual int eval_batch(const common::ObDatum *datums, const int64_t count) override;
virtual int eval_batch_in_group_by(
@ -385,6 +406,7 @@ public:
virtual ~ObMaxAggCell() { reset(); };
virtual void reset() override;
virtual void reuse() override;
virtual int init(const bool is_group_by, sql::ObEvalCtx *eval_ctx) override;
virtual int eval(blocksstable::ObStorageDatum &datum, const int64_t row_count = 1) override;
virtual int eval_batch(const common::ObDatum *datums, const int64_t count) override;
virtual int eval_batch_in_group_by(
@ -406,6 +428,103 @@ private:
common::ObArenaAllocator datum_allocator_;
};
// For statistical information aggregation pushdown.
// Not support cross-partition aggregate, not support group by.
class ObHyperLogLogAggCell : public ObAggCell
{
public:
ObHyperLogLogAggCell(const ObAggCellBasicInfo &basic_info, common::ObIAllocator &allocator);
virtual ~ObHyperLogLogAggCell() { reset(); }
virtual void reset() override;
virtual void reuse() override;
virtual int init(const bool is_group_by, sql::ObEvalCtx *eval_ctx) override;
virtual int eval(blocksstable::ObStorageDatum &storage_datum, const int64_t row_count = 1) override;
virtual int eval_batch(const common::ObDatum *datums, const int64_t count) override;
virtual int eval_index_info(const blocksstable::ObMicroIndexInfo &index_info, const bool is_cg = false) override
{ return OB_NOT_SUPPORTED; }
virtual int eval_batch_in_group_by(
const common::ObDatum *datums,
const int64_t count,
const uint32_t *refs,
const int64_t distinct_cnt,
const bool is_group_by_col = false,
const bool is_default_datum = false) override
{ return OB_NOT_SUPPORTED; }
virtual int copy_output_row(const int32_t datum_offset) override { return OB_NOT_SUPPORTED; }
virtual int copy_output_rows(const int32_t datum_offset) override { return OB_NOT_SUPPORTED; }
virtual int collect_result(sql::ObEvalCtx &ctx) override;
virtual int collect_batch_result_in_group_by(const int64_t distinct_cnt) override { return OB_NOT_SUPPORTED; }
virtual int reserve_group_by_buf(const int64_t size) override { return OB_NOT_SUPPORTED; }
virtual int output_extra_group_by_result(const int64_t start, const int64_t count) override { return OB_NOT_SUPPORTED; }
INHERIT_TO_STRING_KV("ObAggCell", ObAggCell, K_(hash_func), K_(def_hash_value), KPC_(ndv_calculator));
static const int64_t LLC_BUCKET_BITS = 10; // same as ObAggregateProcessor llc bucket bits.
private:
virtual bool can_use_index_info() const override { return false; } // can not use now.
sql::ObExprHashFuncType hash_func_;
ObHyperLogLogCalculator *ndv_calculator_;
uint64_t def_hash_value_;
};
// For statistical information aggregation pushdown.
// Not support cross-partition aggregate, not support group by.
class ObSumOpSizeAggCell : public ObAggCell
{
public:
ObSumOpSizeAggCell(
const ObAggCellBasicInfo &basic_info,
common::ObIAllocator &allocator,
const bool exclude_null);
virtual ~ObSumOpSizeAggCell() { reset(); }
virtual void reset() override;
virtual void reuse() override;
virtual int init(const bool is_group_by, sql::ObEvalCtx *eval_ctx) override;
virtual int eval(blocksstable::ObStorageDatum &storage_datum, const int64_t row_count = 1) override;
virtual int eval_batch(const common::ObDatum *datums, const int64_t count) override;
virtual int eval_micro_block(
const ObTableIterParam &iter_param,
const ObTableAccessContext &context,
const int32_t col_offset,
blocksstable::ObIMicroBlockReader *reader,
const int64_t *row_ids,
const int64_t row_count) override;
virtual int eval_index_info(const blocksstable::ObMicroIndexInfo &index_info, const bool is_cg = false);
virtual int eval_batch_in_group_by(
const common::ObDatum *datums,
const int64_t count,
const uint32_t *refs,
const int64_t distinct_cnt,
const bool is_group_by_col = false,
const bool is_default_datum = false) override
{ return OB_NOT_SUPPORTED; }
virtual int copy_output_row(const int32_t datum_offset) override { return OB_NOT_SUPPORTED; }
virtual int copy_output_rows(const int32_t datum_offset) override { return OB_NOT_SUPPORTED; }
virtual int collect_result(sql::ObEvalCtx &ctx) override;
virtual int collect_batch_result_in_group_by(const int64_t distinct_cnt) override { return OB_NOT_SUPPORTED; }
virtual bool need_access_data() const override
{
ObObjDatumMapType type = basic_info_.agg_expr_->args_[0]->obj_datum_map_;
return type == OBJ_DATUM_STRING || type == OBJ_DATUM_NUMBER || type == OBJ_DATUM_DECIMALINT || exclude_null_;
}
virtual int reserve_group_by_buf(const int64_t size) override { return OB_NOT_SUPPORTED; }
virtual int output_extra_group_by_result(const int64_t start, const int64_t count) override { return OB_NOT_SUPPORTED; }
INHERIT_TO_STRING_KV("ObAggCell", ObAggCell, K_(total_size), K_(op_size), K_(def_op_size), K_(exclude_null));
private:
int set_op_size();
int get_datum_op_size(const ObDatum &datum, int64_t &length);
virtual bool can_use_index_info() const override { return is_fixed_length_type(); }
OB_INLINE bool is_valid_op_size() const { return op_size_ >= 0; }
OB_INLINE bool is_fixed_length_type() const
{
ObObjDatumMapType type = basic_info_.agg_expr_->args_[0]->obj_datum_map_;
return type != OBJ_DATUM_STRING && type != OBJ_DATUM_NUMBER && type != OBJ_DATUM_DECIMALINT;
}
int64_t op_size_;
int64_t def_op_size_;
uint64_t total_size_;
bool exclude_null_;
};
class ObSumAggCell : public ObAggCell
{
typedef int (ObSumAggCell::*ObSumEvalAggFuncType)(const common::ObDatum &datum, const int32_t datum_offset);
@ -432,7 +551,7 @@ public:
virtual int copy_output_row(const int32_t datum_offset) override;
virtual int copy_output_rows(const int32_t datum_offset) override;
virtual int copy_single_output_row(sql::ObEvalCtx &ctx) override;
virtual int collect_result(sql::ObEvalCtx &ctx, bool need_padding) override;
virtual int collect_result(sql::ObEvalCtx &ctx) override;
virtual int collect_batch_result_in_group_by(const int64_t distinct_cnt) override;
virtual int reserve_group_by_buf(const int64_t size) override;
virtual int output_extra_group_by_result(const int64_t start, const int64_t count) override;
@ -564,7 +683,7 @@ public:
UNUSED(ctx);
return OB_SUCCESS;
}
virtual int collect_result(sql::ObEvalCtx &ctx, bool need_padding) override;
virtual int collect_result(sql::ObEvalCtx &ctx) override;
virtual int collect_batch_result_in_group_by(const int64_t distinct_cnt) override;
virtual bool need_access_data() const override { return !finished(); }
virtual bool finished() const override { return aggregated_; }
@ -614,8 +733,8 @@ public:
~ObGroupByCell() { reset(); }
void reset();
void reuse();
int init(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ctx);
int init_for_single_row(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ctx);
int init(const ObTableAccessParam &param, const ObTableAccessContext &context, sql::ObEvalCtx &eval_ctx);
int init_for_single_row(const ObTableAccessParam &param, const ObTableAccessContext &context, sql::ObEvalCtx &eval_ctx);
// do group by for aggregate cell indicated by 'agg_idx'
// datums: batch of datums of this column
// count: batch size
@ -700,7 +819,7 @@ public:
const bool init_output = true);
DECLARE_TO_STRING;
private:
int init_agg_cells(const ObTableAccessParam &param, sql::ObEvalCtx &eval_ctx, const bool is_for_single_row);
int init_agg_cells(const ObTableAccessParam &param, const ObTableAccessContext &context, sql::ObEvalCtx &eval_ctx, const bool is_for_single_row);
static const int64_t DEFAULT_AGG_CELL_CNT = 2;
static const int64_t USE_GROUP_BY_READ_CNT_FACTOR = 2;
static constexpr double USE_GROUP_BY_DISTINCT_RATIO = 0.5;

View File

@ -156,7 +156,7 @@ int ObVectorStore::init(const ObTableAccessParam &param)
ret = common::OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Failed to alloc datum buf", K(ret));
} else if (FALSE_IT(group_by_cell_ = new (buf) ObGroupByCell(batch_size_, *context_.stmt_allocator_))) {
} else if (OB_FAIL(group_by_cell_->init(param, eval_ctx_))) {
} else if (OB_FAIL(group_by_cell_->init(param, context_, eval_ctx_))) {
LOG_WARN("Failed to init group by cell", K(ret));
}
}

View File

@ -966,6 +966,22 @@ public:
return not_over;
}
static OB_INLINE void get_datum_sign_and_size(const ObObjType obj_type,
const ObDatum &datum,
bool &is_signed,
uint64_t &value,
int64_t &value_size)
{
if (ob_obj_type_class(obj_type) == ObDecimalIntTC) {
value_size = sizeof(int64_t);
is_signed = true;
value = datum.len_ > sizeof(int32_t) ? datum.get_decimal_int64() : datum.get_decimal_int32();
} else {
is_signed = is_signed_object_type(obj_type);
value_size = get_type_size_map()[obj_type];
value = get_uint64_from_buf_by_len(datum.ptr_, value_size);
}
}
static OB_INLINE bool is_less_than(const uint64_t lval, const int64_t lval_size, const bool is_lval_signed,
const uint64_t rval, const int64_t rval_size, const bool is_rval_signed)
{

View File

@ -180,10 +180,9 @@ int ObDictColumnDecoder::get_null_count(
const uint32_t width_size = ctx.ref_ctx_->meta_.get_uint_width_size();
if (ctx.dict_meta_->is_const_encoding_ref()) {
ObConstEncodingRefDesc ref_desc(ctx.ref_data_, width_size);
if (0 == ref_desc.exception_cnt_) {
if (ref_desc.const_ref_ == ctx.dict_meta_->distinct_val_cnt_) {
null_count = row_cap;
}
if (OB_UNLIKELY(0 == ref_desc.exception_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("const encoding can not be all null", KR(ret), K(ref_desc));
} else if OB_FAIL(extract_ref_and_null_count_(ref_desc,
ctx.dict_meta_->distinct_val_cnt_, row_ids, row_cap, nullptr, null_count)) {
LOG_WARN("Failed to extrace null count", K(ret));
@ -1070,6 +1069,7 @@ int ObDictColumnDecoder::in_operator(
int ret = OB_SUCCESS;
int64_t datums_cnt = 0;
const uint64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
const bool is_sorted_dict = ctx.dict_meta_->is_sorted();
if (OB_UNLIKELY((datums_cnt = filter.get_datums().count()) < 1)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", KR(ret), K(datums_cnt));
@ -1078,38 +1078,13 @@ int ObDictColumnDecoder::in_operator(
} else {
const sql::ObWhiteFilterOperatorType op_type = filter.get_op_type();
sql::ObBitVector *ref_bitset = nullptr;
int64_t matched_ref_cnt = 0;
bool matched_ref_exist = false;
BUILD_REF_BITSET(ctx, (dict_val_cnt + 1), ref_bitset);
if (OB_FAIL(ret)) {
} else if (ctx.col_header_->is_integer_dict()) {
const ObIntegerStreamMeta &stream_meta = ctx.int_ctx_->meta_;
const uint64_t dict_val_base = stream_meta.is_use_base() * stream_meta.base_value();
bool is_col_signed = false;
if (ObCSDecodingUtil::can_convert_to_integer(ctx.col_header_->get_store_obj_type(), is_col_signed)) {
const int64_t MAX_DATUM_CNT = 200;
if (datums_cnt <= MAX_DATUM_CNT) {
bool filter_vals_valid[datums_cnt];
uint64_t filter_vals[datums_cnt];
integer_dict_val_in_op(ctx, filter, dict_val_base, dict_val_cnt, filter_vals_valid,
filter_vals, datums_cnt, ref_bitset, matched_ref_cnt);
} else {
bool *filter_vals_valid = static_cast<bool *>(ctx.allocator_->alloc(datums_cnt * (1 + sizeof(uint64_t))));
uint64_t *filter_vals = reinterpret_cast<uint64_t *>(filter_vals_valid + datums_cnt);
integer_dict_val_in_op(ctx, filter, dict_val_base, dict_val_cnt, filter_vals_valid,
filter_vals, datums_cnt, ref_bitset, matched_ref_cnt);
}
} else {
datum_dict_val_in_op(ctx, dict_val_cnt, filter, ref_bitset, matched_ref_cnt);
}
} else {
if (OB_FAIL(datum_dict_val_in_op(ctx, dict_val_cnt, filter, ref_bitset, matched_ref_cnt))) {
LOG_WARN("fail to exe datum_dict_val_in_op", KR(ret), K(dict_val_cnt));
}
}
if (OB_SUCC(ret) && (matched_ref_cnt > 0)) {
} else if (OB_FAIL(datum_dict_val_in_op(ctx, filter, is_sorted_dict, ref_bitset, matched_ref_exist))) {
LOG_WARN("Failed to exe datum_dict_val_in_op", KR(ret), K(dict_val_cnt));
} else if (matched_ref_exist) {
const uint32_t ref_width_size = ctx.ref_ctx_->meta_.get_uint_width_size();
if (OB_FAIL(set_bitmap_with_bitset(ref_width_size, ctx.ref_data_, ref_bitset,
pd_filter_info.start_, pd_filter_info.count_, false/*has_null*/, 0, parent, result_bitmap))) {
@ -1120,73 +1095,232 @@ int ObDictColumnDecoder::in_operator(
return ret;
}
void ObDictColumnDecoder::integer_dict_val_in_op(
int ObDictColumnDecoder::datum_dict_val_in_op(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
const uint64_t dict_val_base,
const uint64_t dict_val_cnt,
bool *filter_vals_valid,
uint64_t *filter_vals,
const int64_t datums_cnt,
const bool is_sorted_dict,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt)
bool &matched_ref_exist,
const bool is_const_result_set)
{
MEMSET(filter_vals_valid, 1, datums_cnt);
const sql::ObPushdownWhiteFilterNode &filter_node = filter.get_filter_node();
const uint32_t val_width_size =ctx.int_ctx_->meta_.get_uint_width_size();
const ObObjType col_type = ctx.col_header_->get_store_obj_type();
const bool is_col_signed = ObCSDecodingUtil::is_signed_object_type(col_type);
int ret = OB_SUCCESS;
const uint64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
ObDictValueIterator min_it = ObDictValueIterator(&ctx, 0);
ObDictValueIterator max_it = ObDictValueIterator(&ctx, dict_val_cnt > 0 ? dict_val_cnt - 1 : 0);
for (int64_t i = 0; i < datums_cnt; ++i) {
const ObObjType filter_type = filter_node.get_filter_arg_obj_type(i);
uint64_t filter_val = 0;
int64_t filter_val_size = 0;
bool is_filter_signed = false;
const common::ObDatum &filter_datum = filter.get_datums().at(i);
if (ObCSDecodingUtil::check_datum_not_over_8bytes(filter_type, filter_datum, is_filter_signed, filter_val, filter_val_size)) {
uint64_t filter_base_diff = 0;
if (ObCSDecodingUtil::is_less_than_with_diff(filter_val, filter_val_size, is_filter_signed, dict_val_base, 8,
is_col_signed, filter_base_diff)) {
filter_vals_valid[i] = false;
} else if (~INTEGER_MASK_TABLE[val_width_size] & filter_base_diff) {
filter_vals_valid[i] = false;
} else {
// filter_vals_valid[i] = ture;
filter_vals[i] = filter_val;
}
} else {
filter_vals_valid[i] = false;
bool hit_shortcut = false;
int cmp_ret = 0;
if (is_sorted_dict) {
if (OB_FAIL(filter.cmp_func_(*min_it, filter.get_max_param(), cmp_ret))) {
LOG_WARN("Failed to compare min dict value and max param", KR(ret), K(*min_it), K(filter.get_max_param()));
} else if (cmp_ret > 0) {
hit_shortcut = true;
} else if (OB_FAIL(filter.cmp_func_(*max_it, filter.get_min_param(), cmp_ret))) {
LOG_WARN("Failed to compare max dict value and min param", KR(ret), K(*max_it), K(filter.get_min_param()));
} else if (cmp_ret < 0) {
hit_shortcut = true;
}
}
ObCSFilterFunctionFactory::instance().dict_val_in_tranverse(ctx.int_data_, val_width_size,
dict_val_base, filter_vals_valid, filter_vals, datums_cnt, dict_val_cnt, matched_ref_cnt, ref_bitset);
if (OB_FAIL(ret)) {
} else if (hit_shortcut) {
if (is_const_result_set) {
matched_ref_exist = true;
ref_bitset->bit_not(dict_val_cnt + 1); // all 1
}
LOG_DEBUG("Hit shortcut to judge IN filter");
} else {
const ObFilterInCmpType cmp_type = get_filter_in_cmp_type(dict_val_cnt, filter.get_datums().count(), is_sorted_dict);
int64_t matched_ref_cnt = 0;
switch (cmp_type) {
case ObFilterInCmpType::MERGE_SEARCH: {
if (OB_FAIL(in_operator_merge_search(ctx, filter, ref_bitset,
matched_ref_cnt,
is_const_result_set))) {
LOG_WARN("Failed to merge search in IN operator", KR(ret));
}
break;
}
case ObFilterInCmpType::BINARY_SEARCH_DICT: {
if (OB_FAIL(in_operator_binary_search_dict(ctx, filter, ref_bitset,
matched_ref_cnt,
is_const_result_set))) {
LOG_WARN("Failed to binary search dict in IN operator", KR(ret));
}
break;
}
case ObFilterInCmpType::BINARY_SEARCH: {
if (OB_FAIL(in_operator_binary_search(ctx, filter, ref_bitset,
matched_ref_cnt,
is_const_result_set))) {
LOG_WARN("Failed to binary search in IN operator", KR(ret));
}
break;
}
case ObFilterInCmpType::HASH_SEARCH: {
if (OB_FAIL(in_operator_hash_search(ctx, filter, ref_bitset,
matched_ref_cnt,
is_const_result_set))) {
LOG_WARN("Failed to hash search in IN operator", KR(ret));
}
break;
}
default: {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected compare type", K(ret), K(cmp_type));
}
}
if (OB_SUCC(ret)) {
if (is_const_result_set) {
matched_ref_exist = matched_ref_cnt < dict_val_cnt;
ref_bitset->bit_not(dict_val_cnt + 1);
} else {
matched_ref_exist = matched_ref_cnt > 0;
}
}
}
return ret;
}
int ObDictColumnDecoder::datum_dict_val_in_op(
int ObDictColumnDecoder::in_operator_merge_search(
const ObDictColumnDecoderCtx &ctx,
const uint64_t dict_val_cnt,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt)
int64_t &matched_ref_cnt,
const bool is_const_result_set)
{
int ret = OB_SUCCESS;
const uint64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
ObDictValueIterator begin_it = ObDictValueIterator(&ctx, 0);
ObDictValueIterator end_it = ObDictValueIterator(&ctx, dict_val_cnt);
int64_t dict_ref = 0;
while (OB_SUCC(ret) && (begin_it != end_it)) {
ObObj cur_obj;
bool is_exist = false;
if (OB_FAIL((*begin_it).to_obj(cur_obj, ctx.obj_meta_))) {
LOG_WARN("fail to convert datum to obj", KR(ret), K(*begin_it));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, is_exist))) {
LOG_WARN("fail to check object in hashset", KR(ret), K(*begin_it));
} else if (is_exist) {
ref_bitset->set(dict_ref);
++matched_ref_cnt;
ObDictValueIterator trav_it = begin_it;
const ObFixedArray<ObDatum, ObIAllocator> *params
= static_cast<const ObFixedArray<ObDatum, ObIAllocator> *>(&(filter.get_datums()));
array::Iterator<ObFixedArrayImpl<ObDatum, ObIAllocator>, const ObDatum> param_it = params->begin();
int cmp_ret = 0;
bool equal = false;
ObDatumComparator cmp(filter.cmp_func_, ret, equal);
ObDatumComparator cmp_rev(filter.cmp_func_rev_, ret, equal);
while (OB_SUCC(ret) && trav_it != end_it && param_it != params->end()) {
const ObDatum dict_datum = *trav_it;
const ObDatum param_datum = *param_it;
if (equal) {
cmp_ret = 0;
} else if (OB_FAIL(filter.cmp_func_(dict_datum, param_datum, cmp_ret))) {
LOG_WARN("Failed to compare dict and param datum", K(ret));
}
++begin_it;
++dict_ref;
if (OB_SUCC(ret)) {
equal = false;
if (cmp_ret == 0) {
++matched_ref_cnt;
ref_bitset->set(trav_it - begin_it);
// The values in the dictionary and array are unique.
++trav_it;
++param_it;
} else if (cmp_ret > 0) {
++param_it;
param_it = std::lower_bound(param_it, params->end(), dict_datum, cmp_rev);
} else {
++trav_it;
trav_it = std::lower_bound(trav_it, end_it, param_datum, cmp);
}
if (OB_FAIL(ret)) {
LOG_WARN("Failed to find next compare positions", K(ret));
}
}
}
return ret;
}
int ObDictColumnDecoder::in_operator_binary_search_dict(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt,
const bool is_const_result_set)
{
int ret = OB_SUCCESS;
const uint64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
ObDictValueIterator begin_it = ObDictValueIterator(&ctx, 0);
ObDictValueIterator end_it = ObDictValueIterator(&ctx, dict_val_cnt);
ObDictValueIterator trav_it = begin_it;
const ObFixedArray<ObDatum, ObIAllocator> *params
= static_cast<const ObFixedArray<ObDatum, ObIAllocator> *>(&(filter.get_datums()));
array::Iterator<ObFixedArrayImpl<ObDatum, ObIAllocator>, const ObDatum> param_it = params->begin();
bool is_exist = false;
ObDatumComparator cmp(filter.cmp_func_, ret, is_exist);
while (OB_SUCC(ret) && trav_it != end_it && param_it != params->end()) {
const ObDatum param_datum = *param_it;
is_exist = false;
trav_it = std::lower_bound(trav_it, end_it, param_datum, cmp);
if (OB_FAIL(ret)) {
LOG_WARN("Failed to get lower_bound in dict", K(ret), K(param_datum));
} else if (trav_it == end_it) {
} else if (is_exist) {
++matched_ref_cnt;
ref_bitset->set(trav_it - begin_it);
++trav_it;
}
++param_it;
}
return ret;
}
int ObDictColumnDecoder::in_operator_binary_search(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt,
const bool is_const_result_set)
{
int ret = OB_SUCCESS;
const uint64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
ObDictValueIterator begin_it = ObDictValueIterator(&ctx, 0);
ObDictValueIterator end_it = ObDictValueIterator(&ctx, dict_val_cnt);
ObDictValueIterator trav_it = begin_it;
const ObFixedArray<ObDatum, ObIAllocator> *params
= static_cast<const ObFixedArray<ObDatum, ObIAllocator> *>(&(filter.get_datums()));
array::Iterator<ObFixedArrayImpl<ObDatum, ObIAllocator>, const ObDatum> param_it = params->begin();
bool is_exist = false;
while (OB_SUCC(ret) && trav_it != end_it && param_it != params->end()) {
const ObDatum dict_datum = *trav_it;
if (OB_FAIL(filter.exist_in_datum_array(dict_datum, is_exist, param_it - params->begin()))) {
LOG_WARN("Failed to check dict datum in param array", K(ret), K(dict_datum));
} else if (is_exist) {
++matched_ref_cnt;
ref_bitset->set(trav_it - begin_it);
++param_it;
}
++trav_it;
}
return ret;
}
int ObDictColumnDecoder::in_operator_hash_search(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt,
const bool is_const_result_set)
{
int ret = OB_SUCCESS;
const uint64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
ObDictValueIterator begin_it = ObDictValueIterator(&ctx, 0);
ObDictValueIterator end_it = ObDictValueIterator(&ctx, dict_val_cnt);
ObDictValueIterator trav_it = begin_it;
const ObFixedArray<ObDatum, ObIAllocator> *params
= static_cast<const ObFixedArray<ObDatum, ObIAllocator> *>(&(filter.get_datums()));
array::Iterator<ObFixedArrayImpl<ObDatum, ObIAllocator>, const ObDatum> param_it = params->begin();
bool is_exist = false;
while (OB_SUCC(ret) && trav_it != end_it && param_it != params->end()) {
const ObDatum dict_datum = *trav_it;
if (OB_FAIL(filter.exist_in_datum_set(dict_datum, is_exist))) {
LOG_WARN("Failed to check dict datum in param set", K(ret), K(dict_datum));
} else if (is_exist) {
++matched_ref_cnt;
ref_bitset->set(trav_it - begin_it);
}
++trav_it;
}
return ret;
}
@ -1487,12 +1621,8 @@ int ObDictColumnDecoder::do_const_only_operator(
} else if (1 == const_ref_desc.const_ref_) {
} else {
bool is_existed = false;
// Check const object in hashset or not
ObObj const_obj;
if (OB_FAIL(const_datum.to_obj(const_obj, ctx.obj_meta_))) {
LOG_WARN("fail to convert datum to obj", KR(ret), K(const_datum), K(ctx.obj_meta_));
} else if (OB_FAIL(filter.exist_in_obj_set(const_obj, is_existed))) {
LOG_WARN("fail to check object in hashset", KR(ret), K(const_obj));
if (OB_FAIL(filter.exist_in_datum_set(const_datum, is_existed))) {
LOG_WARN("fail to check object in hashset", KR(ret), K(const_datum));
} else if (is_existed) {
if (OB_FAIL(result_bitmap.bit_not())) {
LOG_WARN("fail to execute bit_not", KR(ret));
@ -1726,6 +1856,7 @@ int ObDictColumnDecoder::in_const_operator(
int ret = OB_SUCCESS;
const int64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
const bool has_null = ctx.dict_meta_->has_null();
const bool is_sorted_dict = ctx.dict_meta_->is_sorted();
bool is_const_result_set = false;
if (const_ref_desc.const_ref_ == dict_val_cnt) {
@ -1733,11 +1864,8 @@ int ObDictColumnDecoder::in_const_operator(
} else {
ObDictValueIterator dict_iter = ObDictValueIterator(&ctx, 0);
ObStorageDatum &const_datum = *(dict_iter + const_ref_desc.const_ref_);
ObObj const_obj;
if (OB_FAIL(const_datum.to_obj(const_obj, ctx.obj_meta_))) {
LOG_WARN("fail to convert datum to obj", KR(ret), K(const_datum), K(ctx));
} else if (OB_FAIL(filter.exist_in_obj_set(const_obj, is_const_result_set))) {
LOG_WARN("fail to check whether const value is in set", KR(ret), K(const_obj));
if (OB_FAIL(filter.exist_in_datum_set(const_datum, is_const_result_set))) {
LOG_WARN("fail to check whether const value is in set", KR(ret), K(const_datum));
} else if (is_const_result_set) {
if (OB_FAIL(result_bitmap.bit_not())) {
LOG_WARN("fail to execute bit_not", KR(ret));
@ -1745,36 +1873,19 @@ int ObDictColumnDecoder::in_const_operator(
}
}
const bool need_filter_null = is_const_result_set && has_null;
bool matched_ref_exist = false;
if (OB_SUCC(ret)) {
sql::ObBitVector *ref_bitset = nullptr;
BUILD_REF_BITSET(ctx, (dict_val_cnt + 1), ref_bitset);
bool exist_matched_ref = false;
ObDictValueIterator mv_iter = ObDictValueIterator(&ctx, 0);
ObDictValueIterator end_iter = ObDictValueIterator(&ctx, dict_val_cnt);
int64_t tmp_idx = 0;
while (OB_SUCC(ret) && (mv_iter != end_iter)) {
ObObj cur_obj;
bool cur_is_result_set = false;
if (OB_FAIL((*mv_iter).to_obj(cur_obj, ctx.obj_meta_))) {
LOG_WARN("fail to convert datum to obj", KR(ret), K((*mv_iter)), K(ctx.obj_meta_));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, cur_is_result_set))) {
LOG_WARN("fail to check whether const value is in set", KR(ret), K(cur_obj));
} else if ((!is_const_result_set) == cur_is_result_set) {
exist_matched_ref = true;
ref_bitset->set(tmp_idx);
}
++tmp_idx;
++mv_iter;
}
const bool need_filter_null = is_const_result_set && has_null;
if (OB_FAIL(ret)) {
} else if ((exist_matched_ref || need_filter_null) && OB_FAIL(set_bitmap_with_bitset_const(
} else if (OB_FAIL(datum_dict_val_in_op(ctx, filter, is_sorted_dict, ref_bitset, matched_ref_exist, is_const_result_set))) {
LOG_WARN("Failed to to exe datum_dict_val_in_op", KR(ret));
} else if ((matched_ref_exist || need_filter_null) && OB_FAIL(set_bitmap_with_bitset_const(
const_ref_desc.width_size_, const_ref_desc.exception_row_id_buf_, const_ref_desc.exception_ref_buf_,
ref_bitset, const_ref_desc.exception_cnt_, pd_filter_info.start_, pd_filter_info.count_,
need_filter_null, dict_val_cnt, result_bitmap, !is_const_result_set))) {
LOG_WARN("fail to set bitmap with bitset", KR(ret), K(const_ref_desc), K(exist_matched_ref),
LOG_WARN("fail to set bitmap with bitset", KR(ret), K(const_ref_desc), K(matched_ref_exist),
K(need_filter_null), K(dict_val_cnt), K(pd_filter_info));
}
}
@ -2102,5 +2213,66 @@ int ObDictColumnDecoder::read_reference(
return ret;
}
int ObDictColumnDecoder::get_aggregate_result(
const ObColumnCSDecoderCtx &col_ctx,
const int64_t *row_ids,
const int64_t row_cap,
storage::ObAggCell &agg_cell) const
{
int ret = OB_SUCCESS;
const ObDictColumnDecoderCtx &dict_ctx = col_ctx.dict_ctx_;
const bool is_const_encoding = dict_ctx.dict_meta_->is_const_encoding_ref();
const uint64_t dict_val_cnt = dict_ctx.dict_meta_->distinct_val_cnt_;
if (OB_UNLIKELY(nullptr == row_ids || row_cap <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_DEBUG("Invalid arguments to get aggregate result", KR(ret), KP(row_ids), K(row_cap));
} else {
bool all_null = false;
if (dict_val_cnt == 0) {
// skip if all null
} else if (row_cap == dict_ctx.micro_block_header_->row_count_) { // cover whole microblock
if (dict_ctx.dict_meta_->is_sorted() && (agg_cell.is_min_agg() || agg_cell.is_max_agg())) { // int dict must be sorted
ObDictValueIterator res_iter = ObDictValueIterator(&dict_ctx, agg_cell.is_min_agg() ? 0 : dict_val_cnt - 1);
if (OB_FAIL(agg_cell.eval(*res_iter))) {
LOG_WARN("Failed to eval agg cell", KR(ret), K(*res_iter), K(agg_cell));
}
} else if (OB_FAIL(traverse_datum_dict_agg(dict_ctx, agg_cell))) {
LOG_WARN("Failed to traverse datum dict to aggregate", KR(ret), K(dict_ctx));
}
} else { // cover partial microblock
int64_t row_id = 0;
ObStorageDatum storage_datum;
if (OB_FAIL(agg_cell.reserve_bitmap(dict_val_cnt))) {
LOG_WARN("Failed to reserve memory for bitmap", KR(ret), K(row_cap));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < row_cap; ++i) {
row_id = row_ids[i];
if (OB_FAIL(decode_and_aggregate(col_ctx, row_id, storage_datum, agg_cell))) {
LOG_WARN("Failed to decode", KR(ret), K(row_id));
}
}
}
}
}
return ret;
}
int ObDictColumnDecoder::traverse_datum_dict_agg(
const ObDictColumnDecoderCtx &ctx,
storage::ObAggCell &agg_cell)
{
int ret = OB_SUCCESS;
const uint64_t dict_val_cnt = ctx.dict_meta_->distinct_val_cnt_;
ObDictValueIterator mv_iter = ObDictValueIterator(&ctx, 0);
ObDictValueIterator end_iter = ObDictValueIterator(&ctx, dict_val_cnt);
while (OB_SUCC(ret) && mv_iter != end_iter) {
if (OB_FAIL(agg_cell.eval(*mv_iter))) {
LOG_WARN("Failed to eval agg cell", KR(ret), K(*mv_iter), K(agg_cell));
}
++mv_iter;
}
return ret;
}
} // namespace blocksstable
} // namespace oceanbase

View File

@ -48,6 +48,12 @@ public:
ObBitmap &result_bitmap,
bool &filter_applied) const override;
virtual int get_aggregate_result(
const ObColumnCSDecoderCtx &col_ctx,
const int64_t *row_ids,
const int64_t row_cap,
storage::ObAggCell &agg_cell) const override;
bool fast_decode_valid(const ObColumnCSDecoderCtx &ctx) const;
virtual int get_distinct_count(const ObColumnCSDecoderCtx &ctx, int64_t &distinct_count) const override;
@ -92,6 +98,15 @@ public:
protected:
const static int64_t MAX_STACK_BUF_SIZE = 4 << 10; // 4K
virtual int decode_and_aggregate(
const ObColumnCSDecoderCtx &ctx,
const int64_t row_id,
ObStorageDatum &datum,
storage::ObAggCell &agg_cell) const
{
UNUSEDx(ctx, row_id, datum, agg_cell);
return OB_NOT_SUPPORTED;
}
static int extract_ref_and_null_count_(
const ObConstEncodingRefDesc &ref_desc,
const int64_t dict_count,
@ -175,23 +190,41 @@ protected:
const sql::PushdownFilterInfo &pd_filter_info,
common::ObBitmap &result_bitmap);
static void integer_dict_val_in_op(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
const uint64_t dict_val_base,
const uint64_t dict_val_cnt,
bool *filter_vals_valid,
uint64_t *filter_vals,
const int64_t datums_cnt,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt);
static int datum_dict_val_in_op(
const ObDictColumnDecoderCtx &ctx,
const uint64_t dict_val_cnt,
const sql::ObWhiteFilterExecutor &filter,
const bool is_sorted_dict,
sql::ObBitVector *ref_bitset,
bool &matched_ref_exist,
const bool is_const_result_set = false);
static int in_operator_merge_search(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt);
int64_t &matched_ref_cnt,
const bool is_const_result_set);
static int in_operator_binary_search_dict(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt,
const bool is_const_result_set);
static int in_operator_binary_search(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt,
const bool is_const_result_set);
static int in_operator_hash_search(
const ObDictColumnDecoderCtx &ctx,
const sql::ObWhiteFilterExecutor &filter,
sql::ObBitVector *ref_bitset,
int64_t &matched_ref_cnt,
const bool is_const_result_set);
static int bt_operator(
const ObDictColumnDecoderCtx &ctx,
@ -241,6 +274,10 @@ protected:
const sql::PushdownFilterInfo &pd_filter_info,
ObBitmap &result_bitmap);
static int traverse_datum_dict_agg(
const ObDictColumnDecoderCtx &ctx,
storage::ObAggCell &agg_cell);
static int cmp_ref_and_set_result(const uint32_t ref_width_size, const char *ref_buf,
const int64_t dict_ref, const bool has_null, const uint64_t null_replaced_val,
const sql::ObWhiteFilterOperatorType &op_type, const int64_t row_start,

View File

@ -27,6 +27,7 @@ namespace oceanbase
{
namespace storage
{
class ObAggCell;
class ObGroupByCell;
}
namespace blocksstable
@ -90,6 +91,16 @@ public:
return common::OB_NOT_SUPPORTED;
}
virtual int get_aggregate_result(
const ObColumnCSDecoderCtx &ctx,
const int64_t *row_ids,
const int64_t row_cap,
storage::ObAggCell &agg_cell) const
{
UNUSEDx(ctx, row_ids, row_cap, agg_cell);
return common::OB_NOT_SUPPORTED;
}
virtual int get_null_count(
const ObColumnCSDecoderCtx &ctx,
const int64_t *row_ids,

View File

@ -16,6 +16,7 @@
#include "ob_integer_stream_decoder.h"
#include "ob_integer_stream_vector_decoder.h"
#include "ob_cs_vector_decoding_util.h"
#include "storage/access/ob_pushdown_aggregate.h"
namespace oceanbase
{
@ -48,7 +49,47 @@ int ObIntDictColumnDecoder::decode(
*dict_ctx.int_ctx_, nullptr/*ref_data*/, nullptr/*row_ids*/, 1, &datum);
}
}
return ret;
}
int ObIntDictColumnDecoder::decode_and_aggregate(
const ObColumnCSDecoderCtx &ctx,
const int64_t row_id,
ObStorageDatum &datum,
storage::ObAggCell &agg_cell) const
{
int ret = OB_SUCCESS;
const ObDictColumnDecoderCtx &dict_ctx = ctx.dict_ctx_;
const uint64_t distinct_cnt = dict_ctx.dict_meta_->distinct_val_cnt_;
if (OB_UNLIKELY(0 == distinct_cnt)) {
datum.set_null(); // empty dict, all datum is null
} else {
if (dict_ctx.dict_meta_->is_const_encoding_ref()) {
GET_CONST_ENCODING_REF(dict_ctx.ref_ctx_->meta_.width_, dict_ctx.ref_data_, row_id, datum.pack_);
} else {
GET_REF_FROM_REF_ARRAY(dict_ctx.ref_ctx_->meta_.width_, dict_ctx.ref_data_, row_id, datum.pack_);
}
ObBitmap &bitmap = agg_cell.get_bitmap();
if (datum.pack_ == distinct_cnt) {
datum.set_null();
} else if (bitmap.test(datum.pack_)) {
// has been evaluated.
} else if (OB_FAIL(bitmap.set(datum.pack_))) {
LOG_WARN("Failed to set bitmap", KR(ret), K(datum.pack_));
} else {
ConvertUnitToDatumFunc convert_func = convert_uint_to_datum_funcs
[dict_ctx.int_ctx_->meta_.width_] /*val_store_width_V*/
[ObRefStoreWidthV::REF_IN_DATUMS] /*ref_store_width_V*/
[get_width_tag_map()[dict_ctx.datum_len_]] /*datum_width_V*/
[ObBaseColumnDecoderCtx::ObNullFlag::HAS_NO_NULL] /*null has been processed, so here set HAS_NO_NULL*/
[dict_ctx.int_ctx_->meta_.is_decimal_int()];
convert_func(dict_ctx, dict_ctx.int_data_,
*dict_ctx.int_ctx_, nullptr/*ref_data*/, nullptr/*row_ids*/, 1, &datum);
if (!datum.is_null() && OB_FAIL(agg_cell.eval(datum))) {
LOG_WARN("Failed to eval agg cell", KR(ret), K(datum), K(agg_cell));
}
}
}
return ret;
}

View File

@ -34,6 +34,11 @@ public:
virtual int batch_decode(const ObColumnCSDecoderCtx &ctx, const int64_t *row_ids,
const int64_t row_cap, common::ObDatum *datums) const override;
virtual int decode_vector(const ObColumnCSDecoderCtx &ctx, ObVectorDecodeCtx &vector_ctx) const override;
virtual int decode_and_aggregate(
const ObColumnCSDecoderCtx &ctx,
const int64_t row_id,
ObStorageDatum &datum,
storage::ObAggCell &agg_cell) const override;
virtual ObCSColumnHeader::Type get_type() const override { return type_; }
};

View File

@ -15,6 +15,7 @@
#include "ob_integer_stream_decoder.h"
#include "ob_integer_stream_vector_decoder.h"
#include "ob_cs_decoding_util.h"
#include "storage/access/ob_pushdown_aggregate.h"
#include "storage/blocksstable/encoding/ob_raw_decoder.h"
namespace oceanbase
@ -241,7 +242,8 @@ int ObIntegerColumnDecoder::comparison_operator(
} else {
ObDatumCmpFuncType type_cmp_func = filter.cmp_func_;
ObGetFilterCmpRetFunc get_cmp_ret = get_filter_cmp_ret_func(op_type);
auto eval = [&] (const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)
ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> eval =
[&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
int cmp_ret = 0;
@ -389,7 +391,8 @@ int ObIntegerColumnDecoder::between_operator(
ObDatumCmpFuncType type_cmp_func = filter.cmp_func_;
ObGetFilterCmpRetFunc get_le_cmp_ret = get_filter_cmp_ret_func(sql::WHITE_OP_LE);
ObGetFilterCmpRetFunc get_ge_cmp_ret = get_filter_cmp_ret_func(sql::WHITE_OP_GE);
auto eval = [&] (const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)
ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> eval =
[&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
int ge_ret = 0;
@ -520,23 +523,42 @@ int ObIntegerColumnDecoder::in_operator(
}
}
} else {
auto eval = [&] (const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
ObObj cur_obj;
bool is_exist = false;
if (OB_TMP_FAIL(cur_datum.to_obj(cur_obj, obj_meta))) {
LOG_WARN("fail to convert datum to obj", KR(tmp_ret), K(cur_datum), K(obj_meta));
} else if (OB_TMP_FAIL(filter.exist_in_obj_set(cur_obj, is_exist))) {
LOG_WARN("fail to check obj in hashset", KR(tmp_ret), K(cur_obj));
} else if (is_exist) {
if (OB_TMP_FAIL(result_bitmap.set(idx))) {
LOG_WARN("fail to set result bitmap", KR(tmp_ret), K(idx));
ObFilterInCmpType cmp_type = get_filter_in_cmp_type(pd_filter_info.count_, filter.get_datums().count(), false);
ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> eval;
if (cmp_type == ObFilterInCmpType::BINARY_SEARCH) {
eval = [&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
bool is_exist = false;
if (OB_TMP_FAIL(filter.exist_in_datum_array(cur_datum, is_exist))) {
LOG_WARN("fail to check datum in array", KR(tmp_ret), K(cur_datum));
} else if (is_exist) {
if (OB_TMP_FAIL(result_bitmap.set(idx))) {
LOG_WARN("fail to set result bitmap", KR(tmp_ret), K(idx));
}
}
}
return tmp_ret;
};
if (OB_FAIL(tranverse_datum_all_op(ctx, pd_filter_info, result_bitmap, eval))) {
return tmp_ret;
};
} else if (cmp_type == ObFilterInCmpType::HASH_SEARCH) {
eval = [&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
bool is_exist = false;
if (OB_TMP_FAIL(filter.exist_in_datum_set(cur_datum, is_exist))) {
LOG_WARN("fail to check datum in hashset", KR(tmp_ret), K(cur_datum));
} else if (is_exist) {
if (OB_TMP_FAIL(result_bitmap.set(idx))) {
LOG_WARN("fail to set result bitmap", KR(tmp_ret), K(idx));
}
}
return tmp_ret;
};
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter in compare type", KR(ret), K(cmp_type));
}
if (OB_SUCC(ret) && OB_FAIL(tranverse_datum_all_op(ctx, pd_filter_info, result_bitmap, eval))) {
LOG_WARN("fail to tranverse datum with base in in_op", KR(ret), K(ctx));
}
}
@ -641,7 +663,7 @@ int ObIntegerColumnDecoder::tranverse_datum_all_op(
} else {
cur_datum.pack_ = ctx.datum_len_;
cur_datum.ptr_ = reinterpret_cast<const char*>(&cur_datum_val);
if (OB_FAIL(eval(ctx.obj_meta_, cur_datum, i))) {
if (OB_FAIL(eval(cur_datum, i))) {
LOG_WARN("fail to exe eval", KR(ret), K(i), K(cur_datum));
}
}
@ -660,7 +682,7 @@ int ObIntegerColumnDecoder::tranverse_datum_all_op(
ENCODING_ADAPT_MEMCPY(&cur_datum_val, ctx.data_ + row_id * store_width_size, store_width_size);
cur_datum_val += base;
cur_datum.ptr_ = reinterpret_cast<const char*>(&cur_datum_val);
if (OB_FAIL(eval(ctx.obj_meta_, cur_datum, i))) {
if (OB_FAIL(eval(cur_datum, i))) {
LOG_WARN("fail to exe eval", KR(ret), K(i), K(cur_datum));
}
}
@ -673,7 +695,7 @@ int ObIntegerColumnDecoder::tranverse_datum_all_op(
ENCODING_ADAPT_MEMCPY(&cur_datum_val, ctx.data_ + row_id * store_width_size, store_width_size);
cur_datum_val += base;
cur_datum.ptr_ = reinterpret_cast<const char*>(&cur_datum_val);
if (OB_FAIL(eval(ctx.obj_meta_, cur_datum, i))) {
if (OB_FAIL(eval(cur_datum, i))) {
LOG_WARN("fail to exe eval", KR(ret), K(i), K(cur_datum));
}
}
@ -682,5 +704,130 @@ int ObIntegerColumnDecoder::tranverse_datum_all_op(
return ret;
}
int ObIntegerColumnDecoder::get_aggregate_result(
const ObColumnCSDecoderCtx &ctx,
const int64_t *row_ids,
const int64_t row_cap,
storage::ObAggCell &agg_cell) const
{
int ret = OB_SUCCESS;
const ObIntegerColumnDecoderCtx &integer_ctx = ctx.integer_ctx_;
bool is_col_signed = false;
const ObObjType store_col_type = integer_ctx.col_header_->get_store_obj_type();
const bool can_convert = ObCSDecodingUtil::can_convert_to_integer(store_col_type, is_col_signed);
if (OB_UNLIKELY(nullptr == row_ids || row_cap <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid arguments to get aggregate result", KR(ret), KP(row_ids), K(row_cap));
} else {
const bool is_reverse = row_cap > 1 && row_ids[2] < row_ids[1];
int64_t row_id_start = is_reverse ? row_ids[row_cap - 1] : row_ids[0];
if (integer_ctx.has_null_bitmap()) {
if (OB_FAIL(agg_cell.reserve_bitmap(row_cap))) {
LOG_WARN("Failed to reserve memory for null bitmap", KR(ret));
} else {
ObBitmap &null_bitmap = agg_cell.get_bitmap();
for (int64_t i = 0; OB_SUCC(ret) && i < row_cap; ++i) {
const int64_t row_id = is_reverse ? row_ids[row_cap - 1 - i] : row_ids[i];
if (ObCSDecodingUtil::test_bit(integer_ctx.null_bitmap_, row_id) &&
OB_FAIL(null_bitmap.set(i))) {
LOG_WARN("Fail to set null bitmap", KR(ret), K(i), K(row_id));
}
}
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(traverse_integer_in_agg(integer_ctx, is_col_signed, row_id_start, row_cap, agg_cell))){
LOG_WARN("Failed to traverse integer to aggregate", KR(ret), K(integer_ctx), K(is_col_signed));
}
}
return ret;
}
int ObIntegerColumnDecoder::traverse_integer_in_agg(
const ObIntegerColumnDecoderCtx &ctx,
const bool is_col_signed,
const int64_t row_start,
const int64_t row_count,
storage::ObAggCell &agg_cell)
{
int ret = OB_SUCCESS;
const bool use_null_replace_val = ctx.is_null_replaced();
const bool exist_null_bitmap = ctx.has_null_bitmap();
const ObIntegerStreamMeta stream_meta = ctx.ctx_->meta_;
const uint64_t base_value = stream_meta.is_use_base() * stream_meta.base_value_;
const uint32_t store_width_tag = stream_meta.get_width_tag();
bool is_less_than_base = false;
bool is_exceed_range = false;
if (!agg_cell.get_result_datum().is_null()) {
const uint32_t store_width_size = stream_meta.get_uint_width_size();
uint64_t agg_base_diff = 0;
uint64_t agg_val = 0;
int64_t agg_val_size = 0;
bool is_agg_signed = false;
ObCSDecodingUtil::get_datum_sign_and_size(agg_cell.get_obj_type(), agg_cell.get_result_datum(),
is_agg_signed, agg_val, agg_val_size);
is_less_than_base = ObCSDecodingUtil::is_less_than_with_diff(
agg_val, agg_val_size, is_agg_signed,
base_value, 8, is_col_signed, agg_base_diff);
is_exceed_range = ~INTEGER_MASK_TABLE[store_width_size] & agg_base_diff;
}
if ((is_less_than_base && agg_cell.is_min_agg()) ||
(is_exceed_range && agg_cell.is_max_agg())) {
// if agg_val less than base, no need to update min
// if agg_val larger than RANGE_MAX_VALUE, no need to update max
} else {
uint64_t result = 0;
bool result_is_null = false;
if (use_null_replace_val) {
const uint64_t null_replaced_val_base_diff = ctx.null_replaced_value_ - base_value;
raw_min_max_function_with_null min_max_func =
RawAggFunctionFactory::instance().get_cs_min_max_function_with_null(store_width_tag, agg_cell.is_min_agg());
if (OB_ISNULL(min_max_func)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null min_max function", KR(ret), K(store_width_tag), K(agg_cell));
} else {
min_max_func(reinterpret_cast<const unsigned char *>(ctx.data_), null_replaced_val_base_diff,
row_start, row_start + row_count, result);
result_is_null = result == null_replaced_val_base_diff;
}
} else if (exist_null_bitmap) {
ObBitmap &null_bitmap = agg_cell.get_bitmap();
if (null_bitmap.is_all_true()) {
result_is_null = true;
} else {
raw_min_max_function_with_null_bitmap min_max_func =
RawAggFunctionFactory::instance().get_cs_min_max_function_with_null_bitmap(store_width_tag, agg_cell.is_min_agg());
if (OB_ISNULL(min_max_func)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null min_max function", KR(ret), K(store_width_tag), K(agg_cell));
} else {
min_max_func(reinterpret_cast<const unsigned char *>(ctx.data_), agg_cell.get_bitmap().get_data(),
row_start, row_start + row_count, result);
}
}
} else {
raw_min_max_function min_max_func = RawAggFunctionFactory::instance().get_min_max_function(
store_width_tag, agg_cell.is_min_agg());
if (OB_ISNULL(min_max_func)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected null min_max function", KR(ret), K(store_width_tag), K(agg_cell));
} else {
min_max_func(reinterpret_cast<const unsigned char *>(ctx.data_), row_start, row_start + row_count, result);
}
}
if (OB_SUCC(ret) && !result_is_null) {
result += base_value;
ObStorageDatum storage_datum;
storage_datum.set_uint(result);
if (OB_FAIL(agg_cell.eval(storage_datum))) {
LOG_WARN("Failed to eval agg_cell", KR(ret), K(storage_datum), K(agg_cell));
}
}
}
return ret;
}
}
}

View File

@ -47,6 +47,12 @@ public:
const sql::PushdownFilterInfo &pd_filter_info,
common::ObBitmap &result_bitmap) const override;
virtual int get_aggregate_result(
const ObColumnCSDecoderCtx &ctx,
const int64_t *row_ids,
const int64_t row_cap,
storage::ObAggCell &agg_cell) const override;
private:
static int nu_nn_operator(const ObIntegerColumnDecoderCtx &ctx,
const sql::ObPushdownFilterExecutor *parent,
@ -102,6 +108,12 @@ private:
common::ObBitmap &result_bitmap,
Operator const &eval);
static int traverse_integer_in_agg(const ObIntegerColumnDecoderCtx &ctx,
const bool is_col_signed,
const int64_t row_start,
const int64_t row_count,
storage::ObAggCell &agg_cell);
};
} // end namespace blocksstable

View File

@ -13,6 +13,7 @@
#define USING_LOG_PREFIX STORAGE
#include "ob_micro_block_cs_decoder.h"
#include "common/sql_mode/ob_sql_mode_utils.h"
#include "lib/container/ob_array_iterator.h"
#include "ob_dict_column_decoder.h"
#include "ob_integer_column_decoder.h"
@ -1436,8 +1437,7 @@ int ObMicroBlockCSDecoder::filter_pushdown_filter(
col_cs_decoder->ctx_->get_base_ctx().set_col_param(col_params.at(0));
if (filter.null_param_contained()
&& (op_type != sql::WHITE_OP_NU)
&& (op_type != sql::WHITE_OP_NN)
&& (op_type != sql::WHITE_OP_IN)) {
&& (op_type != sql::WHITE_OP_NN)) {
} else if (!transform_helper_.get_micro_block_header()->all_lob_in_row_) {
// In the column store scenario, the pushdown row range is split by row store.
// This means that the row store has determined that there is no out_row lob,
@ -1512,7 +1512,7 @@ int ObMicroBlockCSDecoder::filter_pushdown_retro(const sql::ObPushdownFilterExec
bool filtered = false;
if (OB_FAIL(ret)) {
} else if (OB_FAIL(filter_white_filter(filter, obj_meta, decoded_datum, filtered))) {
} else if (OB_FAIL(filter_white_filter(filter, decoded_datum, filtered))) {
LOG_WARN("Failed to filter row with white filter", K(ret), K(row_id), K(decoded_datum));
} else if (!filtered && OB_FAIL(result_bitmap.set(offset))) {
LOG_WARN("Failed to set result bitmap", K(ret), K(offset));
@ -1642,6 +1642,38 @@ int ObMicroBlockCSDecoder::get_column_datum(
return ret;
}
bool ObMicroBlockCSDecoder::can_pushdown_decoder(
const ObColumnCSDecoderCtx &ctx,
const int64_t *row_ids,
const int64_t row_cap,
const ObAggCell &agg_cell) const
{
bool bret = false;
const ObPDAggType agg_type = agg_cell.get_type();
switch (ctx.type_) {
case ObCSColumnHeader::INTEGER : {
const ObIntegerColumnDecoderCtx &integer_ctx = ctx.integer_ctx_;
bool is_col_signed = false;
const ObObjType store_col_type = integer_ctx.col_header_->get_store_obj_type();
const bool can_convert = ObCSDecodingUtil::can_convert_to_integer(store_col_type, is_col_signed);
const int64_t row_gap = std::abs(row_ids[0] - row_ids[row_cap - 1] + 1);
bret = ((PD_MIN == agg_type || PD_MAX == agg_type) &&
row_cap == row_gap &&
can_convert);
break;
}
case ObCSColumnHeader::INT_DICT:
case ObCSColumnHeader::STR_DICT: {
bret = (PD_MIN == agg_type || PD_MAX == agg_type || PD_HLL == agg_type);
break;
}
default: {
bret = false;
}
}
return bret;
}
int ObMicroBlockCSDecoder::get_aggregate_result(
const ObTableIterParam &iter_param,
const ObTableAccessContext &context,
@ -1654,16 +1686,46 @@ int ObMicroBlockCSDecoder::get_aggregate_result(
{
int ret = OB_SUCCESS;
decoder_allocator_.reuse();
ObDatum *datum_buf = agg_datum_buf.get_datums();
if (OB_FAIL(get_col_datums(col_offset, row_ids, row_cap, datum_buf))) {
LOG_WARN("fail to get col datums", KR(ret), K(col_offset), K(row_cap));
} else if (col_param.get_meta_type().is_lob_storage() && transform_helper_.get_micro_block_header()->has_lob_out_row() &&
OB_FAIL(fill_datums_lob_locator(iter_param, context, col_param, row_cap, datum_buf))) {
LOG_WARN("Fail to fill lob locator", K(ret));
} else if (OB_FAIL(agg_cell.eval_batch(datum_buf, row_cap))) { // TODO @donglou.zl can pushdown into decoder.
LOG_WARN("Failed to eval batch", K(ret));
ObColumnCSDecoder *column_decoder = nullptr;
if (OB_UNLIKELY(nullptr == row_ids || row_cap <= 0)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid arguments to get aggregate result", K(ret), KP(row_ids), K(row_cap));
} else if (OB_ISNULL(column_decoder = decoders_ + col_offset)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Column decoder is null", K(ret), K(col_offset));
} else {
ObDatum *datum_buf = agg_datum_buf.get_datums();
const bool can_pushdown = !(col_param.get_meta_type().is_lob_storage() && has_lob_out_row()) &&
can_pushdown_decoder(*column_decoder->ctx_, row_ids, row_cap, agg_cell);
if (can_pushdown) {
if (OB_FAIL(column_decoder->decoder_->get_aggregate_result(
*column_decoder->ctx_,
row_ids,
row_cap,
agg_cell))) {
LOG_WARN("Failed to get aggregate result", K(ret), K(col_offset));
}
} else {
const bool need_padding = is_pad_char_to_full_length(context.sql_mode_) &&
col_param.get_meta_type().is_fixed_len_char_type();
if (OB_FAIL(get_col_datums(col_offset, row_ids, row_cap, datum_buf))) {
LOG_WARN("fail to get col datums", KR(ret), K(col_offset), K(row_cap));
} else if (need_padding && OB_FAIL(storage::pad_on_datums(
col_param.get_accuracy(),
col_param.get_meta_type().get_collation_type(),
decoder_allocator_.get_inner_allocator(),
row_cap,
datum_buf))) {
LOG_WARN("fail to pad on datums", K(ret), K(row_cap));
} else if (col_param.get_meta_type().is_lob_storage() && has_lob_out_row() &&
OB_FAIL(fill_datums_lob_locator(iter_param, context, col_param, row_cap, datum_buf))) {
LOG_WARN("Fail to fill lob locator", K(ret));
} else if (OB_FAIL(agg_cell.eval_batch(datum_buf, row_cap))) { // TODO @donglou.zl can pushdown into decoder.
LOG_WARN("Failed to eval batch", K(ret));
}
}
LOG_DEBUG("get_aggregate_result", K(ret), K(can_pushdown), K(agg_cell));
}
LOG_DEBUG("get_aggregate_result", K(ret), K(agg_cell));
return ret;
}

View File

@ -282,6 +282,11 @@ private:
int decode_cells(
const uint64_t row_id, const int64_t col_begin, const int64_t col_end, ObStorageDatum *datums);
int get_row_impl(int64_t index, ObDatumRow &row);
bool can_pushdown_decoder(
const ObColumnCSDecoderCtx &ctx,
const int64_t *row_ids,
const int64_t row_cap,
const ObAggCell &agg_cell) const;
OB_INLINE static const ObRowHeader &get_major_store_row_header()
{
static ObRowHeader rh = init_major_store_row_header();

View File

@ -16,6 +16,7 @@
#include "ob_string_stream_decoder.h"
#include "ob_cs_vector_decoding_util.h"
#include "ob_string_stream_vector_decoder.h"
#include "storage/access/ob_pushdown_aggregate.h"
namespace oceanbase
{
@ -52,6 +53,49 @@ int ObStrDictColumnDecoder::decode(
return ret;
}
int ObStrDictColumnDecoder::decode_and_aggregate(
const ObColumnCSDecoderCtx &ctx,
const int64_t row_id,
ObStorageDatum &datum,
storage::ObAggCell &agg_cell) const
{
int ret = OB_SUCCESS;
const ObDictColumnDecoderCtx &dict_ctx = ctx.dict_ctx_;
const uint64_t distinct_cnt = dict_ctx.dict_meta_->distinct_val_cnt_;
if (OB_UNLIKELY(0 == distinct_cnt)) {
datum.set_null(); // empty dict, all datum is null
} else {
if (dict_ctx.dict_meta_->is_const_encoding_ref()) {
GET_CONST_ENCODING_REF(dict_ctx.ref_ctx_->meta_.width_, dict_ctx.ref_data_, row_id, datum.pack_);
} else {
GET_REF_FROM_REF_ARRAY(dict_ctx.ref_ctx_->meta_.width_, dict_ctx.ref_data_, row_id, datum.pack_);
}
ObBitmap &bitmap = agg_cell.get_bitmap();
if (datum.pack_ == distinct_cnt) {
datum.set_null();
} else if (bitmap.test(datum.pack_)) {
// has been evaluated.
} else if (OB_FAIL(bitmap.set(datum.pack_))) {
LOG_WARN("Failed to set bitmap", KR(ret), K(datum.pack_));
} else {
const uint8_t offset_width = dict_ctx.str_ctx_->meta_.is_fixed_len_string() ?
FIX_STRING_OFFSET_WIDTH_V : dict_ctx.offset_ctx_->meta_.width_;
ConvertStringToDatumFunc convert_func = convert_string_to_datum_funcs
[offset_width]
[ObRefStoreWidthV::REF_IN_DATUMS]
[ObBaseColumnDecoderCtx::ObNullFlag::HAS_NO_NULL] /*null has been processed, so here set HAS_NO_NULL*/
[dict_ctx.need_copy_];
convert_func(dict_ctx, dict_ctx.str_data_, *dict_ctx.str_ctx_,
dict_ctx.offset_data_, nullptr/*ref_data*/, nullptr/*row_ids*/, 1, &datum);
// datum will be padded in agg_cell
if (!datum.is_null() && OB_FAIL(agg_cell.eval(datum))) {
LOG_WARN("Failed to eval agg cell", KR(ret), K(datum), K(agg_cell));
}
}
}
return ret;
}
int ObStrDictColumnDecoder::batch_decode(const ObColumnCSDecoderCtx &ctx, const int64_t *row_ids,
const int64_t row_cap, common::ObDatum *datums) const
{

View File

@ -34,6 +34,11 @@ public:
virtual int batch_decode(const ObColumnCSDecoderCtx &ctx, const int64_t *row_ids,
const int64_t row_cap, common::ObDatum *datums) const override;
virtual int decode_vector(const ObColumnCSDecoderCtx &ctx, ObVectorDecodeCtx &vector_ctx) const override;
virtual int decode_and_aggregate(
const ObColumnCSDecoderCtx &ctx,
const int64_t row_id,
ObStorageDatum &datum,
storage::ObAggCell &agg_cell) const override;
virtual ObCSColumnHeader::Type get_type() const override { return type_; }
};

View File

@ -158,7 +158,7 @@ struct FilterTranverseDatum_T
const int64_t row_count,
const sql::ObWhiteFilterExecutor &filter,
common::ObBitmap &result_bitmap,
const ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> &op_handle)
const ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> &op_handle)
{
int ret = OB_INNER_STAT_ERROR;
LOG_ERROR("impossible here", K(offset_width_V), K(null_flag_V), K(need_padding_V));
@ -173,7 +173,7 @@ struct FilterTranverseDatum_T<offset_width_V, ObBaseColumnDecoderCtx::ObNullFlag
const ObStringColumnDecoderCtx &ctx,
const int64_t row_start,
const int64_t row_count,
const ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> &op_handle)
const ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> &op_handle)
{
int ret = OB_SUCCESS;
typedef typename ObCSEncodingStoreTypeInference<offset_width_V>::Type OffsetIntType;
@ -197,7 +197,7 @@ struct FilterTranverseDatum_T<offset_width_V, ObBaseColumnDecoderCtx::ObNullFlag
if (need_padding_V && OB_FAIL(pad_datum(ctx, cur_datum))) {
LOG_WARN("failed to pad datum", K(ret));
} else if (OB_FAIL(op_handle(ctx.obj_meta_, cur_datum, i))) {
} else if (OB_FAIL(op_handle(cur_datum, i))) {
LOG_WARN("fail to handle op", KR(ret), K(i), K(ctx), K(cur_datum));
}
}
@ -212,7 +212,7 @@ struct FilterTranverseDatum_T<offset_width_V, ObBaseColumnDecoderCtx::ObNullFlag
const ObStringColumnDecoderCtx &ctx,
const int64_t row_start,
const int64_t row_count,
const ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> &op_handle)
const ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> &op_handle)
{
int ret = OB_SUCCESS;
typedef typename ObCSEncodingStoreTypeInference<offset_width_V>::Type OffsetIntType;
@ -237,7 +237,7 @@ struct FilterTranverseDatum_T<offset_width_V, ObBaseColumnDecoderCtx::ObNullFlag
}
if (need_padding_V && OB_FAIL(pad_datum(ctx, cur_datum))) {
LOG_WARN("failed to pad datum", K(ret));
} else if (OB_FAIL(op_handle(ctx.obj_meta_, cur_datum, i))) {
} else if (OB_FAIL(op_handle(cur_datum, i))) {
LOG_WARN("fail to handle op", KR(ret), K(i), K(ctx), K(cur_datum));
}
}
@ -252,7 +252,7 @@ struct FilterTranverseDatum_T<offset_width_V, ObBaseColumnDecoderCtx::ObNullFlag
const ObStringColumnDecoderCtx &ctx,
const int64_t row_start,
const int64_t row_count,
const ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> &op_handle)
const ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> &op_handle)
{
int ret = OB_SUCCESS;
typedef typename ObCSEncodingStoreTypeInference<offset_width_V>::Type OffsetIntType;
@ -284,7 +284,7 @@ struct FilterTranverseDatum_T<offset_width_V, ObBaseColumnDecoderCtx::ObNullFlag
}
if (need_padding_V && OB_FAIL(pad_datum(ctx, cur_datum))) {
LOG_WARN("failed to pad datum", K(ret));
} else if (OB_FAIL(op_handle(ctx.obj_meta_, cur_datum, i))) {
} else if (OB_FAIL(op_handle(cur_datum, i))) {
LOG_WARN("fail to handle op", KR(ret), K(i), K(ctx), K(cur_datum));
}
}
@ -302,7 +302,7 @@ struct FilterTranverseDatum_T<FIX_STRING_OFFSET_WIDTH_V,
const ObStringColumnDecoderCtx &ctx,
const int64_t row_start,
const int64_t row_count,
const ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> &op_handle)
const ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> &op_handle)
{
int ret = OB_SUCCESS;
const char *start = ctx.str_data_;
@ -323,7 +323,7 @@ struct FilterTranverseDatum_T<FIX_STRING_OFFSET_WIDTH_V,
if (need_padding_V && OB_FAIL(pad_datum(ctx, cur_datum))) {
LOG_WARN("failed to pad datum", K(ret));
} else if (OB_FAIL(op_handle(ctx.obj_meta_, cur_datum, i))) {
} else if (OB_FAIL(op_handle(cur_datum, i))) {
LOG_WARN("fail to handle op", KR(ret), K(i), K(ctx), K(cur_datum));
}
}
@ -340,7 +340,7 @@ struct FilterTranverseDatum_T<FIX_STRING_OFFSET_WIDTH_V,
const ObStringColumnDecoderCtx &ctx,
const int64_t row_start,
const int64_t row_count,
const ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> &op_handle)
const ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> &op_handle)
{
int ret = OB_SUCCESS;
const char *start = ctx.str_data_;
@ -361,7 +361,7 @@ struct FilterTranverseDatum_T<FIX_STRING_OFFSET_WIDTH_V,
}
if (need_padding_V && OB_FAIL(pad_datum(ctx, cur_datum))) {
LOG_WARN("failed to pad datum", K(ret));
} else if (OB_FAIL(op_handle(ctx.obj_meta_, cur_datum, i))) {
} else if (OB_FAIL(op_handle(cur_datum, i))) {
LOG_WARN("fail to handle op", KR(ret), K(i), K(ctx), K(cur_datum));
}
}
@ -373,7 +373,7 @@ typedef int (*FilterTranverseDatum) (
const ObStringColumnDecoderCtx &ctx,
const int64_t row_start,
const int64_t row_count,
const ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> &op_handle);
const ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> &op_handle);
static ObMultiDimArray_T<FilterTranverseDatum, 5, 3, 2> filter_tranverse_datum_;
template <int32_t offset_width_V, int32_t null_flag_V, int32_t need_padding_V>
@ -470,8 +470,8 @@ int ObStringColumnDecoder::nunn_operator(
const bool is_fixed_len_str = ctx.str_ctx_->meta_.is_fixed_len_string();
const bool need_padding = (ctx.obj_meta_.is_fixed_len_char_type() && nullptr != ctx.col_param_);
ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> op_handle =
[&] (const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)
ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> op_handle =
[&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
if (cur_datum.is_null()) {
@ -518,8 +518,8 @@ int ObStringColumnDecoder::comparison_operator(
const common::ObCmpOp &cmp_op = sql::ObPushdownWhiteFilterNode::WHITE_OP_TO_CMP_OP[op_type];
const bool need_padding = (ctx.obj_meta_.is_fixed_len_char_type() && nullptr != ctx.col_param_);
ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> op_handle =
[&] (const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)
ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> op_handle =
[&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
bool cmp_ret = false;
@ -558,33 +558,52 @@ int ObStringColumnDecoder::in_operator(
const bool need_padding = (ctx.obj_meta_.is_fixed_len_char_type() && nullptr != ctx.col_param_);
const bool is_fixed_len_str = ctx.str_ctx_->meta_.is_fixed_len_string();
ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> op_handle =
[&] (const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
ObObj cur_obj;
bool is_exist = false;
if (cur_datum.is_null()) {
} else if (OB_TMP_FAIL(cur_datum.to_obj(cur_obj, obj_meta))) {
LOG_WARN("fail to convert datum to obj", KR(tmp_ret), K(cur_datum), K(obj_meta));
} else if (OB_TMP_FAIL(filter.exist_in_obj_set(cur_obj, is_exist))) {
LOG_WARN("fail to check obj in hashset", KR(tmp_ret), K(cur_obj));
} else if (is_exist) {
if (OB_TMP_FAIL(result_bitmap.set(idx))) {
LOG_WARN("fail to set", KR(tmp_ret), K(idx), K(row_start));
ObFilterInCmpType cmp_type = get_filter_in_cmp_type(row_count, filter.get_datums().count(), false);
ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> op_handle;
if (cmp_type == ObFilterInCmpType::BINARY_SEARCH) {
op_handle = [&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
bool is_exist = false;
if (cur_datum.is_null()) {
} else if (OB_TMP_FAIL(filter.exist_in_datum_array(cur_datum, is_exist))) {
LOG_WARN("fail to check datum in array", KR(tmp_ret), K(cur_datum));
} else if (is_exist) {
if (OB_TMP_FAIL(result_bitmap.set(idx))) {
LOG_WARN("fail to set", KR(tmp_ret), K(idx), K(row_start));
}
}
}
return tmp_ret;
};
return tmp_ret;
};
} else if (cmp_type == ObFilterInCmpType::HASH_SEARCH) {
op_handle = [&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
bool is_exist = false;
if (cur_datum.is_null()) {
} else if (OB_TMP_FAIL(filter.exist_in_datum_set(cur_datum, is_exist))) {
LOG_WARN("fail to check datum in hashset", KR(tmp_ret), K(cur_datum));
} else if (is_exist) {
if (OB_TMP_FAIL(result_bitmap.set(idx))) {
LOG_WARN("fail to set", KR(tmp_ret), K(idx), K(row_start));
}
}
return tmp_ret;
};
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter in compare type", KR(ret), K(cmp_type));
}
if (is_fixed_len_str) {
if (OB_FAIL(ret)) {
} else if (is_fixed_len_str) {
ret = filter_tranverse_datum_[FIX_STRING_OFFSET_WIDTH_V]
[ctx.null_flag_]
[need_padding] (ctx, row_start, row_count, op_handle);
[ctx.null_flag_]
[need_padding] (ctx, row_start, row_count, op_handle);
} else {
ret = filter_tranverse_datum_[ctx.offset_ctx_->meta_.width_]
[ctx.null_flag_]
[need_padding] (ctx, row_start, row_count, op_handle);
[ctx.null_flag_]
[need_padding] (ctx, row_start, row_count, op_handle);
}
return ret;
}
@ -603,8 +622,8 @@ int ObStringColumnDecoder::bt_operator(
const bool need_padding = (ctx.obj_meta_.is_fixed_len_char_type() && nullptr != ctx.col_param_);
const bool is_fixed_len_str = ctx.str_ctx_->meta_.is_fixed_len_string();
ObFunction<int(const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)> op_handle =
[&] (const ObObjMeta &obj_meta, const ObDatum &cur_datum, const int64_t idx)
ObFunction<int(const ObDatum &cur_datum, const int64_t idx)> op_handle =
[&] (const ObDatum &cur_datum, const int64_t idx)
{
int tmp_ret = OB_SUCCESS;
int left_cmp_ret = 0;

View File

@ -485,12 +485,9 @@ int ObConstDecoder::const_only_operator(
} else if (ref == 1) {
} else {
bool is_existed = false;
// Check const object in hashset or not
ObObj const_obj;
if (OB_FAIL(const_datum.to_obj(const_obj, col_ctx.obj_meta_))) {
LOG_WARN("convert datum to obj failed", K(ret), K(const_datum), K(col_ctx.obj_meta_));
} else if (OB_FAIL(filter.exist_in_obj_set(const_obj, is_existed))) {
LOG_WARN("Failed to check object in hashset", K(ret));
// Check const datum in hashset or not
if (OB_FAIL(filter.exist_in_datum_set(const_datum, is_existed))) {
LOG_WARN("Failed to check datum in hashset", K(ret));
} else if (is_existed) {
if (OB_FAIL(result_bitmap.bit_not())) {
LOG_WARN("Failed to do bitwise not on result bitmap", K(ret));
@ -776,11 +773,8 @@ int ObConstDecoder::in_operator(
LOG_WARN("Failed to pad column", K(ret));
}
}
ObObj const_obj;
if (OB_FAIL(ret)) {
} else if (OB_FAIL(const_datum.to_obj(const_obj, col_ctx.obj_meta_))) {
LOG_WARN("convert datum to obj failed", K(ret), K(const_datum), K(col_ctx.obj_meta_));
} else if (OB_FAIL(filter.exist_in_obj_set(const_obj, const_in_result_set))) {
} else if (OB_FAIL(filter.exist_in_datum_set(const_datum, const_in_result_set))) {
LOG_WARN("Failed to check whether const value is in set", K(ret));
} else if (const_in_result_set) {
if (OB_FAIL(result_bitmap.bit_not())) {
@ -799,16 +793,13 @@ int ObConstDecoder::in_operator(
ref_bitset->init(ref_bitset_size);
int64_t dict_ref = 0;
while (OB_SUCC(ret) && trav_it != end_it) {
ObObj cur_obj;
bool cur_in_result_set = false;
if (OB_UNLIKELY((((*trav_it).is_null() || (col_ctx.obj_meta_.is_character_type()
&& (0 == (*trav_it).len_))) && lib::is_oracle_mode())
|| ((*trav_it).is_null() && lib::is_mysql_mode()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("There should not be null datum in dictionary", K(ret));
} else if (OB_FAIL((*trav_it).to_obj(cur_obj, col_ctx.obj_meta_))) {
LOG_WARN("convert datum to obj failed", K(ret), K((*trav_it)), K(col_ctx.obj_meta_));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, cur_in_result_set))) {
} else if (OB_FAIL(filter.exist_in_datum_set(*trav_it, cur_in_result_set))) {
LOG_WARN("Failed to check wheter current value is in set", K(ret));
} else if (!const_in_result_set == cur_in_result_set) {
found = true;

View File

@ -1313,6 +1313,7 @@ int ObDictDecoder::bt_operator(
return ret;
}
// TODO(@wenye): optimize in operator
int ObDictDecoder::in_operator(
const sql::ObPushdownFilterExecutor *parent,
const ObColumnDecoderCtx &col_ctx,
@ -1341,10 +1342,7 @@ int ObDictDecoder::in_operator(
int64_t dict_ref = 0;
bool is_exist = false;
while (OB_SUCC(ret) && traverse_it != end_it) {
ObObj cur_obj;
if (OB_FAIL((*traverse_it).to_obj(cur_obj, col_ctx.obj_meta_))) {
LOG_WARN("convert datum to obj failed", K(ret), K(*traverse_it), K(col_ctx.obj_meta_));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, is_exist))) {
if (OB_FAIL(filter.exist_in_datum_set(*traverse_it, is_exist))) {
LOG_WARN("Failed to check object in hashset", K(ret), K(*traverse_it));
} else if (is_exist) {
found = true;

View File

@ -630,8 +630,7 @@ int ObIntegerBaseDiffDecoder::bt_operator(
|| ObIntSC == get_store_class_map()[col_ctx.obj_meta_.get_type_class()]) {
if (OB_FAIL(traverse_all_data(parent, col_ctx, col_data,
filter, pd_filter_info, result_bitmap,
[](const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
[](const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) -> int {
int ret = OB_SUCCESS;
@ -672,16 +671,12 @@ int ObIntegerBaseDiffDecoder::in_operator(
K(ret), K(col_ctx), K(pd_filter_info), K(result_bitmap.size()), K(filter));
} else if (OB_FAIL(traverse_all_data(parent, col_ctx, col_data,
filter, pd_filter_info, result_bitmap,
[](const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
[](const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) -> int {
int ret = OB_SUCCESS;
ObObj cur_obj;
if (OB_FAIL(cur_datum.to_obj(cur_obj, obj_meta))) {
LOG_WARN("convert datum to obj failed", K(ret), K(cur_datum), K(obj_meta));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, result))) {
LOG_WARN("Failed to check object in hashset", K(ret), K(cur_obj));
if (OB_FAIL(filter.exist_in_datum_set(cur_datum, result))) {
LOG_WARN("Failed to check datum in hashset", K(ret), K(cur_datum));
}
return ret;
}))) {
@ -698,7 +693,6 @@ int ObIntegerBaseDiffDecoder::traverse_all_data(
const sql::PushdownFilterInfo &pd_filter_info,
ObBitmap &result_bitmap,
int (*lambda)(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result)) const
@ -748,7 +742,7 @@ int ObIntegerBaseDiffDecoder::traverse_all_data(
cur_datum.ptr_ = reinterpret_cast<char *> (&cur_int);
// use lambda here to filter and set result bitmap
bool result = false;
if (OB_FAIL(lambda(col_ctx.obj_meta_, cur_datum, filter, result))) {
if (OB_FAIL(lambda(cur_datum, filter, result))) {
LOG_WARN("Failed on trying to filter the row", K(ret), K(row_id), K(cur_int));
} else if (result) {
if (OB_FAIL(result_bitmap.set(offset))) {

View File

@ -126,7 +126,6 @@ private:
const sql::PushdownFilterInfo &pd_filter_info,
ObBitmap &result_bitmap,
int (*lambda)(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result)) const;

View File

@ -12,6 +12,7 @@
#define USING_LOG_PREFIX STORAGE
#include "ob_micro_block_decoder.h"
#include "common/sql_mode/ob_sql_mode_utils.h"
#include "share/rc/ob_tenant_base.h"
#include "storage/access/ob_pushdown_aggregate.h"
#include "storage/access/ob_table_access_context.h"
@ -1797,8 +1798,7 @@ int ObMicroBlockDecoder::filter_pushdown_filter(
column_decoder->ctx_->set_col_param(col_params.at(0));
if (filter.null_param_contained() &&
op_type != sql::WHITE_OP_NU &&
op_type != sql::WHITE_OP_NN &&
op_type != sql::WHITE_OP_IN) {
op_type != sql::WHITE_OP_NN) {
} else if (!header_->all_lob_in_row_) {
if (OB_FAIL(filter_pushdown_retro(parent,
filter,
@ -1894,7 +1894,7 @@ int ObMicroBlockDecoder::filter_pushdown_retro(
bool filtered = false;
if (OB_FAIL(ret)) {
} else if (OB_FAIL(filter_white_filter(filter, obj_meta, decoded_datum, filtered))) {
} else if (OB_FAIL(filter_white_filter(filter, decoded_datum, filtered))) {
LOG_WARN("Failed to filter row with white filter", K(ret), K(row_id), K(decoded_datum));
} else if (!filtered && OB_FAIL(result_bitmap.set(offset))) {
LOG_WARN("Failed to set result bitmap", K(ret), K(row_id));
@ -2025,10 +2025,19 @@ int ObMicroBlockDecoder::get_aggregate_result(
decoder_allocator_.reuse();
const char **cell_datas = agg_datum_buf.get_cell_datas();
ObDatum *datum_buf = agg_datum_buf.get_datums();
const bool need_padding = is_pad_char_to_full_length(context.sql_mode_) &&
col_param.get_meta_type().is_fixed_len_char_type();
if (OB_FAIL(get_col_datums(col_offset, row_ids, cell_datas, row_cap, datum_buf))) {
LOG_WARN("Failed to get col datums", K(ret), K(col_offset), K(row_cap));
} else if (need_padding && OB_FAIL(storage::pad_on_datums(
col_param.get_accuracy(),
col_param.get_meta_type().get_collation_type(),
decoder_allocator_.get_inner_allocator(),
row_cap,
datum_buf))) {
LOG_WARN("fail to pad on datums", K(ret), K(row_cap));
} else if (col_param.get_meta_type().is_lob_storage() && header_->has_lob_out_row() &&
OB_FAIL(fill_datums_lob_locator(iter_param, context, col_param, row_cap, datum_buf))) {
OB_FAIL(fill_datums_lob_locator(iter_param, context, col_param, row_cap, datum_buf))) {
LOG_WARN("Fail to fill lob locator", K(ret));
} else if (OB_FAIL(agg_cell.eval_batch(datum_buf, row_cap))) {
LOG_WARN("Failed to eval batch", K(ret));

View File

@ -1185,7 +1185,7 @@ int ObRawDecoder::traverse_all_data(
cur_datum.pack_ = datum_len;
cur_datum.ptr_ = (const char *)(&value);
bool result = false;
if (OB_FAIL(eval(col_ctx.obj_meta_, cur_datum, filter, result))) {
if (OB_FAIL(eval(cur_datum, filter, result))) {
LOG_WARN("Failed on trying to filter the row", K(ret), K(row_id), K(cur_datum));
} else if (result) {
if (OB_FAIL(result_bitmap.set(offset))) {
@ -1240,7 +1240,7 @@ int ObRawDecoder::traverse_all_data(
if (OB_SUCC(ret)) {
// Run lambda here to filter out the data according to op_type
bool result = false;
if (OB_FAIL(eval(col_ctx.obj_meta_, cur_datum, filter, result))) {
if (OB_FAIL(eval(cur_datum, filter, result))) {
LOG_WARN("Failed on trying to filter the row", K(ret), K(row_id), K(cur_datum));
} else if (result) {
if (OB_FAIL(result_bitmap.set(offset))) {
@ -1256,7 +1256,6 @@ int ObRawDecoder::traverse_all_data(
}
int ObRawDecoder::ObRawDecoderFilterCmpFunc::operator()(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) const
@ -1272,7 +1271,6 @@ int ObRawDecoder::ObRawDecoderFilterCmpFunc::operator()(
}
int ObRawDecoder::ObRawDecoderFilterBetweenFunc::operator()(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) const
@ -1293,17 +1291,13 @@ int ObRawDecoder::ObRawDecoderFilterBetweenFunc::operator()(
}
int ObRawDecoder::ObRawDecoderFilterInFunc::operator()(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) const
{
int ret = OB_SUCCESS;
ObObj cur_obj;
if (OB_FAIL(cur_datum.to_obj(cur_obj, obj_meta))) {
LOG_WARN("convert datum to obj failed", K(ret), K(cur_datum), K(obj_meta));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, result))) {
LOG_WARN("Failed to check obj in hashset", K(ret), K(cur_obj));
if (OB_FAIL(filter.exist_in_datum_set(cur_datum, result))) {
LOG_WARN("Failed to check datum in hashset", K(ret), K(cur_datum));
}
return ret;
}
@ -1620,6 +1614,253 @@ raw_compare_function_with_null RawCompareFunctionFactory::get_cs_cmp_function_wi
return cmp_function;
}
template <typename DataType, typename Op>
class RawAggFunctionImpl
{
public:
// can use SIMD
OB_MULTITARGET_FUNCTION_AVX2_SSE42(
OB_MULTITARGET_FUNCTION_HEADER(static void), raw_min_max_function, OB_MULTITARGET_FUNCTION_BODY((
const unsigned char* raw_data,
uint32_t from,
uint32_t to,
uint64_t &res)
{
const DataType *start_pos = reinterpret_cast<const DataType *>(raw_data);
const DataType *a_end = start_pos + to;
const DataType * __restrict a_pos = start_pos + from;
DataType res_value = *(start_pos + from);
while (a_pos < a_end) {
if (Op::apply(*a_pos, res_value)) {
res_value = *a_pos;
}
++a_pos;
}
res = res_value;
}))
// can not use SIMD
OB_MULTITARGET_FUNCTION_AVX2_SSE42(
OB_MULTITARGET_FUNCTION_HEADER(static void), raw_min_max_function_with_null, OB_MULTITARGET_FUNCTION_BODY((
const unsigned char* raw_data,
const uint64_t null_value,
uint32_t from,
uint32_t to,
uint64_t &res)
{
const DataType *start_pos = reinterpret_cast<const DataType *>(raw_data);
const DataType *a_end = start_pos + to;
const DataType * __restrict a_pos = start_pos + from;
DataType res_value = *(start_pos + from);
while (a_pos < a_end) {
if (*a_pos != null_value && Op::apply(*a_pos, res_value)) {
res_value = *a_pos;
}
++a_pos;
}
res = res_value;
}))
// can use SIMD
OB_MULTITARGET_FUNCTION_AVX2_SSE42(
OB_MULTITARGET_FUNCTION_HEADER(static void), raw_min_max_function_with_null_bitmap, OB_MULTITARGET_FUNCTION_BODY((
const unsigned char* raw_data,
const uint8_t *null_bitmap,
uint32_t from,
uint32_t to,
uint64_t &res)
{
const DataType *start_pos = reinterpret_cast<const DataType *>(raw_data);
const DataType *a_end = start_pos + to;
const DataType * __restrict a_pos = start_pos + from;
const uint8_t * __restrict b_pos = null_bitmap;
DataType res_value = *(start_pos + from);
while (a_pos < a_end) {
if (!*b_pos && Op::apply(*a_pos, res_value)) {
res_value = *a_pos;
}
++a_pos;
++b_pos;
}
res = res_value;
}))
};
template <bool IS_SIGNED, int32_t LEN_TAG>
struct RawAggFunctionProducer
{
static raw_min_max_function produce_min_max(
const uint32_t type)
{
raw_min_max_function min_max_function = nullptr;
typedef typename ObEncodingTypeInference<IS_SIGNED, LEN_TAG>::Type DataType;
const bool is_supported = is_arch_supported(ObTargetArch::AVX2);
switch (type) {
case 0: // min
#if OB_USE_MULTITARGET_CODE
if (is_supported) {
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function_avx2;
} else {
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function;
}
#else
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function;
#endif
break;
case 1:
#if OB_USE_MULTITARGET_CODE
if (is_supported) {
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function_avx2;
} else {
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function;
}
#else
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function;
#endif
break;
default:
LOG_ERROR_RET(OB_ERR_UNEXPECTED, "Invalid min max type", K(type));
break;
}
return min_max_function;
}
static raw_min_max_function_with_null produce_min_max_with_null_for_cs(
const uint32_t type)
{
raw_min_max_function_with_null min_max_function = nullptr;
typedef typename ObEncodingTypeInference<IS_SIGNED, LEN_TAG>::Type DataType;
const bool is_supported = is_arch_supported(ObTargetArch::AVX2);
switch (type) {
case 0: // min
#if OB_USE_MULTITARGET_CODE
if (is_supported) {
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function_with_null_avx2;
} else {
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function_with_null;
}
#else
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function_with_null;
#endif
break;
case 1:
#if OB_USE_MULTITARGET_CODE
if (is_supported) {
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function_with_null_avx2;
} else {
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function_with_null;
}
#else
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function_with_null;
#endif
break;
default:
LOG_ERROR_RET(OB_ERR_UNEXPECTED, "Invalid min max type", K(type));
break;
}
return min_max_function;
}
static raw_min_max_function_with_null_bitmap produce_min_max_with_null_bitmap_for_cs(
const uint32_t type)
{
raw_min_max_function_with_null_bitmap min_max_function = nullptr;
typedef typename ObEncodingTypeInference<IS_SIGNED, LEN_TAG>::Type DataType;
const bool is_supported = is_arch_supported(ObTargetArch::AVX2);
switch (type) {
case 0: // min
#if OB_USE_MULTITARGET_CODE
if (is_supported) {
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function_with_null_bitmap_avx2;
} else {
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function_with_null_bitmap;
}
#else
min_max_function = RawAggFunctionImpl<DataType, RawLessOp<DataType>>::raw_min_max_function_with_null_bitmap;
#endif
break;
case 1:
#if OB_USE_MULTITARGET_CODE
if (is_supported) {
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function_with_null_bitmap_avx2;
} else {
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function_with_null_bitmap;
}
#else
min_max_function = RawAggFunctionImpl<DataType, RawGreaterOp<DataType>>::raw_min_max_function_with_null_bitmap;
#endif
break;
default:
LOG_ERROR_RET(OB_ERR_UNEXPECTED, "Invalid min max type", K(type));
break;
}
return min_max_function;
}
};
// For cs encoding, all value is unsigned
RawAggFunctionFactory::RawAggFunctionFactory()
{
for (uint32_t k = 0; k < MIN_MAX_CNT; ++k) {
min_max_functions_array_[0][k] = RawAggFunctionProducer<0, 0>::produce_min_max(k);
min_max_functions_array_[1][k] = RawAggFunctionProducer<0, 1>::produce_min_max(k);
min_max_functions_array_[2][k] = RawAggFunctionProducer<0, 2>::produce_min_max(k);
min_max_functions_array_[3][k] = RawAggFunctionProducer<0, 3>::produce_min_max(k);
cs_min_max_functions_with_null_array_[0][k] = RawAggFunctionProducer<0, 0>::produce_min_max_with_null_for_cs(k);
cs_min_max_functions_with_null_array_[1][k] = RawAggFunctionProducer<0, 1>::produce_min_max_with_null_for_cs(k);
cs_min_max_functions_with_null_array_[2][k] = RawAggFunctionProducer<0, 2>::produce_min_max_with_null_for_cs(k);
cs_min_max_functions_with_null_array_[3][k] = RawAggFunctionProducer<0, 3>::produce_min_max_with_null_for_cs(k);
cs_min_max_functions_with_null_bitmap_array_[0][k] = RawAggFunctionProducer<0, 0>::produce_min_max_with_null_bitmap_for_cs(k);
cs_min_max_functions_with_null_bitmap_array_[1][k] = RawAggFunctionProducer<0, 1>::produce_min_max_with_null_bitmap_for_cs(k);
cs_min_max_functions_with_null_bitmap_array_[2][k] = RawAggFunctionProducer<0, 2>::produce_min_max_with_null_bitmap_for_cs(k);
cs_min_max_functions_with_null_bitmap_array_[3][k] = RawAggFunctionProducer<0, 3>::produce_min_max_with_null_bitmap_for_cs(k);
}
}
RawAggFunctionFactory &RawAggFunctionFactory::instance()
{
static RawAggFunctionFactory ret;
return ret;
}
raw_min_max_function RawAggFunctionFactory::get_min_max_function(
const int32_t fix_len_tag,
const bool is_min)
{
raw_min_max_function min_max_function = nullptr;
if (OB_UNLIKELY(fix_len_tag < 0 || fix_len_tag >= FIX_LEN_TAG_CNT)) {
} else {
min_max_function = min_max_functions_array_[fix_len_tag][is_min ? 0 : 1];
}
return min_max_function;
}
raw_min_max_function_with_null RawAggFunctionFactory::get_cs_min_max_function_with_null(
const int32_t fix_len_tag,
const bool is_min)
{
raw_min_max_function_with_null min_max_function = nullptr;
if (OB_UNLIKELY(fix_len_tag < 0 || fix_len_tag >= FIX_LEN_TAG_CNT)) {
} else {
min_max_function = cs_min_max_functions_with_null_array_[fix_len_tag][is_min ? 0 : 1];
}
return min_max_function;
}
raw_min_max_function_with_null_bitmap RawAggFunctionFactory::get_cs_min_max_function_with_null_bitmap(
const int32_t fix_len_tag,
const bool is_min)
{
raw_min_max_function_with_null_bitmap min_max_function = nullptr;
if (OB_UNLIKELY(fix_len_tag < 0 || fix_len_tag >= FIX_LEN_TAG_CNT)) {
} else {
min_max_function = cs_min_max_functions_with_null_bitmap_array_[fix_len_tag][is_min ? 0 : 1];
}
return min_max_function;
}
} // end namespace blocksstable
} // end namespace oceanbase

View File

@ -66,6 +66,26 @@ typedef void (*raw_compare_function_with_null)(
uint32_t from,
uint32_t to);
typedef void (*raw_min_max_function)(
const unsigned char *raw_data,
uint32_t from,
uint32_t to,
uint64_t &res);
typedef void (*raw_min_max_function_with_null)(
const unsigned char *raw_data,
const uint64_t null_value,
uint32_t from,
uint32_t to,
uint64_t &res);
typedef void (*raw_min_max_function_with_null_bitmap)(
const unsigned char* raw_data,
const uint8_t *null_bitmap,
uint32_t from,
uint32_t to,
uint64_t &res);
class RawCompareFunctionFactory {
public:
static constexpr uint32_t IS_SIGNED_CNT = 2;
@ -89,6 +109,33 @@ private:
ObMultiDimArray_T<raw_compare_function_with_null, FIX_LEN_TAG_CNT, OP_TYPE_CNT> cs_functions_with_null_array_;
};
class RawAggFunctionFactory
{
public:
static constexpr uint32_t FIX_LEN_TAG_CNT = 4;
static constexpr uint32_t MIN_MAX_CNT = 2;
public:
static RawAggFunctionFactory &instance();
raw_min_max_function get_min_max_function(
const int32_t fix_len_tag,
const bool is_min);
raw_min_max_function_with_null get_cs_min_max_function_with_null(
const int32_t fix_len_tag,
const bool is_min);
raw_min_max_function_with_null_bitmap get_cs_min_max_function_with_null_bitmap(
const int32_t fix_len_tag,
const bool is_min);
private:
RawAggFunctionFactory();
~RawAggFunctionFactory() = default;
DISALLOW_COPY_AND_ASSIGN(RawAggFunctionFactory);
private:
ObMultiDimArray_T<raw_min_max_function, FIX_LEN_TAG_CNT, MIN_MAX_CNT> min_max_functions_array_;
ObMultiDimArray_T<raw_min_max_function_with_null, FIX_LEN_TAG_CNT, MIN_MAX_CNT> cs_min_max_functions_with_null_array_;
ObMultiDimArray_T<raw_min_max_function_with_null_bitmap, FIX_LEN_TAG_CNT, MIN_MAX_CNT> cs_min_max_functions_with_null_bitmap_array_;
};
class ObRawDecoder : public ObIColumnDecoder
{
public:
@ -251,7 +298,6 @@ private:
: type_cmp_func_(type_cmp_func), get_cmp_ret_(get_cmp_ret) {}
~ObRawDecoderFilterCmpFunc() = default;
int operator()(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) const;
@ -269,7 +315,6 @@ private:
}
~ObRawDecoderFilterBetweenFunc() = default;
int operator()(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) const;
@ -284,7 +329,6 @@ private:
ObRawDecoderFilterInFunc() {}
~ObRawDecoderFilterInFunc() = default;
int operator()(
const ObObjMeta &obj_meta,
const ObDatum &cur_datum,
const sql::ObWhiteFilterExecutor &filter,
bool &result) const;

View File

@ -434,10 +434,7 @@ int ObRLEDecoder::in_operator(
int64_t dict_ref = 0;
bool is_exist = false;
while (OB_SUCC(ret) && traverse_it != end_it) {
ObObj cur_obj;
if (OB_FAIL((*traverse_it).to_obj(cur_obj, col_ctx.obj_meta_))) {
LOG_WARN("convert datum to obj failed", K(ret), K(*traverse_it), K(col_ctx.obj_meta_));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, is_exist))) {
if (OB_FAIL(filter.exist_in_datum_set(*traverse_it, is_exist))) {
LOG_WARN("Failed to check object in hashset", K(ret), K(*traverse_it));
} else if (is_exist) {
found = true;

View File

@ -444,39 +444,59 @@ int ObSkipIndexFilterExecutor::in_operator(const sql::ObWhiteFilterExecutor &fil
{
int ret = OB_SUCCESS;
const common::ObIArray<common::ObDatum> &datums = filter.get_datums();
if (OB_UNLIKELY(datums.count() == 0 || filter.null_param_contained())){
const sql::ObExpr *col_expr = filter.get_filter_node().expr_;
if (OB_UNLIKELY(nullptr == col_expr)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("Invalid argument for falsifiable IN operator", K(ret), K(filter));
} else if (filter.null_param_contained()) {
fal_desc.set_always_false();
} else {
const int ref_count = datums.count();
ObDatumCmpFuncType cmp_func = filter.cmp_func_;
int min_cmp_res;
int max_cmp_res;
bool falsifable_true = true;
bool falsifable_false = false;
for (int i = 0; OB_SUCC(ret) && i < ref_count; ++i) {
if (OB_FAIL(cmp_func(min_datum, datums.at(i), min_cmp_res))) {
LOG_WARN("Failed to compare datum", K(ret), K(min_datum), K(i), K(datums.at(i)));
} else if (OB_FAIL(cmp_func(max_datum, datums.at(i), max_cmp_res))) {
LOG_WARN("Failed to compare datum", K(ret), K(max_datum), K(i), K(datums.at(i)));
} else {
if (falsifable_true && ((min_cmp_res < 0 && max_cmp_res > 0) ||
min_cmp_res == 0 || max_cmp_res == 0)) {
falsifable_true = false;
}
if (!falsifable_false && min_cmp_res == 0 && max_cmp_res == 0) {
falsifable_false = true;
}
}
}
if (OB_FAIL(ret)) {
} else if (falsifable_true) {
ObDatumCmpFuncType col_cmp_func = col_expr->args_[0]->basic_funcs_->null_first_cmp_;
int min_cmp_res = 0;
int max_cmp_res = 0;
int cmp_res = 0;
if (OB_FAIL(cmp_func(min_datum, filter.get_max_param(), min_cmp_res))) {
LOG_WARN("Failed to compare min datum with max filter param", K(ret), K(min_datum), K(filter.get_max_param()));
} else if (min_cmp_res > 0) {
fal_desc.set_always_false();
} else if (falsifable_false) {
fal_desc.set_always_true();
} else if (OB_FAIL(cmp_func(max_datum, filter.get_min_param(), max_cmp_res))) {
LOG_WARN("Failed to compare max datum with min filter param", K(ret), K(max_datum), K(filter.get_min_param()));
} else if (max_cmp_res < 0) {
fal_desc.set_always_false();
} else if (OB_FAIL(col_cmp_func(min_datum, max_datum, cmp_res))) {
LOG_WARN("Failed to compare min max datum", K(ret), K(min_datum), K(max_datum));
} else if (cmp_res == 0) {
if (min_cmp_res == 0 || max_cmp_res == 0) {
fal_desc.set_always_true();
} else {
ObFilterInCmpType cmp_type = get_filter_in_cmp_type(1, datums.count(), false);
bool is_exist = false;
if (cmp_type == ObFilterInCmpType::BINARY_SEARCH) {
if (OB_FAIL(filter.exist_in_datum_array(min_datum, is_exist))) {
LOG_WARN("Failed to check datum in array", K(ret), K(min_datum), K(max_datum));
}
} else if (cmp_type == ObFilterInCmpType::HASH_SEARCH) {
if (OB_FAIL(filter.exist_in_datum_set(min_datum, is_exist))) {
LOG_WARN("Failed to check datum in hashset", K(ret), K(min_datum), K(max_datum));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter in compare type", K(ret), K(cmp_type));
}
if (OB_SUCC(ret)) {
if (is_exist) {
fal_desc.set_always_true();
} else {
fal_desc.set_always_false();
}
}
}
} else {
fal_desc.set_uncertain();
}
LOG_DEBUG("check filter in in skip index", K(min_cmp_res), K(max_cmp_res), K(cmp_res), K(fal_desc));
}
return ret;
}

View File

@ -128,7 +128,6 @@ int ObIMicroBlockReader::validate_filter_info(
int ObIMicroBlockReader::filter_white_filter(
const sql::ObWhiteFilterExecutor &filter,
const common::ObObjMeta &obj_meta,
const common::ObDatum &datum,
bool &filtered)
{
@ -199,11 +198,8 @@ int ObIMicroBlockReader::filter_white_filter(
}
case sql::WHITE_OP_IN: {
bool is_existed = false;
ObObj cur_obj;
if (OB_FAIL(datum.to_obj(cur_obj, obj_meta))) {
LOG_WARN("convert datum to obj failed", K(ret), K(datum), K(obj_meta));
} else if (OB_FAIL(filter.exist_in_obj_set(cur_obj, is_existed))) {
LOG_WARN("Failed to check object in hashset", K(ret), K(cur_obj));
if (OB_FAIL(filter.exist_in_datum_set(datum, is_existed))) {
LOG_WARN("Failed to check object in hashset", K(ret), K(datum));
} else if (is_existed) {
filtered = false;
}

View File

@ -387,7 +387,6 @@ public:
int32_t &compare_result) = 0;
static int filter_white_filter(
const sql::ObWhiteFilterExecutor &filter,
const common::ObObjMeta &obj_meta,
const common::ObDatum &datum,
bool &filtered);
virtual bool has_lob_out_row() const = 0;

View File

@ -107,6 +107,7 @@ public:
last_rows_count_(0),
micro_block_merge_verify_level_(MICRO_BLOCK_MERGE_VERIFY_LEVEL::ENCODING_AND_COMPRESSION),
max_merged_trans_version_(0),
min_merged_trans_version_(INT64_MAX),
block_size_upper_bound_(DEFAULT_UPPER_BOUND),
contain_uncommitted_row_(false),
has_string_out_row_(false),
@ -145,6 +146,7 @@ public:
last_rows_count_ = 0;
checksum_helper_.reuse();
max_merged_trans_version_ = 0;
min_merged_trans_version_ = INT64_MAX;
block_size_upper_bound_ = DEFAULT_UPPER_BOUND;
contain_uncommitted_row_ = false;
has_string_out_row_ = false;
@ -162,12 +164,23 @@ public:
int64_t get_micro_block_checksum() const { return checksum_helper_.get_row_checksum(); }
int64_t get_max_merged_trans_version() const { return max_merged_trans_version_; }
inline void update_merged_trans_version(const int64_t merged_trans_version)
{
update_max_merged_trans_version(merged_trans_version);
update_min_merged_trans_version(merged_trans_version);
}
void update_max_merged_trans_version(const int64_t max_merged_trans_version)
{
if (max_merged_trans_version > max_merged_trans_version_) {
max_merged_trans_version_ = max_merged_trans_version;
}
}
inline void update_min_merged_trans_version(const int64_t min_merged_trans_version)
{
if (OB_UNLIKELY(min_merged_trans_version < min_merged_trans_version_)) {
min_merged_trans_version_ = min_merged_trans_version;
}
}
bool is_contain_uncommitted_row() const { return contain_uncommitted_row_; }
inline bool has_string_out_row() const { return has_string_out_row_; }
inline bool has_lob_out_row() const { return has_lob_out_row_; }
@ -217,7 +230,7 @@ protected:
OB_INLINE void calc_column_checksums_ptr(ObMicroBufferWriter &data_buffer)
{
ObMicroBlockHeader *header = reinterpret_cast<ObMicroBlockHeader*>(data_buffer.data());
if (header->column_checksums_ != nullptr) {
if (header->column_checksums_ != nullptr && header->has_column_checksum_) {
header->column_checksums_ = reinterpret_cast<int64_t *>(
data_buffer.data() + ObMicroBlockHeader::COLUMN_CHECKSUM_PTR_OFFSET);
}
@ -232,6 +245,7 @@ protected:
int32_t last_rows_count_;
int64_t micro_block_merge_verify_level_;
int64_t max_merged_trans_version_;
int64_t min_merged_trans_version_;
int64_t block_size_upper_bound_;
bool contain_uncommitted_row_;
bool has_string_out_row_;

View File

@ -951,7 +951,7 @@ int ObMacroBlockWriter::update_micro_commit_info(const ObDatumRow &row)
const int64_t trans_version_col_idx = data_store_desc_->get_schema_rowkey_col_cnt();
const int64_t cur_row_version = row.storage_datums_[trans_version_col_idx].get_int();
if (!data_store_desc_->is_major_merge_type() || data_store_desc_->get_major_working_cluster_version() >= DATA_VERSION_4_3_0_0) {
micro_writer_->update_max_merged_trans_version(-cur_row_version);
micro_writer_->update_merged_trans_version(-cur_row_version);
}
}
return ret;

View File

@ -262,7 +262,7 @@ int ObMicroBlockHeader::deep_copy(
MEMCPY(header->column_checksums_, column_checksums_, column_count_ * sizeof(int64_t));
new_pos += column_count_ * sizeof(int64_t);
}
} else {
} else if (!has_min_merged_trans_version_) {
header->column_checksums_ = nullptr;
}
pos += new_pos;
@ -306,7 +306,7 @@ int ObMicroBlockHeader::deserialize(const char *buf, int64_t buf_len, int64_t &p
column_checksums = reinterpret_cast<const int64_t *>(buf + new_pos);
column_checksums_ = const_cast<int64_t *>(column_checksums);
new_pos += column_count_ * sizeof(int64_t);
} else {
} else if (!has_min_merged_trans_version_) {
column_checksums_ = nullptr;
}
pos += new_pos;

View File

@ -34,7 +34,8 @@ public:
uint16_t all_lob_in_row_ : 1; // compatible with 4.0, we assume that all lob is out row in old data
uint16_t contains_hash_index_ : 1;
uint16_t hash_index_offset_from_end_ : 10;
uint16_t reserved16_ : 2;
uint16_t has_min_merged_trans_version_ : 1;
uint16_t reserved16_ : 1;
};
uint32_t row_count_;
uint8_t row_store_type_;
@ -71,7 +72,10 @@ public:
int32_t data_length_;
int32_t data_zlength_;
int64_t data_checksum_;
int64_t *column_checksums_;
union {
int64_t *column_checksums_;
int64_t min_merged_trans_version_;
};
public:
ObMicroBlockHeader();
~ObMicroBlockHeader() = default;

View File

@ -769,7 +769,7 @@ int ObMicroBlockReader::filter_pushdown_filter(
if (1 != filter.get_col_count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected col_ids count: not 1", K(ret), K(filter));
} else if (OB_FAIL(filter_white_filter(white_filter, cols_desc.at(col_offsets.at(0)).col_type_, datum_buf[0], filtered))) {
} else if (OB_FAIL(filter_white_filter(white_filter, datum_buf[0], filtered))) {
LOG_WARN("Failed to filter row with white filter", K(ret), K(row_idx));
}
}
@ -1051,6 +1051,8 @@ int ObMicroBlockReader::get_aggregate_result(
ObStorageDatum tmp_datum; // used for deep copy decimalint
const bool has_lob_out_row = col_param.get_meta_type().is_lob_storage() && header_->has_lob_out_row();
bool need_reuse_lob_locator = false;
const bool need_padding = is_pad_char_to_full_length(context.sql_mode_) &&
col_param.get_meta_type().is_fixed_len_char_type();
for (int64_t i = 0; OB_SUCC(ret) && i < row_cap; ++i) {
row_idx = row_ids[i];
datum.set_nop();
@ -1063,6 +1065,11 @@ int ObMicroBlockReader::get_aggregate_result(
col_idx,
tmp_datum))) {
LOG_WARN("fail to read column", K(ret), K(i), K(col_idx), K(row_idx));
} else if (need_padding && OB_FAIL(pad_column(col_param.get_meta_type(),
col_param.get_accuracy(),
allocator_.get_inner_allocator(),
tmp_datum))) {
LOG_WARN("Failed to pad column", K(ret), K(col_param), K(datum));
} else if (!tmp_datum.is_nop() && OB_FAIL(datum.from_storage_datum(tmp_datum, map_type))) {
LOG_WARN("Failed to convert storage datum", K(ret), K(i), K(col_offset), K(tmp_datum), K(obj_type), K(map_type));
} else if (has_lob_out_row && !datum.is_nop() && !datum.is_null() && !datum.get_lob_data().in_row_) {
@ -1131,11 +1138,18 @@ int ObMicroBlockReader::get_aggregate_result(
const int32_t col_offset = agg_cells.at(i)->get_col_offset();
tmp_datum.set_nop();
if (OB_COUNT_AGG_PD_COLUMN_ID != col_offset) {
const bool need_padding = is_pad_char_to_full_length(context.sql_mode_) &&
col_params->at(col_offset)->get_meta_type().is_fixed_len_char_type();
const ObObjMeta &obj_meta = cols_desc.at(col_offset).col_type_;
const ObObjDatumMapType map_type = ObDatum::get_obj_datum_map_type(obj_meta.get_type());
if (row_buf.storage_datums_[col_offset].is_nop()) {
} else if (row_buf.storage_datums_[col_offset].is_null()) {
tmp_datum.set_null();
} else if (need_padding && OB_FAIL(pad_column(col_params->at(col_offset)->get_meta_type(),
col_params->at(col_offset)->get_accuracy(),
allocator_.get_inner_allocator(),
row_buf.storage_datums_[col_offset]))) {
LOG_WARN("Failed to pad column", K(ret), K(col_offset), K(row_buf.storage_datums_));
} else if (OB_FAIL(tmp_datum.from_storage_datum(row_buf.storage_datums_[col_offset], map_type))) {
LOG_WARN("Failed to convert storage datum", K(ret), K(i), K(col_offset),
K(row_buf.storage_datums_[col_offset]), K(obj_meta.get_type()), K(map_type));

View File

@ -805,7 +805,7 @@ int ObIMicroBlockRowScanner::filter_pushdown_filter(
*static_cast<sql::ObWhiteFilterExecutor *>(filter),
pd_filter_info,
bitmap))) {
LOG_WARN("Failed to execute black pushdown filter", K(ret));
LOG_WARN("Failed to execute white pushdown filter", K(ret));
}
}
} else {
@ -1155,6 +1155,7 @@ void ObMultiVersionMicroBlockRowScanner::reuse()
is_last_multi_version_row_ = true;
read_row_direct_flag_ = false;
ignore_shadow_row_ = false;
use_pre_micro_row_ = false;
}
void ObMultiVersionMicroBlockRowScanner::inner_reset()
@ -1243,22 +1244,43 @@ int ObMultiVersionMicroBlockRowScanner::open(
} else if (OB_FAIL(set_base_scan_param(is_left_border, is_right_border))) {
LOG_WARN("failed to set base scan param", K(ret), K(is_left_border), K(is_right_border));
} else {
use_pre_micro_row_ = false;
int64_t column_number = block_data.get_micro_header()->column_count_;
int64_t max_col_count = MAX(read_info_->get_request_count(), column_number);
if (OB_FAIL(tmp_row_.reserve(max_col_count))) {
LOG_WARN("fail to reserve memory for tmp datumrow", K(ret), K(max_col_count));
}
}
if (OB_SUCC(ret) && ObIMicroBlockReaderInfo::INVALID_ROW_INDEX != current_) {
if (OB_UNLIKELY(!reverse_scan_ && !is_last_multi_version_row_)) {
const ObRowHeader *row_header = nullptr;
if (OB_FAIL(reader_->get_row_header(current_, row_header))) {
LOG_WARN("failed to get row header", K(ret), K(current_), K_(macro_id));
} else {
ObMultiVersionRowFlag row_flag;
row_flag.flag_ = row_header->get_mvcc_row_flag();
if (row_flag.is_first_multi_version_row()) {
use_pre_micro_row_ = !finish_scanning_cur_rowkey_;
finish_scanning_cur_rowkey_ = true;
is_last_multi_version_row_ = true;
}
}
}
}
if (OB_SUCC(ret)) {
if (reverse_scan_) {
reserved_pos_ = current_;
}
read_row_direct_flag_ = false;
can_ignore_multi_version_ = false;
if (OB_NOT_NULL(sstable_)
if (OB_UNLIKELY(!block_data.get_micro_header()->has_min_merged_trans_version_)) {
LOG_INFO("micro block header not has_min_merged_trans_version_", K(ret), K(block_data));
} else if (OB_NOT_NULL(sstable_)
&& !block_data.get_micro_header()->contain_uncommitted_rows()
&& block_data.get_micro_header()->max_merged_trans_version_ <= context_->trans_version_range_.snapshot_version_
&& 0 == context_->trans_version_range_.base_version_) {
&& block_data.get_micro_header()->min_merged_trans_version_ > context_->trans_version_range_.base_version_) {
// read_row_direct_flag_ can only be effective when read base version is zero
// since we have no idea about the accurate base version of the macro block
read_row_direct_flag_ = true;
@ -1281,7 +1303,10 @@ int ObMultiVersionMicroBlockRowScanner::inner_get_next_row(const ObDatumRow *&ro
{
int ret = OB_SUCCESS;
// TODO(yuanzhe) refactor blockscan opt of multi version sstable
if (can_ignore_multi_version_) {
if (OB_UNLIKELY(use_pre_micro_row_)) {
row = &prev_micro_row_;
use_pre_micro_row_ = false;
} else if (can_ignore_multi_version_) {
if (OB_FAIL(ObIMicroBlockRowScanner::inner_get_next_row(row))) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("Failed to inner get next row", K(ret), K_(start), K_(last), K_(current));
@ -1493,10 +1518,16 @@ int ObMultiVersionMicroBlockRowScanner::locate_cursor_to_read(bool &found_first_
} else {
row_flag.flag_ = row_header->get_mvcc_row_flag();
if (row_flag.is_last_multi_version_row()) {
finish_scanning_cur_rowkey_ = false;
found_first_row = true;
reserved_pos_ = current_;
current_ = current_ + 1;
if (!reverse_scan_ && OB_FAIL(end_of_block())) {
if (OB_UNLIKELY(OB_ITER_END != ret)) {
LOG_WARN("failed to judge end of block or not", K(ret), K_(macro_id));
}
} else {
finish_scanning_cur_rowkey_ = false;
found_first_row = true;
}
break;
} else {
current_ = reverse_scan_ ? current_ - 1 : current_ + 1;

View File

@ -277,7 +277,8 @@ public:
sql_sequence_col_idx_(-1),
cell_cnt_(0),
read_row_direct_flag_(false),
ignore_shadow_row_(false)
ignore_shadow_row_(false),
use_pre_micro_row_(false)
{}
virtual ~ObMultiVersionMicroBlockRowScanner() {}
void reuse() override;
@ -295,7 +296,7 @@ public:
const bool is_left_border,
const bool is_right_border) override final;
virtual int set_ignore_shadow_row() override final { ignore_shadow_row_ = true; return OB_SUCCESS; }
INHERIT_TO_STRING_KV("ObMultiVersionMicroBlockRowScanner", ObIMicroBlockRowScanner, K_(read_row_direct_flag), K_(ignore_shadow_row), K_(version_range));
INHERIT_TO_STRING_KV("ObMultiVersionMicroBlockRowScanner", ObIMicroBlockRowScanner, K_(read_row_direct_flag), K_(ignore_shadow_row), K_(version_range), K_(is_last_multi_version_row), K_(finish_scanning_cur_rowkey), K_(use_pre_micro_row));
protected:
virtual int inner_get_next_row(const ObDatumRow *&row) override;
virtual void inner_reset();
@ -343,6 +344,7 @@ private:
common::ObVersionRange version_range_;
bool read_row_direct_flag_;
bool ignore_shadow_row_;
bool use_pre_micro_row_;
};
// multi version sstable micro block scanner for minor merge

View File

@ -223,6 +223,10 @@ int ObMicroBlockWriter::build_block(char *&buf, int64_t &size)
header->row_index_offset_ = static_cast<int32_t>(data_buffer_.length());
header->contain_uncommitted_rows_ = contain_uncommitted_row_;
header->max_merged_trans_version_ = max_merged_trans_version_;
if (OB_LIKELY(!header->has_column_checksum_)) {
header->has_min_merged_trans_version_ = 1;
header->min_merged_trans_version_ = min_merged_trans_version_;
}
header->has_string_out_row_ = has_string_out_row_;
header->all_lob_in_row_ = !has_lob_out_row_;
header->is_last_row_last_flag_ = is_last_row_last_flag_;

View File

@ -502,7 +502,7 @@ int ObDefaultCGScanner::do_filter(sql::ObPushdownFilterExecutor *filter, const s
}
} else {
sql::ObWhiteFilterExecutor *white_filter = static_cast<sql::ObWhiteFilterExecutor *>(filter);
if (OB_FAIL(blocksstable::ObIMicroBlockReader::filter_white_filter(*white_filter, iter_param_->get_read_info()->get_columns_desc().at(0).col_type_ , default_row_.storage_datums_[0], filtered))) {
if (OB_FAIL(blocksstable::ObIMicroBlockReader::filter_white_filter(*white_filter, default_row_.storage_datums_[0], filtered))) {
LOG_WARN("Failed to filter row with white filter", K(ret), KPC(white_filter), K(default_row_));
}
}

View File

@ -1243,6 +1243,7 @@ int ObPartitionMultiRangeSpliter::get_multi_range_size(
int ret = OB_SUCCESS;
ObSEArray<ObITable *, DEFAULT_STORE_CNT_IN_STORAGE> tables;
total_size = 0;
int64_t estimate_size = 0, range_size = 0;
if (OB_UNLIKELY(0 == table_iter.count() || range_array.empty())) {
ret = OB_INVALID_ARGUMENT;
@ -1253,10 +1254,39 @@ int ObPartitionMultiRangeSpliter::get_multi_range_size(
} else if (tables.empty()) {
// only small tables, can not support arbitrary range split
total_size = 0;
} else if (OB_FAIL(try_estimate_range_size(range_array, tables, estimate_size))) {
STORAGE_LOG(WARN, "fail to estimate range size");
} else {
RangeSplitInfoArray range_info_array;
if (OB_FAIL(get_range_split_infos(tables, index_read_info, range_array, range_info_array, total_size))) {
bool all_single_rowkey = false;
if (OB_FAIL(get_range_split_infos(tables, index_read_info, range_array, range_info_array, range_size, all_single_rowkey))) {
STORAGE_LOG(WARN, "Failed to get range split info array", K(ret));
} else {
total_size = estimate_size + range_size;
}
}
return ret;
}
int ObPartitionMultiRangeSpliter::try_estimate_range_size(
const common::ObIArray<common::ObStoreRange> &range_array,
ObIArray<ObITable *> &tables,
int64_t &total_size)
{
int ret = OB_SUCCESS;
total_size = 0;
if (OB_UNLIKELY(range_array.count() >= RANGE_COUNT_THRESOLD)) {
for (int64_t i = 0; OB_SUCC(ret) && i < tables.count(); i++) {
const ObITable * table = tables.at(i);
if (table->is_sstable()
&& static_cast<const ObSSTable *>(table)->get_data_macro_block_count() * FAST_ESTIMATE_THRESOLD / 100 <= range_array.count()) {
total_size += static_cast<const ObSSTable *>(table)->get_occupy_size();
if (OB_FAIL(tables.remove(i))) {
STORAGE_LOG(WARN, "fail to remove table", K(ret), K(i));
}
}
}
}
@ -1457,34 +1487,44 @@ int ObPartitionMultiRangeSpliter::merge_and_push_range_array(
return ret;
}
int ObPartitionMultiRangeSpliter::build_single_range_array(
int ObPartitionMultiRangeSpliter::fast_build_range_array(
const ObIArray<ObStoreRange> &range_array,
const int64_t expected_task_cnt,
ObIAllocator &allocator,
ObArrayArray<ObStoreRange> &multi_range_split_array)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(range_array.empty())) {
if (OB_UNLIKELY(range_array.empty() || expected_task_cnt > range_array.count())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument to build single range array", K(ret), K(range_array));
} else {
RangeSplitArray range_split_array;
ObStoreRange store_range;
for (int64_t i = 0; OB_SUCC(ret) && i < range_array.count(); i++) {
int64_t avg_range_cnt = range_array.count() / expected_task_cnt, remain_range_cnt = range_array.count() % expected_task_cnt;
for (int64_t i =0 ; OB_SUCC(ret) && i < range_array.count(); i++) {
int64_t task_range_cnt = avg_range_cnt + multi_range_split_array.count() < remain_range_cnt ? 1 : 0;
if (OB_FAIL(range_array.at(i).deep_copy(allocator, store_range))) {
STORAGE_LOG(WARN, "Failed to deep copy store range", K(ret), K(i), K(range_array.at(i)));
} else if (OB_FAIL(range_split_array.push_back(store_range))) {
STORAGE_LOG(WARN, "Failed to push back store range", K(ret), K(store_range));
} else {
store_range.reset();
} else if (range_split_array.count() >= task_range_cnt) {
if (OB_FAIL(multi_range_split_array.push_back(range_split_array))) {
STORAGE_LOG(WARN, "Failed to push range split array", K(ret), K(range_split_array));
} else {
range_split_array.reset();
STORAGE_LOG(DEBUG, "Fast split for single task", K(range_array));
}
}
store_range.reset();
}
if (OB_SUCC(ret)) {
if (OB_FAIL(multi_range_split_array.push_back(range_split_array))) {
STORAGE_LOG(WARN, "Failed to push range split array", K(ret), K(range_split_array));
} else {
STORAGE_LOG(DEBUG, "Fast split for single task", K(range_array));
}
if (OB_FAIL(ret)) {
} else if (!range_split_array.empty() || multi_range_split_array.count() != expected_task_cnt) {
ret = OB_ERR_UNEXPECTED;
STORAGE_LOG(WARN, "unexpected multi_range_split_array cnt", K(ret), K(multi_range_split_array.count()), K(expected_task_cnt), K(range_array.count()), K(range_split_array.empty()));
}
}
@ -1501,7 +1541,7 @@ int ObPartitionMultiRangeSpliter::get_split_multi_ranges(
{
int ret = OB_SUCCESS;
ObSEArray<ObITable *, DEFAULT_STORE_CNT_IN_STORAGE> tables;
bool single_array = false;
int64_t fast_range_array_cnt = 0;
multi_range_split_array.reset();
if (OB_UNLIKELY(0 == table_iter.count() || range_array.empty() || expected_task_count <= 0)) {
@ -1510,33 +1550,36 @@ int ObPartitionMultiRangeSpliter::get_split_multi_ranges(
K(range_array), K(expected_task_count));
} else if (OB_UNLIKELY(expected_task_count == 1)) {
STORAGE_LOG(DEBUG, "Unexpected only one split task", K(expected_task_count), K(range_array));
single_array = true;
fast_range_array_cnt = 1;
} else if (OB_FAIL(get_split_tables(table_iter, tables))) {
STORAGE_LOG(WARN, "Failed to get split tables", K(ret), K(table_iter));
} else if (tables.empty()) {
// only small tables, no need split
STORAGE_LOG(DEBUG, "empty split tables", K(table_iter));
single_array = true;
fast_range_array_cnt = 1;
} else {
RangeSplitInfoArray range_info_array;
int64_t total_size = 0;
if (OB_FAIL(get_range_split_infos(tables, index_read_info, range_array, range_info_array, total_size))) {
bool all_single_rowkey = false;
if (OB_FAIL(get_range_split_infos(tables, index_read_info, range_array, range_info_array, total_size, all_single_rowkey))) {
STORAGE_LOG(WARN, "Failed to get range split info array", K(ret));
} else if (total_size == 0) {
STORAGE_LOG(DEBUG, "too small tables to split range", K(total_size), K(range_info_array));
single_array = true;
fast_range_array_cnt = 1;
} else if (all_single_rowkey) {
fast_range_array_cnt = MIN(range_array.count(), expected_task_count);
} else if (OB_FAIL(split_multi_ranges(range_info_array, expected_task_count, total_size, allocator,
multi_range_split_array))) {
STORAGE_LOG(WARN, "Failed to split multi ranges", K(ret));
}
}
if (OB_SUCC(ret) && single_array) {
if (OB_FAIL(build_single_range_array(range_array, allocator, multi_range_split_array))) {
if (OB_SUCC(ret) && fast_range_array_cnt > 0) {
if (OB_FAIL(fast_build_range_array(range_array, fast_range_array_cnt, allocator, multi_range_split_array))) {
STORAGE_LOG(WARN, "Failed to build single range array", K(ret));
}
}
//TODO:huronghui.hrh delete log before merge into master
STORAGE_LOG(TRACE, "finish split multi ranges", K(ret), K(expected_task_count), K(range_array), K(multi_range_split_array.count()), K(multi_range_split_array));
return ret;
}
@ -1545,19 +1588,34 @@ int ObPartitionMultiRangeSpliter::get_range_split_infos(ObIArray<ObITable *> &ta
const ObITableReadInfo &index_read_info,
const ObIArray<ObStoreRange> &range_array,
RangeSplitInfoArray &range_info_array,
int64_t &total_size)
int64_t &total_size,
bool &all_single_rowkey)
{
int ret = OB_SUCCESS;
all_single_rowkey = true;
if (OB_UNLIKELY(tables.empty() || range_array.empty())) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "Invalid argument to get range split info", K(ret), K(tables), K(range_array));
} else {
ObRangeSplitInfo range_info;
for (int64_t i = 0; OB_SUCC(ret) && i < range_array.count(); i++) {
if (FALSE_IT(range_spliter_.reset())) {
} else if (OB_FAIL(range_spliter_.get_range_split_info(
tables, index_read_info, range_array.at(i), range_info))) {
STORAGE_LOG(WARN, "Failed to get range split info", K(ret), K(i), K(range_array.at(i)));
if (range_array.at(i).is_single_rowkey()) {
range_info.store_range_ = &range_array.at(i);
range_info.tables_ = &tables;
range_info.index_read_info_ = &index_read_info;
range_info.total_size_ = DEFAULT_MICRO_BLOCK_SIZE;
range_info.max_macro_block_count_ = 1;
range_info.max_estimate_micro_block_cnt_ = 1;
} else {
range_spliter_.reset();
all_single_rowkey = false;
if (OB_FAIL(range_spliter_.get_range_split_info(
tables, index_read_info, range_array.at(i), range_info))) {
STORAGE_LOG(WARN, "Failed to get range split info", K(ret), K(i), K(range_array.at(i)));
}
}
if (OB_FAIL(ret)) {
} else if (OB_FAIL(range_info_array.push_back(range_info))) {
STORAGE_LOG(WARN, "Failed to push back range info", K(ret), K(range_info));
} else {

View File

@ -291,6 +291,9 @@ private:
typedef common::ObSEArray<common::ObStoreRange, DEFAULT_STORE_RANGE_ARRAY_SIZE> RangeSplitArray;
typedef common::ObSEArray<ObRangeSplitInfo, DEFAULT_STORE_RANGE_ARRAY_SIZE> RangeSplitInfoArray;
private:
int try_estimate_range_size(const common::ObIArray<common::ObStoreRange> &range_array,
ObIArray<ObITable *> &tables,
int64_t &total_size);
int get_split_tables(ObTableStoreIterator &table_iter, common::ObIArray<ObITable *> &tables);
int split_multi_ranges(RangeSplitInfoArray &range_info_array,
const int64_t expected_task_count,
@ -301,15 +304,20 @@ private:
const ObITableReadInfo &index_read_info,
const common::ObIArray<common::ObStoreRange> &range_array,
RangeSplitInfoArray &range_info_array,
int64_t &total_size);
int64_t &total_size,
bool &all_single_rowkey);
int merge_and_push_range_array(const RangeSplitArray &src_range_split_array,
common::ObIAllocator &allocator,
common::ObArrayArray<common::ObStoreRange> &multi_range_split_array);
int build_single_range_array(const common::ObIArray<common::ObStoreRange> &range_array,
common::ObIAllocator &allocator,
common::ObArrayArray<common::ObStoreRange> &multi_range_split_array);
int fast_build_range_array(const common::ObIArray<common::ObStoreRange> &range_array,
const int64_t expected_task_cnt,
common::ObIAllocator &allocator,
common::ObArrayArray<common::ObStoreRange> &multi_range_split_array);
private:
ObPartitionRangeSpliter range_spliter_;
static const int64_t RANGE_COUNT_THRESOLD = 500;
static const int64_t FAST_ESTIMATE_THRESOLD = 80;
};
class ObPartitionMajorSSTableRangeSpliter

View File

@ -285,6 +285,88 @@ inline static common::ObDatumCmpFuncType get_datum_cmp_func(const common::ObObjM
return cmp_func;
}
struct ObDatumComparator
{
public:
ObDatumComparator(const ObDatumCmpFuncType cmp_func, int &ret, bool &equal)
: cmp_func_(cmp_func),
ret_(ret),
equal_(equal)
{}
~ObDatumComparator() {}
OB_INLINE bool operator() (const ObDatum &datum1, const ObDatum &datum2)
{
int &ret = ret_;
int cmp_ret = 0;
if (OB_FAIL(ret)) {
// do nothing
} else if (OB_FAIL(cmp_func_(datum1, datum2, cmp_ret))) {
STORAGE_LOG(WARN, "Failed to compare datum", K(ret), K(datum1), K(datum2), K_(cmp_func));
} else if (0 == cmp_ret && !equal_) {
equal_ = true;
}
return cmp_ret < 0;
}
private:
ObDatumCmpFuncType cmp_func_;
int &ret_;
bool &equal_;
};
enum class ObFilterInCmpType {
MERGE_SEARCH,
BINARY_SEARCH_DICT,
BINARY_SEARCH,
HASH_SEARCH,
};
inline ObFilterInCmpType get_filter_in_cmp_type(
const int64_t row_count,
const int64_t param_count,
const bool is_sorted_dict)
{
// BINARY_HASH_THRESHOLD: means the threshold to choose BINARY_SEARCH or HASH_SEARCH
// When the dictionary is unordered, the only variable available for iteration is param_count.
// Testing has shown that when the data size is small, the overhead of binary search is
// lower than the overhead of computing hashes.
// Therefore, this threshold is temporarily set to a small value(8).
static constexpr int64_t BINARY_HASH_THRESHOLD = 8;
// HASH_BUCKETS: means the number of buckets(slots) in hashset.
// This value is related to the performance of the hashset.
const int64_t HASH_BUCKETS = hash::cal_next_prime(param_count * 2);
ObFilterInCmpType cmp_type = ObFilterInCmpType::HASH_SEARCH;
if (is_sorted_dict) {
if (row_count > 3 * param_count) {
// row_count >> param_count
if (row_count > HASH_BUCKETS * 4) {
cmp_type = ObFilterInCmpType::BINARY_SEARCH_DICT;
} else {
cmp_type = ObFilterInCmpType::MERGE_SEARCH;
}
} else if (row_count * 3 >= param_count) {
// row_count ~~ param_count
if (row_count > HASH_BUCKETS) {
cmp_type = ObFilterInCmpType::MERGE_SEARCH;
} else {
cmp_type = ObFilterInCmpType::HASH_SEARCH;
}
} else {
// row_count << param_count
cmp_type = ObFilterInCmpType::HASH_SEARCH;
}
} else {
// Unordered dict
if (param_count <= BINARY_HASH_THRESHOLD) {
cmp_type = ObFilterInCmpType::BINARY_SEARCH;
} else {
cmp_type = ObFilterInCmpType::HASH_SEARCH;
}
}
return cmp_type;
}
}
}

View File

@ -53,6 +53,23 @@ public:
void setup_obj(ObObj& obj, int64_t column_idx, int64_t seed);
void setup_obj(ObObj& obj, int64_t column_idx);
void init_filter(
sql::ObWhiteFilterExecutor &filter,
const ObIArray<ObObj> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf,
const ObObjMeta &col_meta);
void init_in_filter(
sql::ObWhiteFilterExecutor &filter,
const ObIArray<ObObj> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf,
const ObObjMeta &col_meta);
template<typename T>
int build_decimal_filter_ref(const int64_t ref_cnt, const T *ref_arr,
const int64_t col_offset, ObArray<ObObj> &ref_objs);
@ -280,6 +297,110 @@ int ObPdFilterTestBase::build_white_filter(
return ret;
}
// col_meta is may not equal to param_meta
void ObPdFilterTestBase::init_filter(
sql::ObWhiteFilterExecutor &filter,
const ObIArray<ObObj> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf,
const ObObjMeta &col_meta)
{
int count = filter_objs.count();
ObWhiteFilterOperatorType op_type = filter.filter_.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = count + 1;
filter.filter_.expr_->args_ = expr_p_buf;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
for (int64_t i = 0; i <= count; ++i) {
filter.filter_.expr_->args_[i] = new (expr_buf + 1 + i) ObExpr();
if (i < count) {
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
filter.filter_.expr_->args_[i]->obj_meta_.set_null();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[i]->obj_meta_ = filter_objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = filter_objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.push_back(datums[i]));
if (filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
filter.null_param_contained_ = true;
}
}
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[i]->obj_meta_ = col_meta;
filter.filter_.expr_->args_[i]->datum_meta_.type_ = col_meta.get_type();
}
}
filter.cmp_func_ = get_datum_cmp_func(col_meta, filter.filter_.expr_->args_[0]->obj_meta_);
}
// col_meta is may not equal to param_meta
void ObPdFilterTestBase::init_in_filter(
sql::ObWhiteFilterExecutor &filter,
const ObIArray<ObObj> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf,
const ObObjMeta &col_meta)
{
int count = filter_objs.count();
ASSERT_TRUE(count > 0);
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = 2;
filter.filter_.expr_->args_ = expr_p_buf;
filter.filter_.expr_->args_[0] = new (expr_buf + 1) ObExpr();
filter.filter_.expr_->args_[1] = new (expr_buf + 2) ObExpr();
filter.filter_.expr_->inner_func_cnt_ = count;
filter.filter_.expr_->args_[1]->args_ = expr_p_buf + 2;
ObObjMeta obj_meta = filter_objs.at(0).get_meta();
sql::ObExprBasicFuncs *basic_funcs = ObDatumFuncs::get_basic_func(
obj_meta.get_type(), obj_meta.get_collation_type(), obj_meta.get_scale(), false, obj_meta.has_lob_header());
sql::ObExprBasicFuncs *col_basic_funcs = ObDatumFuncs::get_basic_func(
col_meta.get_type(), col_meta.get_collation_type(), col_meta.get_scale(), false, col_meta.has_lob_header());
ObDatumCmpFuncType cmp_func = get_datum_cmp_func(col_meta, obj_meta);
ObDatumCmpFuncType cmp_func_rev = get_datum_cmp_func(obj_meta, col_meta);
filter.filter_.expr_->args_[0]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[0]->obj_meta_ = col_meta;
filter.filter_.expr_->args_[0]->datum_meta_.type_ = col_meta.get_type();
filter.filter_.expr_->args_[0]->basic_funcs_ = col_basic_funcs;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
ASSERT_EQ(OB_SUCCESS, filter.param_set_.create(count * 2));
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, basic_funcs->null_first_cmp_);
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[1]->args_[i] = new (expr_buf + 3 + i) ObExpr();
filter.filter_.expr_->args_[1]->args_[i]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[1]->args_[i]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[1]->args_[i]->basic_funcs_ = basic_funcs;
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
if (!filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
ASSERT_EQ(OB_SUCCESS, filter.add_to_param_set_and_array(datums[i], filter.filter_.expr_->args_[1]->args_[i]));
}
}
std::sort(filter.datum_params_.begin(), filter.datum_params_.end(),
[cmp_func] (const ObDatum datum1, const ObDatum datum2) {
int cmp_ret = 0;
cmp_func(datum1, datum2, cmp_ret);
return cmp_ret < 0;
});
filter.cmp_func_ = cmp_func;
filter.cmp_func_rev_ = cmp_func_rev;
filter.param_set_.set_hash_and_cmp_func(col_basic_funcs->murmur_hash_v2_, filter.cmp_func_rev_);
}
int ObPdFilterTestBase::check_column_store_white_filter(
const ObWhiteFilterOperatorType &op_type,
const int64_t row_cnt,
@ -320,78 +441,43 @@ int ObPdFilterTestBase::check_column_store_white_filter(
white_filter->col_offsets_.push_back(col_offset);
white_filter->n_cols_ = 1;
int arg_cnt = ref_objs.count() + 1;
int count = ref_objs.count();
ObWhiteFilterOperatorType op_type = pd_filter_node.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
arg_cnt = 2;
count = 1;
}
int count_expr = WHITE_OP_IN == op_type ? count + 3 : count + 2;
int count_expr_p = WHITE_OP_IN == op_type ? count + 2 : count + 1;
sql::ObExpr *expr_buf = reinterpret_cast<sql::ObExpr *>(allocator_.alloc(sizeof(sql::ObExpr) * count_expr));
sql::ObExpr **expr_p_buf = reinterpret_cast<sql::ObExpr **>(allocator_.alloc(sizeof(sql::ObExpr*) * count_expr_p));
void *datum_buf = allocator_.alloc(sizeof(int8_t) * 128 * count);
ObDatum datums[count];
EXPECT_TRUE(OB_NOT_NULL(expr_buf));
EXPECT_TRUE(OB_NOT_NULL(expr_p_buf));
if (WHITE_OP_IN == op_type) {
init_in_filter(*white_filter, ref_objs, expr_buf, expr_p_buf, datums, datum_buf, col_meta);
} else {
init_filter(*white_filter, ref_objs, expr_buf, expr_p_buf, datums, datum_buf, col_meta);
}
void *expr_ptr = allocator_.alloc(sizeof(sql::ObExpr));
MEMSET(expr_ptr, '\0', sizeof(sql::ObExpr));
void *expr_ptr_arr = allocator_.alloc(sizeof(sql::ObExpr*) * arg_cnt);
MEMSET(expr_ptr_arr, '\0', sizeof(sql::ObExpr*) * arg_cnt);
void *expr_arr = allocator_.alloc(sizeof(sql::ObExpr) * arg_cnt);
MEMSET(expr_arr, '\0', sizeof(sql::ObExpr) * arg_cnt);
EXPECT_TRUE(OB_NOT_NULL(expr_ptr));
EXPECT_TRUE(OB_NOT_NULL(expr_ptr_arr));
EXPECT_TRUE(OB_NOT_NULL(expr_arr));
white_filter->filter_.expr_ = reinterpret_cast<sql::ObExpr *>(expr_ptr);
white_filter->filter_.expr_->arg_cnt_ = arg_cnt;
white_filter->filter_.expr_->args_ = reinterpret_cast<sql::ObExpr **>(expr_ptr_arr);
ObDatum datums[arg_cnt];
white_filter->datum_params_.init(arg_cnt);
const int64_t datum_buf_size = sizeof(int8_t) * 128 * arg_cnt;
void *datum_buf = allocator_.alloc(datum_buf_size);
MEMSET(datum_buf, '\0', datum_buf_size);
EXPECT_TRUE(OB_NOT_NULL(datum_buf));
for (int64_t i = 0; OB_SUCC(ret) && (i < arg_cnt); ++i) {
white_filter->filter_.expr_->args_[i] = reinterpret_cast<sql::ObExpr *>(expr_arr) + i;
if (i < arg_cnt - 1) {
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
white_filter->filter_.expr_->args_[i]->obj_meta_.set_null();
white_filter->filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
white_filter->filter_.expr_->args_[i]->obj_meta_ = ref_objs.at(i).get_meta();
white_filter->filter_.expr_->args_[i]->datum_meta_.type_ = ref_objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
if (OB_FAIL(datums[i].from_obj(ref_objs.at(i)))) {
LOG_WARN("fail to handle datum from obj", KR(ret), K(i), K(ref_objs.at(i)));
} else if (OB_FAIL(white_filter->datum_params_.push_back(datums[i]))) {
LOG_WARN("fail to push back", KR(ret), K(i), K(datums[i]));
}
}
} else {
white_filter->filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
white_filter->filter_.expr_->args_[arg_cnt-1]->obj_meta_ = col_meta;
}
}
if (OB_UNLIKELY(2 > white_filter->filter_.expr_->arg_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), K(white_filter->filter_.expr_->arg_cnt_));
} else if (OB_FAIL(decoder.filter_pushdown_filter(nullptr, *white_filter, pd_filter_info, *res_bitmap))) {
LOG_WARN("fail to filter pushdown filter", KR(ret));
} else {
white_filter->cmp_func_ = get_datum_cmp_func(white_filter->filter_.expr_->args_[arg_cnt-1]->obj_meta_, white_filter->filter_.expr_->args_[0]->obj_meta_);
EXPECT_EQ(res_count, res_bitmap->popcnt());
}
if (OB_SUCC(ret) ) {
if (sql::WHITE_OP_IN == white_filter->get_op_type()) {
if (OB_FAIL(white_filter->init_obj_set())) {
LOG_WARN("fail to init obj_set", KR(ret));
}
}
if (FAILEDx(decoder.filter_pushdown_filter(nullptr, *white_filter, pd_filter_info, *res_bitmap))) {
LOG_WARN("fail to filter pushdown filter", KR(ret));
}
if (nullptr != expr_buf) {
allocator_.free(expr_buf);
}
if (OB_SUCC(ret)) {
if (res_count != res_bitmap->popcnt()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("result count not match", K(res_count), K(res_bitmap->popcnt()), K(ref_objs));
// ::ob_abort();
}
if (nullptr != expr_p_buf) {
allocator_.free(expr_p_buf);
}
if (nullptr != datum_buf) {
allocator_.free(datum_buf);
}
}
return ret;

View File

@ -527,6 +527,12 @@ protected:
void bt_op_test_for_all(const bool is_column_store);
void bt_op_test_for_simple(const bool is_column_store);
void bt_op_test_for_general(const bool is_column_store);
void init_filter(sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf, sql::ObExpr **expr_p_buf, ObDatum *datums, void *datum_buf);
void init_in_filter(sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf, sql::ObExpr **expr_p_buf, ObDatum *datums, void *datum_buf);
int test_filter_pushdown(const bool is_column_store, const uint64_t col_idx,
ObIMicroBlockDecoder *decoder, sql::ObPushdownWhiteFilterNode &filter_node,
const int64_t row_start, const int64_t row_count, common::ObBitmap &result_bitmap,
@ -1882,6 +1888,102 @@ void TestDecoderFilterPerf::bt_op_test_for_general(const bool is_column_store)
}
}
void TestDecoderFilterPerf::init_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ObWhiteFilterOperatorType op_type = filter.filter_.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = count + 1;
filter.filter_.expr_->args_ = expr_p_buf;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
for (int64_t i = 0; i <= count; ++i) {
filter.filter_.expr_->args_[i] = new (expr_buf + 1 + i) ObExpr();
if (i < count) {
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
filter.filter_.expr_->args_[i]->obj_meta_.set_null();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[i]->obj_meta_ = filter_objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = filter_objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.push_back(datums[i]));
if (filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
filter.null_param_contained_ = true;
}
}
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[i]->obj_meta_.set_null(); // unused
}
}
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
}
void TestDecoderFilterPerf::init_in_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ASSERT_TRUE(count > 0);
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = 2;
filter.filter_.expr_->args_ = expr_p_buf;
filter.filter_.expr_->args_[0] = new (expr_buf + 1) ObExpr();
filter.filter_.expr_->args_[1] = new (expr_buf + 2) ObExpr();
filter.filter_.expr_->inner_func_cnt_ = count;
filter.filter_.expr_->args_[1]->args_ = expr_p_buf + 2;
ObObjMeta obj_meta = filter_objs.at(0).get_meta();
sql::ObExprBasicFuncs *basic_funcs = ObDatumFuncs::get_basic_func(
obj_meta.get_type(), obj_meta.get_collation_type(), obj_meta.get_scale(), false, obj_meta.has_lob_header());
ObDatumCmpFuncType cmp_func = get_datum_cmp_func(obj_meta, obj_meta);
filter.filter_.expr_->args_[0]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[0]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[0]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[0]->basic_funcs_ = basic_funcs;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
ASSERT_EQ(OB_SUCCESS, filter.param_set_.create(count * 2));
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, basic_funcs->null_first_cmp_);
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[1]->args_[i] = new (expr_buf + 3 + i) ObExpr();
filter.filter_.expr_->args_[1]->args_[i]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[1]->args_[i]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[1]->args_[i]->basic_funcs_ = basic_funcs;
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
if (!filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
ASSERT_EQ(OB_SUCCESS, filter.add_to_param_set_and_array(datums[i], filter.filter_.expr_->args_[1]->args_[i]));
}
}
std::sort(filter.datum_params_.begin(), filter.datum_params_.end(),
[cmp_func] (const ObDatum datum1, const ObDatum datum2) {
int cmp_ret = 0;
cmp_func(datum1, datum2, cmp_ret);
return cmp_ret < 0;
});
filter.cmp_func_ = cmp_func;
filter.cmp_func_rev_ = cmp_func;
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, filter.cmp_func_rev_);
}
int TestDecoderFilterPerf::test_filter_pushdown(
const bool is_column_store,
const uint64_t col_idx,
@ -1915,66 +2017,39 @@ int TestDecoderFilterPerf::test_filter_pushdown(
pd_filter_info.col_capacity_ = full_column_cnt_;
pd_filter_info.start_ = row_start;
pd_filter_info.count_ = row_count;
int count = objs.count();
int64_t arg_cnt = objs.count() + 1;
if (sql::WHITE_OP_NU == filter_node.get_op_type() ||
sql::WHITE_OP_NN == filter_node.get_op_type()) {
arg_cnt = 2;
ObWhiteFilterOperatorType op_type = filter_node.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
int count_expr = WHITE_OP_IN == op_type ? count + 3 : count + 2;
int count_expr_p = WHITE_OP_IN == op_type ? count + 2 : count + 1;
sql::ObExpr *expr_buf = reinterpret_cast<sql::ObExpr *>(allocator_.alloc(sizeof(sql::ObExpr) * count_expr));
sql::ObExpr **expr_p_buf = reinterpret_cast<sql::ObExpr **>(allocator_.alloc(sizeof(sql::ObExpr*) * count_expr_p));
void *datum_buf = allocator_.alloc(sizeof(int8_t) * 128 * count);
ObDatum datums[count];
EXPECT_TRUE(OB_NOT_NULL(expr_buf));
EXPECT_TRUE(OB_NOT_NULL(expr_p_buf));
void *expr_ptr = cur_allocator.alloc(sizeof(sql::ObExpr));
void *expr_ptr_arr = cur_allocator.alloc(sizeof(sql::ObExpr*) * arg_cnt);
void *expr_arr = cur_allocator.alloc(sizeof(sql::ObExpr) * arg_cnt);
if (OB_ISNULL(expr_ptr) || OB_ISNULL(expr_ptr_arr) || OB_ISNULL(expr_arr)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
if (WHITE_OP_IN == op_type) {
init_in_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
} else {
filter.filter_.expr_ = reinterpret_cast<sql::ObExpr *>(expr_ptr);
filter.filter_.expr_->args_ = reinterpret_cast<sql::ObExpr **>(expr_ptr_arr);
filter.filter_.expr_->arg_cnt_ = arg_cnt;
filter.datum_params_.init(arg_cnt);
int64_t datum_buf_size = sizeof(int8_t) * 128 * arg_cnt;
void *datum_buf = cur_allocator.alloc(datum_buf_size);
MEMSET(datum_buf, '\0', datum_buf_size);
EXPECT_TRUE(datum_buf != nullptr);
ObDatum datums[arg_cnt];
for (int64_t idx = 0; idx < arg_cnt; ++idx) {
filter.filter_.expr_->args_[idx] = reinterpret_cast<sql::ObExpr *>(expr_arr) + idx;
if (idx < arg_cnt - 1) {
if (sql::WHITE_OP_NU == filter_node.get_op_type() ||
sql::WHITE_OP_NN == filter_node.get_op_type()) {
filter.filter_.expr_->args_[idx]->obj_meta_.set_null();
filter.filter_.expr_->args_[idx]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[idx]->obj_meta_ = objs.at(idx).get_meta();
filter.filter_.expr_->args_[idx]->datum_meta_.type_ = objs.at(idx).get_meta().get_type();
datums[idx].ptr_ = reinterpret_cast<char *>(datum_buf) + idx * 128;
datums[idx].from_obj(objs.at(idx));
filter.datum_params_.push_back(datums[idx]);
}
} else {
filter.filter_.expr_->args_[idx]->type_ = T_REF_COLUMN;
}
}
if (OB_UNLIKELY(2 > filter.filter_.expr_->arg_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), K(filter.filter_.expr_->arg_cnt_));
} else {
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
if (sql::WHITE_OP_IN == filter.get_op_type()) {
filter.init_obj_set();
}
uint64_t start_ns = ObTimeUtility::current_time_ns();
ret = decoder->filter_pushdown_filter(nullptr, filter, pd_filter_info, result_bitmap);
filter_cost_ns = ObTimeUtility::current_time_ns() - start_ns;
if (nullptr != storage_datum_buf) { cur_allocator.free(storage_datum_buf); }
if (nullptr != expr_ptr) { cur_allocator.free(expr_ptr); }
if (nullptr != expr_ptr_arr) { cur_allocator.free(expr_ptr_arr); }
if (nullptr != expr_arr) { cur_allocator.free(expr_arr); }
if (nullptr != datum_buf) { cur_allocator.free(datum_buf); }
}
init_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
}
if (OB_UNLIKELY(2 > filter.filter_.expr_->arg_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), K(filter.filter_.expr_->arg_cnt_));
} else {
uint64_t start_ns = ObTimeUtility::current_time_ns();
ret = decoder->filter_pushdown_filter(nullptr, filter, pd_filter_info, result_bitmap);
filter_cost_ns = ObTimeUtility::current_time_ns() - start_ns;
}
if (nullptr != storage_datum_buf) { cur_allocator.free(storage_datum_buf); }
if (nullptr != expr_buf) { cur_allocator.free(expr_buf); }
if (nullptr != expr_p_buf) { cur_allocator.free(expr_p_buf); }
if (nullptr != datum_buf) { cur_allocator.free(datum_buf); }
return ret;
}

View File

@ -110,6 +110,22 @@ public:
inline void setup_obj(ObObj& obj, int64_t column_id, int64_t seed);
void init_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf);
void init_in_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf);
int test_filter_pushdown(
const uint64_t col_idx,
bool is_retro,
@ -436,6 +452,102 @@ inline void TestColumnDecoder::setup_obj(ObObj& obj, int64_t column_id, int64_t
}
}
void TestColumnDecoder::init_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ObWhiteFilterOperatorType op_type = filter.filter_.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = count + 1;
filter.filter_.expr_->args_ = expr_p_buf;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
for (int64_t i = 0; i <= count; ++i) {
filter.filter_.expr_->args_[i] = new (expr_buf + 1 + i) ObExpr();
if (i < count) {
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
filter.filter_.expr_->args_[i]->obj_meta_.set_null();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[i]->obj_meta_ = filter_objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = filter_objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.push_back(datums[i]));
if (filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
filter.null_param_contained_ = true;
}
}
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[i]->obj_meta_.set_null(); // unused
}
}
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
}
void TestColumnDecoder::init_in_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ASSERT_TRUE(count > 0);
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = 2;
filter.filter_.expr_->args_ = expr_p_buf;
filter.filter_.expr_->args_[0] = new (expr_buf + 1) ObExpr();
filter.filter_.expr_->args_[1] = new (expr_buf + 2) ObExpr();
filter.filter_.expr_->inner_func_cnt_ = count;
filter.filter_.expr_->args_[1]->args_ = expr_p_buf + 2;
ObObjMeta obj_meta = filter_objs.at(0).get_meta();
sql::ObExprBasicFuncs *basic_funcs = ObDatumFuncs::get_basic_func(
obj_meta.get_type(), obj_meta.get_collation_type(), obj_meta.get_scale(), false, obj_meta.has_lob_header());
ObDatumCmpFuncType cmp_func = get_datum_cmp_func(obj_meta, obj_meta);
filter.filter_.expr_->args_[0]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[0]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[0]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[0]->basic_funcs_ = basic_funcs;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
ASSERT_EQ(OB_SUCCESS, filter.param_set_.create(count * 2));
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, basic_funcs->null_first_cmp_);
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[1]->args_[i] = new (expr_buf + 3 + i) ObExpr();
filter.filter_.expr_->args_[1]->args_[i]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[1]->args_[i]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[1]->args_[i]->basic_funcs_ = basic_funcs;
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
if (!filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
ASSERT_EQ(OB_SUCCESS, filter.add_to_param_set_and_array(datums[i], filter.filter_.expr_->args_[1]->args_[i]));
}
}
std::sort(filter.datum_params_.begin(), filter.datum_params_.end(),
[cmp_func] (const ObDatum datum1, const ObDatum datum2) {
int cmp_ret = 0;
cmp_func(datum1, datum2, cmp_ret);
return cmp_ret < 0;
});
filter.cmp_func_ = cmp_func;
filter.cmp_func_rev_ = cmp_func;
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, filter.cmp_func_rev_);
}
int TestColumnDecoder::test_filter_pushdown(
const uint64_t col_idx,
bool is_retro,
@ -463,48 +575,31 @@ int TestColumnDecoder::test_filter_pushdown(
pd_filter_info.col_capacity_ = full_column_cnt_;
pd_filter_info.start_ = 0;
pd_filter_info.count_ = decoder.row_count_;
int count = objs.count() + 1;
if (sql::WHITE_OP_NU == filter_node.get_op_type() ||
sql::WHITE_OP_NN == filter_node.get_op_type()) {
count = 2;
}
void *expr_buf1 = allocator_.alloc(sizeof(sql::ObExpr));
void *expr_buf2 = allocator_.alloc(sizeof(sql::ObExpr*) * count);
void *expr_buf3 = allocator_.alloc(sizeof(sql::ObExpr) * count);
filter.filter_.expr_ = reinterpret_cast<sql::ObExpr *>(expr_buf1);
filter.filter_.expr_->args_ = reinterpret_cast<sql::ObExpr **>(expr_buf2);
filter.filter_.expr_->arg_cnt_ = count;
filter.datum_params_.init(count);
int count = objs.count();
ObWhiteFilterOperatorType op_type = filter_node.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
int count_expr = WHITE_OP_IN == op_type ? count + 3 : count + 2;
int count_expr_p = WHITE_OP_IN == op_type ? count + 2 : count + 1;
sql::ObExpr *expr_buf = reinterpret_cast<sql::ObExpr *>(allocator_.alloc(sizeof(sql::ObExpr) * count_expr));
sql::ObExpr **expr_p_buf = reinterpret_cast<sql::ObExpr **>(allocator_.alloc(sizeof(sql::ObExpr*) * count_expr_p));
void *datum_buf = allocator_.alloc(sizeof(int8_t) * 128 * count);
ObDatum datums[count];
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[i] = reinterpret_cast<sql::ObExpr *>(expr_buf3) + i;
if (i < count - 1) {
if (sql::WHITE_OP_NU == filter_node.get_op_type() ||
sql::WHITE_OP_NN == filter_node.get_op_type()) {
filter.filter_.expr_->args_[i]->obj_meta_.set_null();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[i]->obj_meta_ = objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(objs.at(i));
filter.datum_params_.push_back(datums[i]);
}
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
}
EXPECT_TRUE(OB_NOT_NULL(expr_buf));
EXPECT_TRUE(OB_NOT_NULL(expr_p_buf));
if (WHITE_OP_IN == op_type) {
init_in_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
} else {
init_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
}
if (OB_UNLIKELY(2 > filter.filter_.expr_->arg_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), K(filter.filter_.expr_->arg_cnt_));
} else {
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
if (sql::WHITE_OP_IN == filter.get_op_type()) {
filter.init_obj_set();
}
if (is_retro) {
ret = decoder.filter_pushdown_retro(nullptr, filter, pd_filter_info, col_idx, filter.col_params_.at(0), pd_filter_info.datum_buf_[0], result_bitmap);
} else {
@ -514,14 +609,11 @@ int TestColumnDecoder::test_filter_pushdown(
if (nullptr != storage_datum_buf) {
allocator_.free(storage_datum_buf);
}
if (nullptr != expr_buf1) {
allocator_.free(expr_buf1);
if (nullptr != expr_buf) {
allocator_.free(expr_buf);
}
if (nullptr != expr_buf2) {
allocator_.free(expr_buf2);
}
if (nullptr != expr_buf3) {
allocator_.free(expr_buf3);
if (nullptr != expr_p_buf) {
allocator_.free(expr_p_buf);
}
if (nullptr != datum_buf) {
allocator_.free(datum_buf);
@ -557,48 +649,31 @@ int TestColumnDecoder::test_filter_pushdown_with_pd_info(
pd_filter_info.col_capacity_ = full_column_cnt_;
pd_filter_info.start_ = start;
pd_filter_info.count_ = end - start;
int count = objs.count() + 1;
if (sql::WHITE_OP_NU == filter_node.get_op_type() ||
sql::WHITE_OP_NN == filter_node.get_op_type()) {
count = 2;
}
void *expr_buf1 = allocator_.alloc(sizeof(sql::ObExpr));
void *expr_buf2 = allocator_.alloc(sizeof(sql::ObExpr*) * count);
void *expr_buf3 = allocator_.alloc(sizeof(sql::ObExpr) * count);
filter.filter_.expr_ = reinterpret_cast<sql::ObExpr *>(expr_buf1);
filter.filter_.expr_->args_ = reinterpret_cast<sql::ObExpr **>(expr_buf2);
filter.filter_.expr_->arg_cnt_ = count;
filter.datum_params_.init(count);
int count = objs.count();
ObWhiteFilterOperatorType op_type = filter_node.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
int count_expr = WHITE_OP_IN == op_type ? count + 3 : count + 2;
int count_expr_p = WHITE_OP_IN == op_type ? count + 2 : count + 1;
sql::ObExpr *expr_buf = reinterpret_cast<sql::ObExpr *>(allocator_.alloc(sizeof(sql::ObExpr) * count_expr));
sql::ObExpr **expr_p_buf = reinterpret_cast<sql::ObExpr **>(allocator_.alloc(sizeof(sql::ObExpr*) * count_expr_p));
void *datum_buf = allocator_.alloc(sizeof(int8_t) * 128 * count);
ObDatum datums[count];
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[i] = reinterpret_cast<sql::ObExpr *>(expr_buf3) + i;
if (i < count - 1) {
if (sql::WHITE_OP_NU == filter_node.get_op_type() ||
sql::WHITE_OP_NN == filter_node.get_op_type()) {
filter.filter_.expr_->args_[i]->obj_meta_.set_null();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[i]->obj_meta_ = objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(objs.at(i));
filter.datum_params_.push_back(datums[i]);
}
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
}
EXPECT_TRUE(OB_NOT_NULL(expr_buf));
EXPECT_TRUE(OB_NOT_NULL(expr_p_buf));
if (WHITE_OP_IN == op_type) {
init_in_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
} else {
init_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
}
if (OB_UNLIKELY(2 > filter.filter_.expr_->arg_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), K(filter.filter_.expr_->arg_cnt_));
} else {
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
if (sql::WHITE_OP_IN == filter.get_op_type()) {
filter.init_obj_set();
}
if (is_retro) {
ret = decoder.filter_pushdown_retro(nullptr, filter, pd_filter_info, col_idx, filter.col_params_.at(0), pd_filter_info.datum_buf_[0], result_bitmap);
} else {
@ -608,14 +683,11 @@ int TestColumnDecoder::test_filter_pushdown_with_pd_info(
if (nullptr != storage_datum_buf) {
allocator_.free(storage_datum_buf);
}
if (nullptr != expr_buf1) {
allocator_.free(expr_buf1);
if (nullptr != expr_buf) {
allocator_.free(expr_buf);
}
if (nullptr != expr_buf2) {
allocator_.free(expr_buf2);
}
if (nullptr != expr_buf3) {
allocator_.free(expr_buf3);
if (nullptr != expr_p_buf) {
allocator_.free(expr_p_buf);
}
if (nullptr != datum_buf) {
allocator_.free(datum_buf);

View File

@ -251,6 +251,20 @@ public:
virtual ~TestRawDecoder() {}
void setup_obj(ObObj& obj, int64_t column_id, int64_t seed);
void init_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf);
void init_in_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf);
int test_filter_pushdown(
const uint64_t col_id,
ObMicroBlockDecoder& decoder,
@ -394,6 +408,102 @@ void TestRawDecoder::setup_obj(ObObj& obj, int64_t column_id, int64_t seed)
}
}
void TestRawDecoder::init_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ObWhiteFilterOperatorType op_type = filter.filter_.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = count + 1;
filter.filter_.expr_->args_ = expr_p_buf;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
for (int64_t i = 0; i <= count; ++i) {
filter.filter_.expr_->args_[i] = new (expr_buf + 1 + i) ObExpr();
if (i < count) {
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
filter.filter_.expr_->args_[i]->obj_meta_.set_null();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[i]->obj_meta_ = filter_objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = filter_objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.push_back(datums[i]));
if (filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
filter.null_param_contained_ = true;
}
}
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[i]->obj_meta_.set_null(); // unused
}
}
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
}
void TestRawDecoder::init_in_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ASSERT_TRUE(count > 0);
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = 2;
filter.filter_.expr_->args_ = expr_p_buf;
filter.filter_.expr_->args_[0] = new (expr_buf + 1) ObExpr();
filter.filter_.expr_->args_[1] = new (expr_buf + 2) ObExpr();
filter.filter_.expr_->inner_func_cnt_ = count;
filter.filter_.expr_->args_[1]->args_ = expr_p_buf + 2;
ObObjMeta obj_meta = filter_objs.at(0).get_meta();
sql::ObExprBasicFuncs *basic_funcs = ObDatumFuncs::get_basic_func(
obj_meta.get_type(), obj_meta.get_collation_type(), obj_meta.get_scale(), false, obj_meta.has_lob_header());
ObDatumCmpFuncType cmp_func = get_datum_cmp_func(obj_meta, obj_meta);
filter.filter_.expr_->args_[0]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[0]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[0]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[0]->basic_funcs_ = basic_funcs;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
ASSERT_EQ(OB_SUCCESS, filter.param_set_.create(count * 2));
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, basic_funcs->null_first_cmp_);
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[1]->args_[i] = new (expr_buf + 3 + i) ObExpr();
filter.filter_.expr_->args_[1]->args_[i]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[1]->args_[i]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[1]->args_[i]->basic_funcs_ = basic_funcs;
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
if (!filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
ASSERT_EQ(OB_SUCCESS, filter.add_to_param_set_and_array(datums[i], filter.filter_.expr_->args_[1]->args_[i]));
}
}
std::sort(filter.datum_params_.begin(), filter.datum_params_.end(),
[cmp_func] (const ObDatum datum1, const ObDatum datum2) {
int cmp_ret = 0;
cmp_func(datum1, datum2, cmp_ret);
return cmp_ret < 0;
});
filter.cmp_func_ = cmp_func;
filter.cmp_func_rev_ = cmp_func;
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, filter.cmp_func_rev_);
}
int TestRawDecoder::test_filter_pushdown(
const uint64_t col_idx,
ObMicroBlockDecoder& decoder,
@ -417,35 +527,31 @@ int TestRawDecoder::test_filter_pushdown(
void *storage_datum_buf = allocator_.alloc(sizeof(ObStorageDatum) * COLUMN_CNT);
EXPECT_TRUE(storage_datum_buf != nullptr);
pd_filter_info.datum_buf_ = new (storage_datum_buf) ObStorageDatum [COLUMN_CNT]();
int count = objs.count() + 1;
void *expr_buf1 = allocator_.alloc(sizeof(sql::ObExpr));
void *expr_buf2 = allocator_.alloc(sizeof(sql::ObExpr*) * count);
void *expr_buf3 = allocator_.alloc(sizeof(sql::ObExpr) * count);
filter.filter_.expr_ = reinterpret_cast<sql::ObExpr *>(expr_buf1);
filter.filter_.expr_->args_ = reinterpret_cast<sql::ObExpr **>(expr_buf2);
filter.filter_.expr_->arg_cnt_ = count;
filter.datum_params_.init(objs.count());
int count = objs.count();
ObWhiteFilterOperatorType op_type = filter_node.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
int count_expr = WHITE_OP_IN == op_type ? count + 3 : count + 2;
int count_expr_p = WHITE_OP_IN == op_type ? count + 2 : count + 1;
sql::ObExpr *expr_buf = reinterpret_cast<sql::ObExpr *>(allocator_.alloc(sizeof(sql::ObExpr) * count_expr));
sql::ObExpr **expr_p_buf = reinterpret_cast<sql::ObExpr **>(allocator_.alloc(sizeof(sql::ObExpr*) * count_expr_p));
void *datum_buf = allocator_.alloc(sizeof(int8_t) * 128 * count);
ObDatum datums[count];
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[i] = reinterpret_cast<sql::ObExpr *>(expr_buf3) + i;
if (i < count - 1) {
filter.filter_.expr_->args_[i]->obj_meta_ = objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(objs.at(i));
filter.datum_params_.push_back(datums[i]);
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
}
EXPECT_TRUE(OB_NOT_NULL(expr_buf));
EXPECT_TRUE(OB_NOT_NULL(expr_p_buf));
if (WHITE_OP_IN == op_type) {
init_in_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
} else {
init_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
}
if (OB_UNLIKELY(2 > filter.filter_.expr_->arg_cnt_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("Unexpected filter expr", K(ret), K(filter.filter_.expr_->arg_cnt_));
} else {
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
filter.init_obj_set();
pd_filter_info.col_capacity_ = full_column_cnt_;
pd_filter_info.start_ = 0;
pd_filter_info.count_ = decoder.row_count_;
@ -455,14 +561,11 @@ int TestRawDecoder::test_filter_pushdown(
if (nullptr != storage_datum_buf) {
allocator_.free(storage_datum_buf);
}
if (nullptr != expr_buf1) {
allocator_.free(expr_buf1);
if (nullptr != expr_buf) {
allocator_.free(expr_buf);
}
if (nullptr != expr_buf2) {
allocator_.free(expr_buf2);
}
if (nullptr != expr_buf3) {
allocator_.free(expr_buf3);
if (nullptr != expr_p_buf) {
allocator_.free(expr_p_buf);
}
if (nullptr != datum_buf) {
allocator_.free(datum_buf);
@ -495,33 +598,27 @@ int TestRawDecoder::test_filter_pushdown_with_pd_info(
void *storage_datum_buf = allocator_.alloc(sizeof(ObStorageDatum) * COLUMN_CNT);
EXPECT_TRUE(storage_datum_buf != nullptr);
pd_filter_info.datum_buf_ = new (storage_datum_buf) ObStorageDatum [COLUMN_CNT]();
int count = objs.count() + 1;
void *expr_buf1 = allocator_.alloc(sizeof(sql::ObExpr));
void *expr_buf2 = allocator_.alloc(sizeof(sql::ObExpr*) * count);
void *expr_buf3 = allocator_.alloc(sizeof(sql::ObExpr) * count);
filter.filter_.expr_ = reinterpret_cast<sql::ObExpr *>(expr_buf1);
filter.filter_.expr_->args_ = reinterpret_cast<sql::ObExpr **>(expr_buf2);
filter.filter_.expr_->arg_cnt_ = count;
filter.datum_params_.init(objs.count());
int count = objs.count();
ObWhiteFilterOperatorType op_type = filter_node.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
int count_expr = WHITE_OP_IN == op_type ? count + 3 : count + 2;
int count_expr_p = WHITE_OP_IN == op_type ? count + 2 : count + 1;
sql::ObExpr *expr_buf = reinterpret_cast<sql::ObExpr *>(allocator_.alloc(sizeof(sql::ObExpr) * count_expr));
sql::ObExpr **expr_p_buf = reinterpret_cast<sql::ObExpr **>(allocator_.alloc(sizeof(sql::ObExpr*) * count_expr_p));
void *datum_buf = allocator_.alloc(sizeof(int8_t) * 128 * count);
ObDatum datums[count];
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[i] = reinterpret_cast<sql::ObExpr *>(expr_buf3) + i;
if (i < count - 1) {
filter.filter_.expr_->args_[i]->obj_meta_ = objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(objs.at(i));
filter.datum_params_.push_back(datums[i]);
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[i]->obj_meta_.set_null(); // unused
}
EXPECT_TRUE(OB_NOT_NULL(expr_buf));
EXPECT_TRUE(OB_NOT_NULL(expr_p_buf));
if (WHITE_OP_IN == op_type) {
init_in_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
} else {
init_filter(filter, objs, expr_buf, expr_p_buf, datums, datum_buf);
}
if (count >= 1) {
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
}
filter.init_obj_set();
pd_filter_info.col_capacity_ = full_column_cnt_;
pd_filter_info.start_ = start;
pd_filter_info.count_ = end - start;
@ -530,14 +627,11 @@ int TestRawDecoder::test_filter_pushdown_with_pd_info(
if (nullptr != storage_datum_buf) {
allocator_.free(storage_datum_buf);
}
if (nullptr != expr_buf1) {
allocator_.free(expr_buf1);
if (nullptr != expr_buf) {
allocator_.free(expr_buf);
}
if (nullptr != expr_buf2) {
allocator_.free(expr_buf2);
}
if (nullptr != expr_buf3) {
allocator_.free(expr_buf3);
if (nullptr != expr_p_buf) {
allocator_.free(expr_p_buf);
}
if (nullptr != datum_buf) {
allocator_.free(datum_buf);

View File

@ -60,6 +60,22 @@ public:
void setup_obj(ObObj& obj, int64_t column_id, int64_t seed);
void init_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf);
void init_in_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf);
int test_filter_pushdown(const uint64_t col_idx,
sql::ObPushdownWhiteFilterNode &filter_node,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
@ -189,6 +205,102 @@ void TestSkipIndexFilter::TearDown()
{
}
void TestSkipIndexFilter::init_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ObWhiteFilterOperatorType op_type = filter.filter_.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = count + 1;
filter.filter_.expr_->args_ = expr_p_buf;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
for (int64_t i = 0; i <= count; ++i) {
filter.filter_.expr_->args_[i] = new (expr_buf + 1 + i) ObExpr();
if (i < count) {
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
filter.filter_.expr_->args_[i]->obj_meta_.set_null();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = ObNullType;
} else {
filter.filter_.expr_->args_[i]->obj_meta_ = filter_objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = filter_objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.push_back(datums[i]));
if (filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
filter.null_param_contained_ = true;
}
}
} else {
filter.filter_.expr_->args_[i]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[i]->obj_meta_.set_null(); // unused
}
}
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
}
void TestSkipIndexFilter::init_in_filter(
sql::ObWhiteFilterExecutor &filter,
common::ObFixedArray<ObObj, ObIAllocator> &filter_objs,
sql::ObExpr *expr_buf,
sql::ObExpr **expr_p_buf,
ObDatum *datums,
void *datum_buf)
{
int count = filter_objs.count();
ASSERT_TRUE(count > 0);
filter.filter_.expr_ = new (expr_buf) ObExpr();
filter.filter_.expr_->arg_cnt_ = 2;
filter.filter_.expr_->args_ = expr_p_buf;
filter.filter_.expr_->args_[0] = new (expr_buf + 1) ObExpr();
filter.filter_.expr_->args_[1] = new (expr_buf + 2) ObExpr();
filter.filter_.expr_->inner_func_cnt_ = count;
filter.filter_.expr_->args_[1]->args_ = expr_p_buf + 2;
ObObjMeta obj_meta = filter_objs.at(0).get_meta();
sql::ObExprBasicFuncs *basic_funcs = ObDatumFuncs::get_basic_func(
obj_meta.get_type(), obj_meta.get_collation_type(), obj_meta.get_scale(), false, obj_meta.has_lob_header());
ObDatumCmpFuncType cmp_func = get_datum_cmp_func(obj_meta, obj_meta);
filter.filter_.expr_->args_[0]->type_ = T_REF_COLUMN;
filter.filter_.expr_->args_[0]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[0]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[0]->basic_funcs_ = basic_funcs;
ASSERT_EQ(OB_SUCCESS, filter.datum_params_.init(count));
ASSERT_EQ(OB_SUCCESS, filter.param_set_.create(count * 2));
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, basic_funcs->null_first_cmp_);
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[1]->args_[i] = new (expr_buf + 3 + i) ObExpr();
filter.filter_.expr_->args_[1]->args_[i]->obj_meta_ = obj_meta;
filter.filter_.expr_->args_[1]->args_[i]->datum_meta_.type_ = obj_meta.get_type();
filter.filter_.expr_->args_[1]->args_[i]->basic_funcs_ = basic_funcs;
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
if (!filter.is_null_param(datums[i], filter_objs.at(i).get_meta())) {
ASSERT_EQ(OB_SUCCESS, filter.add_to_param_set_and_array(datums[i], filter.filter_.expr_->args_[1]->args_[i]));
}
}
std::sort(filter.datum_params_.begin(), filter.datum_params_.end(),
[cmp_func] (const ObDatum datum1, const ObDatum datum2) {
int cmp_ret = 0;
cmp_func(datum1, datum2, cmp_ret);
return cmp_ret < 0;
});
filter.cmp_func_ = cmp_func;
filter.cmp_func_rev_ = cmp_func;
filter.param_set_.set_hash_and_cmp_func(basic_funcs->murmur_hash_v2_, filter.cmp_func_rev_);
}
int TestSkipIndexFilter::test_skip_index_filter_pushdown (
const uint64_t col_idx,
sql::ObPushdownWhiteFilterNode &filter_node,
@ -211,28 +323,26 @@ int TestSkipIndexFilter::test_skip_index_filter_pushdown (
filter.col_params_.push_back(col_param);
filter.col_offsets_.push_back(col_idx);
filter.n_cols_ = 1;
int count = filter_objs.count();
void *expr_buf1 = allocator_.alloc(sizeof(sql::ObExpr));
void *expr_buf2 = allocator_.alloc(sizeof(sql::ObExpr*) * count);
void *expr_buf3 = allocator_.alloc(sizeof(sql::ObExpr) * count);
filter.filter_.expr_ = reinterpret_cast<sql::ObExpr *>(expr_buf1);
filter.filter_.expr_->arg_cnt_ = count;
filter.filter_.expr_->args_ = reinterpret_cast<sql::ObExpr **>(expr_buf2);
filter.datum_params_.init(count);
int count = filter_objs.count();
ObWhiteFilterOperatorType op_type = filter_node.get_op_type();
if (sql::WHITE_OP_NU == op_type || sql::WHITE_OP_NN == op_type) {
count = 1;
}
int count_expr = WHITE_OP_IN == op_type ? count + 3 : count + 2;
int count_expr_p = WHITE_OP_IN == op_type ? count + 2 : count + 1;
sql::ObExpr *expr_buf = reinterpret_cast<sql::ObExpr *>(allocator_.alloc(sizeof(sql::ObExpr) * count_expr));
sql::ObExpr **expr_p_buf = reinterpret_cast<sql::ObExpr **>(allocator_.alloc(sizeof(sql::ObExpr*) * count_expr_p));
void *datum_buf = allocator_.alloc(sizeof(int8_t) * 128 * count);
ObDatum datums[count];
for (int64_t i = 0; i < count; ++i) {
filter.filter_.expr_->args_[i] = reinterpret_cast<sql::ObExpr *>(expr_buf3) + i;
filter.filter_.expr_->args_[i]->obj_meta_ = filter_objs.at(i).get_meta();
filter.filter_.expr_->args_[i]->datum_meta_.type_ = filter_objs.at(i).get_meta().get_type();
datums[i].ptr_ = reinterpret_cast<char *>(datum_buf) + i * 128;
datums[i].from_obj(filter_objs.at(i));
filter.datum_params_.push_back(datums[i]);
EXPECT_TRUE(OB_NOT_NULL(expr_buf));
EXPECT_TRUE(OB_NOT_NULL(expr_p_buf));
if (WHITE_OP_IN == op_type) {
init_in_filter(filter, filter_objs, expr_buf, expr_p_buf, datums, datum_buf);
} else {
init_filter(filter, filter_objs, expr_buf, expr_p_buf, datums, datum_buf);
}
filter.init_obj_set();
filter.check_null_params();
filter.cmp_func_ = get_datum_cmp_func(filter.filter_.expr_->args_[0]->obj_meta_, filter.filter_.expr_->args_[0]->obj_meta_);
// generate agg_row_writer and reader
ObArray<ObSkipIndexColMeta> agg_cols;
@ -277,14 +387,11 @@ int TestSkipIndexFilter::test_skip_index_filter_pushdown (
fal_desc = filter.get_filter_bool_mask();
if (nullptr != expr_buf1) {
allocator_.free(expr_buf1);
if (nullptr != expr_buf) {
allocator_.free(expr_buf);
}
if (nullptr != expr_buf2) {
allocator_.free(expr_buf2);
}
if (nullptr != expr_buf3) {
allocator_.free(expr_buf3);
if (nullptr != expr_p_buf) {
allocator_.free(expr_p_buf);
}
if (nullptr != buf) {
allocator_.free(buf);

View File

@ -270,7 +270,6 @@ ObPushdownFilterExecutor* TestSSTableIndexFilter::create_lt_white_filter(uint64_
datum.ptr_ = reinterpret_cast<char *>(datum_buf) + 128;
datum.from_obj(ref_obj);
filter->datum_params_.push_back(datum);
filter->init_obj_set();
filter->cmp_func_ = get_datum_cmp_func(obj_meta, obj_meta);
return filter;
}