diff --git a/deps/oblib/src/lib/compress/ob_compress_util.h b/deps/oblib/src/lib/compress/ob_compress_util.h index 257c10a03f..70a9485163 100644 --- a/deps/oblib/src/lib/compress/ob_compress_util.h +++ b/deps/oblib/src/lib/compress/ob_compress_util.h @@ -83,6 +83,16 @@ const char *const syslog_compress_funcs[] = "zstd_1.0", "zstd_1.3.8", }; + +const char *const sql_temp_store_compress_funcs[] = +{ + "none", + "zstd", + "lz4", + "snappy", + "zlib" +}; + } /* namespace common */ } /* namespace oceanbase */ diff --git a/deps/oblib/src/lib/utility/ob_tracepoint_def.h b/deps/oblib/src/lib/utility/ob_tracepoint_def.h index 1ffc527347..50c7632e80 100644 --- a/deps/oblib/src/lib/utility/ob_tracepoint_def.h +++ b/deps/oblib/src/lib/utility/ob_tracepoint_def.h @@ -539,8 +539,11 @@ GLOBAL_ERRSIM_POINT_DEF(2212, EN_LOCK_CONFLICT_RETRY_THEN_REROUTE, "force rerout // WR && ASH GLOBAL_ERRSIM_POINT_DEF(2301, EN_CLOSE_ASH, ""); GLOBAL_ERRSIM_POINT_DEF(2302, EN_DISABLE_HASH_BASE_DISTINCT, ""); -GLOBAL_ERRSIM_POINT_DEF(2304, EN_TRACEPOINT_TEST, "For testing new versions of tracepoint"); +GLOBAL_ERRSIM_POINT_DEF(2303, EN_DISABLE_VEC_WINDOW_FUNCTION, "Disable window function operator of vectorization 2.0"); +GLOBAL_ERRSIM_POINT_DEF(2304, EN_FORCE_WINFUNC_STORE_DUMP, "Force to dump row store of window function operator"); +GLOBAL_ERRSIM_POINT_DEF(2305, EN_TRACEPOINT_TEST, "For testing new versions of tracepoint"); +GLOBAL_ERRSIM_POINT_DEF(2306, EN_DISABLE_VEC_MERGE_DISTINCT, "Used to control whether to turn off the vectorization 2.0 merge distinct operator. It is turned on by default."); // force dump GLOBAL_ERRSIM_POINT_DEF(2400, EN_SQL_FORCE_DUMP, "For testing force dump once"); GLOBAL_ERRSIM_POINT_DEF(2401, EN_TEST_FOR_HASH_UNION, "Used to control whether to turn off the vectorization 2.0 hash set operator. It is turned on by default."); diff --git a/src/share/CMakeLists.txt b/src/share/CMakeLists.txt index d03e2d40d9..364199c56f 100644 --- a/src/share/CMakeLists.txt +++ b/src/share/CMakeLists.txt @@ -8,6 +8,7 @@ ob_set_subtarget(ob_share ALONE aggregate/sum.cpp aggregate/processor.cpp aggregate/single_row.cpp + aggregate/approx_count_distinct.cpp ) file(GLOB SCHEMA_CPPS "inner_table/ob_inner_table_schema.*.cpp") @@ -567,6 +568,7 @@ ob_set_subtarget(ob_share aggregate aggregate/iaggregate.cpp aggregate/count.cpp aggregate/min_max.cpp + aggregate/sys_bit.cpp ) ob_add_new_object_target(ob_share ob_share) diff --git a/src/share/aggregate/agg_ctx.h b/src/share/aggregate/agg_ctx.h index 95a5cdfb58..50ad893c75 100644 --- a/src/share/aggregate/agg_ctx.h +++ b/src/share/aggregate/agg_ctx.h @@ -33,11 +33,11 @@ using AggregateExtras = ObAggregateProcessor::ExtraResult **; // --------------------------------------------------------------------------------------------------------------------------------- // | int64 | | int64 | | | int32 (idx) | bits | // --------------------------------------------------------------------------------------------------------------------------------- -// | -// | -// | ... | | -// |-------------------------------------| | -// extra_info array | |<--------- +// | +// | +// | ... | | +// |-------------------------------------| | +// extra_info array | |<------------------------ // |-------------------------------------| // | ... | // @@ -110,6 +110,11 @@ struct AggrRowMeta return *reinterpret_cast(agg_row + nullbits_offset_); } + inline bool is_var_len(const int32_t col_id) const + { + return (use_var_len_ != nullptr && use_var_len_->at(col_id)); + } + TO_STRING_KV(K_(row_size), K_(col_cnt), K_(extra_cnt), K_(nullbits_offset), K_(extra_idx_offset)); int32_t row_size_; int32_t col_cnt_; @@ -159,16 +164,49 @@ struct RowSelector const int32_t size_; }; +struct RemovalInfo +{ + RemovalInfo() : + enable_removal_opt_(false), is_max_min_idx_changed_(false), max_min_index_(-1), + is_inverse_agg_(false), null_cnt_(0) + {} + bool enable_removal_opt_; + bool is_max_min_idx_changed_; + int64_t max_min_index_; // used to record index of min/max value + bool is_inverse_agg_; // used to determine which interface is called: add_batch_rows/remove_batch_rows + int32_t null_cnt_; + TO_STRING_KV(K_(enable_removal_opt), K_(is_max_min_idx_changed), K_(max_min_index), + K_(is_inverse_agg), K_(null_cnt)); + + void reset() + { + enable_removal_opt_ = false; + max_min_index_ = -1; + is_inverse_agg_ = false; + null_cnt_ = 0; + is_max_min_idx_changed_ = false; + } + + void reset_for_new_frame() + { + max_min_index_ = -1; + is_inverse_agg_ = false; + null_cnt_ = 0; + is_max_min_idx_changed_ = false; + } +}; + struct RuntimeContext { RuntimeContext(sql::ObEvalCtx &eval_ctx, uint64_t tenant_id, ObIArray &aggr_infos, - const lib::ObLabel &label, ObIAllocator &struct_allocator) : + const lib::ObLabel &label) : eval_ctx_(eval_ctx), aggr_infos_(aggr_infos), allocator_(label, OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id, ObCtxIds::WORK_AREA), op_monitor_info_(nullptr), io_event_observer_(nullptr), agg_row_meta_(), agg_rows_(ModulePageAllocator(label, tenant_id, ObCtxIds::WORK_AREA)), - agg_extras_(ModulePageAllocator(label, tenant_id, ObCtxIds::WORK_AREA)) + agg_extras_(ModulePageAllocator(label, tenant_id, ObCtxIds::WORK_AREA)), + removal_info_(), win_func_agg_(false) {} inline const AggrRowMeta &row_meta() const @@ -276,6 +314,7 @@ struct RuntimeContext } // end for agg_extras_.reuse(); allocator_.reset_remain_one_page(); + removal_info_.reset(); } void destroy() { @@ -294,6 +333,22 @@ struct RuntimeContext op_monitor_info_ = nullptr; io_event_observer_ = nullptr; agg_row_meta_.reset(); + removal_info_.reset(); + win_func_agg_ = false; + } + + inline void enable_removal_opt() + { + removal_info_.enable_removal_opt_ = true; + } + inline void set_inverse_agg(bool is_inverse) + { + removal_info_.is_inverse_agg_ = is_inverse; + } + + inline void disable_inverse_agg() + { + removal_info_.reset(); } int init_row_meta(ObIArray &aggr_infos, ObIAllocator &alloc); @@ -308,6 +363,8 @@ struct RuntimeContext agg_rows_; ObSegmentArray agg_extras_; + RemovalInfo removal_info_; + bool win_func_agg_; }; /* diff --git a/src/share/aggregate/approx_count_distinct.cpp b/src/share/aggregate/approx_count_distinct.cpp new file mode 100644 index 0000000000..e5542a848c --- /dev/null +++ b/src/share/aggregate/approx_count_distinct.cpp @@ -0,0 +1,89 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#define USING_LOG_PREFIX SQL_ENG + +#include "approx_count_distinct.h" + +namespace oceanbase +{ +namespace share +{ +namespace aggregate +{ +namespace helper +{ +int init_approx_count_distinct_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id, + ObIAllocator &allocator, IAggregate *&agg) +{ +#define INIT_CASE(func_type, vec_tc) \ + case (vec_tc): { \ + if (func_type == T_FUN_APPROX_COUNT_DISTINCT) { \ + if (lib::is_oracle_mode()) { \ + ret = init_agg_func>( \ + agg_ctx, agg_col_id, has_distinct, allocator, agg); \ + } else { \ + ret = init_agg_func>( \ + agg_ctx, agg_col_id, has_distinct, allocator, agg); \ + } \ + } else { \ + ret = init_agg_func>( \ + agg_ctx, agg_col_id, has_distinct, allocator, agg); \ + } \ + } break + +#define INIT_APP_CNT_DISTINCT_CASE(vec_tc) INIT_CASE(T_FUN_APPROX_COUNT_DISTINCT, vec_tc) +#define INIT_APP_CNT_DIST_SYN_CASE(vec_tc) INIT_CASE(T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS, vec_tc) +#define INIT_APP_CNT_DIST_MERGE_CASE(vec_tc) INIT_CASE(T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE, vec_tc) + + int ret = OB_SUCCESS; + ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id); + bool has_distinct = aggr_info.has_distinct_; + if (T_FUN_APPROX_COUNT_DISTINCT == aggr_info.get_expr_type()) { + ObDatumMeta ¶m_meta = aggr_info.param_exprs_.at(0)->datum_meta_; + VecValueTypeClass in_tc = + get_vec_value_tc(param_meta.type_, param_meta.scale_, param_meta.precision_); + switch (in_tc) { + LST_DO_CODE(INIT_APP_CNT_DISTINCT_CASE, AGG_VEC_TC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid param format", K(ret), K(in_tc)); + } + } + } else if (T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE == aggr_info.get_expr_type()) { + ObDatumMeta ¶m_meta = aggr_info.param_exprs_.at(0)->datum_meta_; + VecValueTypeClass in_tc = + get_vec_value_tc(param_meta.type_, param_meta.scale_, param_meta.precision_); + switch (in_tc) { + LST_DO_CODE(INIT_APP_CNT_DIST_MERGE_CASE, AGG_VEC_TC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid param format", K(ret), K(in_tc)); + } + } + } else if (T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS == aggr_info.get_expr_type()) { + VecValueTypeClass vec_tc = + get_vec_value_tc(aggr_info.get_first_child_type(), aggr_info.get_first_child_datum_scale(), + aggr_info.get_first_child_datum_precision()); + switch (vec_tc) { + LST_DO_CODE(INIT_APP_CNT_DIST_SYN_CASE, AGG_VEC_TC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid param format", K(ret), K(vec_tc)); + } + } + } + return ret; +} +} +} // end aggregate +} // end share +} // end oceanbase \ No newline at end of file diff --git a/src/share/aggregate/approx_count_distinct.h b/src/share/aggregate/approx_count_distinct.h new file mode 100644 index 0000000000..f0c6d49468 --- /dev/null +++ b/src/share/aggregate/approx_count_distinct.h @@ -0,0 +1,436 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SHARE_AGGREGATE_APPROX_COUNT_DISTINCT_H +#define OCEANBASE_SHARE_AGGREGATE_APPROX_COUNT_DISTINCT_H + +#include "share/aggregate/iaggregate.h" +#include "src/sql/engine/expr/ob_expr_estimate_ndv.h" + +namespace oceanbase +{ +namespace share +{ +namespace aggregate +{ +template +class ApproxCountDistinct final + : public BatchAggregateWrapper> +{ + static const constexpr int8_t LLC_BUCKET_BITS = 10; + static const constexpr int64_t LLC_NUM_BUCKETS = (1 << LLC_BUCKET_BITS); +public: + static const constexpr VecValueTypeClass IN_TC = in_tc; + static const constexpr VecValueTypeClass OUT_TC = out_tc; +public: + ApproxCountDistinct() {} + template + inline int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + if (agg_func == T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE) { + char *llc_bitmap_buf = EXTRACT_MEM_ADDR(agg_cell); + const char *payload = nullptr; + int32_t len = 0; + columns.get_payload(row_num, payload, len); + if (OB_UNLIKELY(len != LLC_NUM_BUCKETS)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "unexpected length of input", K(ret), K(len)); + } else if (OB_ISNULL(llc_bitmap_buf)) { + // not calculated before, copy from payload + llc_bitmap_buf = (char *)agg_ctx.allocator_.alloc(LLC_NUM_BUCKETS); + if (OB_ISNULL(llc_bitmap_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(WARN, "allocate memory failed", K(ret)); + } else { + MEMSET(llc_bitmap_buf, 0, LLC_NUM_BUCKETS); + MEMCPY(llc_bitmap_buf, payload, LLC_NUM_BUCKETS); + STORE_MEM_ADDR(llc_bitmap_buf, agg_cell); + *reinterpret_cast(agg_cell + sizeof(char *)) = LLC_NUM_BUCKETS; + } + } else { + for (int i = 0; i < LLC_NUM_BUCKETS; i++) { + llc_bitmap_buf[i] = + std::max(static_cast(llc_bitmap_buf[i]), static_cast(payload[i])); + } + } + } else { + const char *payload = nullptr; + int32_t len = 0; + ObExprHashFuncType hash_func = reinterpret_cast(calc_info); + ObDatum tmp_datum; + uint64_t hash_val = 0; + if (OB_ISNULL(tmp_res)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid null tmp result", K(ret)); + } else { + columns.get_payload(row_num, payload, len); + tmp_datum.ptr_ = payload; + tmp_datum.pack_ = len; + if (OB_FAIL(hash_func(tmp_datum, hash_val, hash_val))) { + SQL_LOG(WARN, "hash func failed", K(ret)); + } else if (OB_FAIL( + llc_add_value(hash_val, reinterpret_cast(tmp_res), LLC_NUM_BUCKETS))) { + SQL_LOG(WARN, "llc add value failed", K(ret)); + } + } + } + return ret; + } + + template + inline int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + if (OB_UNLIKELY(columns.is_null(row_num))) { + SQL_LOG(DEBUG, "add null row", K(ret), K(row_num)); + } else if (OB_FAIL( + add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } else { + NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); + not_nulls.set(agg_col_id); + } + return ret; + } + + template + int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr, + const int32_t agg_col_id, const char *agg_cell, + const int32_t agg_cell_len) + { + int ret = OB_SUCCESS; + const NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); + int64_t output_idx = agg_ctx.eval_ctx_.get_batch_idx(); + const char *llc_bitmap_buf = nullptr; + ColumnFmt *res_vec = static_cast(agg_expr.get_vector(agg_ctx.eval_ctx_)); + if (agg_func != T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE) { + llc_bitmap_buf = (const char *)get_tmp_res(agg_ctx, agg_col_id, const_cast(agg_cell)); + } else { + llc_bitmap_buf = reinterpret_cast(*reinterpret_cast(agg_cell)); + } + if (agg_func == T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE) { + if (OB_LIKELY(not_nulls.at(agg_col_id))) { + char *res_buf = agg_expr.get_str_res_mem(agg_ctx.eval_ctx_, LLC_NUM_BUCKETS); + if (OB_ISNULL(res_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(WARN, "allocate memory failed", K(ret)); + } else { + MEMCPY(res_buf, llc_bitmap_buf, LLC_NUM_BUCKETS); + res_vec->set_payload_shallow(output_idx, res_buf, LLC_NUM_BUCKETS); + } + } else { + res_vec->set_null(output_idx); + } + } else if (OB_ISNULL(llc_bitmap_buf)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN , "invalid null llc bitmap", K(ret)); + } else if (agg_func == T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS) { + char *res_buf = agg_expr.get_str_res_mem(agg_ctx.eval_ctx_, LLC_NUM_BUCKETS); + if (OB_ISNULL(res_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(WARN, "allocate memory failed", K(ret)); + } else { + MEMCPY(res_buf, llc_bitmap_buf, LLC_NUM_BUCKETS); + res_vec->set_payload_shallow(output_idx, res_buf, LLC_NUM_BUCKETS); + } + } else if (OB_LIKELY(not_nulls.at(agg_col_id))) { + ObString llc_str(LLC_NUM_BUCKETS, llc_bitmap_buf); + int64_t tmp_result = OB_INVALID_COUNT; + ObExprEstimateNdv::llc_estimate_ndv(tmp_result, llc_str); + if (tmp_result >= 0) { + if (lib::is_mysql_mode()) { + res_vec->set_int(output_idx, tmp_result); + } else { + ObNumStackOnceAlloc tmp_alloc; + number::ObNumber res_nmb; + if (OB_FAIL(res_nmb.from(tmp_result, tmp_alloc))) { + SQL_LOG(WARN, "convert int to number failed", K(ret)); + } else { + res_vec->set_number(output_idx, res_nmb); + } + } + } + } else { + if (lib::is_oracle_mode()) { + // set zero number + number::ObNumber zero_nmb; + zero_nmb.set_zero(); + res_vec->set_payload(output_idx, &zero_nmb, sizeof(ObNumberDesc)); + } else { + res_vec->set_int(output_idx, 0); + } + } + return ret; + } + + template + int add_param_batch(RuntimeContext &agg_ctx, const ObBitVector &skip, ObBitVector &pvt_skip, + const EvalBound &bound, const RowSelector &row_sel, const int32_t agg_col_id, + const int32_t param_id, ColumnFmt ¶m_vec, char *aggr_cell) + { + int ret = OB_SUCCESS; + ob_assert(agg_func == T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS); + char *llc_bitmap_buf = EXTRACT_MEM_ADDR(aggr_cell); + if (OB_ISNULL(llc_bitmap_buf)) { + llc_bitmap_buf = (char *)agg_ctx.allocator_.alloc(LLC_NUM_BUCKETS); + if (OB_ISNULL(llc_bitmap_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(WARN, "allocate memory failed", K(ret)); + } else { + MEMSET(llc_bitmap_buf, 0, LLC_NUM_BUCKETS); + STORE_MEM_ADDR(llc_bitmap_buf, aggr_cell); + } + } + if (OB_FAIL(ret)) { + } else if (row_sel.is_empty()) { + for (int i = bound.start(); i < bound.end(); i++) { + if (pvt_skip.at(i)) { continue; } + if (param_vec.is_null(i)) { + pvt_skip.set(i); + } + } + } else { + for (int i = 0; i < row_sel.size(); i++) { + if (param_vec.is_null(row_sel.index(i))) { pvt_skip.set(row_sel.index(i)); } + } + } + if (param_id == agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.count() - 1) { + // last param, calculate hash values if possible + ObIArray ¶m_exprs = agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_; + const char *payload = nullptr; + int32_t len = 0; + ObDatum tmp_datum; + if (row_sel.is_empty()) { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + uint64_t hash_val = 0; + if (pvt_skip.at(i)) { + continue; + } + for (int j = 0; OB_SUCC(ret) && j < param_exprs.count(); j++) { + ObExpr *expr = param_exprs.at(j); + ObExprHashFuncType hash_func = expr->basic_funcs_->murmur_hash_; + expr->get_vector(agg_ctx.eval_ctx_)->get_payload(i, payload, len); + tmp_datum.ptr_ = payload; + tmp_datum.pack_ = len; + if (OB_FAIL(hash_func(tmp_datum, hash_val, hash_val))) { + SQL_LOG(WARN, "hash calculation failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(llc_add_value(hash_val, llc_bitmap_buf, LLC_NUM_BUCKETS))) { + SQL_LOG(WARN, "add llc value failed"); + } + } + } + } else { + for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) { + if (pvt_skip.at(row_sel.index(i))) { + continue; + } + int64_t row_num = row_sel.index(i); + uint64_t hash_val = 0; + for (int j = 0; OB_SUCC(ret) && j < param_exprs.count(); j++) { + ObExpr *expr = param_exprs.at(j); + ObExprHashFuncType hash_func = expr->basic_funcs_->murmur_hash_; + expr->get_vector(agg_ctx.eval_ctx_)->get_payload(row_num, payload, len); + tmp_datum.ptr_ = payload; + tmp_datum.pack_ = len; + if (OB_FAIL(hash_func(tmp_datum, hash_val, hash_val))) { + SQL_LOG(WARN, "hash calculation failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(llc_add_value(hash_val, llc_bitmap_buf, LLC_NUM_BUCKETS))) { + SQL_LOG(WARN, "add llc value failed"); + } + } + } + } + } + return ret; + } + + int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size, + const bool is_null, const char *data, const int32_t data_len, int32_t agg_col_idx, + char *agg_cell) override + { + int ret = OB_SUCCESS; + void *llc_bitmap_buf = get_tmp_res(agg_ctx, agg_col_idx, agg_cell); + + if (agg_func != T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE && OB_ISNULL(llc_bitmap_buf)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid null llc bitmap buf", K(ret), K(*this)); + } else if (OB_LIKELY(agg_ctx.aggr_infos_.at(agg_col_idx).param_exprs_.count() == 1)) { + if (agg_func == T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE) { + if (OB_UNLIKELY(data_len != LLC_NUM_BUCKETS)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "unexpected input length", K(ret)); + } else if (!is_null) { + char *llc_bitmap_buf = EXTRACT_MEM_ADDR(agg_cell); + if (OB_ISNULL(llc_bitmap_buf)) { + // not calculated before + llc_bitmap_buf = (char *)agg_ctx.allocator_.alloc(LLC_NUM_BUCKETS); + if (OB_ISNULL(llc_bitmap_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(WARN, "allocate memory failed", K(ret)); + } else { + MEMSET(llc_bitmap_buf, 0, LLC_NUM_BUCKETS); + MEMCPY(llc_bitmap_buf, data, LLC_NUM_BUCKETS); + STORE_MEM_ADDR(llc_bitmap_buf, agg_cell); + *reinterpret_cast(agg_cell + sizeof(char *)) = LLC_NUM_BUCKETS; + } + } else { + for (int i = 0; i < LLC_NUM_BUCKETS; i++) { + llc_bitmap_buf[i] = + std::max(static_cast(llc_bitmap_buf[i]), static_cast(data[i])); + } + } + agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell).set(agg_col_idx); + } + } else { + const ObExpr *param_expr = agg_ctx.aggr_infos_.at(agg_col_idx).param_exprs_.at(0); + ObExprHashFuncType hash_func = param_expr->basic_funcs_->murmur_hash_; + if (OB_LIKELY(!is_null)) { + ObDatum tmp_datum; + tmp_datum.ptr_ = data; + tmp_datum.pack_ = data_len; + uint64_t hash_val = 0; + if (OB_FAIL(hash_func(tmp_datum, hash_val, hash_val))) { + SQL_LOG(WARN, "hash calculation failed", K(ret)); + } else if (OB_FAIL(llc_add_value(hash_val, reinterpret_cast(llc_bitmap_buf), + LLC_NUM_BUCKETS))) { + SQL_LOG(WARN, "llc add value failed", K(ret)); + } else { + NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell); + not_nulls.set(agg_col_idx); + } + } + } + } else if (agg_ctx.aggr_infos_.at(agg_col_idx).param_exprs_.count() > 1) { + ob_assert(T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS == agg_func); + ObIArray ¶m_exprs = agg_ctx.aggr_infos_.at(agg_col_idx).param_exprs_; + bool has_null = false; + for (int i = 0; !has_null && i < param_exprs.count(); i++) { + has_null = param_exprs.at(i)->get_vector(agg_ctx.eval_ctx_)->is_null(batch_idx); + } + if (!has_null) { + uint64_t hash_val = 0; + ObDatum tmp_datum; + const char *payload = nullptr; + int32_t len = 0; + for (int i = 0; OB_SUCC(ret) && i < param_exprs.count(); i++) { + ObExpr *expr = param_exprs.at(i); + ObExprHashFuncType hash_func = expr->basic_funcs_->murmur_hash_; + expr->get_vector(agg_ctx.eval_ctx_)->get_payload(batch_idx, payload, len); + tmp_datum.ptr_ = payload; + tmp_datum.pack_ = len; + if (OB_FAIL(hash_func(tmp_datum, hash_val, hash_val))) { + SQL_LOG(WARN, "hash calculation failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(llc_add_value(hash_val, (char *)llc_bitmap_buf, LLC_NUM_BUCKETS))) { + SQL_LOG(WARN, "llc add value failed", K(ret)); + } else { + agg_ctx.locate_notnulls_bitmap(agg_col_idx, agg_cell).set(agg_col_idx); + } + } + } + } + return ret; + } + + inline void *get_tmp_res(RuntimeContext &agg_ctx, int32_t agg_col_id, char *agg_cell) override + { + int ret = OB_SUCCESS; + void *ret_ptr = nullptr; + if (agg_func == T_FUN_APPROX_COUNT_DISTINCT) { + int32_t cell_len = agg_ctx.get_cell_len(agg_col_id, agg_cell); + char *llc_bitmap_buf = EXTRACT_MEM_ADDR((agg_cell + cell_len)); + if (OB_ISNULL(llc_bitmap_buf)) { + // allocate tmp res memory + llc_bitmap_buf = (char *)agg_ctx.allocator_.alloc(LLC_NUM_BUCKETS); + if (OB_ISNULL(llc_bitmap_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(ERROR, "allocate memory failed", K(ret)); + } else { + MEMSET(llc_bitmap_buf, 0, LLC_NUM_BUCKETS); + STORE_MEM_ADDR(llc_bitmap_buf, (agg_cell + cell_len)); + ret_ptr = llc_bitmap_buf; + } + } else { + ret_ptr = llc_bitmap_buf; + } + } else if (agg_func == T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS) { + char *llc_bitmap_buf = EXTRACT_MEM_ADDR(agg_cell); + if (OB_ISNULL(llc_bitmap_buf)) { + llc_bitmap_buf = (char *)agg_ctx.allocator_.alloc(LLC_NUM_BUCKETS); + if (OB_ISNULL(llc_bitmap_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(ERROR, "allocate memory failed", K(ret)); + } else { + MEMSET(llc_bitmap_buf, 0, LLC_NUM_BUCKETS); + STORE_MEM_ADDR(llc_bitmap_buf, agg_cell); + // set result length of approx_count_distinct_synopsis + *reinterpret_cast(agg_cell + sizeof(char *)) = LLC_NUM_BUCKETS; + } + } + if (OB_SUCC(ret)) { + ret_ptr = llc_bitmap_buf; + } + } else { + } + return ret_ptr; + } + + inline int64_t get_batch_calc_info(RuntimeContext &agg_ctx, int32_t agg_col_id, + char *agg_cell) override + { + OB_ASSERT(agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.count() == 1); + const ObExpr *expr = agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.at(0); + OB_ASSERT(expr != NULL); + OB_ASSERT(expr->basic_funcs_ != NULL); + ObExprHashFuncType hash_func = expr->basic_funcs_->murmur_hash_; + return reinterpret_cast(hash_func); + } + TO_STRING_KV("aggregate", "approx_count_distinct", K(in_tc), K(out_tc), K(agg_func)); +private: + int llc_add_value(const uint64_t value, char *llc_bitmap_buf, int64_t size) + { + // copy from `ObAggregateProcessor::llv_add_value` + int ret = OB_SUCCESS; + uint64_t bucket_index = value >> (64 - LLC_BUCKET_BITS); + uint64_t pmax = 0; + if (0 == value << LLC_BUCKET_BITS) { + // do nothing + } else { + pmax = ObExprEstimateNdv::llc_leading_zeros(value << LLC_BUCKET_BITS, 64 - LLC_BUCKET_BITS) + 1; + } + ObString::obstr_size_t llc_num_buckets = size; + OB_ASSERT(size == LLC_NUM_BUCKETS); + OB_ASSERT(ObExprEstimateNdv::llc_is_num_buckets_valid(llc_num_buckets)); + OB_ASSERT(llc_num_buckets > bucket_index); + if (pmax > static_cast(llc_bitmap_buf[bucket_index])) { + // 理论上pmax不会超过65. + llc_bitmap_buf[bucket_index] = static_cast(pmax); + } + return ret; + } +}; + +} // end aggregate +} // end share +} // end oceanbase +#endif // OCEANBASE_SHARE_AGGREGATE_APPROX_COUNT_DISTINCT_H \ No newline at end of file diff --git a/src/share/aggregate/count.cpp b/src/share/aggregate/count.cpp index 967ee56c40..1c5d03b7ef 100644 --- a/src/share/aggregate/count.cpp +++ b/src/share/aggregate/count.cpp @@ -73,7 +73,22 @@ int quick_add_batch_rows_for_count(IAggregate *agg, RuntimeContext &agg_ctx, LOG_WARN("invalid null aggregate", K(ret)); } else if (!is_single_row_agg) { auto &count_agg = *static_cast *>(agg); - if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) { + if (OB_UNLIKELY(agg_ctx.removal_info_.enable_removal_opt_)) { + int64_t &count = *reinterpret_cast(agg_cell); + if (!agg_ctx.removal_info_.is_inverse_agg_) { + if (row_sel.is_empty()) { + count += bound.range_size() - skip.accumulate_bit_cnt(bound); + } else { + count += row_sel.size(); + } + } else { + if (row_sel.is_empty()) { + count -= bound.range_size() - skip.accumulate_bit_cnt(bound); + } else { + count -= row_sel.size(); + } + } + } else if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) { for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { ret = count_agg.add_row(agg_ctx, mock_cols, i, agg_col_id, agg_cell, nullptr, fake_calc_info); diff --git a/src/share/aggregate/count.h b/src/share/aggregate/count.h index 744d042fba..96c8b19031 100644 --- a/src/share/aggregate/count.h +++ b/src/share/aggregate/count.h @@ -33,7 +33,6 @@ public: CountAggregate() {} template - // TODO: impl add_batch_rows, using null mpas to optimize counting logic int add_param_batch(RuntimeContext &agg_ctx, const ObBitVector &skip, ObBitVector &pvt_skip, const EvalBound &bound, const RowSelector &row_sel, const int32_t agg_col_id, const int32_t param_id, ColumnFmt ¶m_vec, char *aggr_cell) @@ -55,10 +54,19 @@ public: if (param_id == agg_ctx.aggr_infos_.at(agg_col_id).param_exprs_.count() - 1) { // last param int64_t &count = *reinterpret_cast(aggr_cell); - if (row_sel.is_empty()) { - for (int i = bound.start(); i < bound.end(); i++) { count += !pvt_skip.at(i); } + if (OB_LIKELY(!agg_ctx.removal_info_.enable_removal_opt_) + || !agg_ctx.removal_info_.is_inverse_agg_) { + if (row_sel.is_empty()) { + for (int i = bound.start(); i < bound.end(); i++) { count += !pvt_skip.at(i); } + } else { + for (int i = 0; i < row_sel.size(); i++) { count += !pvt_skip.at(row_sel.index(i)); } + } } else { - for (int i = 0; i < row_sel.size(); i++) { count += !pvt_skip.at(row_sel.index(i)); } + if (row_sel.is_empty()) { + for (int i = bound.start(); i < bound.end(); i++) { count -= !pvt_skip.at(i); } + } else { + for (int i = 0; i < row_sel.size(); i++) { count -= !pvt_skip.at(row_sel.index(i)); } + } } } return ret; @@ -132,6 +140,23 @@ public: } return OB_SUCCESS; } + + template + int add_or_sub_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + bool is_trans = !agg_ctx.removal_info_.is_inverse_agg_; + if (!columns.is_null(row_num)) { + int64_t &count = *reinterpret_cast(agg_cell); + if (is_trans) { + count++; + } else { + count--; + } + } + return ret; + } TO_STRING_KV("aggregate", "count"); }; diff --git a/src/share/aggregate/iaggregate.cpp b/src/share/aggregate/iaggregate.cpp index b2de7c47d9..2c734b06b6 100644 --- a/src/share/aggregate/iaggregate.cpp +++ b/src/share/aggregate/iaggregate.cpp @@ -13,9 +13,6 @@ #define USING_LOG_PREFIX SQL_ENG #include "iaggregate.h" #include "share/datum/ob_datum_util.h" -#include "share/aggregate/count.h" -#include "share/aggregate/min_max.h" -#include "share/aggregate/sum.h" #include "share/aggregate/first_row.h" namespace oceanbase @@ -40,9 +37,13 @@ extern int init_sum_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator, IAggregate *&agg, int32 *tmp_res_size = NULL); extern int init_count_sum_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id, ObIAllocator &allocator, IAggregate *&agg); -#define INIT_AGGREGATE_CASE(OP_TYPE, func_name) \ +extern int init_approx_count_distinct_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id, + ObIAllocator &allocator, IAggregate *&agg); +extern int init_sysbit_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id, + ObIAllocator &allocator, IAggregate *&agg); +#define INIT_AGGREGATE_CASE(OP_TYPE, func_name, col_id) \ case (OP_TYPE): { \ - ret = init_##func_name##_aggregate(agg_ctx, i, allocator, agg); \ + ret = init_##func_name##_aggregate(agg_ctx, col_id, allocator, aggregate); \ } break int init_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator, ObIArray &aggregates) @@ -54,18 +55,27 @@ int init_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator, } for (int i = 0; OB_SUCC(ret) && i < agg_ctx.aggr_infos_.count(); i++) { ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(i); - IAggregate *agg = nullptr; + IAggregate *aggregate = nullptr; if (aggr_info.is_implicit_first_aggr()) { - if (OB_FAIL(init_first_row_aggregate(agg_ctx, i, allocator, agg))) { + if (OB_FAIL(init_first_row_aggregate(agg_ctx, i, allocator, aggregate))) { SQL_LOG(WARN, "init first row aggregate failed", K(ret)); } } else { - switch (aggr_info.expr_->type_) { - INIT_AGGREGATE_CASE(T_FUN_MIN, min); - INIT_AGGREGATE_CASE(T_FUN_MAX, max); - INIT_AGGREGATE_CASE(T_FUN_COUNT, count); - INIT_AGGREGATE_CASE(T_FUN_SUM, sum); - INIT_AGGREGATE_CASE(T_FUN_COUNT_SUM, count_sum); + ObExprOperatorType fun_type = + (aggr_info.expr_->type_ == T_WINDOW_FUNCTION ? aggr_info.real_aggr_type_ : + aggr_info.expr_->type_); + switch (fun_type) { + INIT_AGGREGATE_CASE(T_FUN_MIN, min, i); + INIT_AGGREGATE_CASE(T_FUN_MAX, max, i); + INIT_AGGREGATE_CASE(T_FUN_COUNT, count, i); + INIT_AGGREGATE_CASE(T_FUN_SUM, sum, i); + INIT_AGGREGATE_CASE(T_FUN_COUNT_SUM, count_sum, i); + INIT_AGGREGATE_CASE(T_FUN_APPROX_COUNT_DISTINCT, approx_count_distinct, i); + INIT_AGGREGATE_CASE(T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS, approx_count_distinct, i); + INIT_AGGREGATE_CASE(T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE, approx_count_distinct, i); + INIT_AGGREGATE_CASE(T_FUN_SYS_BIT_OR, sysbit, i); + INIT_AGGREGATE_CASE(T_FUN_SYS_BIT_AND, sysbit, i); + INIT_AGGREGATE_CASE(T_FUN_SYS_BIT_XOR, sysbit, i); default: { ret = OB_NOT_SUPPORTED; SQL_LOG(WARN, "not supported aggregate function", K(ret), K(aggr_info.expr_->type_)); @@ -73,17 +83,18 @@ int init_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator, } if (OB_FAIL(ret)) { SQL_LOG(WARN, "init aggregate failed", K(ret)); - } else if (OB_ISNULL(agg)) { + } else if (OB_ISNULL(aggregate)) { ret = OB_ERR_UNEXPECTED; SQL_LOG(WARN, "unexpected null aggregate", K(ret)); } } - if (OB_SUCC(ret) && OB_FAIL(aggregates.push_back(agg))) { + if (OB_SUCC(ret) && OB_FAIL(aggregates.push_back(aggregate))) { SQL_LOG(WARN, "push back element failed", K(ret)); } } // end for return ret; } + #undef INIT_AGGREGATE_CASE static int32_t agg_cell_tmp_res_size(RuntimeContext &agg_ctx, int64_t agg_col_id) @@ -110,6 +121,8 @@ static int32_t agg_cell_tmp_res_size(RuntimeContext &agg_ctx, int64_t agg_col_id } else { ret_size = tmp_res_size; } + } else if (info.get_expr_type() == T_FUN_APPROX_COUNT_DISTINCT) { + ret_size = sizeof(char *); } return ret_size; } diff --git a/src/share/aggregate/iaggregate.h b/src/share/aggregate/iaggregate.h index 4cabc7ca59..9ae4ce3ba7 100644 --- a/src/share/aggregate/iaggregate.h +++ b/src/share/aggregate/iaggregate.h @@ -400,72 +400,99 @@ protected: { int ret = OB_SUCCESS; ObEvalCtx &ctx = agg_ctx.eval_ctx_; - auto &columns = *static_cast(param_expr.get_vector(ctx)); + ColumnFmt &columns = *static_cast(param_expr.get_vector(ctx)); bool all_not_null = !columns.has_null(); Derived &derived = *static_cast(this); void *tmp_res = derived.get_tmp_res(agg_ctx, agg_col_id, agg_cell); int64_t calc_info = derived.get_batch_calc_info(agg_ctx, agg_col_id, agg_cell); - if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) { - if (all_not_null) { - for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { - if (OB_FAIL( - derived.add_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res, calc_info))) { - SQL_LOG(WARN, "add row failed", K(ret)); + if (OB_LIKELY(!agg_ctx.removal_info_.enable_removal_opt_)) { + if (OB_LIKELY(row_sel.is_empty() && bound.get_all_rows_active())) { + if (all_not_null) { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + if (OB_FAIL( + derived.add_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res, calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } + } // end for + if (agg_cell != nullptr) { + NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); + not_nulls.set(agg_col_id); } - } // end for - if (agg_cell != nullptr) { - NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); - not_nulls.set(agg_col_id); + } else { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res, + calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } + } // end for + } + } else if (!row_sel.is_empty()) { + if (all_not_null) { + for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) { + if (OB_FAIL(derived.add_row(agg_ctx, columns, row_sel.index(i), agg_col_id, agg_cell, + tmp_res, calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } + } // end for + if (agg_cell != nullptr) { + NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); + not_nulls.set(agg_col_id); + } + } else { + for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) { + if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, row_sel.index(i), agg_col_id, + agg_cell, tmp_res, calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } + } // end for } } else { - for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { - if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res, + if (all_not_null) { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + if (skip.at(i)) { + } else if (OB_FAIL(derived.add_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res, calc_info))) { - SQL_LOG(WARN, "add row failed", K(ret)); + SQL_LOG(WARN, "add row failed", K(ret)); + } + } // end for + if (agg_cell != nullptr) { + NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); + not_nulls.set(agg_col_id); } - } // end for - } - } else if (!row_sel.is_empty()) { - if (all_not_null) { - for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) { - if (OB_FAIL(derived.add_row(agg_ctx, columns, row_sel.index(i), agg_col_id, agg_cell, - tmp_res, calc_info))) { - SQL_LOG(WARN, "add row failed", K(ret)); - } - } // end for - if (agg_cell != nullptr) { - NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); - not_nulls.set(agg_col_id); + } else { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + if (skip.at(i)) { + } else if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, i, agg_col_id, agg_cell, + tmp_res, calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } + } // end for } - } else { - for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) { - if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, row_sel.index(i), agg_col_id, - agg_cell, tmp_res, calc_info))) { - SQL_LOG(WARN, "add row failed", K(ret)); - } - } // end for } } else { - if (all_not_null) { - for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { - if (skip.at(i)) { - } else if (OB_FAIL(derived.add_row(agg_ctx, columns, i, agg_col_id, agg_cell, tmp_res, - calc_info))) { - SQL_LOG(WARN, "add row failed", K(ret)); + if (row_sel.is_empty()) { + if (bound.get_all_rows_active()) { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + ret = removal_opt::add_or_sub_row(derived, agg_ctx, columns, i, agg_col_id, agg_cell, + tmp_res, calc_info); + if (OB_FAIL(ret)) { SQL_LOG(WARN, "add or sub row failed", K(ret)); } + } + } else { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + if (skip.at(i)) { + } else { + ret = removal_opt::add_or_sub_row(derived, agg_ctx, columns, i, agg_col_id, agg_cell, + tmp_res, calc_info); + if (OB_FAIL(ret)) { SQL_LOG(WARN, "add or sub row failed", K(ret)); } + } } - } // end for - if (agg_cell != nullptr) { - NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); - not_nulls.set(agg_col_id); } } else { - for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { - if (skip.at(i)) { - } else if (OB_FAIL(derived.add_nullable_row(agg_ctx, columns, i, agg_col_id, agg_cell, - tmp_res, calc_info))) { - SQL_LOG(WARN, "add row failed", K(ret)); - } - } // end for + for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) { + ret = removal_opt::add_or_sub_row(derived, agg_ctx, columns, i, agg_col_id, agg_cell, + tmp_res, calc_info); + if (OB_FAIL(ret)) { SQL_LOG(WARN, "add or sub row failed", K(ret)); } + } } } if (OB_SUCC(ret)) { @@ -595,7 +622,7 @@ protected: } else { MEMSET(skip_buf, 0, skip_size); ObBitVector *tmp_skip = to_bit_vector(skip_buf); - tmp_skip->deep_copy(skip, skip_size); + tmp_skip->deep_copy(skip, bound.batch_size()); Derived &derived = *static_cast(this); for (int param_id = 0; OB_SUCC(ret) && param_id < param_exprs.count(); param_id++) { ObIVector *param_vec = param_exprs.at(param_id)->get_vector(agg_ctx.eval_ctx_); @@ -836,9 +863,8 @@ namespace helper int init_aggregates(RuntimeContext &agg_ctx, ObIAllocator &allocator, ObIArray &aggregates); -template -int init_aggregate(sql::ObEvalCtx &ctx, RuntimeContext &agg_ctx, const int64_t agg_col_id, - ObIAllocator &allocator, IAggregate *&agg); +int init_aggregate(RuntimeContext &agg_ctx, ObIAllocator &allocator, const int64_t col_id, + IAggregate *&aggregate); inline constexpr bool is_var_len_agg_cell(VecValueTypeClass vec_tc) { diff --git a/src/share/aggregate/min_max.cpp b/src/share/aggregate/min_max.cpp index 690a1f608e..eeb4505e62 100644 --- a/src/share/aggregate/min_max.cpp +++ b/src/share/aggregate/min_max.cpp @@ -33,20 +33,20 @@ inline static int init_min_max_agg(RuntimeContext &agg_ctx, if (OB_FAIL(ret)) { SQL_LOG(WARN, "init aggregate failed", K(ret)); } \ } break -int ret = OB_SUCCESS; -agg = nullptr; + int ret = OB_SUCCESS; + agg = nullptr; -ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id); -if (OB_ISNULL(aggr_info.expr_)) { - ret = OB_ERR_UNEXPECTED; - SQL_LOG(WARN, "invalid null expr", K(ret)); -} else { - VecValueTypeClass res_vec = - get_vec_value_tc(aggr_info.expr_->datum_meta_.type_, aggr_info.expr_->datum_meta_.scale_, - aggr_info.expr_->datum_meta_.precision_); + ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id); + if (OB_ISNULL(aggr_info.expr_)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid null expr", K(ret)); + } else { + VecValueTypeClass res_vec = + get_vec_value_tc(aggr_info.expr_->datum_meta_.type_, aggr_info.expr_->datum_meta_.scale_, + aggr_info.expr_->datum_meta_.precision_); - switch (res_vec) { - LST_DO_CODE(INIT_AGGREGATE_CASE, AGG_VEC_TC_LIST); + switch (res_vec) { + LST_DO_CODE(INIT_AGGREGATE_CASE, AGG_VEC_TC_LIST); default: { ret = OB_ERR_UNEXPECTED; SQL_LOG(WARN, "unexpected result type of min/max aggregate", K(ret), K(res_vec)); diff --git a/src/share/aggregate/min_max.h b/src/share/aggregate/min_max.h index b0e40664c9..e4604b91e2 100644 --- a/src/share/aggregate/min_max.h +++ b/src/share/aggregate/min_max.h @@ -28,19 +28,30 @@ using namespace sql; struct CmpCalcInfo { CmpCalcInfo(ObObjMeta obj_meta, int16_t cell_len) : - obj_meta_(obj_meta), agg_cell_len_(cell_len), calculated_(0) {} - CmpCalcInfo() : obj_meta_(), agg_cell_len_(0), calculated_(0) {} + obj_meta_(obj_meta), agg_cell_len_(cell_len), calc_flags_(0) {} + CmpCalcInfo() : obj_meta_(), agg_cell_len_(0), calc_flags_(0) {} operator int64_t() const { return flags_; } inline void set_calculated() { - calculated_ = static_cast(1); + calculated_ = true; + } + inline void set_min_max_idx_changed() + { + min_max_idx_changed_ = true; } inline bool calculated() const { return calculated_ == 1; } union { struct { ObObjMeta obj_meta_; int16_t agg_cell_len_; // for fixed length type only - int16_t calculated_; + union { + struct { + uint16_t calculated_: 1; + uint16_t min_max_idx_changed_: 1; + uint16_t reserved_: 14; + }; + uint16_t calc_flags_; + }; }; int64_t flags_; }; @@ -157,10 +168,12 @@ public: SQL_LOG(WARN, "compare failed", K(ret)); } else if ((is_min && cmp_ret > 0) || (!is_min && cmp_ret < 0)) { MEMCPY(agg_cell, row_data, row_len); + cmp_info.set_min_max_idx_changed(); } } else { MEMCPY(agg_cell, row_data, row_len); cmp_info.set_calculated(); + cmp_info.set_min_max_idx_changed(); } } else { int32_t agg_cell_len = *reinterpret_cast(agg_cell + sizeof(char *)); @@ -173,11 +186,13 @@ public: } else if ((is_min && cmp_ret > 0) || (!is_min && cmp_ret < 0)) { *reinterpret_cast(agg_cell) = reinterpret_cast(row_data); *reinterpret_cast(agg_cell + sizeof(char *)) = row_len; + cmp_info.set_min_max_idx_changed(); } } else { *reinterpret_cast(agg_cell) = reinterpret_cast(row_data); *reinterpret_cast(agg_cell + sizeof(char *)) = row_len; cmp_info.set_calculated(); + cmp_info.set_min_max_idx_changed(); } } return ret; @@ -237,7 +252,10 @@ public: NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); // if result value has variable result length, e.g. string value, // we need copy value to tmp buffer in case value ptr stored was changed after next batch loop. - if (not_nulls.at(agg_col_id) && helper::is_var_len_agg_cell(vec_tc)) { + + if (agg_ctx.win_func_agg_) { + // do nothing + } else if (not_nulls.at(agg_col_id) && helper::is_var_len_agg_cell(vec_tc)) { if (OB_FAIL(set_tmp_var_agg_data(agg_ctx, agg_col_id, agg_cell))) { SQL_LOG(WARN, "set variable aggregate data failed", K(ret)); } @@ -268,6 +286,31 @@ public: } TO_STRING_KV("aggregate", (is_min ? "min" : "max"), K(vec_tc)); + + template + int add_or_sub_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + bool is_trans = !agg_ctx.removal_info_.is_inverse_agg_; + if (!columns.is_null(row_num)) { + CmpCalcInfo &cmp_info = reinterpret_cast(calc_info); + cmp_info.min_max_idx_changed_ = 0; + if (OB_FAIL(add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } else if (cmp_info.min_max_idx_changed_) { + agg_ctx.removal_info_.max_min_index_ = row_num; + agg_ctx.removal_info_.is_max_min_idx_changed_ = true; + } + agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell).set(0); + } else if (is_trans) { + agg_ctx.removal_info_.null_cnt_++; + } else { + agg_ctx.removal_info_.null_cnt_--; + } + // do nothing + return ret; + } private: int set_tmp_var_agg_data(RuntimeContext &agg_ctx, const int32_t agg_col_id, char *agg_cell) { diff --git a/src/share/aggregate/processor.cpp b/src/share/aggregate/processor.cpp index 911070cce6..fe7eccc9ff 100644 --- a/src/share/aggregate/processor.cpp +++ b/src/share/aggregate/processor.cpp @@ -323,23 +323,23 @@ int Processor::collect_group_results(const RowMeta &row_meta, } int Processor::setup_rt_info(AggrRowPtr row, - const int32_t row_size) + RuntimeContext &agg_ctx) { static const int constexpr extra_arr_buf_size = 16; int ret = OB_SUCCESS; - OB_ASSERT(row_size == agg_ctx_.row_meta().row_size_); + int32_t row_size = agg_ctx.row_meta().row_size_; char *extra_array_buf = nullptr; MEMSET(row, 0, row_size); - for (int col_id = 0; col_id < agg_ctx_.aggr_infos_.count(); col_id++) { - ObDatumMeta &res_meta = agg_ctx_.aggr_infos_.at(col_id).expr_->datum_meta_; + for (int col_id = 0; col_id < agg_ctx.aggr_infos_.count(); col_id++) { + ObDatumMeta &res_meta = agg_ctx.aggr_infos_.at(col_id).expr_->datum_meta_; VecValueTypeClass res_tc = get_vec_value_tc(res_meta.type_, res_meta.scale_, res_meta.precision_); char *cell = nullptr; int32_t cell_len = 0; - agg_ctx_.row_meta().locate_cell_payload(col_id, row, cell, cell_len); + agg_ctx.row_meta().locate_cell_payload(col_id, row, cell, cell_len); // oracle mode use ObNumber as result type for count aggregation // we use int64_t as result type for count aggregation in aggregate row // and cast int64_t to ObNumber during `collect_group_result` - if (res_tc == VEC_TC_NUMBER && agg_ctx_.aggr_infos_.at(col_id).get_expr_type() != T_FUN_COUNT) { + if (res_tc == VEC_TC_NUMBER && agg_ctx.aggr_infos_.at(col_id).get_expr_type() != T_FUN_COUNT) { ObNumberDesc &d = *reinterpret_cast(cell); // set zero number d.len_ = 0; @@ -351,27 +351,19 @@ int Processor::setup_rt_info(AggrRowPtr row, *reinterpret_cast(cell) = double(); } } - int extra_size = agg_ctx_.row_meta().extra_cnt_ * sizeof(char *); - if (agg_ctx_.row_meta().extra_cnt_ > 0) { - if (cur_extra_rt_info_idx_ % extra_arr_buf_size == 0) { - cur_extra_rt_info_idx_ = 0; - extra_rt_info_buf_ = nullptr; - extra_rt_info_buf_ = (char *)agg_ctx_.allocator_.alloc(extra_size * extra_arr_buf_size); - if (OB_ISNULL(extra_rt_info_buf_)) { - ret = OB_ALLOCATE_MEMORY_FAILED; - SQL_LOG(WARN, "allocate memory failed", K(ret)); - } - } - if (OB_FAIL(ret)) { + int extra_size = agg_ctx.row_meta().extra_cnt_ * sizeof(char *); + if (agg_ctx.row_meta().extra_cnt_ > 0) { + void *extra_array_buf = agg_ctx.allocator_.alloc(extra_size); + if (OB_ISNULL(extra_array_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); } else { - extra_array_buf = extra_rt_info_buf_ + (cur_extra_rt_info_idx_ * extra_size); - cur_extra_rt_info_idx_++; MEMSET(extra_array_buf, 0, extra_size); - if (OB_FAIL(agg_ctx_.agg_extras_.push_back((AggregateExtras)extra_array_buf))) { + if (OB_FAIL(agg_ctx.agg_extras_.push_back((AggregateExtras)extra_array_buf))) { SQL_LOG(WARN, "push back element failed", K(ret)); } else { - *reinterpret_cast(row + agg_ctx_.row_meta().extra_idx_offset_) = - static_cast(agg_ctx_.agg_extras_.count()) - 1; + *reinterpret_cast(row + agg_ctx.row_meta().extra_idx_offset_) = + static_cast(agg_ctx.agg_extras_.count()) - 1; } } } @@ -396,7 +388,7 @@ int Processor::single_row_agg_batch(AggrRowPtr *agg_rows, const int64_t batch_si sql::EvalBound bound(1, true); for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { if (skip.at(i)) { - } else if (OB_FAIL(setup_rt_info(agg_rows[i], get_aggregate_row_size()))) { + } else if (OB_FAIL(setup_rt_info(agg_rows[i], agg_ctx_))) { SQL_LOG(WARN, "setup runtime info failed", K(ret)); } else { AggrRowPtr row = agg_rows[i]; @@ -591,7 +583,7 @@ int Processor::init_scalar_aggregate_row(ObCompactRow *&row, RowMeta &row_meta, row = new(row_buf)ObCompactRow(); row->init(row_meta); AggrRowPtr agg_row = (char *)row->get_extra_payload(row_meta); - if (OB_FAIL(setup_rt_info(agg_row, agg_ctx_.row_meta().row_size_))) { + if (OB_FAIL(setup_rt_info(agg_row, agg_ctx_))) { LOG_WARN("setup rt info failed", K(ret)); } } @@ -632,7 +624,8 @@ int Processor::add_one_aggregate_row(AggrRowPtr data, const int32_t row_size, bool push_agg_row /*true*/) { int ret = OB_SUCCESS; - if (OB_FAIL(setup_rt_info(data, row_size))) { + UNUSED(row_size); + if (OB_FAIL(setup_rt_info(data, agg_ctx_))) { SQL_LOG(WARN, "setup runtime info failed", K(ret)); } else if (push_agg_row && OB_FAIL(agg_ctx_.agg_rows_.push_back(data))) { SQL_LOG(WARN, "push back element failed", K(ret)); diff --git a/src/share/aggregate/processor.h b/src/share/aggregate/processor.h index f28ddf8e53..b02a67cbb6 100644 --- a/src/share/aggregate/processor.h +++ b/src/share/aggregate/processor.h @@ -33,7 +33,7 @@ public: support_fast_single_row_agg_(false), has_distinct_(false), has_order_by_(false), dir_id_(-1), op_eval_infos_(nullptr), allocator_(label, OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id, ObCtxIds::WORK_AREA), - agg_ctx_(eval_ctx, tenant_id, aggr_infos, label, allocator_), + agg_ctx_(eval_ctx, tenant_id, aggr_infos, label), aggregates_(allocator_, aggr_infos.count()), fast_single_row_aggregates_(allocator_, aggr_infos.count()), extra_rt_info_buf_(nullptr), cur_extra_rt_info_idx_(0), add_one_row_fns_(allocator_, aggr_infos.count()), row_selector_(nullptr) @@ -212,9 +212,14 @@ public: int init_scalar_aggregate_row(ObCompactRow *&row, RowMeta &row_meta, ObIAllocator &allocator); int init_fast_single_row_aggs(); -private: - int setup_rt_info(AggrRowPtr data, const int32_t row_size); + RuntimeContext *get_rt_ctx() { return &agg_ctx_; } + + static int setup_rt_info(AggrRowPtr data, RuntimeContext &agg_ctx); + + ObIArray &get_aggregates() { return aggregates_; } + +private: int setup_bypass_rt_infos(const int64_t batch_size); int llc_init_empty(ObExpr &expr, ObEvalCtx &eval_ctx) const; @@ -249,4 +254,4 @@ private: } // end aggregate } // end share } // end oceanbase -#endif // OCEANBASE_SHARE_AGGREGATE_PROCESSOR_H_ \ No newline at end of file +#endif // OCEANBASE_SHARE_AGGREGATE_PROCESSOR_H_ diff --git a/src/share/aggregate/single_row.cpp b/src/share/aggregate/single_row.cpp index 12dd48591b..a0beb6ca87 100644 --- a/src/share/aggregate/single_row.cpp +++ b/src/share/aggregate/single_row.cpp @@ -19,6 +19,8 @@ namespace share { namespace aggregate { + // TODO: add approx_distinct_count + // TODO: add sysbit_ops #define INIT_GENERAL_CASE(func_type, vec_tc) \ case (vec_tc): { \ ret = \ diff --git a/src/share/aggregate/single_row.h b/src/share/aggregate/single_row.h index 8e566f37fb..2bc8130192 100644 --- a/src/share/aggregate/single_row.h +++ b/src/share/aggregate/single_row.h @@ -36,7 +36,7 @@ public: const int32_t agg_col_id, const char *agg_cell, const int32_t agg_cell_len) { - int ret = OB_SUCCESS; + int ret = OB_SUCCESS; const NotNullBitVector ¬nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); int64_t output_idx = agg_ctx.eval_ctx_.get_batch_idx(); ResultFmt *res_vec = static_cast(agg_expr.get_vector(agg_ctx.eval_ctx_)); @@ -194,7 +194,7 @@ public: return ret; } void set_first_row(bool is_first_row) { is_first_row_ = is_first_row; } - TO_STRING_KV("aggregate", "single_row", K(in_tc), K(out_tc), K(is_first_row_)); + TO_STRING_KV("aggregate", "single_row", K(in_tc), K(out_tc), K(is_first_row_), K(agg_func)); private: bool is_first_row_; }; diff --git a/src/share/aggregate/sum.h b/src/share/aggregate/sum.h index 6b4cfb3151..e318ead126 100644 --- a/src/share/aggregate/sum.h +++ b/src/share/aggregate/sum.h @@ -87,6 +87,24 @@ public: return ret; } + template + int sub_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + const char* param_payload = nullptr; + int32_t param_len = 0; + columns.get_payload(row_num, param_payload, param_len); + const ParamType *lparam = reinterpret_cast(param_payload); + const ResultType &rparam = *reinterpret_cast(agg_cell); + SQL_LOG(DEBUG, "sum sub row", K(agg_col_id)); + ret = sub_values(rparam, *lparam, agg_cell, sizeof(ResultType)); + if (OB_FAIL(ret)) { + SQL_LOG(WARN, "sub value failed", K(ret)); + } + return ret; + } + template int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) @@ -104,6 +122,31 @@ public: return ret; } + template + int add_or_sub_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + bool is_trans = !agg_ctx.removal_info_.is_inverse_agg_; + if (!columns.is_null(row_num)) { + if (is_trans) { + if (OB_FAIL(add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info))) { + SQL_LOG(WARN, "add row failed", K(ret)); + } + } else if (OB_FAIL( + sub_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info))) { + SQL_LOG(WARN, "sub row failed", K(ret)); + } + } else { + if (is_trans) { + agg_ctx.removal_info_.null_cnt_++; + } else { + agg_ctx.removal_info_.null_cnt_--; + } + } + return ret; + } + template int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr, const int32_t agg_col_id, const char *agg_cell, @@ -116,7 +159,7 @@ public: ColumnFmt *res_vec = static_cast(agg_expr.get_vector(agg_ctx.eval_ctx_)); if (OB_LIKELY(not_nulls.at(agg_col_id))) { CellWriter::set(agg_cell, agg_cell_len, res_vec, output_idx, nullptr); - } else if (agg_expr.type_ == T_FUN_COUNT_SUM) { + } else if (agg_ctx.locate_aggr_info(agg_col_id).get_expr_type() == T_FUN_COUNT_SUM) { if (VEC_TC_INTEGER == out_tc) { int64_t res = 0; res_vec->set_payload(output_idx, &res, sizeof(int64_t)); @@ -243,21 +286,49 @@ public: int ret = OB_SUCCESS; TmpStore &tmp_res = *reinterpret_cast(get_tmp_res(agg_ctx, agg_col_id, agg_cell)); if (OB_LIKELY(tmp_res != 0)) { - ResultType &res = *reinterpret_cast(agg_cell); - const int32_t agg_cell_len = agg_ctx.row_meta().get_cell_len(agg_col_id, agg_cell); - ObScale scale = agg_ctx.get_first_param_scale(agg_col_id); - if (out_tc == VEC_TC_NUMBER) { - if (OB_FAIL(add_value_to_nmb(tmp_res, scale, agg_cell))) { - SQL_LOG(WARN, "add value to nmb failed", K(ret)); + if (OB_LIKELY(!agg_ctx.removal_info_.enable_removal_opt_) + || !agg_ctx.removal_info_.is_inverse_agg_) { + ResultType &res = *reinterpret_cast(agg_cell); + const int32_t agg_cell_len = agg_ctx.row_meta().get_cell_len(agg_col_id, agg_cell); + ObScale scale = agg_ctx.get_first_param_scale(agg_col_id); + if (out_tc == VEC_TC_NUMBER) { + if (OB_FAIL(add_value_to_nmb(tmp_res, scale, agg_cell))) { + SQL_LOG(WARN, "add value to nmb failed", K(ret)); + } + } else { + ResultType *res_val = reinterpret_cast(agg_cell); + ret = add_values(tmp_res, *res_val, agg_cell, agg_ctx.get_cell_len(agg_col_id, agg_cell)); + } + if (OB_FAIL(ret)) { + SQL_LOG(WARN, "do op failed", K(ret)); + } else { + tmp_res = 0; } } else { - ResultType *res_val = reinterpret_cast(agg_cell); - ret = add_values(tmp_res, *res_val, agg_cell, agg_ctx.get_cell_len(agg_col_id, agg_cell)); - } - if (OB_FAIL(ret)) { - SQL_LOG(WARN, "do op failed", K(ret)); - } else { - tmp_res = 0; + if (out_tc == VEC_TC_NUMBER) { + ObScale scale = agg_ctx.get_first_param_scale(agg_col_id); + number::ObNumber param_nmb, res_nmb; + number::ObNumber cur_nmb(*reinterpret_cast(agg_cell)); + number::ObCompactNumber *res_cnum = reinterpret_cast(agg_cell); + ObNumStackAllocator<3> tmp_alloc; + if (OB_FAIL(to_nmb(tmp_res, scale, tmp_alloc, param_nmb))) { + SQL_LOG(WARN, "to number failed", K(ret)); + } else if (OB_FAIL(cur_nmb.sub(param_nmb, res_nmb, tmp_alloc))) { + SQL_LOG(WARN, "number::sub failed", K(ret)); + } else { + res_cnum->desc_ = res_nmb.d_; + MEMCPY(&(res_cnum->digits_[0]), res_nmb.get_digits(), sizeof(uint32_t) * res_nmb.d_.len_); + tmp_res = 0; + } + } else { + constexpr unsigned res_bits = + (out_tc == VEC_TC_NUMBER ? 128 : sizeof(ResultType) * CHAR_BIT); + using res_int_type = wide::ObWideInteger; + res_int_type &res_val = *reinterpret_cast(agg_cell); + // overflow is impossible + sub_values(res_val, tmp_res, agg_cell, sizeof(res_int_type)); + tmp_res = 0; + } } } return ret; @@ -271,14 +342,16 @@ public: if (!is_null) { const ParamType *row_param = reinterpret_cast(data); TmpStore *tmp_res = reinterpret_cast(get_tmp_res(agg_ctx, agg_col_idx, agg_cell)); + TmpStore copied_tmp_res = *tmp_res; SQL_LOG(DEBUG, "sum with tmp::add_one_row ", K(*row_param), K(*tmp_res), K(batch_idx)); - ret = add_overflow(*row_param, *tmp_res, reinterpret_cast(tmp_res), sizeof(TmpStore)); + ret = add_overflow(*row_param, copied_tmp_res, reinterpret_cast(tmp_res), + sizeof(TmpStore)); if (OB_FAIL(ret)) { if (OB_LIKELY(OB_OPERATE_OVERFLOW == ret)) { ret = OB_SUCCESS; if (out_tc == VEC_TC_NUMBER) { ObScale scale = agg_ctx.get_first_param_scale(agg_col_idx); - if (OB_FAIL(add_value_to_nmb(*tmp_res, scale, agg_cell))) { + if (OB_FAIL(add_value_to_nmb(copied_tmp_res, scale, agg_cell))) { SQL_LOG(WARN, "add value to nmb failed", K(ret)); } else if (OB_FAIL(add_value_to_nmb(*row_param, scale, agg_cell))) { SQL_LOG(WARN, "add value to nmb failed", K(ret)); @@ -315,16 +388,19 @@ public: int32_t param_len = 0; columns.get_payload(row_num, param_payload, param_len); const ParamType *row_param = reinterpret_cast(param_payload); + // add_overflow may overwrite `tmp_res`, deep copy is needed TmpStore &tmp_res = *reinterpret_cast(tmp_res_ptr); + TmpStore copied_tmp_res = tmp_res; SQL_LOG(DEBUG, "sum add row", K(row_param), K(row_num), K(tmp_res)); - ret = add_overflow(*row_param, tmp_res, reinterpret_cast(&tmp_res), sizeof(TmpStore)); + ret = add_overflow(*row_param, copied_tmp_res, reinterpret_cast(&tmp_res), + sizeof(TmpStore)); if (OB_FAIL(ret)) { if (OB_LIKELY(ret == OB_OPERATE_OVERFLOW)) { ret = OB_SUCCESS; if (out_tc == VEC_TC_NUMBER) { ObScale in_scale = agg_ctx.get_first_param_scale(agg_col_id); - if (OB_FAIL(add_value_to_nmb(tmp_res, in_scale, agg_cell))) { + if (OB_FAIL(add_value_to_nmb(copied_tmp_res, in_scale, agg_cell))) { SQL_LOG(WARN, "add_value_to_nmb failed", K(ret)); } else if (OB_FAIL(add_value_to_nmb(*row_param, in_scale, agg_cell))) { SQL_LOG(WARN, "add_value_to_nmb failed", K(ret)); @@ -333,7 +409,7 @@ public: } } else { ResultType &res_val = *reinterpret_cast(agg_cell); - if (OB_FAIL(add_values(tmp_res, res_val, agg_cell, sizeof(ResultType)))) { + if (OB_FAIL(add_values(copied_tmp_res, res_val, agg_cell, sizeof(ResultType)))) { SQL_LOG(WARN, "add values failed", K(ret)); } else if (OB_FAIL(add_values(*row_param, res_val, agg_cell, sizeof(ResultType)))) { SQL_LOG(WARN, "add values failed", K(ret)); @@ -367,6 +443,87 @@ public: return ret; } + template + int sub_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + // for substraction with tmp result, we added all substractions in tmp results + // and do substraction when overflow or tmp result collecting happended. + const char *payload = columns.get_payload(row_num); + const ParamType ¶m = *reinterpret_cast(payload); + TmpStore &tmp_val = *reinterpret_cast(tmp_res); + TmpStore copied_tmp_val = tmp_val; + ret = add_overflow(param, copied_tmp_val, (char *)tmp_res, sizeof(TmpStore)); + if (OB_FAIL(ret)) { + if (OB_LIKELY(ret == OB_OPERATE_OVERFLOW)) { + ret = OB_SUCCESS; + if (out_tc == VEC_TC_NUMBER) { + ObNumStackAllocator<4> tmp_alloc; + number::ObNumber sub1, sub2, tmp_sum, res_nmb; + number::ObNumber cur_nmb(*reinterpret_cast(agg_cell)); + ObScale scale = agg_ctx.get_first_param_scale(agg_col_id); + if (OB_FAIL(to_nmb(tmp_val, scale, tmp_alloc, sub1))) { + SQL_LOG(WARN, "to number failed", K(ret)); + } else if (OB_FAIL(to_nmb(param, scale, tmp_alloc, sub2))) { + SQL_LOG(WARN, "to number failed", K(ret)); + } else if (OB_FAIL(sub1.add(sub2, tmp_sum, tmp_alloc))) { + SQL_LOG(WARN, "number::add failed", K(ret)); + } else if (OB_FAIL(cur_nmb.sub(tmp_sum, res_nmb, tmp_alloc))) { + SQL_LOG(WARN, "number::sub failed", K(ret)); + } else { + number::ObCompactNumber *res_cnum = reinterpret_cast(agg_cell); + res_cnum->desc_ = res_nmb.d_; + MEMCPY(&(res_cnum->digits_[0]), res_nmb.get_digits(), res_nmb.d_.len_ * sizeof(uint32_t)); + tmp_val = 0; + } + } else { + constexpr unsigned res_bits = + (out_tc == VEC_TC_NUMBER ? 128 : sizeof(ResultType) * CHAR_BIT); + using res_int_type = wide::ObWideInteger; + res_int_type &res_val = *reinterpret_cast(agg_cell); + if (OB_FAIL(sub_values(res_val, tmp_val, agg_cell, sizeof(ResultType)))) { + SQL_LOG(WARN, "sub values failed", K(ret)); + } else if (OB_FAIL(sub_values(res_val, param, agg_cell, sizeof(ResultType)))) { + SQL_LOG(WARN, "sub values failed", K(ret)); + } else { + tmp_val = 0; + } + } + } else { + SQL_LOG(WARN, "add overflow", K(ret)); + } + } + return ret; + } + + template + int add_or_sub_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + bool is_trans = !agg_ctx.removal_info_.is_inverse_agg_; + if (!columns.is_null(row_num)) { + if (is_trans) { + ret = add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info); + if (OB_FAIL(ret)) { + SQL_LOG(WARN, "add row failed", K(ret)); + } + } else { + if (OB_FAIL(sub_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info))) { + SQL_LOG(WARN, "sub row failed", K(ret)); + } + } + } else { + if (is_trans) { + agg_ctx.removal_info_.null_cnt_++; + } else { + agg_ctx.removal_info_.null_cnt_--; + } + } + return ret; + } + template int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr, const int32_t agg_col_id, const char *agg_cell, const int32_t agg_len) @@ -395,7 +552,7 @@ public: CellWriter::set(agg_cell, agg_len, res_vec, output_idx, nullptr); } } - } else if (agg_expr.type_ == T_FUN_COUNT_SUM) { + } else if (agg_ctx.locate_aggr_info(agg_col_id).get_expr_type() == T_FUN_COUNT_SUM) { if (lib::is_oracle_mode() || out_tc == VEC_TC_NUMBER) { number::ObNumber res_nmb; res_nmb.set_zero(); @@ -483,6 +640,27 @@ private: } } } + + template + int to_nmb(const T &v, ObScale in_scale, ObIAllocator &alloc, number::ObNumber &res_nmb) + { + int ret = OB_SUCCESS; + if (sizeof(T) > sizeof(int64_t)) { + if (OB_FAIL(wide::to_number(v, in_scale, alloc, res_nmb))) { + SQL_LOG(WARN, "wide::to_number failed", K(ret)); + } + } else if (sizeof(T) <= sizeof(int32_t) || std::is_same::value) { + int64_t tmp_v = v; + if (OB_FAIL(wide::to_number(tmp_v, in_scale, alloc, res_nmb))) { + SQL_LOG(WARN, "wide::to_number failed", K(ret)); + } + } else if (std::is_same::value) { + if (OB_FAIL(wide::to_number(static_cast(v), in_scale, alloc, res_nmb))) { + SQL_LOG(WARN, "wide::to_number failed", K(ret)); + } + } + return ret; + } template int add_value_to_nmb(const T &v, const ObScale in_scale, char *agg_cell) { @@ -490,23 +668,8 @@ private: ObNumStackAllocator<2> tmp_alloc; number::ObNumber res_nmb, param_nmb; number::ObNumber tmp(*reinterpret_cast(agg_cell)); - if (sizeof(T) > sizeof(int64_t)) { - if (OB_FAIL(wide::to_number(v, in_scale, tmp_alloc, param_nmb))) { - SQL_LOG(WARN, "to_number failed", K(ret)); - } - } else if (sizeof(T) <= sizeof(int32_t)) { - int64_t tmp_v = v; - if (OB_FAIL(wide::to_number(tmp_v, in_scale, tmp_alloc, param_nmb))) { - SQL_LOG(WARN, "wide::to_number failed", K(ret)); - } - } else if (std::is_same::value - && OB_FAIL( - wide::to_number(static_cast(v), in_scale, tmp_alloc, param_nmb))) { - SQL_LOG(WARN, "wide::to_number failed", K(ret)); - } else if (OB_FAIL(wide::to_number(static_cast(v), in_scale, tmp_alloc, param_nmb))) { - SQL_LOG(WARN, "wide::to_number failed", K(ret)); - } - if (OB_FAIL(ret)) { + if (OB_FAIL(to_nmb(v, in_scale, tmp_alloc, param_nmb))) { + SQL_LOG(WARN, "to_nmb failed", K(ret)); } else if (OB_FAIL(tmp.add_v3(param_nmb, res_nmb, tmp_alloc))) { SQL_LOG(WARN, "ObNumber::add_v3 failed", K(ret)); } else { diff --git a/src/share/aggregate/sum_nmb.h b/src/share/aggregate/sum_nmb.h index 0ad9e03fbc..7f5a7d41c2 100644 --- a/src/share/aggregate/sum_nmb.h +++ b/src/share/aggregate/sum_nmb.h @@ -35,6 +35,25 @@ } \ } +#define ADD_OR_SUB_NMB(fmt) \ + do { \ + fmt *columns = static_cast(input_vec); \ + if (row_sel.is_empty()) { \ + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { \ + if (skip.at(i)) { \ + } else { \ + ret = add_or_sub_row(agg_ctx, *columns, i, agg_col_id, agg_cell, nullptr, \ + mock_calc_info); \ + } \ + } \ + } else { \ + for (int i = 0; OB_SUCC(ret) && i < row_sel.size(); i++) { \ + ret = add_or_sub_row(agg_ctx, *columns, row_sel.index(i), agg_col_id, agg_cell, \ + nullptr, mock_calc_info); \ + } \ + } \ + } while (false) + namespace oceanbase { namespace share @@ -87,8 +106,8 @@ public: return ret; } inline int add_batch_rows(RuntimeContext &agg_ctx, const int32_t agg_col_id, - const sql::ObBitVector &skip, const sql::EvalBound &bound, char *agg_cell, - const RowSelector row_sel = RowSelector{}) override + const sql::ObBitVector &skip, const sql::EvalBound &bound, + char *agg_cell, const RowSelector row_sel = RowSelector{}) override { int ret = OB_SUCCESS; #ifndef NDEBUG @@ -100,45 +119,70 @@ public: OB_ASSERT(param_expr != NULL); ObIVector *input_vec = param_expr->get_vector(agg_ctx.eval_ctx_); VectorFormat in_fmt = param_expr->get_format(agg_ctx.eval_ctx_); - uint32_t sum_digits[number::ObNumber::OB_CALC_BUFFER_SIZE] = {0}; - char buf_alloc1[number::ObNumber::MAX_CALC_BYTE_LEN] = {0}; - char buf_alloc2[number::ObNumber::MAX_CALC_BYTE_LEN] = {0}; - ObDataBuffer allocator1(buf_alloc1, number::ObNumber::MAX_CALC_BYTE_LEN); - ObDataBuffer allocator2(buf_alloc2, number::ObNumber::MAX_CALC_BYTE_LEN); - number::ObNumber result(*reinterpret_cast(agg_cell)); - bool all_skip = true; - switch (in_fmt) { - case common::VEC_UNIFORM: { - ACCUMULATE_NMB(ObUniformFormat); - break; + if (OB_LIKELY(!agg_ctx.removal_info_.enable_removal_opt_)) { + uint32_t sum_digits[number::ObNumber::OB_CALC_BUFFER_SIZE] = {0}; + char buf_alloc1[number::ObNumber::MAX_CALC_BYTE_LEN] = {0}; + char buf_alloc2[number::ObNumber::MAX_CALC_BYTE_LEN] = {0}; + ObDataBuffer allocator1(buf_alloc1, number::ObNumber::MAX_CALC_BYTE_LEN); + ObDataBuffer allocator2(buf_alloc2, number::ObNumber::MAX_CALC_BYTE_LEN); + number::ObNumber result(*reinterpret_cast(agg_cell)); + bool all_skip = true; + switch (in_fmt) { + case common::VEC_UNIFORM: { + ACCUMULATE_NMB(ObUniformFormat); + break; + } + case common::VEC_UNIFORM_CONST: { + ACCUMULATE_NMB(ObUniformFormat); + break; + } + case common::VEC_DISCRETE: { + ACCUMULATE_NMB(ObDiscreteFormat); + break; + } + case common::VEC_CONTINUOUS: { + ACCUMULATE_NMB(ObContinuousFormat); + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "unexpected format for sum aggregate", K(ret), K(*this), K(in_fmt)); + } + } + if (OB_FAIL(ret)) { + SQL_LOG(WARN, "accumulate number failed", K(ret)); + } else if (OB_LIKELY(!all_skip)) { + NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); + not_nulls.set(agg_col_id); + number::ObCompactNumber *cnum = reinterpret_cast(agg_cell); + cnum->desc_ = result.d_; + MEMCPY(&(cnum->digits_[0]), result.get_digits(), result.d_.len_ * sizeof(uint32_t)); + } + SQL_LOG(DEBUG, "number result", K(result), K(all_skip), K(ret)); + } else { + int64_t mock_calc_info = 0; + switch(in_fmt) { + case common::VEC_UNIFORM: { + ADD_OR_SUB_NMB(ObUniformFormat); + break; + } + case common::VEC_DISCRETE: { + ADD_OR_SUB_NMB(ObDiscreteFormat); + break; + } + case common::VEC_UNIFORM_CONST: { + ADD_OR_SUB_NMB(ObUniformFormat); + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid input format", K(ret), K(in_fmt)); + } + } + if (OB_FAIL(ret)) { + SQL_LOG(WARN, "add or sub rows failed", K(ret)); + } } - case common::VEC_UNIFORM_CONST: { - ACCUMULATE_NMB(ObUniformFormat); - break; - } - case common::VEC_DISCRETE: { - ACCUMULATE_NMB(ObDiscreteFormat); - break; - } - case common::VEC_CONTINUOUS: { - ACCUMULATE_NMB(ObContinuousFormat); - break; - } - default: { - ret = OB_ERR_UNEXPECTED; - SQL_LOG(WARN, "unexpected format for sum aggregate", K(ret), K(*this), K(in_fmt)); - } - } - if (OB_FAIL(ret)) { - SQL_LOG(WARN, "accumulate number failed", K(ret)); - } else if (OB_LIKELY(!all_skip)) { - NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); - not_nulls.set(agg_col_id); - number::ObCompactNumber *cnum = reinterpret_cast(agg_cell); - cnum->desc_ = result.d_; - MEMCPY(&(cnum->digits_[0]), result.get_digits(), result.d_.len_ * sizeof(uint32_t)); - } - SQL_LOG(DEBUG, "number result", K(result), K(all_skip), K(ret)); return ret; } @@ -182,6 +226,40 @@ public: return ret; } + template + int add_or_sub_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + UNUSEDx(tmp_res, calc_info); + bool is_trans = !agg_ctx.removal_info_.is_inverse_agg_; + if (!columns.is_null(row_num)) { + const number::ObCompactNumber *param_cnum = + reinterpret_cast(columns.get_payload(row_num)); + number::ObCompactNumber *res_cnum = reinterpret_cast(agg_cell); + number::ObNumber param1(*param_cnum); + number::ObNumber param2(*reinterpret_cast(agg_cell)); + number::ObNumber res_nmb; + ObNumStackOnceAlloc tmp_alloc; + if (is_trans) { + if (OB_FAIL(param2.add_v3(param1, res_nmb, tmp_alloc))) { + SQL_LOG(WARN, "add number failed", K(ret)); + } + } else if (OB_FAIL(param2.sub_v3(param1, res_nmb, tmp_alloc))){ + SQL_LOG(WARN, "sub number failed", K(ret)); + } + if (OB_SUCC(ret)) { + res_cnum->desc_ = res_nmb.d_; + MEMCPY(&(res_cnum->digits_[0]), res_nmb.get_digits(), res_nmb.d_.len_ * sizeof(uint32_t)); + } + } else if (is_trans){ + agg_ctx.removal_info_.null_cnt_++; + } else { + agg_ctx.removal_info_.null_cnt_--; + } + return ret; + } + TO_STRING_KV("aggregate", "sum_nmb", K_(is_ora_count_sum)); private: template @@ -275,4 +353,5 @@ private: } // end share } // end oceanbase #undef ACCUMULATE_NMB +#undef ADD_OR_SUB_NMB #endif // OCEANBASE_SHARE_AGGREGATE_SUM_NMB_H_ \ No newline at end of file diff --git a/src/share/aggregate/sys_bit.cpp b/src/share/aggregate/sys_bit.cpp new file mode 100644 index 0000000000..2edb8e756a --- /dev/null +++ b/src/share/aggregate/sys_bit.cpp @@ -0,0 +1,71 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#define USING_LOG_PREFIX SQL_ENG +#include "sys_bit.h" + +namespace oceanbase +{ +namespace share +{ +namespace aggregate +{ +namespace helper +{ +int init_sysbit_aggregate(RuntimeContext &agg_ctx, const int64_t agg_col_id, + ObIAllocator &allocator, IAggregate *&agg) +{ +#define INIT_SYS_BIT_AGG(op_type) \ + do { \ + if (in_tc == VEC_TC_INTEGER) { \ + ret = init_agg_func>( \ + agg_ctx, agg_col_id, has_distinct, allocator, agg); \ + } else { \ + ret = init_agg_func>( \ + agg_ctx, agg_col_id, has_distinct, allocator, agg); \ + } \ + } while (false) + + int ret = OB_SUCCESS; + ObAggrInfo &aggr_info = agg_ctx.locate_aggr_info(agg_col_id); + bool has_distinct = aggr_info.has_distinct_; + if (OB_UNLIKELY(aggr_info.param_exprs_.count() != 1 + || (aggr_info.param_exprs_.at(0)->get_vec_value_tc() != VEC_TC_UINTEGER + && aggr_info.param_exprs_.at(0)->get_vec_value_tc() != VEC_TC_INTEGER) + && aggr_info.param_exprs_.at(0)->get_vec_value_tc() != VEC_TC_BIT)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected param exprs", K(ret), K(aggr_info)); + } else if (OB_UNLIKELY(aggr_info.expr_->get_vec_value_tc() != VEC_TC_UINTEGER)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected result expr", K(ret), K(aggr_info)); + } else { + VecValueTypeClass in_tc = aggr_info.param_exprs_.at(0)->get_vec_value_tc(); + ObExprOperatorType fn_type = aggr_info.get_expr_type(); + if (fn_type == T_FUN_SYS_BIT_OR) { + INIT_SYS_BIT_AGG(T_FUN_SYS_BIT_OR); + } else if (fn_type == T_FUN_SYS_BIT_AND) { + INIT_SYS_BIT_AGG(T_FUN_SYS_BIT_AND); + } else if (fn_type == T_FUN_SYS_BIT_XOR) { + INIT_SYS_BIT_AGG(T_FUN_SYS_BIT_XOR); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected sysbit operator", K(ret), K(fn_type)); + } + if (OB_FAIL(ret)) { + LOG_WARN("init sysbit functions failed", K(ret)); + } + } + return ret; +} +} +} // end aggregate +} // end share +} // end oceanbase \ No newline at end of file diff --git a/src/share/aggregate/sys_bit.h b/src/share/aggregate/sys_bit.h new file mode 100644 index 0000000000..7b258c761e --- /dev/null +++ b/src/share/aggregate/sys_bit.h @@ -0,0 +1,116 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#ifndef OCEANBASE_SHARE_SYSBIT_AGGREGATE_H_ +#define OCEANBASE_SHARE_SYSBIT_AGGREGATE_H_ + +#include "share/aggregate/iaggregate.h" + +namespace oceanbase +{ +namespace share +{ +namespace aggregate +{ +template +struct SysBitAggregate final + : public BatchAggregateWrapper> +{ +public: + static const constexpr VecValueTypeClass IN_TC = in_tc; + static const constexpr VecValueTypeClass OUT_TC = out_tc; +public: + SysBitAggregate() {} + + template + inline int add_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + setup_inital_value(agg_ctx, agg_cell, agg_col_id); + uint64_t &res_uint = *reinterpret_cast(agg_cell); + uint64_t cur_uint = *reinterpret_cast(columns.get_payload(row_num)); + if (agg_func == T_FUN_SYS_BIT_AND) { + res_uint &= cur_uint; + } else if (agg_func == T_FUN_SYS_BIT_OR) { + res_uint |= cur_uint; + } else if (agg_func == T_FUN_SYS_BIT_XOR) { + res_uint ^= cur_uint; + } else { + // impossible + ob_assert(false); + } + return ret; + } + + template + inline int add_nullable_row(RuntimeContext &agg_ctx, ColumnFmt &columns, const int32_t row_num, + const int32_t agg_col_id, char *agg_cell, void *tmp_res, int64_t &calc_info) + { + int ret = OB_SUCCESS; + setup_inital_value(agg_ctx, agg_cell, agg_col_id); + if (OB_LIKELY(!columns.is_null(row_num))) { + ret = add_row(agg_ctx, columns, row_num, agg_col_id, agg_cell, tmp_res, calc_info); + } + return ret; + } + + template + int collect_group_result(RuntimeContext &agg_ctx, const sql::ObExpr &agg_expr, + const int32_t agg_col_id, const char *agg_cell, + const int32_t agg_cell_len) + { + int ret = OB_SUCCESS; + int64_t output_idx = agg_ctx.eval_ctx_.get_batch_idx(); + ColumnFmt *res_vec = static_cast(agg_expr.get_vector(agg_ctx.eval_ctx_)); + res_vec->set_payload(output_idx, agg_cell, sizeof(uint64_t)); + return ret; + } + + int add_one_row(RuntimeContext &agg_ctx, int64_t batch_idx, int64_t batch_size, + const bool is_null, const char *data, const int32_t data_len, int32_t agg_col_idx, + char *agg_cell) override + { + int ret = OB_SUCCESS; + setup_inital_value(agg_ctx, agg_cell, agg_col_idx); + if (!is_null) { + uint64_t cur_uint = *reinterpret_cast(data); + uint64_t &res_uint = *reinterpret_cast(agg_cell); + if (agg_func == T_FUN_SYS_BIT_AND) { + res_uint &= cur_uint; + } else if (agg_func == T_FUN_SYS_BIT_OR) { + res_uint |= cur_uint; + } else if (agg_func == T_FUN_SYS_BIT_XOR) { + res_uint ^= cur_uint; + } else { + // impossible + ob_assert(false); + } + } + return ret; + } + TO_STRING_KV("aggregate", "sysbit_ops", K(in_tc), K(out_tc), K(agg_func)); +private: + void setup_inital_value(RuntimeContext &agg_ctx, char *agg_cell, const int32_t agg_col_id) + { + NotNullBitVector ¬_nulls = agg_ctx.locate_notnulls_bitmap(agg_col_id, agg_cell); + if (OB_UNLIKELY(!not_nulls.at(agg_col_id))) { + uint64_t initial_val = (agg_func == T_FUN_SYS_BIT_AND ? UINT_MAX_VAL[ObUInt64Type] : 0); + *reinterpret_cast(agg_cell) = initial_val; + not_nulls.set(agg_col_id); + } + } +}; +} // namespace aggregate +} // namespace share +} // namespace oceanbase + +#endif // OCEANBASE_SHARE_SYSBIT_AGGREGATE_H_ \ No newline at end of file diff --git a/src/share/aggregate/util.h b/src/share/aggregate/util.h index 196049c6dc..450fd584b2 100644 --- a/src/share/aggregate/util.h +++ b/src/share/aggregate/util.h @@ -21,6 +21,12 @@ #include "sql/engine/ob_bit_vector.h" #include "share/vector/ob_vector_define.h" +#define EXTRACT_MEM_ADDR(ptr) (reinterpret_cast(*reinterpret_cast((ptr)))) +#define STORE_MEM_ADDR(addr, dst) \ + do { \ + *reinterpret_cast((dst)) = reinterpret_cast((addr)); \ + } while (false) + namespace oceanbase { namespace sql @@ -115,6 +121,37 @@ struct add_param_batch } }; +template > +struct defined_removal_func: std::false_type {}; + +template +struct defined_removal_func< + T, std::void_t)>> + : std::true_type +{}; + +template::value> +struct removal_opt +{ + template + inline static int add_or_sub_row(T &v, Args&&... args) + { + return v.add_or_sub_row(std::forward(args)...); + } +}; + +template +struct removal_opt +{ + template + inline static int add_or_sub_row(T &v, Args&&... args) + { + int ret = OB_NOT_SUPPORTED; + SQL_LOG(WARN, "removal opt not suppported", K(ret)); + return ret; + } +}; + // ================ // read/writer helper @@ -409,6 +446,15 @@ inline int add_values(const int32_t &l, const number::ObCompactNumber &r, char * return ret; } +template +inline int sub_values(const L &l, const R &r, char *res_buf, const int32_t res_len) +{ + int ret = OB_SUCCESS; + L &res = *reinterpret_cast(res_buf) ; + res = l - r; + return ret; +} + template struct Caster { @@ -555,7 +601,13 @@ inline bool supported_aggregate_function(const ObItemType agg_op) case T_FUN_MIN: case T_FUN_MAX: case T_FUN_COUNT_SUM: - case T_FUN_SUM: { + case T_FUN_SUM: + case T_FUN_APPROX_COUNT_DISTINCT: + case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS: + case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE: + case T_FUN_SYS_BIT_OR: + case T_FUN_SYS_BIT_AND: + case T_FUN_SYS_BIT_XOR: { return true; } default: @@ -563,6 +615,23 @@ inline bool supported_aggregate_function(const ObItemType agg_op) } } +inline bool agg_res_not_null(const ObItemType agg_op) +{ + // TODO: add other functions + bool ret = false; + switch (agg_op) { + case T_FUN_COUNT: + case T_FUN_COUNT_SUM: { + ret = true; + break; + } + default: { + break; + } + } + return ret; +} + #define AGG_FIXED_TC_LIST \ VEC_TC_INTEGER, \ VEC_TC_UINTEGER, \ @@ -598,6 +667,7 @@ inline bool supported_aggregate_function(const ObItemType agg_op) VEC_TC_TIME, \ VEC_TC_YEAR, \ VEC_TC_STRING, \ + VEC_TC_EXTEND, \ VEC_TC_BIT, \ VEC_TC_ENUM_SET, \ VEC_TC_ENUM_SET_INNER, \ diff --git a/src/share/config/ob_config_helper.cpp b/src/share/config/ob_config_helper.cpp index 89c1556ac0..d4b4519aa8 100644 --- a/src/share/config/ob_config_helper.cpp +++ b/src/share/config/ob_config_helper.cpp @@ -1256,6 +1256,17 @@ bool ObConfigArchiveLagTargetChecker::check(const uint64_t tenant_id, const ObAd return is_valid; } +bool ObConfigSQLSpillCompressionCodecChecker::check(const ObConfigItem &t) const +{ + bool is_valid = false; + for (int i = 0; i < ARRAYSIZEOF(common::sql_temp_store_compress_funcs) && !is_valid; ++i) { + if (0 == ObString::make_string(sql_temp_store_compress_funcs[i]).case_compare(t.str())) { + is_valid = true; + } + } + return is_valid; +} + bool ObParallelDDLControlParser::parse(const char *str, uint8_t *arr, int64_t len) { bool bret = true; diff --git a/src/share/config/ob_config_helper.h b/src/share/config/ob_config_helper.h index 6c4ced8e1e..24cae0c3dd 100644 --- a/src/share/config/ob_config_helper.h +++ b/src/share/config/ob_config_helper.h @@ -747,6 +747,17 @@ private: DISALLOW_COPY_AND_ASSIGN(ObConfigSQLTlsVersionChecker); }; +class ObConfigSQLSpillCompressionCodecChecker + : public ObConfigChecker +{ +public: + ObConfigSQLSpillCompressionCodecChecker() {} + virtual ~ObConfigSQLSpillCompressionCodecChecker() {} + bool check(const ObConfigItem &t) const; +private: + DISALLOW_COPY_AND_ASSIGN(ObConfigSQLSpillCompressionCodecChecker); +}; + class ObModeConfigParserUitl { public: diff --git a/src/share/datum/ob_datum_funcs.h b/src/share/datum/ob_datum_funcs.h index 4e7726fff2..2d43a0bc63 100644 --- a/src/share/datum/ob_datum_funcs.h +++ b/src/share/datum/ob_datum_funcs.h @@ -68,6 +68,7 @@ public: ObCmpFunc() : cmp_func_(NULL) {} union { common::ObDatumCmpFuncType cmp_func_; + sql::NullSafeRowCmpFunc row_cmp_func_; sql::serializable_function ser_cmp_func_; }; TO_STRING_KV(KP_(cmp_func)); diff --git a/src/share/parameter/ob_parameter_seed.ipp b/src/share/parameter/ob_parameter_seed.ipp index 5613586314..9316586b38 100644 --- a/src/share/parameter/ob_parameter_seed.ipp +++ b/src/share/parameter/ob_parameter_seed.ipp @@ -303,6 +303,16 @@ DEF_INT(_max_malloc_sample_interval, OB_CLUSTER_PARAMETER, "256", "[1, 10000]", DEF_BOOL(_enable_values_table_folding, OB_CLUSTER_PARAMETER, "True", "whether enable values statement folds self params", ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); + +// +DEF_STR_WITH_CHECKER(spill_compression_codec, OB_TENANT_PARAMETER, "NONE", + common::ObConfigSQLSpillCompressionCodecChecker, + "specific the compression algorithm type to compress the spilled data in temp block store "\ + "during the sql execution phase. "\ + "The supported compression codecs are: ZSTD, LZ4, SNAPPY, ZLIB. NONE means no compression."\ + "The default value is NONE.", + ObParameterAttr(Section::OBSERVER, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); + //// tenant config DEF_TIME_WITH_CHECKER(max_stale_time_for_weak_consistency, OB_TENANT_PARAMETER, "5s", common::ObConfigStaleTimeChecker, diff --git a/src/share/vector/ob_continuous_base.cpp b/src/share/vector/ob_continuous_base.cpp index f7051b6eca..73cd749021 100644 --- a/src/share/vector/ob_continuous_base.cpp +++ b/src/share/vector/ob_continuous_base.cpp @@ -35,5 +35,19 @@ namespace common } } } + + void ObContinuousBase::to_rows(const sql::RowMeta &row_meta, + sql::ObCompactRow **stored_rows, + const int64_t size, + const int64_t col_idx) const + { + for (int i = 0; i < size; i++) { + if (nulls_->at(i)) { + stored_rows[i]->set_null(row_meta, col_idx); + } else { + stored_rows[i]->set_cell_payload(row_meta, col_idx, data_ + offsets_[i], get_length(i)); + } + } + } } } \ No newline at end of file diff --git a/src/share/vector/ob_continuous_base.h b/src/share/vector/ob_continuous_base.h index 56b0930d51..ee060632ef 100644 --- a/src/share/vector/ob_continuous_base.h +++ b/src/share/vector/ob_continuous_base.h @@ -53,6 +53,9 @@ public: const uint16_t selector[], const int64_t size, const int64_t col_idx) const override final; + + virtual void to_rows(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, + const int64_t size, const int64_t col_idx) const override final; inline void from(uint32_t *offsets, char *data) { offsets_ = offsets; diff --git a/src/share/vector/ob_continuous_vector.cpp b/src/share/vector/ob_continuous_vector.cpp index b15d3c893a..536412fdf9 100644 --- a/src/share/vector/ob_continuous_vector.cpp +++ b/src/share/vector/ob_continuous_vector.cpp @@ -64,6 +64,12 @@ int ObContinuousVector::null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const return VecOpUtil::template ns_cmp(expr.obj_meta_, *this, row_idx, r_null, r_v, r_len, cmp_ret); } +template +int ObContinuousVector::no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const +{ + return VecOpUtil::Op::cmp(expr.obj_meta_, get_payload(row_idx1), get_length(row_idx1), get_payload(row_idx2), get_length(row_idx2), cmp_ret); +} + template class ObContinuousVector>; template class ObContinuousVector>; template class ObContinuousVector>; diff --git a/src/share/vector/ob_continuous_vector.h b/src/share/vector/ob_continuous_vector.h index f03da99e73..e906431fc3 100644 --- a/src/share/vector/ob_continuous_vector.h +++ b/src/share/vector/ob_continuous_vector.h @@ -40,6 +40,7 @@ public: int murmur_hash_v3_for_one_row(EVAL_HASH_ARGS_FOR_ROW) const override; int null_first_cmp(VECTOR_ONE_COMPARE_ARGS) const override; int null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const override; + int no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const override final; }; } diff --git a/src/share/vector/ob_discrete_base.cpp b/src/share/vector/ob_discrete_base.cpp index 5cf31f9d8e..72572f79d4 100644 --- a/src/share/vector/ob_discrete_base.cpp +++ b/src/share/vector/ob_discrete_base.cpp @@ -34,6 +34,18 @@ namespace common } } + void ObDiscreteBase::to_rows(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, + const int64_t size, const int64_t col_idx) const + { + for (int64_t row_idx = 0; row_idx < size; row_idx++) { + if (nulls_->at(row_idx)) { + stored_rows[row_idx]->set_null(row_meta, col_idx); + } else { + stored_rows[row_idx]->set_cell_payload(row_meta, col_idx, ptrs_[row_idx], lens_[row_idx]); + } + } + } + DEF_TO_STRING(ObDiscreteBase) { int64_t pos = 0; diff --git a/src/share/vector/ob_discrete_base.h b/src/share/vector/ob_discrete_base.h index ffcb0f7f7a..cd4596c4d4 100644 --- a/src/share/vector/ob_discrete_base.h +++ b/src/share/vector/ob_discrete_base.h @@ -39,6 +39,10 @@ public: const uint16_t selector[], const int64_t size, const int64_t col_idx) const override final; + + virtual void to_rows(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, + const int64_t size, const int64_t col_idx) const override final; + protected: ObLength *lens_; char **ptrs_; diff --git a/src/share/vector/ob_discrete_vector.cpp b/src/share/vector/ob_discrete_vector.cpp index 2aeb79e3f1..1f8e13dc04 100644 --- a/src/share/vector/ob_discrete_vector.cpp +++ b/src/share/vector/ob_discrete_vector.cpp @@ -65,6 +65,12 @@ int ObDiscreteVector::null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const return VecOpUtil::template ns_cmp(expr.obj_meta_, *this, row_idx, r_null, r_v, r_len, cmp_ret); } +template +int ObDiscreteVector::no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const +{ + return VecOpUtil::Op::cmp(expr.obj_meta_, get_payload(row_idx1), get_length(row_idx1), get_payload(row_idx2), get_length(row_idx2), cmp_ret); +} + template class ObDiscreteVector>; template class ObDiscreteVector>; template class ObDiscreteVector>; diff --git a/src/share/vector/ob_discrete_vector.h b/src/share/vector/ob_discrete_vector.h index 1384d503cc..5ca8145e96 100644 --- a/src/share/vector/ob_discrete_vector.h +++ b/src/share/vector/ob_discrete_vector.h @@ -40,6 +40,7 @@ public: int murmur_hash_v3_for_one_row(EVAL_HASH_ARGS_FOR_ROW) const override; int null_first_cmp(VECTOR_ONE_COMPARE_ARGS) const override; int null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const override; + int no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const override final; }; } // end namespace common diff --git a/src/share/vector/ob_fixed_length_base.cpp b/src/share/vector/ob_fixed_length_base.cpp index 9f17d5322b..d65745940f 100644 --- a/src/share/vector/ob_fixed_length_base.cpp +++ b/src/share/vector/ob_fixed_length_base.cpp @@ -46,6 +46,18 @@ namespace common } } + void ObFixedLengthBase::to_rows(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, + const int64_t size, const int64_t col_idx) const + { + for (int64_t row_idx = 0; row_idx < size; row_idx++) { + if (nulls_->at(row_idx)) { + stored_rows[row_idx]->set_null(row_meta, col_idx); + } else { + stored_rows[row_idx]->set_cell_payload(row_meta, col_idx, data_ + len_ * row_idx, len_); + } + } + } + DEF_TO_STRING(ObFixedLengthBase) { int64_t pos = 0; diff --git a/src/share/vector/ob_fixed_length_base.h b/src/share/vector/ob_fixed_length_base.h index 38aab5e946..d2b7ec4f31 100644 --- a/src/share/vector/ob_fixed_length_base.h +++ b/src/share/vector/ob_fixed_length_base.h @@ -65,6 +65,10 @@ public: const uint16_t selector[], const int64_t size, const int64_t col_idx) const override final; + + virtual void to_rows(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, + const int64_t size, const int64_t col_idx) const override final; + inline void from(ObLength len, char *data) { len_ = len; diff --git a/src/share/vector/ob_fixed_length_vector.cpp b/src/share/vector/ob_fixed_length_vector.cpp index 06a75eb224..fd3007ac04 100644 --- a/src/share/vector/ob_fixed_length_vector.cpp +++ b/src/share/vector/ob_fixed_length_vector.cpp @@ -66,6 +66,12 @@ int ObFixedLengthVector::null_last_cmp(VECTOR_ONE_COMPARE_AR return VecOpUtil::template ns_cmp(expr.obj_meta_, *this, row_idx, r_null, r_v, r_len, cmp_ret); } +template +int ObFixedLengthVector::no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const +{ + return VecOpUtil::Op::cmp(expr.obj_meta_, this->get_payload(row_idx1), this->get_length(row_idx1), this->get_payload(row_idx2), this->get_length(row_idx2), cmp_ret); +} + template class ObFixedLengthVector>; template class ObFixedLengthVector>; template class ObFixedLengthVector>; diff --git a/src/share/vector/ob_fixed_length_vector.h b/src/share/vector/ob_fixed_length_vector.h index 0190d42b04..18d0961898 100644 --- a/src/share/vector/ob_fixed_length_vector.h +++ b/src/share/vector/ob_fixed_length_vector.h @@ -37,7 +37,9 @@ public: int murmur_hash_v3(BATCH_EVAL_HASH_ARGS) const override; int murmur_hash_v3_for_one_row(EVAL_HASH_ARGS_FOR_ROW) const override; int null_first_cmp(VECTOR_ONE_COMPARE_ARGS) const override; - int null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const override; + int null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const override final; + int no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const override final; + // TODO: add not null safe cmp function }; diff --git a/src/share/vector/ob_i_vector.h b/src/share/vector/ob_i_vector.h index 6385dfe06e..8803d30103 100644 --- a/src/share/vector/ob_i_vector.h +++ b/src/share/vector/ob_i_vector.h @@ -52,6 +52,10 @@ namespace common const ObLength r_len, \ int &cmp_ret +#define VECTOR_NOT_NULL_COMPARE_ARGS const sql::ObExpr &expr, \ + const int64_t row_idx1, \ + const int64_t row_idx2, \ + int &cmp_ret #define DEF_VEC_READ_INTERFACES(Derived) \ public: \ OB_INLINE bool is_false(const int64_t idx) const \ @@ -490,6 +494,7 @@ public: // the type of other value must be same as this vector virtual int null_first_cmp(VECTOR_ONE_COMPARE_ARGS) const = 0; virtual int null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const = 0; + virtual int no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const = 0; // append values to this vector from idx-th column of rows virtual int from_rows(const sql::RowMeta &row_meta, @@ -514,6 +519,12 @@ public: const uint16_t selector[], const int64_t size, const int64_t col_idx) const = 0; + + virtual void to_rows(const sql::RowMeta &row_meta, + sql::ObCompactRow **stored_rows, + const int64_t size, + const int64_t col_idx) const = 0; + virtual int to_row(const sql::RowMeta &row_meta, sql::ObCompactRow *stored_row, const uint64_t row_idx, diff --git a/src/share/vector/ob_uniform_base.cpp b/src/share/vector/ob_uniform_base.cpp index 6202948fd0..9c551ec5f3 100644 --- a/src/share/vector/ob_uniform_base.cpp +++ b/src/share/vector/ob_uniform_base.cpp @@ -49,6 +49,30 @@ namespace common } } + void ObUniformBase::to_rows(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, + const int64_t size, const int64_t col_idx) const + { + if (get_format() == VEC_UNIFORM) { + for (int64_t row_idx = 0; row_idx < size; row_idx++) { + if (datums_[row_idx].is_null()) { + stored_rows[row_idx]->set_null(row_meta, col_idx); + } else { + stored_rows[row_idx]->set_cell_payload(row_meta, col_idx, datums_[row_idx].ptr_, + datums_[row_idx].len_); + } + } + } else { + for (int64_t i = 0, row_idx = 0; i < size; i++) { + if (datums_[row_idx].is_null()) { + stored_rows[i]->set_null(row_meta, col_idx); + } else { + stored_rows[i]->set_cell_payload(row_meta, col_idx, datums_[row_idx].ptr_, + datums_[row_idx].len_); + } + } + } + } + DEF_TO_STRING(ObUniformBase) { int64_t pos = 0; diff --git a/src/share/vector/ob_uniform_base.h b/src/share/vector/ob_uniform_base.h index fe1323484d..dc9befffc2 100644 --- a/src/share/vector/ob_uniform_base.h +++ b/src/share/vector/ob_uniform_base.h @@ -45,6 +45,9 @@ public: const int64_t size, const int64_t col_idx) const override final; + virtual void to_rows(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, + const int64_t size, const int64_t col_idx) const override final; + protected: ObDatum *datums_; sql::ObEvalInfo *eval_info_; // just used for maintain has_null flag in uniform vector diff --git a/src/share/vector/ob_uniform_vector.cpp b/src/share/vector/ob_uniform_vector.cpp index e26a009f57..776ce82a8f 100644 --- a/src/share/vector/ob_uniform_vector.cpp +++ b/src/share/vector/ob_uniform_vector.cpp @@ -68,6 +68,12 @@ int ObUniformVector::null_last_cmp(VECTOR_ONE_COMPARE_ARGS) c return expr.basic_funcs_->null_last_cmp_(this->get_datum(row_idx), ObDatum(r_v, r_len, r_null), cmp_ret); } +template +int ObUniformVector::no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const +{ + return expr.basic_funcs_->null_last_cmp_(this->get_datum(row_idx1), this->get_datum(row_idx2), cmp_ret); +} + template class ObUniformVector>; template class ObUniformVector>; template class ObUniformVector>; diff --git a/src/share/vector/ob_uniform_vector.h b/src/share/vector/ob_uniform_vector.h index a29876680e..19b66d4f62 100644 --- a/src/share/vector/ob_uniform_vector.h +++ b/src/share/vector/ob_uniform_vector.h @@ -43,6 +43,7 @@ public: int murmur_hash_v3_for_one_row(EVAL_HASH_ARGS_FOR_ROW) const override; int null_first_cmp(VECTOR_ONE_COMPARE_ARGS) const override; int null_last_cmp(VECTOR_ONE_COMPARE_ARGS) const override; + int no_null_cmp(VECTOR_NOT_NULL_COMPARE_ARGS) const override; }; } // end namespace common diff --git a/src/share/vector/ob_vector_define.h b/src/share/vector/ob_vector_define.h index b7e586d940..428756fa29 100644 --- a/src/share/vector/ob_vector_define.h +++ b/src/share/vector/ob_vector_define.h @@ -24,6 +24,29 @@ using UniformFormat = ObUniformFormat; #define DEFINE_FIXED_VECTOR(vector_name, vec_tc) \ using vector_name = ObFixedLengthVector, VectorBasicOp>; +#define FIXED_VEC_LIST \ + VEC_TC_INTEGER, \ + VEC_TC_UINTEGER, \ + VEC_TC_FLOAT, \ + VEC_TC_DOUBLE, \ + VEC_TC_FIXED_DOUBLE, \ + VEC_TC_DATETIME, \ + VEC_TC_DATE, \ + VEC_TC_TIME, \ + VEC_TC_YEAR, \ + VEC_TC_ENUM_SET, \ + VEC_TC_UNKNOWN, \ + VEC_TC_BIT, \ + VEC_TC_TIMESTAMP_TZ, \ + VEC_TC_TIMESTAMP_TINY, \ + VEC_TC_INTERVAL_YM, \ + VEC_TC_INTERVAL_DS, \ + VEC_TC_DEC_INT32, \ + VEC_TC_DEC_INT64, \ + VEC_TC_DEC_INT128, \ + VEC_TC_DEC_INT256, \ + VEC_TC_DEC_INT512 + DEFINE_FIXED_VECTOR(IntegerFixedVec, VEC_TC_INTEGER); DEFINE_FIXED_VECTOR(UIntegerFixedVec, VEC_TC_UINTEGER); DEFINE_FIXED_VECTOR(FloatFixedVec, VEC_TC_FLOAT); @@ -82,7 +105,7 @@ constexpr bool is_decint_vec(const VecValueTypeClass tc) DEFINE_CONTINUOUS_VECTOR(NumberContVec, VEC_TC_NUMBER); DEFINE_CONTINUOUS_VECTOR(ExtendContVec, VEC_TC_EXTEND); -DEFINE_CONTINUOUS_VECTOR(TextContVec, VEC_TC_STRING); +DEFINE_CONTINUOUS_VECTOR(StrContVec, VEC_TC_STRING); DEFINE_CONTINUOUS_VECTOR(ESInnerContVec, VEC_TC_ENUM_SET_INNER); DEFINE_CONTINUOUS_VECTOR(RawContVec, VEC_TC_RAW); DEFINE_CONTINUOUS_VECTOR(RowidContVec, VEC_TC_ROWID); @@ -112,7 +135,7 @@ constexpr bool is_continuous_vec(const VecValueTypeClass tc) using vector_name = ObDiscreteVector>; DEFINE_DISCRETE_VECTOR(NumberDiscVec, VEC_TC_NUMBER); DEFINE_DISCRETE_VECTOR(ExtendDiscVec, VEC_TC_EXTEND); -DEFINE_DISCRETE_VECTOR(TextDiscVec, VEC_TC_STRING); +DEFINE_DISCRETE_VECTOR(StrDiscVec, VEC_TC_STRING); DEFINE_DISCRETE_VECTOR(ESInnerDiscVec, VEC_TC_ENUM_SET_INNER); DEFINE_DISCRETE_VECTOR(RawDiscVec, VEC_TC_RAW); DEFINE_DISCRETE_VECTOR(RowidDiscVec, VEC_TC_ROWID); @@ -164,7 +187,7 @@ DEFINE_UNIFORM_VECTOR(DecInt256UniVec, VEC_TC_DEC_INT256); DEFINE_UNIFORM_VECTOR(DecInt512UniVec, VEC_TC_DEC_INT512); DEFINE_UNIFORM_VECTOR(NumberUniVec, VEC_TC_NUMBER); DEFINE_UNIFORM_VECTOR(ExtendUniVec, VEC_TC_EXTEND); -DEFINE_UNIFORM_VECTOR(TextUniVec, VEC_TC_STRING); +DEFINE_UNIFORM_VECTOR(StrUniVec, VEC_TC_STRING); DEFINE_UNIFORM_VECTOR(ESInnerUniVec, VEC_TC_ENUM_SET_INNER); DEFINE_UNIFORM_VECTOR(RawUniVec, VEC_TC_RAW); DEFINE_UNIFORM_VECTOR(RowidUniVec, VEC_TC_ROWID); @@ -206,7 +229,7 @@ DEFINE_UNIFORM_CONST_VECTOR(DecInt256UniCVec, VEC_TC_DEC_INT256); DEFINE_UNIFORM_CONST_VECTOR(DecInt512UniCVec, VEC_TC_DEC_INT512); DEFINE_UNIFORM_CONST_VECTOR(NumberUniCVec, VEC_TC_NUMBER); DEFINE_UNIFORM_CONST_VECTOR(ExtendUniCVec, VEC_TC_EXTEND); -DEFINE_UNIFORM_CONST_VECTOR(TextUniCVec, VEC_TC_STRING); +DEFINE_UNIFORM_CONST_VECTOR(StrUniCVec, VEC_TC_STRING); DEFINE_UNIFORM_CONST_VECTOR(ESInnerUniCVec, VEC_TC_ENUM_SET_INNER); DEFINE_UNIFORM_CONST_VECTOR(RawUniCVec, VEC_TC_RAW); DEFINE_UNIFORM_CONST_VECTOR(RowidUniCVec, VEC_TC_ROWID); diff --git a/src/share/vector/vector_op_util.h b/src/share/vector/vector_op_util.h index 65a99dc5c2..fc6b622cb5 100644 --- a/src/share/vector/vector_op_util.h +++ b/src/share/vector/vector_op_util.h @@ -146,6 +146,195 @@ private: } }; +struct VectorRangeUtil +{ + template + static int lower_bound(sql::ObExpr *expr, sql::ObEvalCtx &ctx, const sql::EvalBound &bound, + const sql::ObBitVector &skip, OP cmp_op, int64_t &iter) + { + return common_bound(expr, ctx, bound, skip, cmp_op, iter); + } + + template + static int upper_bound(sql::ObExpr *expr, sql::ObEvalCtx &ctx, const sql::EvalBound &bound, + const sql::ObBitVector &skip, OP cmp_op, int64_t &iter) + { + return common_bound(expr, ctx, bound, skip, cmp_op, iter); + } + + struct NullSafeCmp + { + NullSafeCmp(ObObjMeta &obj_meta, const sql::NullSafeRowCmpFunc cmp, const char *value, + const int32_t len, bool is_null, bool is_ascending) : + obj_meta_(obj_meta), + cmp_(cmp), value_(value), len_(len), is_null_(is_null), is_ascending_(is_ascending) + {} + + OB_INLINE int operator()(const ObObjMeta &other_meta, const char *other, int32_t other_len, + const bool other_null, int &cmp_ret) const + { + int ret = OB_SUCCESS; + cmp_ret = 0; + if (OB_FAIL(cmp_(other_meta, obj_meta_, + other, other_len, other_null, + value_, len_, is_null_, + cmp_ret))) { + COMMON_LOG(WARN, "compare failed", K(ret)); + } else { + cmp_ret *= (is_ascending_ ? 1 : -1); + } + return ret; + } + private: + ObObjMeta &obj_meta_; + const sql::NullSafeRowCmpFunc cmp_; + const char *value_; + const int32_t len_; + const bool is_null_; + const bool is_ascending_; + }; + +private: + template + static int common_bound(sql::ObExpr *expr, sql::ObEvalCtx &ctx, const sql::EvalBound &bound, + const sql::ObBitVector &skip, OP cmp_op, int64_t &iter); + template + static int find_bound(const ObObjMeta &obj_meta, ObIVector *ivector, sql::ObEvalCtx &ctx, + const sql::EvalBound &bound, const sql::ObBitVector &skip, OP cmp_op, + int64_t &iter); +}; + +class ObDiscreteFormat; +template +class ObUniformFormat; +class ObContinuousFormat; +template +class ObFixedLengthFormat; + +#define FIND_BOUND_USE_FMT(fmt) \ + ret = find_bound(expr->obj_meta_, expr->get_vector(ctx), ctx, bound, skip, \ + cmp_op, iter) + +#define FIND_BOUND_USE_FIXED_FMT(vec_tc) \ + case (vec_tc): { \ + ret = FIND_BOUND_USE_FMT(ObFixedLengthFormat>); \ + } break + +template +int VectorRangeUtil::common_bound(sql::ObExpr *expr, sql::ObEvalCtx &ctx, + const sql::EvalBound &bound, const sql::ObBitVector &skip, + OP cmp_op, int64_t &iter) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(expr->eval_vector(ctx, skip, bound))) { + COMMON_LOG(WARN, "eval vector failed", K(ret)); + } else { + iter = -1; + VectorFormat fmt = expr->get_format(ctx); + VecValueTypeClass vec_tc = expr->get_vec_value_tc(); + switch (fmt) { + case VEC_DISCRETE: { + FIND_BOUND_USE_FMT(ObDiscreteFormat); + break; + } + case VEC_FIXED: { + switch(vec_tc) { + LST_DO_CODE(FIND_BOUND_USE_FIXED_FMT, FIXED_VEC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + COMMON_LOG(WARN, "unexpected type class", K(vec_tc)); + } + } + break; + } + case VEC_CONTINUOUS: { + FIND_BOUND_USE_FMT(ObContinuousFormat); + break; + } + case VEC_UNIFORM: { + FIND_BOUND_USE_FMT(ObUniformFormat); + break; + } + case VEC_UNIFORM_CONST: { + FIND_BOUND_USE_FMT(ObUniformFormat); + break; + } + default: { + ret = OB_ERR_UNDEFINED; + COMMON_LOG(WARN, "unexpected data format", K(ret), K(fmt)); + } + } + if (OB_FAIL(ret)) { + COMMON_LOG(WARN, "find bound failed", K(ret)); + } + } + return ret; +} + +#undef FIND_BOUND_USE_FMT +#undef FIND_BOUND_USE_FIXED_FMT + +template +int VectorRangeUtil::find_bound(const ObObjMeta &obj_meta, ObIVector *ivector, sql::ObEvalCtx &ctx, + const sql::EvalBound &bound, const sql::ObBitVector &skip, + OP cmp_op, int64_t &iter) +{ + // TODO: use binary search + int ret = OB_SUCCESS; + int cmp_ret = 0; + const char *payload = nullptr; + int32_t len = 0; + bool is_null = false; + iter = -1; + VecFmt *data = static_cast(ivector); + if (std::is_same>::value) { + bool skip_all = (skip.accumulate_bit_cnt(bound) == bound.range_size()); + if (skip_all) { + } else { + is_null = data->is_null(0); + data->get_payload(0, payload, len); + if(OB_FAIL(cmp_op(obj_meta, payload,len, is_null, cmp_ret))) { + COMMON_LOG(WARN, "compare failed", K(ret)); + } else if (is_lower && cmp_ret >= 0) { + iter = 0; + } else if (!is_lower && cmp_ret > 0) { + iter = 0; + } + } + } else if (OB_LIKELY(bound.get_all_rows_active())) { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + is_null = data->is_null(i); + data->get_payload(i, payload, len); + if (OB_FAIL(cmp_op(obj_meta, payload, len, is_null, cmp_ret))) { + COMMON_LOG(WARN, "compare failed", K(ret)); + } else if (is_lower && cmp_ret >= 0) { + iter = i; + break; + } else if (!is_lower && cmp_ret > 0) { + iter = i; + break; + } + } + } else { + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + if (skip.at(i)) { + } else { + is_null = data->is_null(i); + data->get_payload(i, payload, len); + if (OB_FAIL(cmp_op(obj_meta, payload, len, is_null, cmp_ret))) { + COMMON_LOG(WARN, "compare failed", K(ret)); + } else if (is_lower && cmp_ret >= 0) { + iter = i; + break; + } else if (!is_lower && cmp_ret > 0) { + iter = i; + break; + } + } + } + } + return ret; +} } // end namespace common } // end namespace oceanbase #endif // OCEANBASE_SHARE_VECTOR_VECTOR_OP_UTIL_H_ diff --git a/src/sql/CMakeLists.txt b/src/sql/CMakeLists.txt index 2ddff40342..15b2127112 100644 --- a/src/sql/CMakeLists.txt +++ b/src/sql/CMakeLists.txt @@ -141,6 +141,11 @@ ob_set_subtarget(ob_sql engine engine/sort/ob_sort_vec_op_eager_filter.cpp engine/sort/ob_sort_key_fetcher_vec_op.cpp engine/ob_subschema_ctx.cpp + engine/sort/ob_pd_topn_sort_filter.cpp + engine/window_function/ob_window_function_vec_op.cpp + engine/window_function/row_store.cpp + engine/window_function/win_expr.cpp + engine/window_function/row_store.cpp ) ob_set_subtarget(ob_sql engine_aggregate @@ -151,6 +156,7 @@ ob_set_subtarget(ob_sql engine_aggregate engine/aggregate/ob_hash_distinct_op.cpp engine/aggregate/ob_hash_groupby_op.cpp engine/aggregate/ob_merge_distinct_op.cpp + engine/aggregate/ob_merge_distinct_vec_op.cpp engine/aggregate/ob_merge_groupby_op.cpp engine/aggregate/ob_scalar_aggregate_op.cpp engine/aggregate/ob_adaptive_bypass_ctrl.cpp @@ -738,6 +744,7 @@ ob_set_subtarget(ob_sql engine_expr engine/expr/ob_expr_inner_row_cmp_val.cpp engine/expr/ob_expr_last_refresh_scn.cpp engine/expr/ob_expr_json_utils.cpp + engine/expr/ob_expr_topn_filter.cpp engine/expr/ob_expr_inner_table_option_printer.cpp engine/expr/ob_expr_rb_build_empty.cpp engine/expr/ob_expr_rb_is_empty.cpp @@ -853,6 +860,10 @@ ob_set_subtarget(ob_sql engine_set engine/set/ob_merge_intersect_op.cpp engine/set/ob_merge_set_op.cpp engine/set/ob_merge_union_op.cpp + engine/set/ob_hash_set_vec_op.cpp + engine/set/ob_hash_union_vec_op.cpp + engine/set/ob_hash_except_vec_op.cpp + engine/set/ob_hash_intersect_vec_op.cpp engine/set/ob_set_op.cpp ) diff --git a/src/sql/code_generator/ob_static_engine_cg.cpp b/src/sql/code_generator/ob_static_engine_cg.cpp index 6940046574..e89ffd6c3c 100644 --- a/src/sql/code_generator/ob_static_engine_cg.cpp +++ b/src/sql/code_generator/ob_static_engine_cg.cpp @@ -72,6 +72,7 @@ #include "sql/engine/table/ob_table_scan_op.h" #include "sql/engine/aggregate/ob_hash_distinct_op.h" #include "sql/engine/aggregate/ob_merge_distinct_op.h" +#include "sql/engine/aggregate/ob_merge_distinct_vec_op.h" #include "share/schema/ob_schema_getter_guard.h" #include "sql/engine/basic/ob_expr_values_op.h" #include "sql/engine/dml/ob_table_insert_op.h" @@ -115,6 +116,7 @@ #include "sql/engine/dml/ob_table_insert_up_op.h" #include "sql/engine/dml/ob_table_replace_op.h" #include "sql/engine/window_function/ob_window_function_op.h" +#include "sql/engine/window_function/ob_window_function_vec_op.h" #include "sql/engine/table/ob_row_sample_scan_op.h" #include "sql/engine/table/ob_block_sample_scan_op.h" #include "sql/engine/table/ob_table_scan_with_index_back_op.h" @@ -154,6 +156,10 @@ #include "share/aggregate/processor.h" #include "share/vector/expr_cmp_func.h" #include "sql/engine/sort/ob_sort_vec_op.h" +#include "sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h" +#include "sql/engine/set/ob_hash_union_vec_op.h" +#include "sql/engine/set/ob_hash_intersect_vec_op.h" +#include "sql/engine/set/ob_hash_except_vec_op.h" #ifdef OB_BUILD_TDE_SECURITY #include "share/ob_master_key_getter.h" #endif @@ -199,12 +205,15 @@ int ObStaticEngineCG::generate(const ObLogPlan &log_plan, ObPhysicalPlan &phy_pl const bool in_root_job = true; const bool is_subplan = false; bool check_eval_once = true; + ObCompressorType compress_type = NONE_COMPRESSOR; if (OB_ISNULL(log_plan.get_plan_root())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("no logical plan root", K(ret)); + } else if (OB_FAIL(get_query_compress_type(log_plan, compress_type))) { + LOG_WARN("fail to get query compress type", K(ret)); } else if (OB_FAIL(postorder_generate_op( *log_plan.get_plan_root(), root_spec, in_root_job, is_subplan, - check_eval_once, need_check_output_datum))) { + check_eval_once, need_check_output_datum, compress_type))) { LOG_WARN("failed to generate plan", K(ret)); } else if (OB_ISNULL(root_spec)) { ret = OB_ERR_UNEXPECTED; @@ -223,7 +232,8 @@ int ObStaticEngineCG::postorder_generate_op(ObLogicalOperator &op, const bool in_root_job, const bool is_subplan, bool &check_eval_once, - const bool need_check_output_datum) + const bool need_check_output_datum, + const ObCompressorType compress_type) { int ret = OB_SUCCESS; const int64_t child_num = op.get_num_of_child(); @@ -254,7 +264,8 @@ int ObStaticEngineCG::postorder_generate_op(ObLogicalOperator &op, } else if (OB_FAIL(SMART_CALL(postorder_generate_op(*child_op, child_spec, in_root_job && is_exchange, is_subplan, child_op_check_eval_once, - need_check_output_datum)))) { + need_check_output_datum, + compress_type)))) { LOG_WARN("generate child op failed", K(ret), K(op.get_name())); } else if (OB_ISNULL(child_spec)) { ret = OB_ERR_UNEXPECTED; @@ -321,7 +332,8 @@ int ObStaticEngineCG::postorder_generate_op(ObLogicalOperator &op, } else if (OB_FAIL(ObOperatorFactory::generate_spec(*this, op, *spec, in_root_job))) { LOG_WARN("generate operator spec failed", K(ret), KP(phy_plan_), K(ob_phy_operator_type_str(type))); - } else if (OB_FAIL(generate_spec_basic(op, *spec, check_eval_once, need_check_output_datum))) { + } else if (OB_FAIL(generate_spec_basic(op, *spec, check_eval_once, need_check_output_datum, + compress_type))) { LOG_WARN("generate operator spec basic failed", K(ret)); } else if (OB_FAIL(generate_spec_final(op, *spec))) { LOG_WARN("generate operator spec final failed", K(ret)); @@ -775,7 +787,8 @@ int ObStaticEngineCG::generate_rt_exprs(const ObIArray &src, int ObStaticEngineCG::generate_spec_basic(ObLogicalOperator &op, ObOpSpec &spec, const bool check_eval_once, - const bool need_check_output_datum) + const bool need_check_output_datum, + const common::ObCompressorType compress_type) { int ret = OB_SUCCESS; if (0 == spec.rows_) { @@ -785,6 +798,7 @@ int ObStaticEngineCG::generate_spec_basic(ObLogicalOperator &op, spec.width_ = op.get_width(); spec.plan_depth_ = op.get_plan_depth(); spec.px_est_size_factor_ = op.get_px_est_size_factor(); + spec.compress_type_ = compress_type; OZ(generate_rt_exprs(op.get_startup_exprs(), spec.startup_filters_)); @@ -880,6 +894,52 @@ int ObStaticEngineCG::generate_spec_basic(ObLogicalOperator &op, return ret; } +int ObStaticEngineCG::get_query_compress_type(const ObLogPlan &log_plan, + ObCompressorType &compress_type) +{ + int ret = OB_SUCCESS; + ObString codec_str; + const int64_t tenant_id = + log_plan.get_optimizer_context().get_session_info()->get_effective_tenant_id(); + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); + if (OB_ISNULL(log_plan.get_stmt()) || OB_ISNULL(log_plan.get_stmt()->get_query_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("stmt or query ctx is null", K(ret)); + } else { + const ObOptParamHint *opt_params = + &log_plan.get_stmt()->get_query_ctx()->get_global_hint().opt_params_; + ObObj hint_val; + if (OB_FAIL(opt_params->get_opt_param(ObOptParamHint::SPILL_COMPRESSION_CODEC, hint_val))) { + LOG_WARN("fail to get compression algorithm opt param from hint", K(ret)); + } else if (hint_val.is_nop_value()) { // get compression algorithm from configure + codec_str = ObString::make_string(tenant_config->spill_compression_codec.get_value()); + } else if (tenant_config.is_valid()) { // get compression algorithm from hint + codec_str = hint_val.get_varchar(); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected tenant config", K(ret)); + } + } + compress_type = NONE_COMPRESSOR; + if (OB_FAIL(ret)) { + } else if (0 == ObString::make_string("none").case_compare(codec_str)) { + compress_type = NONE_COMPRESSOR; + } else if (0 == ObString::make_string("zstd").case_compare(codec_str)) { + compress_type = ZSTD_COMPRESSOR; + } else if (0 == ObString::make_string("lz4").case_compare(codec_str)) { + compress_type = LZ4_COMPRESSOR; + } else if (0 == ObString::make_string("snappy").case_compare(codec_str)) { + compress_type = SNAPPY_COMPRESSOR; + } else if (0 == ObString::make_string("zlib").case_compare(codec_str)) { + compress_type = ZLIB_COMPRESSOR; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected compression algorithm", K(ret)); + } + LOG_TRACE("check query compress type", K(ret), K(compress_type)); + return ret; +} + int ObStaticEngineCG::extract_all_mview_ids(const ObIArray &exprs) { int ret = OB_SUCCESS; @@ -1174,6 +1234,53 @@ int ObStaticEngineCG::generate_spec( return ret; } +int ObStaticEngineCG::generate_spec( + ObLogDistinct &op, ObMergeDistinctVecSpec &spec, const bool in_root_job) +{ + int ret = OB_SUCCESS; + UNUSED(in_root_job); + spec.by_pass_enabled_ = false; + if (op.get_block_mode()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("merge distinct has no block mode", K(op.get_algo()), K(op.get_block_mode()), K(ret)); + } else if (OB_FAIL(spec.cmp_funcs_.init(op.get_distinct_exprs().count()))) { + LOG_WARN("failed to init sort functions", K(ret)); + } else if (OB_FAIL(spec.distinct_exprs_.init(op.get_distinct_exprs().count()))) { + LOG_WARN("failed to init distinct exprs", K(ret)); + } else { + ObExpr *expr = nullptr; + ARRAY_FOREACH(op.get_distinct_exprs(), i) { + const ObRawExpr* raw_expr = op.get_distinct_exprs().at(i); + if (OB_ISNULL(raw_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("null pointer", K(ret)); + } else if (is_oracle_mode() && OB_UNLIKELY(ObLongTextType == raw_expr->get_data_type() + || ObLobType == raw_expr->get_data_type())) { + ret = OB_ERR_INVALID_TYPE_FOR_OP; + LOG_WARN("select distinct lob not allowed", K(ret)); + } else if (is_oracle_mode() && OB_UNLIKELY(ObJsonType == raw_expr->get_data_type())) { + ret = OB_ERR_INVALID_CMP_OP; + LOG_WARN("select distinct json not allowed", K(ret)); + } else if (raw_expr->is_const_expr()) { + // distinct const value, 这里需要注意:distinct 1被跳过了, + // 但ObMergeDistinct中,如果没有distinct列,则默认所有值都相等,这个语义正好是符合预期的。 + continue; + } else if (OB_FAIL(generate_rt_expr(*raw_expr, expr))) { + LOG_WARN("failed to generate rt expr", K(ret)); + } else if (OB_FAIL(spec.distinct_exprs_.push_back(expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } else { + ObCmpFunc cmp_func; + // no matter null first or null last. + cmp_func.cmp_func_ = expr->basic_funcs_->null_last_cmp_; + CK(NULL != cmp_func.cmp_func_); + OZ(spec.cmp_funcs_.push_back(cmp_func)); + } + } + } + return ret; +} + void ObStaticEngineCG::set_murmur_hash_func( ObHashFunc &hash_func, const ObExprBasicFuncs *basic_funcs_) { @@ -1569,6 +1676,92 @@ int ObStaticEngineCG::generate_spec(ObLogSet &op, ObHashExceptSpec &spec, const return ret; } +int ObStaticEngineCG::generate_spec(ObLogSet &op, ObHashUnionVecSpec &spec, const bool in_root_job) +{ + int ret = OB_SUCCESS; + UNUSED(in_root_job); + if (OB_FAIL(generate_hash_set_spec(op, spec))) { + LOG_WARN("failed to generate spec set", K(ret)); + } + return ret; +} + +int ObStaticEngineCG::generate_spec(ObLogSet &op, ObHashIntersectVecSpec &spec, const bool in_root_job) +{ + int ret = OB_SUCCESS; + UNUSED(in_root_job); + if (OB_FAIL(generate_hash_set_spec(op, spec))) { + LOG_WARN("failed to generate spec set", K(ret)); + } + return ret; +} + +int ObStaticEngineCG::generate_spec(ObLogSet &op, ObHashExceptVecSpec &spec, const bool in_root_job) +{ + int ret = OB_SUCCESS; + UNUSED(in_root_job); + if (OB_FAIL(generate_hash_set_spec(op, spec))) { + LOG_WARN("failed to generate spec set", K(ret)); + } + return ret; +} + +int ObStaticEngineCG::generate_hash_set_spec(ObLogSet &op, ObHashSetVecSpec &spec) +{ + int ret = OB_SUCCESS; + ObSEArray out_raw_exprs; + if (OB_FAIL(op.get_pure_set_exprs(out_raw_exprs))) { + LOG_WARN("failed to get output exprs", K(ret)); + } else if (OB_FAIL(mark_expr_self_produced(out_raw_exprs))) { // set expr + LOG_WARN("fail to mark exprs self produced", K(ret)); + } else if (OB_FAIL(spec.set_exprs_.init(out_raw_exprs.count()))) { + LOG_WARN("failed to init set exprs", K(ret)); + } else if (OB_FAIL(generate_rt_exprs(out_raw_exprs, spec.set_exprs_))) { + LOG_WARN("failed to generate rt exprs", K(ret)); + } else if (OB_FAIL(spec.sort_collations_.init(spec.set_exprs_.count()))) { + LOG_WARN("failed to init sort collations", K(ret)); + } else { + for (int64_t i = 0; i < spec.set_exprs_.count() && OB_SUCC(ret); ++i) { + ObRawExpr *raw_expr = out_raw_exprs.at(i); + ObExpr *expr = spec.set_exprs_.at(i); + ObOrderDirection order_direction = default_asc_direction(); + bool is_ascending = is_ascending_direction(order_direction); + ObSortFieldCollation field_collation(i, + expr->datum_meta_.cs_type_, + is_ascending, + (is_null_first(order_direction) ^ is_ascending) ? NULL_LAST : NULL_FIRST); + if (raw_expr->get_expr_type() != expr->type_ || + !(T_OP_SET < expr->type_ && expr->type_ <= T_OP_EXCEPT)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: expr type is not match", + K(raw_expr->get_expr_type()), K(expr->type_)); + } else if (OB_ISNULL(expr->basic_funcs_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: basic funcs is not init", K(ret)); + } else if (ob_is_user_defined_sql_type(expr->datum_meta_.type_) || ob_is_user_defined_pl_type(expr->datum_meta_.type_)) { + // other udt types not supported, xmltype does not have order or map member function + ret = OB_ERR_NO_ORDER_MAP_SQL; + LOG_WARN("cannot ORDER objects without MAP or ORDER method", K(ret)); + } else if (OB_FAIL(spec.sort_collations_.push_back(field_collation))) { + LOG_WARN("failed to push back sort collation", K(ret)); + } + } + } + if (OB_SUCC(ret)) { + ObOpSpec *left = nullptr; + ObOpSpec *right = nullptr; + if (OB_UNLIKELY(spec.get_child_cnt() != 2) + || OB_ISNULL(left = spec.get_child(0)) + || OB_ISNULL(right = spec.get_child(1)) + || OB_UNLIKELY(left->get_output_count() != spec.set_exprs_.count()) + || OB_UNLIKELY(right->get_output_count() != spec.set_exprs_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("cg check failed", K(ret), K(spec.get_child_cnt()), K(spec.set_exprs_.count())); + } + } + return ret; +} + int ObStaticEngineCG::generate_hash_set_spec(ObLogSet &op, ObHashSetSpec &spec) { int ret = OB_SUCCESS; @@ -2104,6 +2297,11 @@ int ObStaticEngineCG::generate_spec(ObLogSort &op, ObSortSpec &spec, const bool ret = OB_ERR_UNEXPECTED; LOG_WARN("topn must be int", K(ret), K(*spec.topn_expr_)); } + if (OB_SUCC(ret) && op.enable_pd_topn_filter()) { + if (OB_FAIL(prepare_topn_runtime_filter_info(op, spec))) { + LOG_WARN("failed to prepare topn runtime_filter info"); + } + } } if (OB_NOT_NULL(op.get_topk_limit_expr())) { OZ(generate_rt_expr(*op.get_topk_limit_expr(), spec.topk_limit_expr_)); @@ -2318,6 +2516,59 @@ int ObStaticEngineCG::generate_sort_exprs(const bool is_store_sortkey_separately return ret; } +template +int ObStaticEngineCG::prepare_topn_runtime_filter_info(ObLogSort &op, ObOpSpec &spec) +{ + int ret = OB_SUCCESS; + int64_t max_batch_size = spec.max_batch_size_; + const ObRawExpr *pd_topn_filter_expr = op.get_pushdown_topn_filter_expr(); + double adaptive_filter_ratio = 0.5; + if (pd_topn_filter_expr->is_white_runtime_filter_expr()) { + adaptive_filter_ratio = 0.1; + } + ObExpr *pd_topn_filter_rt_expr = nullptr; + if (OB_ISNULL(pd_topn_filter_rt_expr = reinterpret_cast( + ObStaticEngineExprCG::get_left_value_rt_expr(*pd_topn_filter_expr)))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null rt_expr"); + } else { + int64_t effective_sk_cnt = op.get_effective_sk_cnt_of_topn_filter(); + int64_t total_sk_cnt = op.get_sort_keys().count(); + bool is_shuffle_pd_topn_filter = op.is_shuffle_pd_topn_filter(); + // TODO XUNSI: in shuffled scene, we can share the msg in create dfo, not support now + bool is_shared_pd_topn_filter = false; + ObSEArray cmp_metas; + ObTopNFilterCmpMeta cmp_meta; + for (int64_t i = 0; i < effective_sk_cnt && OB_SUCC(ret); ++i) { + bool is_null_first = op.get_sort_keys().at(i).is_null_first(); + const ObExpr *sort_key = pd_topn_filter_rt_expr->args_[i]; + const sql::ObDatumMeta &meta = sort_key->datum_meta_; + NullSafeRowCmpFunc null_first_cmp = nullptr, null_last_cmp = nullptr; + VectorCmpExprFuncsHelper::get_cmp_set(meta, meta, null_first_cmp, null_last_cmp); + cmp_meta.cmp_func_ = is_null_first ? null_first_cmp : null_last_cmp; + cmp_meta.obj_meta_ = sort_key->obj_meta_; + if (OB_FAIL(cmp_metas.push_back(cmp_meta))) { + LOG_WARN("failed to push back cmp meta"); + } + } + ObPushDownTopNFilterInfo *topn_filter_info = nullptr; + if (USE_RICH_FORMAT) { + topn_filter_info = &static_cast(spec).pd_topn_filter_info_; + } else { + topn_filter_info = &static_cast(spec).pd_topn_filter_info_; + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(topn_filter_info->init( + op.get_p2p_sequence_id(), effective_sk_cnt, total_sk_cnt, cmp_metas, + ObP2PDatahubMsgBase::PD_TOPN_FILTER_MSG, pd_topn_filter_rt_expr->expr_ctx_id_, + is_shared_pd_topn_filter, is_shuffle_pd_topn_filter, max_batch_size, + adaptive_filter_ratio))) { + LOG_WARN("failed to init topn_filter_info"); + } + } + return ret; +} + int ObStaticEngineCG::generate_spec(ObLogSort &op, ObSortVecSpec &spec, const bool in_root_job) { int ret = OB_SUCCESS; @@ -2333,6 +2584,11 @@ int ObStaticEngineCG::generate_spec(ObLogSort &op, ObSortVecSpec &spec, const bo ret = OB_ERR_UNEXPECTED; LOG_WARN("topn must be int", K(ret), K(*spec.topn_expr_)); } + if (OB_SUCC(ret) && op.enable_pd_topn_filter()) { + if (OB_FAIL(prepare_topn_runtime_filter_info(op, spec))) { + LOG_WARN("failed to prepare topn runtime_filter info"); + } + } } if (OB_NOT_NULL(op.get_topk_limit_expr())) { OZ(generate_rt_expr(*op.get_topk_limit_expr(), spec.topk_limit_expr_)); @@ -7260,6 +7516,8 @@ int ObStaticEngineCG::generate_spec(ObLogWindowFunction &op, ObWindowFunctionSpe } else { spec.single_part_parallel_ = op.is_single_part_parallel(); spec.range_dist_parallel_ = op.is_range_dist_parallel(); + spec.input_rows_mem_bound_ratio_ = op.get_input_rows_mem_bound_ratio(); + spec.estimated_part_cnt_ = op.get_estimated_part_cnt(); for (int64_t i = 0; OB_SUCC(ret) && i < op.get_window_exprs().count(); ++i) { ObWinFunRawExpr *wf_expr = op.get_window_exprs().at(i); WinFuncInfo &wf_info = spec.wf_infos_.at(i); @@ -7311,6 +7569,57 @@ int ObStaticEngineCG::generate_spec(ObLogWindowFunction &op, ObWindowFunctionSpe return ret; } +int ObStaticEngineCG::generate_spec(ObLogWindowFunction &op, ObWindowFunctionVecSpec &spec, + const bool in_root_job) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(generate_spec(op, static_cast(spec), in_root_job))) { + LOG_WARN("generate window function spec failed", K(ret)); + } else { + NullSafeRowCmpFunc null_first_cmp = nullptr, null_last_cmp = nullptr; + for (int i = 0; OB_SUCC(ret) && i < spec.wf_infos_.count(); i++) { + WinFuncInfo &wf_info = spec.wf_infos_.at(i); + for (int j = 0; OB_SUCC(ret) && j < wf_info.sort_exprs_.count(); j++) { + ObExpr *sort_expr = wf_info.sort_exprs_.at(j); + VectorCmpExprFuncsHelper::get_cmp_set(sort_expr->datum_meta_, sort_expr->datum_meta_, + null_first_cmp, null_last_cmp); + if (OB_ISNULL(null_first_cmp) || OB_ISNULL(null_last_cmp)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null cmp funcs", K(ret), K(null_first_cmp), K(null_last_cmp)); + } else if (wf_info.sort_collations_.at(j).null_pos_ == NULL_FIRST) { + wf_info.sort_cmp_funcs_.at(j).row_cmp_func_ = null_first_cmp; + } else { + wf_info.sort_cmp_funcs_.at(j).row_cmp_func_ = null_last_cmp; + } + } // end inner for + } // end outter for + } + if (OB_FAIL(ret)) { + } else if (op.is_range_dist_parallel()) { + // change sort cmp functions in rd_sort_cmp_funcs_ + for (int i = 0; OB_SUCC(ret) && i < spec.rd_sort_collations_.count(); i++) { + ObExpr *rd_expr = spec.rd_coord_exprs_.at(i); + if (OB_ISNULL(rd_expr) || i >= spec.rd_sort_cmp_funcs_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null rd expr", K(ret), K(i)); + } else { + NullSafeRowCmpFunc null_first_cmp = nullptr, null_last_cmp = nullptr; + VectorCmpExprFuncsHelper::get_cmp_set( + rd_expr->datum_meta_, rd_expr->datum_meta_, null_first_cmp, null_last_cmp); + if (OB_ISNULL(null_first_cmp) || OB_ISNULL(null_last_cmp)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null cmp funcs", K(ret), K(null_first_cmp), K(null_last_cmp)); + } else if (spec.rd_sort_collations_.at(i).null_pos_ == NULL_FIRST) { + spec.rd_sort_cmp_funcs_.at(i).row_cmp_func_ = null_first_cmp; + } else { + spec.rd_sort_cmp_funcs_.at(i).row_cmp_func_ = null_last_cmp; + } + } + } + } + return ret; +} + int ObStaticEngineCG::fill_wf_info(ObIArray &all_expr, ObWinFunRawExpr &win_expr, WinFuncInfo &wf_info, const bool can_push_down) { @@ -8462,7 +8771,7 @@ int ObStaticEngineCG::get_phy_op_type(ObLogicalOperator &log_op, int tmp_ret = OB_SUCCESS; bool use_vec_sort = true; const ObLogSort &sort_op = static_cast(log_op); - if (1 == sort_op.get_sort_keys().count() + if ((1 == sort_op.get_sort_keys().count() && !sort_op.enable_pd_topn_filter()) || (NULL != sort_op.get_topn_expr() && sort_op.get_part_cnt() > 0)) { use_vec_sort = false; } @@ -8585,7 +8894,15 @@ int ObStaticEngineCG::get_phy_op_type(ObLogicalOperator &log_op, case log_op_def::LOG_DISTINCT: { auto &op = static_cast(log_op); if (MERGE_AGGREGATE == op.get_algo()) { - type = PHY_MERGE_DISTINCT; + int tmp_ret = OB_SUCCESS; + tmp_ret = OB_E(EventTable::EN_DISABLE_VEC_MERGE_DISTINCT) OB_SUCCESS; + if (OB_SUCCESS != tmp_ret) { + type = PHY_MERGE_DISTINCT; + } else if (use_rich_format && GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_3_2_0) { + type = PHY_VEC_MERGE_DISTINCT; + } else { + type = PHY_MERGE_DISTINCT; + } } else if (HASH_AGGREGATE == op.get_algo()) { int tmp_ret = OB_SUCCESS; tmp_ret = OB_E(EventTable::EN_DISABLE_VEC_HASH_DISTINCT) OB_SUCCESS; @@ -8660,19 +8977,47 @@ int ObStaticEngineCG::get_phy_op_type(ObLogicalOperator &log_op, } case log_op_def::LOG_SET: { auto &op = static_cast(log_op); + uint64_t min_cluster_version = GET_MIN_CLUSTER_VERSION(); switch (op.get_set_op()) { case ObSelectStmt::UNION: if (op.is_recursive_union()) { type = PHY_RECURSIVE_UNION_ALL; } else { - type = (MERGE_SET == op.get_algo() ? PHY_MERGE_UNION : PHY_HASH_UNION); + if (use_rich_format && (min_cluster_version >= CLUSTER_VERSION_4_3_2_0)) { + bool use_vec = (EVENT_CALL(EventTable::EN_TEST_FOR_HASH_UNION) == OB_SUCCESS); + if (use_vec) { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_UNION : PHY_VEC_HASH_UNION); + } else { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_UNION : PHY_HASH_UNION); + } + } else { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_UNION : PHY_HASH_UNION); + } } break; case ObSelectStmt::INTERSECT: - type = (MERGE_SET == op.get_algo() ? PHY_MERGE_INTERSECT : PHY_HASH_INTERSECT); + if (use_rich_format && (min_cluster_version >= CLUSTER_VERSION_4_3_2_0)) { + bool use_vec = (EVENT_CALL(EventTable::EN_TEST_FOR_HASH_UNION) == OB_SUCCESS); + if (use_vec) { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_INTERSECT : PHY_VEC_HASH_INTERSECT); + } else { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_INTERSECT : PHY_HASH_INTERSECT); + } + } else { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_INTERSECT : PHY_HASH_INTERSECT); + } break; case ObSelectStmt::EXCEPT: - type = (MERGE_SET == op.get_algo() ? PHY_MERGE_EXCEPT : PHY_HASH_EXCEPT); + if (use_rich_format && (min_cluster_version >= CLUSTER_VERSION_4_3_2_0)) { + bool use_vec = (EVENT_CALL(EventTable::EN_TEST_FOR_HASH_UNION) == OB_SUCCESS); + if (use_vec) { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_EXCEPT : PHY_VEC_HASH_EXCEPT); + } else { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_EXCEPT : PHY_HASH_EXCEPT); + } + } else { + type = (MERGE_SET == op.get_algo() ? PHY_MERGE_EXCEPT : PHY_HASH_EXCEPT); + } break; default: break; @@ -8696,7 +9041,16 @@ int ObStaticEngineCG::get_phy_op_type(ObLogicalOperator &log_op, break; } case log_op_def::LOG_WINDOW_FUNCTION: { - type = PHY_WINDOW_FUNCTION; + int tmp_ret = OB_SUCCESS; + tmp_ret = OB_E(EventTable::EN_DISABLE_VEC_WINDOW_FUNCTION) OB_SUCCESS; + bool use_vec2_winfunc = (GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_3_2_0); + if (use_rich_format && use_vec2_winfunc && tmp_ret == OB_SUCCESS + && ObWindowFunctionVecOp::all_supported_winfuncs( + static_cast(&log_op)->get_window_exprs())) { + type = PHY_VEC_WINDOW_FUNCTION; + } else { + type = PHY_WINDOW_FUNCTION; + } break; } case log_op_def::LOG_SELECT_INTO: { diff --git a/src/sql/code_generator/ob_static_engine_cg.h b/src/sql/code_generator/ob_static_engine_cg.h index d2bff3c6d6..608028e911 100644 --- a/src/sql/code_generator/ob_static_engine_cg.h +++ b/src/sql/code_generator/ob_static_engine_cg.h @@ -40,9 +40,13 @@ class ObMergeIntersectSpec; class ObMergeExceptSpec; class ObRecursiveUnionAllSpec; class ObHashSetSpec; +class ObHashSetVecSpec; class ObHashUnionSpec; +class ObHashUnionVecSpec; class ObHashIntersectSpec; +class ObHashIntersectVecSpec; class ObHashExceptSpec; +class ObHashExceptVecSpec; class ObCountSpec; class ObExprValuesSpec; class ObTableMergeSpec; @@ -229,7 +233,8 @@ private: const bool in_root_job, const bool is_subplan, bool &check_eval_once, - const bool need_check_output_datum); + const bool need_check_output_datum, + const common::ObCompressorType compress_type); int clear_all_exprs_specific_flag(const ObIArray &exprs, ObExprInfoFlag flag); int mark_expr_self_produced(ObRawExpr *expr); int mark_expr_self_produced(const ObIArray &exprs); @@ -242,7 +247,8 @@ private: int generate_spec_basic(ObLogicalOperator &op, ObOpSpec &spec, const bool check_eval_once, - const bool need_check_output_datum); + const bool need_check_output_datum, + const common::ObCompressorType compress_type); // Invoked after generate_spec() and generate_spec_basic(), // some operator need this phase to do some special generation. @@ -270,13 +276,18 @@ private: int generate_spec(ObLogLimit &op, ObLimitVecSpec &spec, const bool in_root_job); int generate_spec(ObLogDistinct &op, ObMergeDistinctSpec &spec, const bool in_root_job); + int generate_spec(ObLogDistinct &op, ObMergeDistinctVecSpec &spec, const bool in_root_job); int generate_spec(ObLogDistinct &op, ObHashDistinctSpec &spec, const bool in_root_job); int generate_spec(ObLogDistinct &op, ObHashDistinctVecSpec &spec, const bool in_root_job); int generate_spec(ObLogSet &op, ObHashUnionSpec &spec, const bool in_root_job); + int generate_spec(ObLogSet &op, ObHashUnionVecSpec &spec, const bool in_root_job); int generate_spec(ObLogSet &op, ObHashIntersectSpec &spec, const bool in_root_job); + int generate_spec(ObLogSet &op, ObHashIntersectVecSpec &spec, const bool in_root_job); int generate_spec(ObLogSet &op, ObHashExceptSpec &spec, const bool in_root_job); + int generate_spec(ObLogSet &op, ObHashExceptVecSpec &spec, const bool in_root_job); int generate_hash_set_spec(ObLogSet &op, ObHashSetSpec &spec); + int generate_hash_set_spec(ObLogSet &op, ObHashSetVecSpec &spec); int generate_spec(ObLogSet &op, ObMergeUnionSpec &spec, const bool in_root_job); int generate_spec(ObLogSet &op, ObMergeIntersectSpec &spec, const bool in_root_job); @@ -409,6 +420,8 @@ private: int generate_spec(ObLogWindowFunction &op, ObWindowFunctionSpec &spec, const bool in_root_job); + int generate_spec(ObLogWindowFunction &op, ObWindowFunctionVecSpec &spec, const bool in_root_job); + int generate_spec(ObLogTableScan &op, ObRowSampleScanSpec &spec, const bool in_root_job); int generate_spec(ObLogTableScan &op, ObBlockSampleScanSpec &spec, const bool in_root_job); @@ -454,6 +467,7 @@ private: const ObIArray &sort_exprs); int fill_compress_type(ObLogSort &op, ObCompressorType &compr_type); + int get_query_compress_type(const ObLogPlan &log_plan, ObCompressorType &compress_type); int check_not_support_cmp_type( const ObSortCollations &collations, const ObIArray &sort_exprs); @@ -552,6 +566,10 @@ private: int check_window_functions_order(const ObIArray &winfunc_exprs); int prepare_runtime_filter_cmp_info(ObLogJoinFilter &join_filter_create, ObJoinFilterSpec &spec); + + template + int prepare_topn_runtime_filter_info(ObLogSort &op, ObOpSpec &spec); + int append_child_output_no_dup(const bool is_store_sortkey_separately, const ObIArray &child_output_exprs, ObIArray &sk_exprs, ObIArray &addon_exprs); diff --git a/src/sql/das/ob_das_scan_op.cpp b/src/sql/das/ob_das_scan_op.cpp index b6cf64aed6..29c164aac1 100644 --- a/src/sql/das/ob_das_scan_op.cpp +++ b/src/sql/das/ob_das_scan_op.cpp @@ -1283,10 +1283,11 @@ int ObDASScanResult::init(const ObIDASTaskOp &op, common::ObIAllocator &alloc) if (OB_FAIL(vec_row_store_.init(result_output, max_batch_size, mem_attr, - UINT64_MAX, - false, - 0, - false))) { + UINT64_MAX, /*mem_limit*/ + false, /*enable_dump*/ + 0, /*row_extra_size*/ + NONE_COMPRESSOR, + false /*reorder_fixed_expr*/))) { LOG_WARN("init vec row store failed", K(ret)); } } diff --git a/src/sql/das/ob_das_task_result.cpp b/src/sql/das/ob_das_task_result.cpp index b94dd13c29..6f72ea5da5 100644 --- a/src/sql/das/ob_das_task_result.cpp +++ b/src/sql/das/ob_das_task_result.cpp @@ -333,9 +333,10 @@ int ObDASTaskResultMgr::save_task_result_by_vector(int64_t &read_rows, max_batch_size, mem_attr, 4 * 1024 * 1024, // 4MB - true, - 0, - false))) { + true, /*enable_dump*/ + 0, /*row_extra_size*/ + NONE_COMPRESSOR, + false/*reorder_fixed_expr*/))) { LOG_WARN("init vec row store failed", K(ret)); } else if (OB_FAIL(vec_row_store.alloc_dir_id())) { LOG_WARN("datum store alloc dir id failed", KR(ret)); @@ -602,8 +603,9 @@ int ObDASTaskResultMgr::fetch_result_by_vector(ObDASTCB *tcb, && OB_FAIL(vec_row_store.init(tcb->vec_row_store_.get_row_meta(), tcb->vec_row_store_.get_max_batch_size(), mem_attr, - INT64_MAX, - false))) { + INT64_MAX, /* mem_limit */ + false, /* enable_dump */ + NONE_COMPRESSOR /* compressor_type */))) { LOG_WARN("init datum store failed", KR(ret)); } else { bool added = false; diff --git a/src/sql/dtl/ob_dtl_basic_channel.cpp b/src/sql/dtl/ob_dtl_basic_channel.cpp index b9e92054f5..cba2073914 100644 --- a/src/sql/dtl/ob_dtl_basic_channel.cpp +++ b/src/sql/dtl/ob_dtl_basic_channel.cpp @@ -1376,11 +1376,12 @@ int ObDtlVectorRowMsgWriter::init(ObDtlLinkedBuffer *buffer, uint64_t tenant_id) } else { reset(); ObTempBlockStore::Block *blk = NULL; - if (OB_FAIL(ObTempBlockStore::init_block_buffer(buffer->buf(), buffer->size(), blk))) { + if (OB_FAIL(ObTempBlockStore::init_dtl_block_buffer(buffer->buf(), buffer->size(), blk))) { LOG_WARN("fail to init block buffer", K(ret)); } else { - block_ = static_cast(blk); - block_buffer_ = block_->get_buffer(); + block_ = static_cast(blk); + block_buffer_ = static_cast( + static_cast(reinterpret_cast(blk) + blk->buf_off_)); write_buffer_ = buffer; } } @@ -1398,13 +1399,13 @@ int ObDtlVectorRowMsgWriter::need_new_buffer( const ObPxNewRow &px_row = static_cast(msg); const ObIArray *row = px_row.get_exprs(); if (nullptr == row) { - serialize_need_size = ObTempRowStore::Block::min_blk_size(0); + serialize_need_size = ObTempRowStore::Block::min_blk_size(0); need_size = serialize_need_size; } else { if (OB_FAIL(ObTempRowStore::RowBlock::calc_row_size(*row, row_meta_, *ctx, serialize_need_size))) { LOG_WARN("failed to calc row store size", K(ret)); } - need_size = ObTempRowStore::Block::min_blk_size(serialize_need_size); + need_size = ObTempRowStore::Block::min_blk_size(serialize_need_size); } need_new = nullptr == write_buffer_ || (remain() < serialize_need_size); if(need_new && nullptr != write_buffer_) { diff --git a/src/sql/dtl/ob_dtl_basic_channel.h b/src/sql/dtl/ob_dtl_basic_channel.h index 95fada438a..3c60c38bba 100644 --- a/src/sql/dtl/ob_dtl_basic_channel.h +++ b/src/sql/dtl/ob_dtl_basic_channel.h @@ -107,7 +107,10 @@ static DtlWriterType msg_writer_map[] = VECTOR_WRITER, //PX_VECTOR, VECTOR_FIXED_WRITER, //PX_FIXED_VECTOR VECTOR_ROW_WRITER, //PX_VECTOR_ROW, - + CONTROL_WRITER, // DH_SP_WINFUNC_PX_PIECE_MSG + CONTROL_WRITER, // DH_SP_WINFUNC_PX_WHOLE_MSG + CONTROL_WRITER, // DH_RD_WINFUNC_PX_PIECE_MSG + CONTROL_WRITER, // DH_RD_WINFUNC_PX_WHOLE_MSG }; static_assert(ARRAYSIZEOF(msg_writer_map) == ObDtlMsgType::MAX, "invalid ms_writer_map size"); @@ -372,7 +375,7 @@ OB_INLINE int ObDtlVectorRowMsgWriter::try_append_row(const common::ObIArrayadd_row(exprs, write_buffer_->get_row_meta(), + } else if (OB_FAIL(block_->add_row(*block_buffer_, exprs, write_buffer_->get_row_meta(), ctx, new_row))) { if (OB_BUF_NOT_ENOUGH != ret) { SQL_DTL_LOG(WARN, "failed to add row", K(ret)); @@ -413,7 +416,7 @@ OB_INLINE int ObDtlVectorRowMsgWriter::try_append_batch(const common::ObIArrayadd_batch(vectors, write_buffer_->get_row_meta(), selector, + if (OB_FAIL(block_->add_batch(*block_buffer_, vectors, write_buffer_->get_row_meta(), selector, size, row_size_arr, sum_size, new_rows))) { if (OB_BUF_NOT_ENOUGH != ret) { diff --git a/src/sql/dtl/ob_dtl_msg_type.h b/src/sql/dtl/ob_dtl_msg_type.h index dbd0654e21..52c71cb97f 100644 --- a/src/sql/dtl/ob_dtl_msg_type.h +++ b/src/sql/dtl/ob_dtl_msg_type.h @@ -53,6 +53,10 @@ enum ObDtlMsgType { PX_VECTOR, PX_VECTOR_FIXED, PX_VECTOR_ROW, + DH_SP_WINFUNC_PX_PIECE_MSG, // SP stands for single partition + DH_SP_WINFUNC_PX_WHOLE_MSG, // 45 + DH_RD_WINFUNC_PX_PIECE_MSG, + DH_RD_WINFUNC_PX_WHOLE_MSG, MAX }; diff --git a/src/sql/engine/aggregate/ob_exec_hash_struct_vec.h b/src/sql/engine/aggregate/ob_exec_hash_struct_vec.h index 3f5d99a7a7..2607c072f4 100644 --- a/src/sql/engine/aggregate/ob_exec_hash_struct_vec.h +++ b/src/sql/engine/aggregate/ob_exec_hash_struct_vec.h @@ -743,7 +743,8 @@ public: group_store_.set_io_event_observer(observer); ObMemAttr attr(tenant_id_, ObModIds::OB_HASH_NODE_GROUP_ROWS, ObCtxIds::WORK_AREA); int64_t extra_size = calc_extra_size(agg_row_size); - return group_store_.init(*gby_exprs_, max_batch_size, attr, 0, false, extra_size); + return group_store_.init(*gby_exprs_, max_batch_size, attr, 0/*mem_limit*/, + false/* enable_dump*/, extra_size, NONE_COMPRESSOR); } int get_next_batch(const ObCompactRow **rows, const int64_t max_rows, int64_t &read_rows) @@ -1171,7 +1172,7 @@ public: return ret; } - bool exist(const uint64_t hash_val) const + OB_INLINE bool exist(const uint64_t hash_val) const { return bits_.has_member(h1(hash_val) & (cnt_ - 1)) && bits_.has_member(h2(hash_val) & (cnt_ - 1)); diff --git a/src/sql/engine/aggregate/ob_hash_distinct_vec_op.cpp b/src/sql/engine/aggregate/ob_hash_distinct_vec_op.cpp index c1588b8c26..9ae1827685 100644 --- a/src/sql/engine/aggregate/ob_hash_distinct_vec_op.cpp +++ b/src/sql/engine/aggregate/ob_hash_distinct_vec_op.cpp @@ -161,7 +161,8 @@ int ObHashDistinctVecOp::init_hash_partition_infras() LOG_WARN("failed to init sql mem processor", K(ret)); } else if (OB_FAIL(hp_infras_.init(tenant_id_, enable_sql_dumped_, - true, true, 2, MY_SPEC.max_batch_size_, MY_SPEC.distinct_exprs_, &sql_mem_processor_))) { + true, true, 2, MY_SPEC.max_batch_size_, MY_SPEC.distinct_exprs_, &sql_mem_processor_, + MY_SPEC.compress_type_))) { LOG_WARN("failed to init hash partition infrastructure", K(ret)); } else { hp_infras_.set_io_event_observer(&io_event_observer_); diff --git a/src/sql/engine/aggregate/ob_hash_groupby_vec_op.cpp b/src/sql/engine/aggregate/ob_hash_groupby_vec_op.cpp index 302f64dd36..424a8c4def 100644 --- a/src/sql/engine/aggregate/ob_hash_groupby_vec_op.cpp +++ b/src/sql/engine/aggregate/ob_hash_groupby_vec_op.cpp @@ -260,6 +260,7 @@ int ObHashGroupByVecOp::inner_open() } } } + if (OB_SUCC(ret)) { if (ObThreeStageAggrStage::FIRST_STAGE == MY_SPEC.aggr_stage_) { no_non_distinct_aggr_ = (0 == MY_SPEC.aggr_infos_.count()); @@ -321,6 +322,7 @@ int ObHashGroupByVecOp::inner_close() sql_mem_processor_.unregister_profile(); distinct_sql_mem_processor_.unregister_profile(); group_expr_fixed_lengths_.destroy(); + dump_vectors_.destroy(); curr_group_id_ = common::OB_INVALID_INDEX; return ObGroupByVecOp::inner_close(); } @@ -501,7 +503,8 @@ int ObHashGroupByVecOp::init_distinct_info(bool is_part) LOG_WARN("unexpected status: distinct origin exprs is empty", K(ret)); } else if (OB_FAIL(distinct_data_set_.init( tenant_id, GCONF.is_sql_operator_dump_enabled(), true, true, 1, - MY_SPEC.max_batch_size_, distinct_origin_exprs_, &distinct_sql_mem_processor_))) { + MY_SPEC.max_batch_size_, distinct_origin_exprs_, &distinct_sql_mem_processor_, + MY_SPEC.compress_type_))) { LOG_WARN("failed to init hash partition infrastructure", K(ret)); } else if (FALSE_IT(distinct_data_set_.set_io_event_observer(&io_event_observer_))) { } else if (OB_FAIL(hash_funcs_.init(n_distinct_expr_))) {// TODO: remove hash_funcs_ @@ -816,7 +819,8 @@ int ObHashGroupByVecOp::setup_dump_env(const int64_t part_id, const int64_t inpu attr, 1/* memory limit, dump immediately */, true, - extra_size))) { + extra_size, + MY_SPEC.compress_type_))) { LOG_WARN("init temp row store failed", K(ret)); } else { parts[i]->row_store_.set_dir_id(sql_mem_processor_.get_dir_id()); @@ -835,6 +839,39 @@ int ObHashGroupByVecOp::setup_dump_env(const int64_t part_id, const int64_t inpu } LOG_TRACE("trace setup dump", K(part_cnt), K(pre_part_cnt), K(part_id)); } + + if (OB_SUCC(ret) && part_cnt <= MAX_BATCH_DUMP_PART_CNT && need_reinit_vectors_) { + // only_init_once + dump_vectors_.set_allocator(&mem_context_->get_allocator()); + if (OB_ISNULL(dump_rows_ = static_cast + (mem_context_->get_arena_allocator().alloc(sizeof(ObCompactRow *) * MY_SPEC.max_batch_size_)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to alloc dump_rows_", K(ret), K(MY_SPEC.max_batch_size_)); + } else if (OB_FAIL(dump_vectors_.prepare_allocate(child_->get_spec().output_.count()))) { + LOG_WARN("failed to init dump_vectors prepare_allocate", K(ret), K(child_->get_spec().output_.count())); + } else if (OB_ISNULL(dump_add_row_selectors_ = + static_cast (mem_context_->get_arena_allocator().alloc(sizeof(uint16_t *) * MAX_BATCH_DUMP_PART_CNT)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to init temp_add_row_selectors_", K(ret)); + } else if (OB_ISNULL(dump_add_row_selectors_item_cnt_ = + static_cast (mem_context_->get_arena_allocator().alloc(sizeof(uint16_t) * MAX_BATCH_DUMP_PART_CNT)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to init temp_add_row_selectors_item_cnt_", K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < MAX_BATCH_DUMP_PART_CNT; ++i) { + if (OB_ISNULL(dump_add_row_selectors_[i] = static_cast (mem_context_->get_arena_allocator().alloc(sizeof(uint16_t) + * MY_SPEC.max_batch_size_)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to init temp_add_row_selectors_item_cnt_[index]", K(ret), K(i)); + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < child_->get_spec().output_.count() ; i++) { + ObIVector *col_vec = child_->get_spec().output_.at(i)->get_vector(eval_ctx_); + dump_vectors_.at(i) = col_vec; + } + need_reinit_vectors_ = false; + } + } return ret; } @@ -1585,29 +1622,56 @@ int ObHashGroupByVecOp::batch_process_duplicate_data( } } // need dump - for (int64_t i = 0; OB_SUCC(ret) && i < child_brs.size_; ++i) { - if (child_brs.skip_->exist(i) - || nullptr != batch_new_rows_[i] - || nullptr != batch_old_rows_[i]) { - continue; + if (1 != nth_dup_data) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: dump is not expected", K(ret)); + } else if (part_cnt <= MAX_BATCH_DUMP_PART_CNT) { + reuse_dump_selectors(); + for (int64_t i = 0; OB_SUCC(ret) && i < child_brs.size_; i++) { + if (child_brs.skip_->exist(i) + || nullptr != batch_new_rows_[i] + || nullptr != batch_old_rows_[i]) { + continue; + } + is_dumped_[i] = true; + ++agged_dumped_cnt_; + const int64_t part_idx = (hash_vals_[i] >> part_shift) & (part_cnt - 1); + dump_add_row_selectors_[part_idx][dump_add_row_selectors_item_cnt_[part_idx]++] = i; } - is_dumped_[i] = true; - ++agged_dumped_cnt_; - const int64_t part_idx = (hash_vals_[i] >> part_shift) & (part_cnt - 1); - ObCompactRow *stored_row = nullptr; - if (OB_FAIL(parts[part_idx]->row_store_.add_row(child_->get_spec().output_, - i, - eval_ctx_, - stored_row))) { - LOG_WARN("add row failed", K(ret)); - } else if (1 != nth_dup_data) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected status: dump is not expected", K(ret)); - } else { - *static_cast(stored_row[0].get_extra_payload(parts[part_idx]->row_store_.get_row_meta())) = hash_vals_[i]; + for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt ; i++) { + if (OB_FAIL(parts[i]->row_store_.add_batch(dump_vectors_, dump_add_row_selectors_[i], + dump_add_row_selectors_item_cnt_[i], dump_rows_, nullptr))) { + LOG_WARN("add row batch failed", K(ret)); + } else { + // set dump rows hash_val + for (int64_t j = 0; j < dump_add_row_selectors_item_cnt_[i]; j++) { + *static_cast(dump_rows_[j][0].get_extra_payload(parts[i]->row_store_.get_row_meta())) = + hash_vals_[dump_add_row_selectors_[i][j]]; + } + } + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < child_brs.size_; ++i) { + if (child_brs.skip_->exist(i) + || nullptr != batch_new_rows_[i] + || nullptr != batch_old_rows_[i]) { + continue; + } + is_dumped_[i] = true; + ++agged_dumped_cnt_; + const int64_t part_idx = (hash_vals_[i] >> part_shift) & (part_cnt - 1); + ObCompactRow *stored_row = nullptr; + if (OB_FAIL(parts[part_idx]->row_store_.add_row(child_->get_spec().output_, + i, + eval_ctx_, + stored_row))) { + LOG_WARN("add row failed", K(ret)); + } else { + *static_cast(stored_row[0].get_extra_payload(parts[part_idx]->row_store_.get_row_meta())) = hash_vals_[i]; + } + LOG_DEBUG("finish dump", K(part_idx), K(agged_dumped_cnt_), K(agged_row_cnt_), + K(parts[part_idx]->row_store_.get_row_cnt()), K(i)); } - LOG_DEBUG("finish dump", K(part_idx), K(agged_dumped_cnt_), K(agged_row_cnt_), - K(parts[part_idx]->row_store_.get_row_cnt()), K(i)); } } force_check_dump = false; @@ -1758,6 +1822,12 @@ int ObHashGroupByVecOp::get_next_batch_distinct_rows( return ret; } +void ObHashGroupByVecOp::reuse_dump_selectors() { + if (nullptr != dump_add_row_selectors_item_cnt_) { + memset(dump_add_row_selectors_item_cnt_, 0, sizeof(uint16_t) * MAX_BATCH_DUMP_PART_CNT); + } +} + int ObHashGroupByVecOp::group_child_batch_rows(const ObCompactRow **store_rows, const RowMeta *meta, const int64_t input_rows, @@ -1865,30 +1935,57 @@ int ObHashGroupByVecOp::group_child_batch_rows(const ObCompactRow **store_rows, sql_mem_processor_.set_number_pass(part_id + 1); } } - for (int64_t i = 0; OB_SUCC(ret) && i < child_brs.size_; i++) { - if (child_brs.skip_->exist(i) - || is_dumped_[i] - || nullptr != batch_new_rows_[i] - || nullptr != batch_old_rows_[i]) { - continue; - } - if (OB_SUCC(ret)) { + if (process_check_dump) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: check dump is processed", K(ret)); + } else if (part_cnt <= MAX_BATCH_DUMP_PART_CNT) { + reuse_dump_selectors(); + for (int64_t i = 0; OB_SUCC(ret) && i < child_brs.size_; i++) { + if (child_brs.skip_->exist(i) + || is_dumped_[i] + || nullptr != batch_new_rows_[i] + || nullptr != batch_old_rows_[i]) { + continue; + } ++agged_dumped_cnt_; const int64_t part_idx = (hash_vals_[i] >> part_shift) & (part_cnt - 1); - ObCompactRow *stored_row = nullptr; - if (OB_FAIL(parts[part_idx]->row_store_.add_row(child_->get_spec().output_, - i, - eval_ctx_, - stored_row))) { - LOG_WARN("add row failed", K(ret)); - } else if (process_check_dump) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected status: check dump is processed", K(ret)); + dump_add_row_selectors_[part_idx][dump_add_row_selectors_item_cnt_[part_idx]++] = i; + } + for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt ; i++) { + if (OB_FAIL(parts[i]->row_store_.add_batch(dump_vectors_, dump_add_row_selectors_[i], + dump_add_row_selectors_item_cnt_[i], dump_rows_, nullptr))) { + LOG_WARN("add row batch failed", K(ret)); } else { - *static_cast(stored_row[0].get_extra_payload(parts[part_idx]->row_store_.get_row_meta())) = hash_vals_[i]; + // set dump rows hash_val + for (int64_t j = 0; j < dump_add_row_selectors_item_cnt_[i]; j++) { + *static_cast(dump_rows_[j][0].get_extra_payload(parts[i]->row_store_.get_row_meta())) = + hash_vals_[dump_add_row_selectors_[i][j]]; + } + } + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < child_brs.size_; i++) { + if (child_brs.skip_->exist(i) + || is_dumped_[i] + || nullptr != batch_new_rows_[i] + || nullptr != batch_old_rows_[i]) { + continue; + } + if (OB_SUCC(ret)) { + ++agged_dumped_cnt_; + const int64_t part_idx = (hash_vals_[i] >> part_shift) & (part_cnt - 1); + ObCompactRow *stored_row = nullptr; + if (OB_FAIL(parts[part_idx]->row_store_.add_row(child_->get_spec().output_, + i, + eval_ctx_, + stored_row))) { + LOG_WARN("add row failed", K(ret)); + } else { + *static_cast(stored_row[0].get_extra_payload(parts[part_idx]->row_store_.get_row_meta())) = hash_vals_[i]; + } + LOG_DEBUG("finish dump", K(part_idx), K(agged_dumped_cnt_), K(agged_row_cnt_), + K(parts[part_idx]->row_store_.get_row_cnt()), K(i)); } - LOG_DEBUG("finish dump", K(part_idx), K(agged_dumped_cnt_), K(agged_row_cnt_), - K(parts[part_idx]->row_store_.get_row_cnt()), K(i)); } } } diff --git a/src/sql/engine/aggregate/ob_hash_groupby_vec_op.h b/src/sql/engine/aggregate/ob_hash_groupby_vec_op.h index 1ee6048330..4efdc07bd3 100644 --- a/src/sql/engine/aggregate/ob_hash_groupby_vec_op.h +++ b/src/sql/engine/aggregate/ob_hash_groupby_vec_op.h @@ -109,6 +109,7 @@ public: public: static const int64_t MIN_PARTITION_CNT = 8; static const int64_t MAX_PARTITION_CNT = 256; + static const int64_t MAX_BATCH_DUMP_PART_CNT = 64; static const int64_t INIT_BKT_SIZE_FOR_ADAPTIVE_GBY = 256; // min in memory groups @@ -176,7 +177,12 @@ public: reorder_aggr_rows_(false), old_row_selector_(nullptr), batch_aggr_rows_table_(), - llc_est_() + llc_est_(), + dump_add_row_selectors_(nullptr), + dump_add_row_selectors_item_cnt_(nullptr), + dump_vectors_(nullptr), + dump_rows_(nullptr), + need_reinit_vectors_(true) { } void reset(bool for_rescan); @@ -336,6 +342,7 @@ private: int by_pass_get_next_permutation_batch(int64_t &nth_group, bool &last_group, const ObBatchRows *child_brs, ObBatchRows &my_brs, const int64_t batch_size, bool &insert_group_ht); + void reuse_dump_selectors(); int init_by_pass_op(); int process_multi_groups(aggregate::AggrRowPtr *agg_rows, const ObBatchRows &brs); @@ -419,6 +426,11 @@ private: uint16_t *old_row_selector_; BatchAggrRowsTable batch_aggr_rows_table_; LlcEstimate llc_est_; + uint16_t **dump_add_row_selectors_; + uint16_t *dump_add_row_selectors_item_cnt_; + common::ObFixedArray dump_vectors_; + ObCompactRow **dump_rows_; + bool need_reinit_vectors_; }; } // end namespace sql diff --git a/src/sql/engine/aggregate/ob_merge_distinct_vec_op.cpp b/src/sql/engine/aggregate/ob_merge_distinct_vec_op.cpp new file mode 100644 index 0000000000..6c849c60bd --- /dev/null +++ b/src/sql/engine/aggregate/ob_merge_distinct_vec_op.cpp @@ -0,0 +1,357 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "objit/common/ob_item_type.h" +#include "sql/engine/expr/ob_expr_result_type_util.h" +#include "sql/engine/aggregate/ob_merge_distinct_vec_op.h" +#include "sql/engine/px/ob_px_util.h" +#include "sql/engine/ob_physical_plan.h" +#include "sql/engine/ob_exec_context.h" + +namespace oceanbase +{ +using namespace common; +using namespace share; +namespace sql +{ + +ObMergeDistinctVecSpec::ObMergeDistinctVecSpec(ObIAllocator &alloc, const ObPhyOperatorType type) + : ObDistinctSpec(alloc, type) +{} + +OB_SERIALIZE_MEMBER((ObMergeDistinctVecSpec, ObDistinctSpec)); + +ObMergeDistinctVecOp::ObMergeDistinctVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input) + : ObOperator(exec_ctx, spec, input), + first_got_row_(true), + alloc_(ObModIds::OB_SQL_MERGE_GROUPBY, + OB_MALLOC_NORMAL_BLOCK_SIZE, exec_ctx.get_my_session()->get_effective_tenant_id(), ObCtxIds::WORK_AREA), + last_row_(alloc_) +{ +} + +int ObMergeDistinctVecOp::inner_open() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("child is null", K(ret)); + } else if (MY_SPEC.is_block_mode_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("merge distinct not support block mode", K(ret)); + } else if (OB_FAIL(cmp_.init(&eval_ctx_, &MY_SPEC.cmp_funcs_))) { + LOG_WARN("failed to init cmp", K(ret)); + } else { + last_row_.reuse_ = true; + } + return ret; +} + +int ObMergeDistinctVecOp::inner_rescan() +{ + int ret = OB_SUCCESS; + first_got_row_ = true; + last_row_.reset(); + alloc_.reset(); + if (OB_FAIL(ObOperator::inner_rescan())) { + } + return ret; +} + +int ObMergeDistinctVecOp::inner_get_next_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + const int64_t batch_size = std::min(max_row_cnt, MY_SPEC.max_batch_size_); + clear_evaluated_flag(); + const ObBatchRows *child_brs = nullptr; + bool got_batch = false; + bool has_last = !first_got_row_; + if (OB_FAIL(child_->get_next_batch(batch_size, child_brs))) { + LOG_WARN("failed to get next batch", K(ret)); + } else if (child_brs->end_ && 0 == child_brs->size_) { + ret = OB_ITER_END; + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.distinct_exprs_.count(); ++i) { + if (OB_FAIL(MY_SPEC.distinct_exprs_.at(i)->eval_vector(eval_ctx_, *child_brs))) { + LOG_WARN("failed to eval batch", K(i), K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(deduplicate_for_batch(has_last, child_brs))) { + LOG_WARN("failed to deduplicate batch", K(ret)); + } else { + got_batch = true; + } + } + first_got_row_ = false; + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + brs_.end_ = true; + brs_.size_ = 0; + } + return ret; +} + +template +int ObMergeDistinctVecOp::compare_in_column_with_format(InputVec *vec, const ObBatchRows *child_brs, int64_t first_no_skip_idx, + int64_t col_idx, int64_t &last_idx) { + int ret = OB_SUCCESS; + if (vec->has_null()) { + if (child_brs->all_rows_active_ && col_idx == 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } else if (child_brs->all_rows_active_ && col_idx != 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } else if (!child_brs->all_rows_active_ && col_idx == 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } else if (!child_brs->all_rows_active_ && col_idx != 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } + } else { + if (child_brs->all_rows_active_ && col_idx == 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } else if (child_brs->all_rows_active_ && col_idx != 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } else if (!child_brs->all_rows_active_ && col_idx == 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } else if (!child_brs->all_rows_active_ && col_idx != 0) { + if (OB_FAIL((compare_in_column)(vec, first_no_skip_idx, child_brs, col_idx, last_idx))) { + LOG_WARN("failed to cmp compare in column", K(ret), K(col_idx)); + } + } + } + + return ret; +} + +int ObMergeDistinctVecOp::deduplicate_for_batch(bool has_last, const ObBatchRows *child_brs) +{ + int ret = OB_SUCCESS; + int64_t last_idx = -1; + int64_t curr_idx = 0; + int64_t first_no_skip_idx = -1; + int64_t last_cmp_idx = first_no_skip_idx; + bool equal = false; + + // 1.get first_no_skip + brs_.skip_->set_all(child_brs->size_); // default skip + if (!child_brs->all_rows_active_) { + while (curr_idx < child_brs->size_) { + if (!child_brs->skip_->at(curr_idx)) { + first_no_skip_idx = curr_idx; + break; + } + curr_idx++; + } + } else { + first_no_skip_idx = 0; + } + + brs_.size_ = child_brs->size_; + if (has_last) { + // 2.cmp last and first_no_skip_idx row, let first_no_skip_idx replace the last_row + if (OB_FAIL(cmp_.equal_in_row(&MY_SPEC.distinct_exprs_, &last_row_, first_no_skip_idx, equal))) { + LOG_WARN("failed to cmp row", K(ret)); + } else if (equal) { + // brs_.skip_->set(curr_idx); // no need set, default skip + } else { + brs_.skip_->unset(first_no_skip_idx); + last_idx = first_no_skip_idx; + } + } else { + // no last, out the first_no_skip_idx row + brs_.skip_->unset(first_no_skip_idx); + last_idx = first_no_skip_idx; + } + if (OB_FAIL(ret)) { + } else { + // 3.cmp curr_idx and last_idx col by col + for (int col_idx = 0; col_idx < MY_SPEC.distinct_exprs_.count() && OB_SUCC(ret); col_idx++){ + bool curr_out = false; + int cmp_ret; + ObIVector *vec = MY_SPEC.distinct_exprs_.at(col_idx)->get_vector(eval_ctx_); + last_cmp_idx = first_no_skip_idx; + curr_idx = first_no_skip_idx + 1; + switch (vec->get_format()) { + case VEC_FIXED : { + // 对big_int进行特化 + if (ob_is_integer_type(MY_SPEC.distinct_exprs_.at(col_idx)->datum_meta_.type_)) { + // 测试发现,对于bigint类型,当前列的每一行不判断(是否被上一列的比较输出)直接进行比较性能通常更高(at有位运算导致), + // 首列过滤行较少时,将后续列看作是first_col不进行当前行out的取值性能会更好 + // 而对于第一列就输出大量行,此时仍将后续列视为first_col的话性能会发生较大回退,因此暂时不进行该优化 + ObFixedLengthVector> *fixed_vec = + static_cast> *> (vec); + compare_in_column_with_format(fixed_vec, child_brs, first_no_skip_idx, col_idx, last_idx); + } else { + compare_in_column_with_format(vec, child_brs, first_no_skip_idx, col_idx, last_idx); + } + break; + } + case VEC_DISCRETE : { + if (ob_is_string_tc(MY_SPEC.distinct_exprs_.at(col_idx)->datum_meta_.type_)) { + ObDiscreteVector> *string_vec = static_cast> *> (vec); + compare_in_column_with_format(string_vec, child_brs, first_no_skip_idx, col_idx, last_idx); + } else { + compare_in_column_with_format(vec, child_brs, first_no_skip_idx, col_idx, last_idx); + } + break; + } + default : { + compare_in_column_with_format(vec, child_brs, first_no_skip_idx, col_idx, last_idx); + } + } + if (brs_.skip_->accumulate_bit_cnt(child_brs->size_) <= first_no_skip_idx) { + // 说明brs_.skip_从first_no_skip_idx开始往下全部为false,全部被输出了,不需要再比较下一列 + break; + } + } + } + + if (OB_SUCC(ret)) { + // 4.store the last row + if (last_idx != -1) { + ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx_); + batch_info_guard.set_batch_idx(last_idx); + if (OB_FAIL(last_row_.save_store_row(MY_SPEC.distinct_exprs_, *child_brs, eval_ctx_, 0))) { + LOG_WARN("failed to save last row"); + } + } + } + return ret; +} + +template +int ObMergeDistinctVecOp::compare_in_column(InputVec * vec, int64_t first_no_skip_idx, + const ObBatchRows *child_brs, int64_t col_idx, int64_t &last_idx) { + int ret = OB_SUCCESS; + int null_type = 0; + int cmp_ret = 0; + int64_t last_cmp_idx = first_no_skip_idx; + int64_t curr_idx = first_no_skip_idx + 1; + const sql::ObExpr &col_expr = *MY_SPEC.distinct_exprs_.at(col_idx); + for (; curr_idx < child_brs->size_ && OB_SUCC(ret) ; curr_idx++) { + if (ALL_ROWS_ACTIVE && FIRST_COL) { // skip and out are false, do not continue, need compare + } else if (ALL_ROWS_ACTIVE && !FIRST_COL) { // skip is false, judge out + if (!brs_.skip_->at(curr_idx)) { + // curr_idx row is out, do not need compare, but need update last_cmp_idx + last_cmp_idx = curr_idx; + continue; + } + } else if (!ALL_ROWS_ACTIVE && FIRST_COL) { // out is false, judge skip + if (child_brs->skip_->at(curr_idx)) { continue; } // skip row + } else if (!ALL_ROWS_ACTIVE && !FIRST_COL) { + if (child_brs->skip_->at(curr_idx)) { continue; } + if (!brs_.skip_->at(curr_idx)) { + last_cmp_idx = curr_idx; + continue; + } + } + + // curr_idx row is neither skip nor out, cmp with last_cmp_idx + if (HAS_NULL) { + if (OB_FAIL(vec->null_last_cmp(col_expr, curr_idx, vec->is_null(last_cmp_idx), + vec->get_payload(last_cmp_idx), vec->get_length(last_cmp_idx), cmp_ret))) { + LOG_WARN("null_last_cmp failed", K(curr_idx), K(last_cmp_idx), K(cmp_ret), K(ret)); + } + } else { + if (OB_FAIL(vec->no_null_cmp(col_expr, curr_idx, last_cmp_idx, cmp_ret))) { + LOG_WARN("no_null_cmp failed", K(curr_idx), K(last_cmp_idx), K(cmp_ret), K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (cmp_ret == 0) { // equal, do nothing + } else { + brs_.skip_->unset(curr_idx); + if (last_idx < curr_idx) { + last_idx = curr_idx; + } + } + last_cmp_idx = curr_idx; + } + return ret; +} + +void ObMergeDistinctVecOp::destroy() +{ + last_row_.reset(); + alloc_.reset(); + ObOperator::destroy(); +} + +int ObMergeDistinctVecOp::inner_close() +{ + int ret = OB_SUCCESS; + last_row_.reset(); + alloc_.reset(); + if (OB_FAIL(ObOperator::inner_close())) { + } + return ret; +} + +int ObMergeDistinctVecOp::Compare::init(ObEvalCtx *eval_ctx, const ObIArray *cmp_funcs) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(eval_ctx) || OB_ISNULL(cmp_funcs)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to init compare funcs", K(ret)); + } else { + eval_ctx_ = eval_ctx; + cmp_funcs_ = cmp_funcs; + ret_code_ = OB_SUCCESS; + } + return ret; +} + +int ObMergeDistinctVecOp::Compare::equal_in_row(const common::ObIArray *set_exprs, + const LastCompactRow *r, + const int64_t curr_idx, + bool &equal) +{ + int ret = OB_SUCCESS; + equal = false; + if (0 == set_exprs->count()) { + // 表示是distinct 常量,所以没有distinct列,则永远相等 + // case: select distinct 1 from t1; + equal = true; + } else { + int cmp = 0; + const char *r_v = NULL; + ObLength r_len = 0; + for (int64_t i = 0; OB_SUCC(ret) && 0 == cmp && i < set_exprs->count(); i++) { + ObIVector *vec = set_exprs->at(i)->get_vector(*eval_ctx_); + r->compact_row_->get_cell_payload(r->row_meta_, i, r_v, r_len); + if (OB_FAIL(vec->null_last_cmp(*set_exprs->at(i), curr_idx, r->compact_row_->is_null(i), + r_v, r_len, cmp))) { + LOG_WARN("failed to cmp left and right", K(ret)); + } + } + equal = (0 == cmp); + } + return ret; +} +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/aggregate/ob_merge_distinct_vec_op.h b/src/sql/engine/aggregate/ob_merge_distinct_vec_op.h new file mode 100644 index 0000000000..6b54d7501c --- /dev/null +++ b/src/sql/engine/aggregate/ob_merge_distinct_vec_op.h @@ -0,0 +1,82 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#ifndef OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_MERGE_DISTINCT_VEC_OP_H_ +#define OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_MERGE_DISTINCT_VEC_OP_H_ + +#include "sql/engine/ob_operator.h" +#include "sql/engine/aggregate/ob_distinct_op.h" +#include "sql/engine/basic/ob_chunk_datum_store.h" + + +namespace oceanbase +{ +namespace sql +{ +class ObCompactRow; +class RowMeta; + +class ObMergeDistinctVecSpec : public ObDistinctSpec +{ + OB_UNIS_VERSION_V(1); +public: + ObMergeDistinctVecSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type); +}; + +class ObMergeDistinctVecOp : public ObOperator +{ +public: + ObMergeDistinctVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input); + + virtual int inner_open() override; + virtual int inner_close() override; + virtual int inner_rescan() override; + virtual int inner_get_next_row() override { return common::OB_NOT_IMPLEMENT; } + virtual int inner_get_next_batch(const int64_t max_row_cnt); + virtual void destroy() override; + + template + int compare_in_column(InputVec * vec, int64_t first_no_skip_idx, + const ObBatchRows *child_brs, int64_t col_idx, int64_t &last_idx); + + template + int compare_in_column_with_format(InputVec *vec, const ObBatchRows *child_brs, int64_t first_no_skip_idx, + int64_t col_idx, int64_t &last_idx); + + typedef ObFixedLengthVector> FixedLengthVectorBigInt; + typedef ObDiscreteVector> DiscreteVectorString; + + class Compare + { + public: + Compare() : eval_ctx_(nullptr), cmp_funcs_(nullptr), ret_code_(common::OB_SUCCESS) + {} + + int init(ObEvalCtx *eval_ctx, const common::ObIArray *cmp_funcs); + int equal_in_row(const common::ObIArray *set_exprs, + const sql::LastCompactRow *r, + const int64_t curr_idx, + bool &equal); + ObEvalCtx *eval_ctx_; + const common::ObIArray *cmp_funcs_; + int ret_code_; + }; + int deduplicate_for_batch(bool has_last, const ObBatchRows *child_brs); + bool first_got_row_; // 是否第一次拿数据 + common::ObArenaAllocator alloc_; + LastCompactRow last_row_; + ObBitVector *out_; + Compare cmp_; +}; + +} // end sql +} // end oceanabse +#endif // OCEANBASE_SRC_SQL_ENGINE_AGGREGATE_OB_MERGE_DISTINCT_VEC_OP_H_ \ No newline at end of file diff --git a/src/sql/engine/basic/chunk_store/ob_block_iwriter.h b/src/sql/engine/basic/chunk_store/ob_block_iwriter.h index fbf208e942..e7bf54b740 100644 --- a/src/sql/engine/basic/chunk_store/ob_block_iwriter.h +++ b/src/sql/engine/basic/chunk_store/ob_block_iwriter.h @@ -64,7 +64,7 @@ protected: { int64_t size = 0; if (OB_NOT_NULL(cur_blk_)) { - size = cur_blk_->get_buffer()->head_size(); + size = store_->get_blk_buf().head_size(); } return size; } @@ -72,7 +72,7 @@ protected: { int64_t remain_size = 0; if (OB_NOT_NULL(cur_blk_)) { - remain_size = cur_blk_->get_buffer()->remain(); + remain_size = store_->get_blk_buf().remain(); } return remain_size; } @@ -81,7 +81,7 @@ protected: { char *res = nullptr; if (OB_NOT_NULL(cur_blk_)) { - res = cur_blk_->get_buffer()->head(); + res = store_->get_blk_buf().head(); } return res; } @@ -89,10 +89,10 @@ protected: { int ret = OB_SUCCESS; if (OB_NOT_NULL(cur_blk_)) { - last_row_pos_ = cur_blk_->get_buffer()->head(); + last_row_pos_ = store_->get_blk_buf().head(); cur_blk_->cnt_ += 1; store_->inc_block_id_cnt(1); - ret = cur_blk_->get_buffer()->advance(size); + ret = store_->get_blk_buf().advance(size); } return ret; } diff --git a/src/sql/engine/basic/ob_compact_row.h b/src/sql/engine/basic/ob_compact_row.h index d20e744cc2..bb76b92de5 100644 --- a/src/sql/engine/basic/ob_compact_row.h +++ b/src/sql/engine/basic/ob_compact_row.h @@ -397,7 +397,8 @@ public: int save_store_row(const common::ObIArray &exprs, const ObBatchRows &brs, ObEvalCtx &ctx, - const int32_t extra_size = 0) + const int32_t extra_size = 0, + const bool reorder_fixed_expr = true) { int ret = OB_SUCCESS; int64_t row_size = 0; @@ -405,7 +406,7 @@ public: } else if (OB_UNLIKELY(extra_size < 0 || extra_size > INT32_MAX)) { ret = OB_INVALID_ARGUMENT; SQL_ENG_LOG(ERROR, "invalid extra size", K(ret), K(extra_size)); - } else if (OB_FAIL(init_row_meta(exprs, extra_size))) { + } else if (OB_FAIL(init_row_meta(exprs, extra_size, reorder_fixed_expr))) { SQL_ENG_LOG(WARN, "fail to init row meta", K(ret)); } else if (OB_FAIL(ObCompactRow::calc_row_size(row_meta_, exprs, brs, ctx, row_size))) { SQL_ENG_LOG(WARN, "failed to calc copy size", K(ret)); @@ -447,7 +448,11 @@ public: int ret = OB_SUCCESS; bool reuse = reuse_; const int64_t row_size = row.get_row_size(); - if (OB_FAIL(ensure_compact_row_buffer(row_size))) { + if (is_first_row_) { + // row meta is not inited + ret = OB_NOT_INIT; + SQL_ENG_LOG(WARN, "row meta not inited", K(ret)); + } else if (OB_FAIL(ensure_compact_row_buffer(row_size))) { SQL_ENG_LOG(WARN, "fail to ensure compact row buffer", K(ret)); } else { MEMCPY(compact_row_, reinterpret_cast(&row), row_size); @@ -472,17 +477,18 @@ public: } ObDatum get_datum(const int64_t col_idx) const { return compact_row_->get_datum(row_meta_, col_idx); } -private: - inline int init_row_meta(const common::ObIArray &exprs, const int32_t extra_size) + inline int init_row_meta(const common::ObIArray &exprs, const int32_t extra_size, + const bool reorder_fixed_expr) { int ret = OB_SUCCESS; if (is_first_row_) { - ret = row_meta_.init(exprs, extra_size); + ret = row_meta_.init(exprs, extra_size, reorder_fixed_expr); is_first_row_ = false; } return ret; } +private: inline int ensure_compact_row_buffer(const int64_t row_size) { int ret = OB_SUCCESS; bool reuse = reuse_; diff --git a/src/sql/engine/basic/ob_hp_infras_vec_op.cpp b/src/sql/engine/basic/ob_hp_infras_vec_op.cpp index 3ce2d9ccf0..82b3fe171a 100644 --- a/src/sql/engine/basic/ob_hp_infras_vec_op.cpp +++ b/src/sql/engine/basic/ob_hp_infras_vec_op.cpp @@ -27,7 +27,7 @@ ObIHashPartInfrastructure::~ObIHashPartInfrastructure() int ObIHashPartInfrastructure::init( uint64_t tenant_id, bool enable_sql_dumped, bool unique, bool need_pre_part, int64_t ways, int64_t max_batch_size, const common::ObIArray &exprs, - ObSqlMemMgrProcessor *sql_mem_processor) + ObSqlMemMgrProcessor *sql_mem_processor, const common::ObCompressorType compressor_type) { int ret = OB_SUCCESS; void *buf = nullptr; @@ -53,6 +53,7 @@ int ObIHashPartInfrastructure::init( arena_alloc_->set_label("HashPartInfra"); alloc_ = &mem_context_->get_malloc_allocator(); sql_mem_processor_ = sql_mem_processor; + compressor_type_ = compressor_type; init_part_func_ = &ObIHashPartInfrastructure::init_default_part; insert_row_func_ = &ObIHashPartInfrastructure::direct_insert_row; part_shift_ = sizeof(uint64_t) * CHAR_BIT / 2; @@ -206,8 +207,8 @@ int ObIHashPartInfrastructure::init_default_part( part->part_key_.level_ = cur_level_ + 1; part->part_key_.nth_part_ = nth_part; ObMemAttr attr(tenant_id_, "HashPartInfra", ObCtxIds::WORK_AREA); - if (OB_FAIL(part->store_.init(*exprs_, max_batch_size_, attr, limit, true, - ObHashPartItem::get_extra_size()))) { + if (OB_FAIL(part->store_.init(*exprs_, max_batch_size_, attr, limit, true, /*enable_dump*/ + ObHashPartItem::get_extra_size(), compressor_type_))) { SQL_ENG_LOG(WARN, "failed to init row store", K(ret)); } else if (OB_ISNULL(sql_mem_processor_)) { ret = OB_ERR_UNEXPECTED; @@ -457,6 +458,7 @@ insert_batch_on_partitions(const common::ObIArray &exprs, } else { ObHashPartItem *store_row = static_cast(srow); store_row->set_hash_value(hash_values[idx], cur_dumped_parts_[part_idx]->store_.get_row_meta()); + store_row->set_is_match(cur_dumped_parts_[part_idx]->store_.get_row_meta(), false); } return ret; } @@ -1245,7 +1247,7 @@ int64_t ObHashPartInfrastructureVecImpl::get_bucket_size() const int ObHashPartInfrastructureVecImpl::init(uint64_t tenant_id, bool enable_sql_dumped, bool unique, bool need_pre_part, int64_t ways, int64_t max_batch_size, const common::ObIArray &exprs, - ObSqlMemMgrProcessor *sql_mem_processor) + ObSqlMemMgrProcessor *sql_mem_processor, const common::ObCompressorType compressor_type) { int ret = OB_SUCCESS; if (is_inited_) { @@ -1256,7 +1258,7 @@ int ObHashPartInfrastructureVecImpl::init(uint64_t tenant_id, } else if (OB_FAIL(init_hp_infras(tenant_id, exprs, hp_infras_))) { LOG_WARN("failed to init hash part infras instance", K(ret)); } else if (OB_FAIL(hp_infras_->init(tenant_id, enable_sql_dumped, unique, - need_pre_part, ways, max_batch_size, exprs, sql_mem_processor))) { + need_pre_part, ways, max_batch_size, exprs, sql_mem_processor, compressor_type))) { LOG_WARN("failed to init hash part infras", K(ret)); } else { is_inited_ = true; @@ -1329,6 +1331,54 @@ int ObHashPartInfrastructureVecImpl::end_round() return ret; } +void ObHashPartInfrastructureVecImpl::switch_left() +{ + hp_infras_->switch_left(); +} + +bool ObHashPartInfrastructureVecImpl::has_cur_part(InputSide input_side) +{ + return hp_infras_->has_cur_part(input_side); +} + +int ObHashPartInfrastructureVecImpl::get_right_next_batch( + const common::ObIArray &exprs, + const int64_t max_row_cnt, + int64_t &read_rows) +{ + HP_INFRAS_STATUS_CHECK + { + if (OB_FAIL(hp_infras_->get_right_next_batch(exprs, max_row_cnt, read_rows))) { + LOG_WARN("failed to end round", K(ret)); + } + } + return ret; +} + +int ObHashPartInfrastructureVecImpl::exists_batch( + const common::ObIArray &exprs, + const ObBatchRows &brs, ObBitVector *skip, + uint64_t *hash_values_for_batch) +{ + HP_INFRAS_STATUS_CHECK + { + if (OB_FAIL(hp_infras_->exists_batch(exprs, brs, skip, hash_values_for_batch))) { + LOG_WARN("failed to end round", K(ret)); + } + } + return ret; +} + +const RowMeta &ObHashPartInfrastructureVecImpl::get_hash_store_row_meta() const +{ + return hp_infras_->get_hash_store_row_meta(); +} + +void ObHashPartInfrastructureVecImpl::switch_right() +{ + hp_infras_->switch_right(); +} + int ObHashPartInfrastructureVecImpl::init_hash_table(int64_t bucket_cnt, int64_t min_bucket, int64_t max_bucket) diff --git a/src/sql/engine/basic/ob_hp_infras_vec_op.h b/src/sql/engine/basic/ob_hp_infras_vec_op.h index 3c8a399e82..1c8de588ce 100644 --- a/src/sql/engine/basic/ob_hp_infras_vec_op.h +++ b/src/sql/engine/basic/ob_hp_infras_vec_op.h @@ -36,6 +36,15 @@ namespace sql class ObHashPartItem : public ObCompactRow { public: + // used for hash set. + const static int64_t HASH_VAL_BIT = 63; + const static int64_t HASH_VAL_MASK = UINT64_MAX >> (64 - HASH_VAL_BIT); + struct ExtraInfo + { + uint64_t hash_val_:HASH_VAL_BIT; + uint64_t is_match_:1; + }; + // end. ObHashPartItem() : ObCompactRow() {} ~ObHashPartItem() {} ObHashPartItem *next(const RowMeta &row_meta) @@ -54,9 +63,20 @@ public: } void set_hash_value(const uint64_t hash_val, const RowMeta &row_meta) { - *reinterpret_cast(this->get_extra_payload(row_meta)) = hash_val; + *reinterpret_cast(this->get_extra_payload(row_meta)) = hash_val & HASH_VAL_MASK; } static int64_t get_extra_size() { return sizeof(uint64_t) + sizeof(ObHashPartItem *); } + ExtraInfo &get_extra_info(const RowMeta &row_meta) + { + static_assert(sizeof(ObHashPartItem) == sizeof(ObCompactRow), + "sizeof ObHashPartItem must be the save with ObCompactRow"); + return *reinterpret_cast(get_extra_payload(row_meta)); + } + const ExtraInfo &get_extra_info(const RowMeta &row_meta) const + { return *reinterpret_cast(get_extra_payload(row_meta)); } + + bool is_match(const RowMeta &row_meta) const { return get_extra_info(row_meta).is_match_; } + void set_is_match(const RowMeta &row_meta, bool is_match) { get_extra_info(row_meta).is_match_ = is_match; } }; template @@ -131,7 +151,8 @@ public: has_cur_part_dumped_(false), has_create_part_map_(false), est_part_cnt_(INT64_MAX), cur_level_(0), part_shift_(0), period_row_cnt_(0), left_part_cur_id_(0), right_part_cur_id_(0), my_skip_(nullptr), - is_push_down_(false), exprs_(nullptr), is_inited_vec_(false), max_batch_size_(0) + is_push_down_(false), exprs_(nullptr), is_inited_vec_(false), max_batch_size_(0), + compressor_type_(NONE_COMPRESSOR) {} virtual ~ObIHashPartInfrastructure(); public: @@ -223,9 +244,11 @@ public: virtual int init_hash_table(int64_t bucket_cnt, int64_t min_bucket = MIN_BUCKET_NUM, int64_t max_bucket = MAX_BUCKET_NUM) = 0; + virtual int exists_batch(const common::ObIArray &exprs, const ObBatchRows &brs, ObBitVector *skip, + uint64_t *hash_values_for_batch) = 0; int init(uint64_t tenant_id, bool enable_sql_dumped, bool unique, bool need_pre_part, int64_t ways, int64_t max_batch_size, const common::ObIArray &exprs, - ObSqlMemMgrProcessor *sql_mem_processor); + ObSqlMemMgrProcessor *sql_mem_processor, const common::ObCompressorType compressor_type); void switch_left() { cur_side_ = InputSide::LEFT; } void switch_right() @@ -384,6 +407,7 @@ public: return ret; } int64_t get_hash_store_mem_used() const { return preprocess_part_.store_.get_mem_used(); } + const RowMeta &get_hash_store_row_meta() const { return preprocess_part_.store_.get_row_meta(); } void set_push_down() { is_push_down_ = true; } int process_dump(bool is_block, bool &full_by_pass); bool hash_table_full() { return get_hash_table_size() >= 0.8 * get_hash_bucket_num(); } @@ -429,6 +453,14 @@ protected: if (INT64_MAX == est_part_cnt_) { est_partition_count(); } + + // test for dump once + bool force_dump = !(EVENT_CALL(EventTable::EN_SQL_FORCE_DUMP) == OB_SUCCESS); + if (force_dump && is_test_for_dump_ == true) { + is_test_for_dump_ = false; + return true; + } + return (sql_mem_processor_->get_mem_bound() <= est_part_cnt_ * BLOCK_SIZE + get_mem_used()); } @@ -502,6 +534,8 @@ protected: bool is_inited_vec_; common::ObFixedArray vector_ptrs_; int64_t max_batch_size_; + common::ObCompressorType compressor_type_; + bool is_test_for_dump_ = true; // tracepoint for dump once. }; template @@ -534,6 +568,9 @@ public: int init_hash_table(int64_t bucket_cnt, int64_t min_bucket = MIN_BUCKET_NUM, int64_t max_bucket = MAX_BUCKET_NUM) override; + int exists_batch(const common::ObIArray &exprs, + const ObBatchRows &brs, ObBitVector *skip, + uint64_t *hash_values_for_batch) override; private: int set_distinct_batch(const common::ObIArray &exprs, uint64_t *hash_values_for_batch, @@ -633,7 +670,7 @@ public: bool is_inited() const { return is_inited_; } int init(uint64_t tenant_id, bool enable_sql_dumped, bool unique, bool need_pre_part, int64_t ways, int64_t max_batch_size, const common::ObIArray &exprs, - ObSqlMemMgrProcessor *sql_mem_processor); + ObSqlMemMgrProcessor *sql_mem_processor, const common::ObCompressorType compressor_type); int init_mem_context(uint64_t tenant_id); int decide_hp_infras_type(const common::ObIArray &exprs, BucketType &bkt_type, uint64_t &payload_len); template @@ -653,6 +690,16 @@ public: ObEvalCtx *eval_ctx); int start_round(); int end_round(); + void switch_left(); + void switch_right(); + bool has_cur_part(InputSide input_side); + int get_right_next_batch(const common::ObIArray &exprs, + const int64_t max_row_cnt, + int64_t &read_rows); + int exists_batch(const common::ObIArray &exprs, + const ObBatchRows &brs, ObBitVector *skip, + uint64_t *hash_values_for_batch); + const RowMeta &get_hash_store_row_meta() const; int init_hash_table(int64_t bucket_cnt, int64_t min_bucket = MIN_BUCKET_NUM, int64_t max_bucket = MAX_BUCKET_NUM); @@ -826,6 +873,69 @@ set_distinct_batch(const common::ObIArray &exprs, return ret; } +template +int ObHashPartInfrastructureVec::exists_batch( + const common::ObIArray &exprs, + const ObBatchRows &brs, ObBitVector *skip, + uint64_t *hash_values_for_batch) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(hash_values_for_batch)) { + ret = OB_ERR_UNEXPECTED; + SQL_ENG_LOG(WARN, "hash values vector is not init", K(ret)); + } else if (OB_ISNULL(skip)) { + ret = OB_ERR_UNEXPECTED; + SQL_ENG_LOG(WARN, "skip vector is null", K(ret)); + } else if (OB_FAIL(calc_hash_value_for_batch(exprs, brs, hash_values_for_batch))) { + SQL_ENG_LOG(WARN, "failed to calc hash values", K(ret)); + } else { + const ObHashPartItem *exists_item = nullptr; + ObBitVector &skip_for_dump = *my_skip_; + skip_for_dump.reset(brs.size_); + if (OB_FAIL(prefetch(hash_values_for_batch, brs.size_, brs.skip_))) { + SQL_ENG_LOG(WARN, "failed to prefetch", K(ret)); + } else { + ObEvalCtx::BatchInfoScopeGuard guard(*eval_ctx_); + for (int64_t i = 0; OB_SUCC(ret) && i < brs.size_; ++i) { + //if nullptr, data is from dumped partition + if (OB_NOT_NULL(brs.skip_) && brs.skip_->at(i)) { + skip->set(i); //skip indicates rows need to return, only useful for intersect + skip_for_dump.set(i); //skip_for_dump indicates rows need to dump + continue; + } + guard.set_batch_idx(i); + if (OB_FAIL(hash_table_.get(preprocess_part_.store_.get_row_meta(), + i, hash_values_for_batch[i], + exists_item))) { + SQL_ENG_LOG(WARN, "failed to get item", K(ret)); + } else if (OB_ISNULL(exists_item)) { + skip->set(i); + } else if (exists_item->is_match(preprocess_part_.store_.get_row_meta())) { + skip->set(i); + //we dont need dumped this row + skip_for_dump.set(i); + } else { + const_cast(exists_item)->set_is_match(preprocess_part_.store_.get_row_meta(), true); + //we dont need dumped this row + skip_for_dump.set(i); + } + } + } + if (OB_SUCC(ret) && has_left_dumped()) { + // dump right row if left is dumped + if (!has_right_dumped() + && OB_FAIL(create_dumped_partitions(InputSide::RIGHT))) { + SQL_ENG_LOG(WARN, "failed to create dump partitions", K(ret)); + } else if (OB_FAIL(insert_batch_on_partitions(exprs, skip_for_dump, + brs.size_, hash_values_for_batch))) { + SQL_ENG_LOG(WARN, "failed to insert row into partitions", K(ret)); + } + } + my_skip_->reset(brs.size_); + } + return ret; +} + template int ObHashPartInfrastructureVec:: probe_batch(uint64_t *hash_values_for_batch, diff --git a/src/sql/engine/basic/ob_material_vec_op.cpp b/src/sql/engine/basic/ob_material_vec_op.cpp index 95cff30473..a503ffb3aa 100644 --- a/src/sql/engine/basic/ob_material_vec_op.cpp +++ b/src/sql/engine/basic/ob_material_vec_op.cpp @@ -53,7 +53,8 @@ int ObMaterialVecOp::inner_open() mem_attr, 0 /*mem_limit*/, true /*enable_dump*/, - true /*reuse_vector_array*/))) { + true /*reuse_vector_array*/, + MY_SPEC.compress_type_))) { LOG_WARN("init row store failed"); } else { const int64_t size = OB_INVALID_ID == row_count ? 0 : row_count * MY_SPEC.width_; diff --git a/src/sql/engine/basic/ob_pushdown_filter.cpp b/src/sql/engine/basic/ob_pushdown_filter.cpp index bbce63d313..bb411d74ee 100644 --- a/src/sql/engine/basic/ob_pushdown_filter.cpp +++ b/src/sql/engine/basic/ob_pushdown_filter.cpp @@ -23,6 +23,7 @@ #include "storage/blocksstable/ob_micro_block_row_scanner.h" #include "storage/column_store/ob_column_store_util.h" #include "storage/lob/ob_lob_manager.h" +#include "sql/engine/expr/ob_expr_topn_filter.h" namespace oceanbase { @@ -54,6 +55,13 @@ ObPushdownFilterFactory::FilterExecutorAllocFunc ObPushdownFilterFactory::FILTER ObDynamicFilterExecutor::PreparePushdownDataFunc ObDynamicFilterExecutor::PREPARE_PD_DATA_FUNCS [DynamicFilterType::MAX_DYNAMIC_FILTER_TYPE] = { ObExprJoinFilter::prepare_storage_white_filter_data, + ObExprTopNFilter::prepare_storage_white_filter_data, +}; + +ObDynamicFilterExecutor::UpdatePushdownDataFunc ObDynamicFilterExecutor::UPDATE_PD_DATA_FUNCS + [DynamicFilterType::MAX_DYNAMIC_FILTER_TYPE] = { + nullptr, + ObExprTopNFilter::update_storage_white_filter_data, }; OB_SERIALIZE_MEMBER(ObPushdownFilterNode, type_, n_child_, col_ids_); @@ -223,19 +231,25 @@ int ObPushdownWhiteFilterNode::set_op_type(const ObRawExpr &raw_expr) int ObPushdownDynamicFilterNode::set_op_type(const ObRawExpr &raw_expr) { int ret = OB_SUCCESS; - const RuntimeFilterType type = raw_expr.get_runtime_filter_type(); - switch (type) { - case RANGE: - op_type_ = WHITE_OP_BT; - dynamic_filter_type_ = JOIN_RUNTIME_FILTER; - break; - case IN: - op_type_ = WHITE_OP_IN; - dynamic_filter_type_ = JOIN_RUNTIME_FILTER; - break; - default: - ret = OB_ERR_UNEXPECTED; - break; + if (T_OP_RUNTIME_FILTER == raw_expr.get_expr_type()) { + const RuntimeFilterType type = raw_expr.get_runtime_filter_type(); + switch (type) { + case RANGE: + op_type_ = WHITE_OP_BT; + dynamic_filter_type_ = JOIN_RUNTIME_FILTER; + break; + case IN: + op_type_ = WHITE_OP_IN; + dynamic_filter_type_ = JOIN_RUNTIME_FILTER; + break; + default: + ret = OB_ERR_UNEXPECTED; + break; + } + } else if (T_OP_PUSHDOWN_TOPN_FILTER == raw_expr.get_expr_type()) { + // for topn pushdown filter, we can not sure whether is ascding or not + // so we set the real optype in ObExprTopNFilter::prepare_storage_white_filter_data + dynamic_filter_type_ = PD_TOPN_FILTER; } return ret; } @@ -714,7 +728,10 @@ int ObPushdownFilterConstructor::generate(ObRawExpr *raw_expr, ObPushdownFilterN if (OB_ISNULL(raw_expr) || OB_ISNULL(alloc_)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Invalid null parameter", K(ret), KP(raw_expr), KP(alloc_)); - } else if (use_column_store_ && raw_expr->is_white_runtime_filter_expr()) { + // join runtime filter only in column store can be pushdown as white filter + // topn runtime filter can be pushdown as white filter both in row store and column store + } else if ((use_column_store_ || T_OP_PUSHDOWN_TOPN_FILTER == raw_expr->get_expr_type()) + && raw_expr->is_white_runtime_filter_expr()) { // only in column store, the runtime filter can be pushdown as white filter ObOpRawExpr *op_raw_expr = static_cast(raw_expr); if (op_raw_expr->get_children_count() > 1) { @@ -1469,8 +1486,9 @@ int ObPushdownFilterExecutor::do_filter( int ret = OB_SUCCESS; bool is_needed_to_do_filter = check_sstable_index_filter(); if (!is_needed_to_do_filter) { - } else if (is_filter_dynamic_node() && - OB_FAIL(static_cast(this)->check_runtime_filter(is_needed_to_do_filter))) { + } else if (is_filter_dynamic_node() + && OB_FAIL(static_cast(this)->check_runtime_filter( + parent, is_needed_to_do_filter))) { LOG_WARN("Failed to check runtime filter", K(ret), KPC(this)); } else if (is_filter_dynamic_node() && !is_needed_to_do_filter) { ObDynamicFilterExecutor *dynamic_filter = static_cast(this); @@ -2298,65 +2316,35 @@ int ObBlackFilterExecutor::filter_batch( return ret; } -void ObDynamicFilterExecutor::update_rf_slide_window() -{ - int ret = OB_SUCCESS; - sql::ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = - static_cast(join_filter_ctx_); - if (OB_NOT_NULL(join_filter_ctx)) { - if (join_filter_ctx->cur_pos_ >= join_filter_ctx->next_check_start_pos_ - && join_filter_ctx->need_reset_sample_info_) { - join_filter_ctx->partial_total_count_ = 0; - join_filter_ctx->partial_filter_count_ = 0; - join_filter_ctx->need_reset_sample_info_ = false; - if (join_filter_ctx->dynamic_disable()) { - join_filter_ctx->dynamic_disable_ = false; - } - } else if (join_filter_ctx->cur_pos_ >= - join_filter_ctx->next_check_start_pos_ + join_filter_ctx->window_size_) { - if (join_filter_ctx->partial_total_count_ - - join_filter_ctx->partial_filter_count_ < - join_filter_ctx->partial_filter_count_) { - // partial_filter_count_ / partial_total_count_ > 0.5 - // The optimizer choose the bloom filter when the filter threshold is larger than 0.6 - // 0.5 is a acceptable value - // if enabled, the slide window not needs to expand - join_filter_ctx->window_cnt_ = 0; - join_filter_ctx->next_check_start_pos_ = join_filter_ctx->cur_pos_; - } else { - // if enabled, the slide window needs to expand - join_filter_ctx->window_cnt_++; - join_filter_ctx->next_check_start_pos_ = join_filter_ctx->cur_pos_ + - (join_filter_ctx->window_size_ * join_filter_ctx->window_cnt_); - join_filter_ctx->dynamic_disable_ = true; - } - join_filter_ctx->partial_total_count_ = 0; - join_filter_ctx->partial_filter_count_ = 0; - join_filter_ctx->need_reset_sample_info_ = true; - } - } -} void ObDynamicFilterExecutor::filter_on_bypass(ObPushdownFilterExecutor* parent_filter) { ObPushdownDynamicFilterNode &dynamic_filter_node = static_cast(filter_); + int64_t total_rows_count = 0; + int64_t filter_count = 0; + int64_t check_count = 0; + if (!dynamic_filter_node.is_first_child()) { } else { - int64_t total_rows_count = filter_bitmap_->size(); + total_rows_count = filter_bitmap_->size(); if (parent_filter) { total_rows_count = parent_filter->get_result()->popcnt(); } - sql::ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = - static_cast(join_filter_ctx_); - - if (OB_NOT_NULL(join_filter_ctx)) { - join_filter_ctx->inc_partial_rows_count(0, total_rows_count); - join_filter_ctx->collect_monitor_info(0, 0, total_rows_count); + if (is_filter_always_false() || DynamicFilterAction::FILTER_ALL == filter_action_) { + // is_filter_always_false is set by skip index + // filter_action_ is set by runtime filter msg + filter_count = total_rows_count; + check_count = total_rows_count; + } + if (OB_NOT_NULL(runtime_filter_ctx_)) { + runtime_filter_ctx_->collect_monitor_info(filter_count, check_count, total_rows_count); } } if (dynamic_filter_node.is_last_child()) { - update_rf_slide_window(); + if (OB_NOT_NULL(runtime_filter_ctx_)) { + runtime_filter_ctx_->collect_sample_info(filter_count, total_rows_count); + } } } @@ -2364,8 +2352,6 @@ void ObDynamicFilterExecutor::filter_on_success(ObPushdownFilterExecutor* parent { int64_t total_rows_count = filter_bitmap_->size(); int64_t filtered_rows_count = total_rows_count - filter_bitmap_->popcnt(); - sql::ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = - static_cast(join_filter_ctx_); if (parent_filter) { const int64_t skipped_rows_count = parent_filter->get_skipped_rows(); total_rows_count -= skipped_rows_count; @@ -2378,21 +2364,20 @@ void ObDynamicFilterExecutor::filter_on_success(ObPushdownFilterExecutor* parent if (!dynamic_filter_node.is_first_child()) { total_rows_count = 0; } - if (OB_NOT_NULL(join_filter_ctx)) { - join_filter_ctx->inc_partial_rows_count(filtered_rows_count, total_rows_count); - join_filter_ctx->collect_monitor_info(filtered_rows_count, total_rows_count, total_rows_count); - } - if (dynamic_filter_node.is_last_child()) { - update_rf_slide_window(); + if (OB_NOT_NULL(runtime_filter_ctx_)) { + runtime_filter_ctx_->collect_monitor_info(filtered_rows_count, total_rows_count, + total_rows_count); + if (dynamic_filter_node.is_last_child()) { + runtime_filter_ctx_->collect_sample_info(filtered_rows_count, total_rows_count); + } } } int ObDynamicFilterExecutor::init_evaluated_datums() { int ret = OB_SUCCESS; - sql::ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = - static_cast(join_filter_ctx_); - if (is_data_prepared_ && OB_NOT_NULL(join_filter_ctx) && join_filter_ctx->is_partition_wise_jf_) { + if (is_data_prepared_ && OB_NOT_NULL(runtime_filter_ctx_) + && runtime_filter_ctx_->need_reset_in_rescan()) { is_data_prepared_ = false; batch_cnt_ = 0; datum_params_.clear(); @@ -2400,30 +2385,34 @@ int ObDynamicFilterExecutor::init_evaluated_datums() return ret; } -int ObDynamicFilterExecutor::check_runtime_filter(bool &is_needed) +int ObDynamicFilterExecutor::check_runtime_filter(ObPushdownFilterExecutor *parent_filter, + bool &is_needed) { int ret = OB_SUCCESS; is_needed = false; if (is_first_check_) { - locate_join_filter_ctx(); + locate_runtime_filter_ctx(); is_first_check_ = false; } - if (!is_data_prepared() && (0 == ((batch_cnt_++) % DEFAULT_CHECK_INTERVAL)) && + // If data has prepared, and need continuous update(such as topn runtime filter) + // we check whether the data in runtime filter has a new version and then update it. + // If the data has not prepared, we check whether the runtime filter is ready and + // get data from it. + if (is_data_prepared() && is_data_version_updated() && OB_FAIL(try_updating_data())) { + LOG_WARN("Failed to updating data"); + } else if (!is_data_prepared() /*&& (0 == ((batch_cnt_++) % DEFAULT_CHECK_INTERVAL))*/ && OB_FAIL(try_preparing_data())) { LOG_WARN("Failed to try preparing data", K_(is_data_prepared)); - } - if (OB_SUCC(ret)) { + } else { if (!is_data_prepared()) { filter_bitmap_->reuse(true); } else { if (DynamicFilterAction::PASS_ALL == filter_action_) { filter_bitmap_->reuse(true); } else if (DynamicFilterAction::FILTER_ALL == filter_action_) { - filter_bitmap_->reuse(false); - } else if (OB_NOT_NULL(join_filter_ctx_)) { - sql::ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx = - static_cast(join_filter_ctx_); - is_needed = !join_filter_ctx->dynamic_disable(); + // bitmap is inited as all false, do not fill false again + } else if (DynamicFilterAction::DO_FILTER == filter_action_) { + is_needed = !runtime_filter_ctx_->dynamic_disable(); if (!is_needed) { filter_bitmap_->reuse(true); } @@ -2433,11 +2422,11 @@ int ObDynamicFilterExecutor::check_runtime_filter(bool &is_needed) return ret; } -void ObDynamicFilterExecutor::locate_join_filter_ctx() +void ObDynamicFilterExecutor::locate_runtime_filter_ctx() { const uint64_t op_id = get_filter_node().expr_->expr_ctx_id_; ObEvalCtx &eval_ctx = op_.get_eval_ctx(); - join_filter_ctx_ = eval_ctx.exec_ctx_.get_expr_op_ctx(op_id); + runtime_filter_ctx_ = static_cast(eval_ctx.exec_ctx_.get_expr_op_ctx(op_id)); } int ObDynamicFilterExecutor::try_preparing_data() @@ -2466,6 +2455,39 @@ int ObDynamicFilterExecutor::try_preparing_data() return ret; } +int ObDynamicFilterExecutor::try_updating_data() +{ + int ret = OB_SUCCESS; + bool is_update = false; + ObRuntimeFilterParams runtime_filter_params; + DynamicFilterType dynamic_filter_type = + static_cast(filter_).get_dynamic_filter_type(); + if (dynamic_filter_type >= DynamicFilterType::MAX_DYNAMIC_FILTER_TYPE) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid func type", K(ret), K(dynamic_filter_type)); + } else { + ret = UPDATE_PD_DATA_FUNCS[dynamic_filter_type]( + *filter_.expr_, *this, op_.get_eval_ctx(), runtime_filter_params, is_update); + } + if (OB_FAIL(ret)) { + } else if (is_update) { + if (OB_FAIL(datum_params_.assign(runtime_filter_params))) { + LOG_WARN("Failed to assing params for white filter", K(runtime_filter_params)); + } + } + return ret; +} + +inline bool ObDynamicFilterExecutor::is_data_version_updated() +{ + bool bool_ret = false; + if (!get_filter_node().need_continuous_update()) { + } else if (OB_NOT_NULL(runtime_filter_ctx_)) { + bool_ret = runtime_filter_ctx_->is_data_version_updated(stored_data_version_); + } + return bool_ret; +} + //--------------------- end filter executor ---------------------------- diff --git a/src/sql/engine/basic/ob_pushdown_filter.h b/src/sql/engine/basic/ob_pushdown_filter.h index 5ce9228881..f387c7a9ef 100644 --- a/src/sql/engine/basic/ob_pushdown_filter.h +++ b/src/sql/engine/basic/ob_pushdown_filter.h @@ -51,6 +51,7 @@ namespace sql { class ObRawExpr; class ObOpRawExpr; +class ObExprOperatorCtx; class ObStaticEngineCG; class ObPushdownOperator; struct ObExprFrameInfo; @@ -391,6 +392,8 @@ public: } ~ObPushdownDynamicFilterNode() {} int set_op_type(const ObRawExpr &raw_expr) override; + // for topn filter, we set op_type during runtime rather than compile time + inline void set_op_type(const ObWhiteFilterOperatorType &op_type) { op_type_ = op_type; } inline int64_t get_col_idx() const { return col_idx_; @@ -420,6 +423,11 @@ public: { return dynamic_filter_type_; } + inline bool need_continuous_update() + { + return DynamicFilterType::PD_TOPN_FILTER == dynamic_filter_type_; + } + private: int64_t col_idx_; // mark which column for multi columns runtime filter /* @@ -580,6 +588,7 @@ public: virtual OB_INLINE bool is_logic_or_node() const { return type_ == OR_FILTER_EXECUTOR; } virtual OB_INLINE bool is_logic_op_node() const { return is_logic_and_node() || is_logic_or_node(); } OB_INLINE bool is_filter_dynamic_node() const { return type_ == DYNAMIC_FILTER_EXECUTOR; } + virtual OB_INLINE bool filter_can_continuous_filter() const { return true; } int prepare_skip_filter(); OB_INLINE bool can_skip_filter(int64_t row) const { @@ -788,6 +797,16 @@ public: INHERIT_TO_STRING_KV("ObPushdownBlackFilterExecutor", ObPhysicalFilterExecutor, K_(filter), KP_(skip_bit)); virtual int filter(ObEvalCtx &eval_ctx, const sql::ObBitVector &skip_bit, bool &filtered) override; + OB_INLINE bool filter_can_continuous_filter() const override final { + bool can_continuous_filter = true; + for (int64_t i = 0; i < filter_.filter_exprs_.count();++i) { + if (T_OP_PUSHDOWN_TOPN_FILTER == filter_.filter_exprs_.at(i)->type_) { + can_continuous_filter = false; + break; + } + } + return can_continuous_filter; + } virtual int filter(blocksstable::ObStorageDatum &datum, const sql::ObBitVector &skip_bit, bool &ret_val); virtual int judge_greater_or_less(blocksstable::ObStorageDatum &datum, const sql::ObBitVector &skip_bit, @@ -1012,20 +1031,29 @@ public: : ObWhiteFilterExecutor(alloc, filter, op), is_data_prepared_(false), batch_cnt_(0), - join_filter_ctx_(nullptr), + runtime_filter_ctx_(nullptr), is_first_check_(true), build_obj_type_(ObNullType), - filter_action_(DO_FILTER) + filter_action_(DO_FILTER), + stored_data_version_(0) {} + OB_INLINE ObPushdownDynamicFilterNode &get_filter_node() + { + return static_cast(filter_); + } + OB_INLINE const ObPushdownDynamicFilterNode &get_filter_node() const + { + return static_cast(filter_); + } virtual int init_evaluated_datums() override; - int check_runtime_filter(bool &is_needed); + int check_runtime_filter(ObPushdownFilterExecutor* parent_filter, bool &is_needed); void filter_on_bypass(ObPushdownFilterExecutor* parent_filter); void filter_on_success(ObPushdownFilterExecutor* parent_filter); int64_t get_col_idx() const { return static_cast(&filter_)->get_col_idx(); } - void locate_join_filter_ctx(); + void locate_runtime_filter_ctx(); inline void set_filter_val_meta(const ObObjMeta &val_meta_) { return static_cast(filter_).set_filter_val_meta(val_meta_); @@ -1035,12 +1063,22 @@ public: return static_cast(filter_).get_filter_val_meta(); } inline void set_filter_action(DynamicFilterAction value) {filter_action_ = value; } + inline DynamicFilterAction get_filter_action() const { return filter_action_; } inline bool is_filter_all_data() { return DynamicFilterAction::FILTER_ALL == filter_action_; } inline bool is_pass_all_data() { return DynamicFilterAction::PASS_ALL == filter_action_; } inline bool is_check_all_data() { return DynamicFilterAction::DO_FILTER == filter_action_; } inline bool is_data_prepared() const { return is_data_prepared_; } + inline void set_stored_data_version(int64_t data_version) + { + stored_data_version_ = data_version; + }; + OB_INLINE bool filter_can_continuous_filter() const override final + { + // for topn sort runtime filter, the filter can not do continuouly check + return DynamicFilterType::PD_TOPN_FILTER != get_filter_node().get_dynamic_filter_type(); + } INHERIT_TO_STRING_KV("ObDynamicFilterExecutor", ObWhiteFilterExecutor, K_(is_data_prepared), - K_(batch_cnt), KP_(join_filter_ctx)); + K_(batch_cnt), KP_(runtime_filter_ctx)); public: using ObRuntimeFilterParams = common::ObSEArray; typedef int (*PreparePushdownDataFunc) (const ObExpr &expr, @@ -1048,17 +1086,28 @@ public: ObEvalCtx &eval_ctx, ObRuntimeFilterParams ¶ms, bool &is_data_prepared); + typedef int (*UpdatePushdownDataFunc) (const ObExpr &expr, + ObDynamicFilterExecutor &dynamic_filter, + ObEvalCtx &eval_ctx, + ObRuntimeFilterParams ¶ms, + bool &is_update); static PreparePushdownDataFunc PREPARE_PD_DATA_FUNCS[DynamicFilterType::MAX_DYNAMIC_FILTER_TYPE]; + static UpdatePushdownDataFunc UPDATE_PD_DATA_FUNCS[DynamicFilterType::MAX_DYNAMIC_FILTER_TYPE]; + private: int try_preparing_data(); - void update_rf_slide_window(); + int try_updating_data(); + inline bool is_data_version_updated(); private: bool is_data_prepared_; int64_t batch_cnt_; - void *join_filter_ctx_; + ObExprOperatorCtx *runtime_filter_ctx_; bool is_first_check_; ObObjType build_obj_type_; // for runtime filter, the datum_params_ are from the build table DynamicFilterAction filter_action_; + // for topn runtime filter, we need continuosly update the dynamic filter data + // stored_data_version_ means the data version now the dynamic filter use + int64_t stored_data_version_; }; class ObFilterExecutorConstructor diff --git a/src/sql/engine/basic/ob_temp_block_store.cpp b/src/sql/engine/basic/ob_temp_block_store.cpp index a31d0bce89..ecf87bd820 100644 --- a/src/sql/engine/basic/ob_temp_block_store.cpp +++ b/src/sql/engine/basic/ob_temp_block_store.cpp @@ -47,7 +47,7 @@ int ObTempBlockStore::ShrinkBuffer::init(char *buf, const int64_t buf_size) } ObTempBlockStore::ObTempBlockStore(common::ObIAllocator *alloc /* = NULL */) - : inited_(false), allocator_(NULL == alloc ? &inner_allocator_ : alloc), blk_(NULL), + : inited_(false), allocator_(NULL == alloc ? &inner_allocator_ : alloc), blk_(NULL), blk_buf_(), block_id_cnt_(0), saved_block_id_cnt_(0), dumped_block_id_cnt_(0), enable_dump_(true), enable_trunc_(false), last_trunc_offset_(0), tenant_id_(0), label_(), ctx_id_(0), mem_limit_(0), mem_hold_(0), mem_used_(0), @@ -90,6 +90,7 @@ void ObTempBlockStore::reset() LOG_TRACE("reset temp block store", KP(this), K(*this)); blk_ = NULL; + blk_buf_.reset(); // the last index block may not be linked to `blk_mem_list_` and needs to be released manually if (NULL != idx_blk_) { free_blk_mem(idx_blk_); @@ -104,7 +105,7 @@ void ObTempBlockStore::reset() if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.remove(io_.fd_))) { LOG_WARN("remove file failed", K(ret), K_(io_.fd)); } else { - LOG_INFO("close file success", K(ret), K_(io_.fd)); + LOG_INFO("close file success", K(ret), K_(io_.fd), K_(file_size)); } io_.fd_ = -1; } @@ -127,7 +128,7 @@ void ObTempBlockStore::reuse() if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.remove(io_.fd_))) { LOG_WARN("remove file failed", K(ret), K_(io_.fd)); } else { - LOG_INFO("close file success", K(ret), K_(io_.fd)); + LOG_INFO("close file success", K(ret), K_(io_.fd), K_(file_size)); } io_.fd_ = -1; } @@ -138,7 +139,7 @@ void ObTempBlockStore::reuse() } free_mem_list(alloced_mem_list_); DLIST_FOREACH_REMOVESAFE_NORET(node, blk_mem_list_) { - if (&(*node) + 1 != static_cast(static_cast(blk_->get_buffer()->data()))) { + if (&(*node) + 1 != static_cast(static_cast(blk_buf_.data()))) { node->unlink(); node->~LinkNode(); allocator_->free(node); @@ -147,13 +148,12 @@ void ObTempBlockStore::reuse() set_mem_hold(0); set_mem_used(0); if (NULL != blk_) { - if (OB_FAIL(setup_block(blk_->get_buffer(), blk_))) { + if (OB_FAIL(setup_block(blk_buf_, blk_))) { LOG_WARN("setup block failed", K(ret)); } block_cnt_ = 1; - const ShrinkBuffer *buf = blk_->get_buffer(); - set_mem_hold(buf->capacity() + sizeof(LinkNode)); - max_block_size_ = buf->capacity(); + set_mem_hold(blk_buf_.capacity() + sizeof(LinkNode)); + max_block_size_ = blk_buf_.capacity(); max_hold_mem_ = mem_hold_; } blocks_.reset(); @@ -216,14 +216,14 @@ int ObTempBlockStore::finish_add_row(bool need_dump /*true*/) return ret; } -int ObTempBlockStore::init_block_buffer(void* mem, const int64_t size, Block *&block) +int ObTempBlockStore::init_dtl_block_buffer(void* mem, const int64_t size, Block *&block) { int ret = OB_SUCCESS; ShrinkBuffer *buf = NULL; if (OB_ISNULL(mem)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("mem is null", KP(mem)); - } else if (OB_UNLIKELY(size <= Block::min_blk_size(0))) { + } else if (OB_UNLIKELY(size <= Block::min_blk_size(0))) { ret = OB_BUF_NOT_ENOUGH; LOG_WARN("buffer is not enough", K(size)); } else if (OB_ISNULL(buf = new (Block::buffer_position(mem, size))ShrinkBuffer)) { @@ -240,7 +240,7 @@ int ObTempBlockStore::init_block_buffer(void* mem, const int64_t size, Block *&b } else { block->block_id_ = 0; // unused block->cnt_ = 0; - block->buf_off_ = buf->remain(); + block->buf_off_ = buf->capacity() - buf->tail_size(); } } return ret; @@ -280,14 +280,14 @@ int ObTempBlockStore::append_block_payload(const char *buf, const int64_t size, LOG_WARN("buf is null", K(ret), KP(buf), K(size), K(cnt)); } else if (OB_FAIL(new_block(size, blk_, true))) { LOG_WARN("fail to new block", K(ret)); - } else if (OB_UNLIKELY(size > blk_->remain())) { + } else if (OB_UNLIKELY(size > blk_buf_.remain())) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("blk size is not enough", K(ret), K(size), K(blk_->remain())); + LOG_WARN("blk size is not enough", K(ret), K(size), K(blk_buf_.remain())); } else { blk_->cnt_ = static_cast(cnt); MEMCPY(blk_->payload_, buf, size); block_id_cnt_ = blk_->end(); - blk_->get_buffer()->fast_advance(size); + blk_buf_.fast_advance(size); LOG_DEBUG("append block payload", K(*this), K(*blk_), K(mem_used_), K(mem_hold_)); } // dump data if mem used > 16MB @@ -305,7 +305,7 @@ int ObTempBlockStore::new_block(const int64_t mem_size, const bool strict_mem_size /* false*/) { int ret = OB_SUCCESS; - const int64_t min_blk_size = Block::min_blk_size(mem_size); + const int64_t min_blk_size = Block::min_blk_size(mem_size); if (OB_UNLIKELY(!is_inited())) { ret = OB_NOT_INIT; LOG_WARN("not init", K(ret)); @@ -544,20 +544,16 @@ int ObTempBlockStore::alloc_block(Block *&blk, const int64_t min_size, const boo size -= sizeof(LinkNode); } void *mem = alloc_blk_mem(size, &blk_mem_list_); - ShrinkBuffer *buf = NULL; if (OB_ISNULL(mem)) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("alloc memory failed", K(ret), K(size)); - } else if (OB_ISNULL(buf = new (Block::buffer_position(mem, size))ShrinkBuffer)) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("alloc buffer failed", K(ret)); - } else if (OB_FAIL(buf->init(static_cast(mem), size))) { + } else if (OB_FAIL(blk_buf_.init(static_cast(mem), size))) { LOG_WARN("init shrink buffer failed", K(ret)); - } else if (OB_FAIL(setup_block(buf, blk))) { + } else if (OB_FAIL(setup_block(blk_buf_, blk))) { LOG_WARN("setup block buffer fail", K(ret)); } else { ++block_cnt_; - LOG_TRACE("succ to alloc new block", KP(this), KP(mem), K(*buf), K(*blk)); + LOG_TRACE("succ to alloc new block", KP(this), KP(mem), K_(blk_buf), K(*blk)); } if (OB_FAIL(ret) && !OB_ISNULL(mem)) { free_blk_mem(mem, size); @@ -597,23 +593,24 @@ void *ObTempBlockStore::alloc_blk_mem(const int64_t size, ObDList *lis return blk; } -int ObTempBlockStore::setup_block(ShrinkBuffer *buf, Block *&blk) +int ObTempBlockStore::setup_block(ShrinkBuffer &buf, Block *&blk) { int ret = OB_SUCCESS; - if (OB_UNLIKELY(!buf->is_inited())) { + if (OB_UNLIKELY(!buf.is_inited())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("block buffer not inited", K(ret)); } else { - buf->reuse(); - blk = new (buf->head()) Block; + buf.reuse(); + blk = new (buf.head()) Block; blk->block_id_ = block_id_cnt_; - if (OB_FAIL(buf->fill_head(sizeof(Block)))) { + blk->raw_size_ = buf.capacity(); + if (OB_FAIL(buf.fill_head(sizeof(Block)))) { LOG_WARN("fill buffer head failed", K(ret), K(buf), K(sizeof(Block))); - } else if (OB_FAIL(buf->fill_tail(sizeof(ShrinkBuffer)))) { - LOG_WARN("fill buffer tail failed", K(ret), K(buf), K(sizeof(ShrinkBuffer))); } else { - blk->buf_off_ = buf->remain(); - inc_mem_used(sizeof(Block) + sizeof(ShrinkBuffer)); + inc_mem_used(sizeof(Block)); + } + if (OB_SUCC(ret) && OB_FAIL(prepare_setup_blk(blk))) { + LOG_WARN("fail to prepare setup blk", K(ret)); } } return ret; @@ -627,18 +624,20 @@ int ObTempBlockStore::switch_block(const int64_t min_size, const bool strict_mem } else if (OB_UNLIKELY(min_size < 0)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(min_size)); + } else if (OB_FAIL(prepare_blk_for_switch(blk_))) { + LOG_WARN("block compact failed", K(ret)); } else { const bool finish_add = (0 == min_size); Block *new_blk = NULL; - const ShrinkBuffer *buf = blk_->get_buffer(); BlockIndex bi; bi.is_idx_block_ = false; bi.on_disk_ = false; bi.block_id_ = ~(0b11UL << 62) & saved_block_id_cnt_; bi.blk_ = blk_; - bi.length_ = static_cast(buf->head_size()); - bi.capacity_ = static_cast(buf->capacity()); + bi.length_ = static_cast(blk_buf_.head_size()); + bi.capacity_ = static_cast(blk_buf_.capacity()); blk_->raw_size_ = bi.length_; + blk_buf_.reset(); if (OB_FAIL(add_block_idx(bi))) { LOG_WARN("add block index failed", K(ret)); } else if (!finish_add && OB_FAIL(alloc_block(new_blk, min_size, strict_mem_size))) { @@ -649,7 +648,7 @@ int ObTempBlockStore::switch_block(const int64_t min_size, const bool strict_mem blk_ = new_blk; } if (OB_FAIL(ret) && NULL != new_blk) { - free_blk_mem(new_blk, new_blk->get_buffer()->capacity()); + free_blk_mem(new_blk, blk_buf_.capacity()); } } return ret; @@ -1004,6 +1003,10 @@ int ObTempBlockStore::ensure_reader_buffer(BlockReader &reader, ShrinkBuffer &bu p->next_ = reader.try_free_list_; reader.try_free_list_ = p; } + if (buf.data() == reader.buf_.data()) { + // reset `buf_` of reader if the buffers share common memory ptr. + reader.buf_.reset(); + } buf.reset(); } @@ -1076,7 +1079,7 @@ int ObTempBlockStore::write_file(BlockIndex &bi, void *buf, int64_t size) io_.tenant_id_ = tenant_id_; io_.io_desc_.set_wait_event(ObWaitEventIds::ROW_STORE_DISK_WRITE); io_.io_timeout_ms_ = timeout_ms; - LOG_INFO("open file success", K_(io_.fd), K_(io_.dir_id)); + LOG_INFO("open file success", K_(io_.fd), K_(io_.dir_id), K(get_compressor_type())); } } ret = OB_E(EventTable::EN_8) ret; @@ -1310,7 +1313,7 @@ int ObTempBlockStore::dump_block(Block *blk, int64_t &dumped_size) ++block_cnt_on_disk_; dumped_block_id_cnt_ += blk->cnt_; inc_mem_hold(-(bi->capacity_ + sizeof(LinkNode))); - inc_mem_used(-(dumped_size + sizeof(ShrinkBuffer))); + inc_mem_used(-(dumped_size)); LOG_TRACE("succ to dump block", KP(this), K(*blk), K(*bi), K(dumped_size)); } return ret; @@ -1458,9 +1461,10 @@ void ObTempBlockStore::BlockHolder::release() OB_DEF_SERIALIZE(ObTempBlockStore) { int ret = OB_SUCCESS; - if (inited_ && enable_dump_) { + if (inited_ && (enable_dump_ || need_compress())) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("block store not support serialize if enable dump", K(ret)); + LOG_WARN("block store not support serialize if enable dump", K(ret), K_(enable_dump), + K(get_compressor_type())); } LST_DO_CODE(OB_UNIS_ENCODE, tenant_id_, @@ -1518,7 +1522,8 @@ OB_DEF_DESERIALIZE(ObTempBlockStore) mem_limit_, label); if (!is_inited()) { - if (OB_FAIL(init(mem_limit_, false/*enable_dump*/, tenant_id_, ctx_id_, label))) { + if (OB_FAIL(init(mem_limit_, false/*enable_dump*/, tenant_id_, ctx_id_, label, + NONE_COMPRESSOR))) { LOG_WARN("fail to init Block row store", K(ret)); } } diff --git a/src/sql/engine/basic/ob_temp_block_store.h b/src/sql/engine/basic/ob_temp_block_store.h index 249bc1a367..cb1dc99db4 100644 --- a/src/sql/engine/basic/ob_temp_block_store.h +++ b/src/sql/engine/basic/ob_temp_block_store.h @@ -106,8 +106,8 @@ public: /* * Block, a stucture storing the data uses block id for indexing, * the real data starts from the payload. - * If the block is in the process of data appending, the tail will occupy one `ShrinkBuffer` size - * to record the current writing position information. + * If the block is in the process of data appending in dtl, the tail will occupy one + * `ShrinkBuffer` size to record the current writing position information. * The memory layout is as follows: * +------------------------------------------------------------------+ * | Block Header | Payload | ShrinkBuffer(optional)| @@ -119,9 +119,10 @@ public: Block() : magic_(MAGIC), block_id_(0), cnt_(0), raw_size_(0) {} + template inline static int64_t min_blk_size(const int64_t size) { - return sizeof(Block) + sizeof(ShrinkBuffer) + size; + return sizeof(Block) + size + (WITH_BLK_BUF ? sizeof(ShrinkBuffer) : 0); } inline bool contain(const int64_t block_id) const { @@ -129,21 +130,15 @@ public: } inline int64_t begin() const { return block_id_; } inline int64_t end() const { return block_id_ + cnt_; } - inline int64_t remain() const { return get_buffer()->remain(); } inline int64_t payload_size() const { return raw_size_ - sizeof(Block); } - inline ShrinkBuffer* get_buffer() - { - return static_cast(static_cast(payload_ + buf_off_)); - } - inline const ShrinkBuffer* get_buffer() const - { - return static_cast(static_cast(payload_ + buf_off_)); - } + // We put the buffer at the end of the block. This is only used in dtl scenarios + // to support the self-explanatory ability of the block. inline static char *buffer_position(void *mem, const int64_t size) { return static_cast(mem) + size - sizeof(ShrinkBuffer); } - inline uint64_t checksum() const { + inline uint64_t checksum() const + { ObBatchChecksum bc; bc.fill(payload_, raw_size_ - sizeof(Block)); return bc.calc(); @@ -354,7 +349,7 @@ public: uint64_t tenant_id, int64_t mem_ctx_id, const char *label, - common::ObCompressorType compressor_type = NONE_COMPRESSOR, + common::ObCompressorType compressor_type, const bool enable_trunc = false); void reset(); void reuse(); @@ -401,11 +396,8 @@ public: inline int64_t get_max_hold_mem() const { return max_hold_mem_; } inline common::DefaultPageAllocator& get_inner_allocator() { return inner_allocator_; } inline int64_t has_dumped() const { return block_cnt_on_disk_ > 0; } - inline int64_t get_last_buffer_mem_size() const - { - return nullptr == blk_ ? 0 : blk_->get_buffer()->capacity(); - } - static int init_block_buffer(void* mem, const int64_t size, Block *&block); + inline int64_t get_last_buffer_mem_size() const { return blk_buf_.capacity(); } + static int init_dtl_block_buffer(void* mem, const int64_t size, Block *&block); int append_block(const char *buf, const int64_t size); int append_block_payload(const char *buf, const int64_t size, const int64_t cnt); int alloc_dir_id(); @@ -415,9 +407,12 @@ public: { enable_trunc_ = enable_trunc; } + inline ShrinkBuffer &get_blk_buf() { return blk_buf_; } bool is_truncate() { return enable_trunc_; } inline int64_t get_cur_file_offset() const { return cur_file_offset_; } inline void set_cur_file_offset(int64_t file_offset) { cur_file_offset_ = file_offset; } + + inline bool is_empty_save_block_cnt() const { return saved_block_id_cnt_ == 0; } // include index blocks and data blocks TO_STRING_KV(K_(inited), K_(enable_dump), K_(tenant_id), K_(label), K_(ctx_id), K_(mem_limit), @@ -494,7 +489,7 @@ private: static int get_timeout(int64_t &timeout_ms); int alloc_block(Block *&blk, const int64_t min_size, const bool strict_mem_size); void *alloc_blk_mem(const int64_t size, common::ObDList *list); - int setup_block(ShrinkBuffer *buf, Block *&blk); + int setup_block(ShrinkBuffer &buf, Block *&blk); // new block is not needed if %min_size is zero. (finish add row) int switch_block(const int64_t min_size, const bool strict_mem_size); int add_block_idx(const BlockIndex &bi); @@ -521,8 +516,34 @@ private: void free_mem_list(common::ObDList &list); inline bool has_index_block() const { return index_block_cnt_ > 0; } inline int64_t get_block_raw_size(const Block *blk) const - { return is_last_block(blk) ? blk->get_buffer()->head_size() : blk->raw_size_; } + { return is_last_block(blk) ? blk_buf_.head_size() : blk->raw_size_; } inline bool need_compress() const { return compressor_.get_compressor_type() != NONE_COMPRESSOR; } + inline ObCompressorType get_compressor_type() const { return compressor_.get_compressor_type(); } + +protected: + /** + * These functions are inserted into different stages of the block, and their main function + * is to customize some special operations to meet the needs of different data formats or + * scenarios. The overall calling timing of there functions is as follows: + * new_blk -> prepare_setup_blk -> blk_add_batch -> blk_is_full -> prepare_blk_for_switch -> + * switch_blk -> prepare_blk_for_write -> dump_blk -> prepare_blk_for_read -> read_blk + * + * `prepare_setup_blk`: called when the block has just initialized the initial meta information. + * The block does not yet have any content. You can add some new meta information you need + * by overwriting it. + */ + virtual int prepare_setup_blk(Block *blk) { return OB_SUCCESS; } + /** + * `prepare_blk_for_switch`: The function call occurs when the current block is full and blk + * needs to be switched. Before switching, some customized actions will be performed on `blk`, + * such as data compaction, etc. + */ + virtual int prepare_blk_for_switch(Block *blk) { return OB_SUCCESS; } + /** + * `prepare_blk_for_write/read`: These two functions are used in conjunction. + * `prepare_blk_for_write` is called before the block is dumped on the disk, and + * `prepare_blk_for_read` occurs when the block has just been read from the disk. + */ virtual int prepare_blk_for_write(Block *blk) { return OB_SUCCESS; } virtual int prepare_blk_for_read(Block *blk) { return OB_SUCCESS; } @@ -532,6 +553,7 @@ protected: Block *blk_; // currently operating block // variables related to `block_id`, the total number of `block_id` is the sum of // all block's `cnt_`, and it can also be used to count rows. + ShrinkBuffer blk_buf_; int64_t block_id_cnt_; int64_t saved_block_id_cnt_; int64_t dumped_block_id_cnt_; @@ -579,10 +601,7 @@ private: inline int ObTempBlockStore::ShrinkBuffer::fill_head(int64_t size) { int ret = common::OB_SUCCESS; - if (size < -head_) { - ret = common::OB_INVALID_ARGUMENT; - SQL_ENG_LOG(WARN, "invalid argument", K(size), K_(head)); - } else if (size > remain()) { + if (size > remain()) { ret = common::OB_BUF_NOT_ENOUGH; SQL_ENG_LOG(WARN, "buffer not enough", K(size), "remain", remain()); } else { @@ -594,10 +613,7 @@ inline int ObTempBlockStore::ShrinkBuffer::fill_head(int64_t size) inline int ObTempBlockStore::ShrinkBuffer::fill_tail(int64_t size) { int ret = common::OB_SUCCESS; - if (size < -tail_size()) { - ret = common::OB_INVALID_ARGUMENT; - SQL_ENG_LOG(WARN, "invalid argument", K(size), "tail_size", tail_size()); - } else if (size > remain()) { + if (size > remain()) { ret = common::OB_BUF_NOT_ENOUGH; SQL_ENG_LOG(WARN, "buffer not enough", K(size), "remain", remain()); } else { @@ -613,7 +629,7 @@ inline int ObTempBlockStore::ShrinkBuffer::compact() ret = common::OB_NOT_INIT; SQL_ENG_LOG(WARN, "not inited", K(ret)); } else { - const int64_t tail_data_size = tail_size() - sizeof(ShrinkBuffer); + const int64_t tail_data_size = tail_size(); MEMMOVE(head(), tail(), tail_data_size); head_ += tail_data_size; tail_ += tail_data_size; diff --git a/src/sql/engine/basic/ob_temp_column_store.cpp b/src/sql/engine/basic/ob_temp_column_store.cpp index fc682436c7..56b934df4f 100644 --- a/src/sql/engine/basic/ob_temp_column_store.cpp +++ b/src/sql/engine/basic/ob_temp_column_store.cpp @@ -63,19 +63,19 @@ int ObTempColumnStore::ColumnBlock::calc_rows_size(const IVectorPtrs &vectors, return ret; } -int ObTempColumnStore::ColumnBlock::add_batch(const IVectorPtrs &vectors, +int ObTempColumnStore::ColumnBlock::add_batch(ShrinkBuffer &buf, + const IVectorPtrs &vectors, const uint16_t *selector, const ObArray &lengths, const int64_t size, const int64_t batch_mem_size) { int ret = OB_SUCCESS; - if (OB_UNLIKELY(batch_mem_size > remain())) { + if (OB_UNLIKELY(batch_mem_size > buf.remain())) { ret = OB_BUF_NOT_ENOUGH; - LOG_WARN("block is not enough", K(ret), K(batch_mem_size), K(remain())); + LOG_WARN("block is not enough", K(ret), K(batch_mem_size), K(buf)); } else { - ShrinkBuffer *buf = get_buffer(); - char *head = buf->head(); + char *head = buf.head(); *reinterpret_cast(head) = static_cast(size); // row_count int32_t *vec_offsets = reinterpret_cast(head + sizeof(int32_t)); int64_t pos = get_header_size(vectors.count()); @@ -106,7 +106,7 @@ int ObTempColumnStore::ColumnBlock::add_batch(const IVectorPtrs &vectors, } } vec_offsets[vectors.count()] = pos; // last offset, the size of vector - buf->fast_advance(pos); + buf.fast_advance(pos); if (OB_FAIL(ret)) { } else if (OB_UNLIKELY(pos != batch_mem_size)) { ret = OB_ERR_UNEXPECTED; @@ -317,7 +317,8 @@ int ObTempColumnStore::init(const ObExprPtrIArray &exprs, const lib::ObMemAttr &mem_attr, const int64_t mem_limit, const bool enable_dump, - const bool reuse_vector_array) + const bool reuse_vector_array, + const common::ObCompressorType compressor_type) { int ret = OB_SUCCESS; mem_attr_ = mem_attr; @@ -325,7 +326,7 @@ int ObTempColumnStore::init(const ObExprPtrIArray &exprs, max_batch_size_ = max_batch_size; ObTempBlockStore::set_inner_allocator_attr(mem_attr); OZ(ObTempBlockStore::init(mem_limit, enable_dump, mem_attr.tenant_id_, mem_attr.ctx_id_, - mem_attr_.label_)); + mem_attr_.label_, compressor_type)); reuse_vector_array_ = reuse_vector_array; inited_ = true; return ret; @@ -425,8 +426,8 @@ int ObTempColumnStore::add_batch(const common::ObIArray &exprs, ObEval LOG_WARN("fail to calc rows size", K(ret)); } else if (OB_FAIL(ensure_write_blk(batch_mem_size))) { LOG_WARN("ensure write block failed", K(ret)); - } else if (OB_FAIL(cur_blk_->add_batch(batch_ctx_->vectors_, selector, batch_ctx_->lengths_, - size, batch_mem_size))) { + } else if (OB_FAIL(cur_blk_->add_batch(blk_buf_, batch_ctx_->vectors_, selector, + batch_ctx_->lengths_, size, batch_mem_size))) { LOG_WARN("fail to add batch to column store", K(ret)); } else { block_id_cnt_ += size; diff --git a/src/sql/engine/basic/ob_temp_column_store.h b/src/sql/engine/basic/ob_temp_column_store.h index b488c400ac..c111c19c3e 100644 --- a/src/sql/engine/basic/ob_temp_column_store.h +++ b/src/sql/engine/basic/ob_temp_column_store.h @@ -60,7 +60,8 @@ public: const ObArray &lengths, const int64_t size, int64_t &batch_mem_size); - int add_batch(const IVectorPtrs &vectors, + int add_batch(ShrinkBuffer &buf, + const IVectorPtrs &vectors, const uint16_t *selector, const ObArray &lengths, const int64_t size, @@ -152,7 +153,8 @@ public: const lib::ObMemAttr &mem_attr, const int64_t mem_limit, const bool enable_dump, - const bool reuse_vector_array); + const bool reuse_vector_array, + const common::ObCompressorType compressor_type); int init_batch_ctx(const ObExprPtrIArray &exprs); @@ -172,7 +174,7 @@ private: inline int ensure_write_blk(const int64_t mem_size) { int ret = common::OB_SUCCESS; - if (NULL == cur_blk_ || mem_size > cur_blk_->remain()) { + if (NULL == cur_blk_ || mem_size > blk_buf_.remain()) { if (OB_FAIL(new_block(mem_size))) { SQL_ENG_LOG(WARN, "fail to new block", K(ret), K(mem_size)); } else { diff --git a/src/sql/engine/basic/ob_temp_row_store.cpp b/src/sql/engine/basic/ob_temp_row_store.cpp index 883553a777..2031a55d06 100644 --- a/src/sql/engine/basic/ob_temp_row_store.cpp +++ b/src/sql/engine/basic/ob_temp_row_store.cpp @@ -27,7 +27,10 @@ using namespace common; namespace sql { -int ObTempRowStore::RowBlock::get_store_row(int64_t &cur_pos, const ObCompactRow *&sr) +#define ROW_BLK reinterpret_cast(blk_) + +template<> +int ObTempRowStoreBase::RowBlock::get_store_row(int64_t &cur_pos, const ObCompactRow *&sr) { int ret = OB_SUCCESS; if (cur_pos >= raw_size_) { @@ -41,20 +44,22 @@ int ObTempRowStore::RowBlock::get_store_row(int64_t &cur_pos, const ObCompactRow return ret; } -int ObTempRowStore::RowBlock::add_row(const common::ObIArray &exprs, - const RowMeta &row_meta, - ObEvalCtx &ctx, - ObCompactRow *&stored_row) +template +int ObTempRowStoreBase::RowBlock::add_row( + ShrinkBuffer &buf, + const common::ObIArray &exprs, + const RowMeta &row_meta, + ObEvalCtx &ctx, + ObCompactRow *&stored_row) { int ret = OB_SUCCESS; const int64_t batch_idx = ctx.get_batch_idx(); - ShrinkBuffer *buf = get_buffer(); int64_t row_size = row_meta.get_row_fixed_size(); - int64_t remain_size = buf->remain(); + int64_t remain_size = buf.remain(); if (OB_UNLIKELY(row_size > remain_size)) { ret = OB_BUF_NOT_ENOUGH; } else { - stored_row = new(buf->head())ObCompactRow(); + stored_row = new(buf.head())ObCompactRow(); stored_row->init(row_meta); for (int64_t i = 0; i < exprs.count() && OB_SUCC(ret); ++i) { ObExpr *expr = exprs.at(i); @@ -67,49 +72,58 @@ int ObTempRowStore::RowBlock::add_row(const common::ObIArray &exprs, } } } - if (OB_SUCC(ret)) { - ++cnt_; - stored_row->set_row_size(row_size); - buf->fast_advance(row_size); + } + if (OB_SUCC(ret)) { + if (OB_FAIL(post_add_row(buf, row_size))) { + LOG_WARN("fill index to buffer tail failed", K(ret)); } + ++cnt_; + stored_row->set_row_size(row_size); } return ret; } -int ObTempRowStore::RowBlock::add_row(const ObCompactRow *src_row, ObCompactRow *&stored_row) +template +int ObTempRowStoreBase::RowBlock::add_row( + ShrinkBuffer &buf, const ObCompactRow *src_row, ObCompactRow *&stored_row) { int ret = OB_SUCCESS; if (OB_ISNULL(src_row)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("src row is null", K(ret)); } else { - ShrinkBuffer *buf = get_buffer(); - stored_row = new(buf->head())ObCompactRow(); + stored_row = new(buf.head())ObCompactRow(); MEMCPY(stored_row, src_row, src_row->get_row_size()); - buf->fast_advance(src_row->get_row_size()); + if (RA) { + ret = post_add_row(buf, src_row->get_row_size()); + } else { + buf.fast_advance(src_row->get_row_size()); + } ++cnt_; } return ret; } -int ObTempRowStore::RowBlock::add_batch(const IVectorPtrs &vectors, - const RowMeta &row_meta, - const uint16_t selector[], - const int64_t size, - const uint32_t row_size_arr[], - int64_t batch_mem_size, - ObCompactRow **stored_rows) +template +int ObTempRowStoreBase::RowBlock::add_batch( + ShrinkBuffer &buf, + const IVectorPtrs &vectors, + const RowMeta &row_meta, + const uint16_t selector[], + const int64_t size, + const uint32_t row_size_arr[], + int64_t batch_mem_size, + ObCompactRow **stored_rows) { int ret = OB_SUCCESS; - if (OB_UNLIKELY(batch_mem_size > remain())) { + if (OB_UNLIKELY(batch_mem_size > buf.remain())) { ret = OB_BUF_NOT_ENOUGH; } else { - ShrinkBuffer *buf = get_buffer(); - memset(buf->head(), 0, batch_mem_size); - for (int64_t i = 0; i < size; i++) { - stored_rows[i] = reinterpret_cast (buf->head()); + memset(buf.head(), 0, batch_mem_size); + for (int64_t i = 0; OB_SUCC(ret) && i < size; i++) { + stored_rows[i] = reinterpret_cast (buf.head()); stored_rows[i]->set_row_size(row_size_arr[i]); - buf->fast_advance(row_size_arr[i]); + ret = post_add_row(buf, row_size_arr[i]); } for (int64_t col_idx = 0; OB_SUCC(ret) && col_idx < vectors.count(); col_idx ++) { if (nullptr == vectors.at(col_idx)) { @@ -126,13 +140,24 @@ int ObTempRowStore::RowBlock::add_batch(const IVectorPtrs &vectors, return ret; } -int ObTempRowStore::RowBlock::get_next_batch(Iterator &iter, +template <> +int32_t ObTempRowStoreBase::RowBlock::get_row_location(const int64_t row_id) const +{ + return *reinterpret_cast(reinterpret_cast(this) + raw_size_ + - (row_id - block_id_ + 1) * ROW_INDEX_SIZE); +} + +template +int ObTempRowStoreBase::RowBlock::get_next_batch(ObTempRowStoreBase::ReaderBase &iter, const int64_t max_rows, int64_t &read_rows, const ObCompactRow **stored_rows) const { int ret = OB_SUCCESS; read_rows = 0; + if (RA) { + iter.read_pos_ = get_row_location(iter.cur_blk_id_); + } for (read_rows = 0; read_rows < max_rows && iter.cur_blk_id_ < end(); ++read_rows) { ++iter.cur_blk_id_; stored_rows[read_rows] = reinterpret_cast(iter.read_pos_ + payload_); @@ -144,7 +169,8 @@ int ObTempRowStore::RowBlock::get_next_batch(Iterator &iter, return ret; } -int ObTempRowStore::RowBlock::calc_row_size(const common::ObIArray &exprs, +template +int ObTempRowStoreBase::RowBlock::calc_row_size(const common::ObIArray &exprs, const RowMeta &row_meta, ObEvalCtx &ctx, int64_t &size) @@ -186,7 +212,8 @@ int ObTempRowStore::RowBlock::calc_row_size(const common::ObIArray &exp } // calc need size for this batch -int ObTempRowStore::RowBlock::calc_rows_size(const IVectorPtrs &vectors, +template +int ObTempRowStoreBase::RowBlock::calc_rows_size(const IVectorPtrs &vectors, const RowMeta &row_meta, const uint16_t selector[], const int64_t size, @@ -249,7 +276,23 @@ int ObTempRowStore::RowBlock::calc_rows_size(const IVectorPtrs &vectors, return ret; } -int ObTempRowStore::DtlRowBlock::calc_rows_size(const IVectorPtrs &vectors, +template<> +int ObTempRowStoreBase::RowBlock::get_row(const int64_t row_id, const ObCompactRow *&sr) const +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!contain(row_id))) { + ret = OB_INDEX_OUT_OF_RANGE; + LOG_WARN("invalid index", K(ret), K(row_id), K(*this)); + } else { + int32_t location = get_row_location(row_id); + const ObCompactRow *row = reinterpret_cast(&payload_[location]); + sr = row; + } + return ret; +} + +template +int ObTempRowStoreBase::DtlRowBlock::calc_rows_size(const IVectorPtrs &vectors, const RowMeta &row_meta, const ObBatchRows &brs, uint32_t row_size_arr[]) { @@ -311,14 +354,16 @@ int ObTempRowStore::DtlRowBlock::calc_rows_size(const IVectorPtrs &vectors, return ret; } -int ObTempRowStore::Iterator::init(ObTempRowStore *store) +template +int ObTempRowStoreBase::ReaderBase::init(ObTempRowStoreBase *store) { reset(); row_store_ = store; return BlockReader::init(store); } -int ObTempRowStore::Iterator::get_next_batch(const ObExprPtrIArray &exprs, +template +int ObTempRowStoreBase::ReaderBase::get_next_batch(const ObExprPtrIArray &exprs, ObEvalCtx &ctx, const int64_t max_rows, int64_t &read_rows, @@ -359,7 +404,8 @@ int ObTempRowStore::Iterator::get_next_batch(const ObExprPtrIArray &exprs, return ret; } -int ObTempRowStore::Iterator::get_next_batch(const IVectorPtrs &vectors, +template +int ObTempRowStoreBase::ReaderBase::get_next_batch(const IVectorPtrs &vectors, const int64_t max_rows, int64_t &read_rows, const ObCompactRow **stored_rows) @@ -401,7 +447,8 @@ int ObTempRowStore::Iterator::get_next_batch(const IVectorPtrs &vectors, return ret; } -int ObTempRowStore::Iterator::get_next_batch(const int64_t max_rows, +template +int ObTempRowStoreBase::ReaderBase::get_next_batch(const int64_t max_rows, int64_t &read_rows, const ObCompactRow **stored_rows) { int ret = OB_SUCCESS; @@ -422,7 +469,7 @@ int ObTempRowStore::Iterator::get_next_batch(const int64_t max_rows, const ObCompactRow **srs = &stored_rows[read_rows]; if (OB_UNLIKELY(!cur_blk_->contain(cur_blk_id_))) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("current block is invalid", K(ret)); + LOG_WARN("current block is invalid", K(ret), K(cur_blk_id_), K(*cur_blk_)); } else if (OB_FAIL(cur_blk_->get_next_batch(*this, max_rows - read_rows, read_rows_in_blk, srs))) { LOG_WARN("fail to get batch from block", K(ret)); @@ -443,7 +490,8 @@ int ObTempRowStore::Iterator::get_next_batch(const int64_t max_rows, return ret; } -int ObTempRowStore::Iterator::attach_rows(const ObExprPtrIArray &exprs, +template<> +int ObTempRowStoreBase::Iterator::attach_rows(const ObExprPtrIArray &exprs, ObEvalCtx &ctx, const RowMeta &row_meta, const ObCompactRow **srows, @@ -464,7 +512,8 @@ int ObTempRowStore::Iterator::attach_rows(const ObExprPtrIArray &exprs, return ret; } -int ObTempRowStore::Iterator::next_block() +template +int ObTempRowStoreBase::ReaderBase::next_block() { int ret = OB_SUCCESS; const Block *read_blk = NULL; @@ -481,43 +530,89 @@ int ObTempRowStore::Iterator::next_block() return ret; } -ObTempRowStore::ObTempRowStore(common::ObIAllocator *alloc /* = NULL */) - : ObTempBlockStore(alloc), cur_blk_(NULL), col_cnt_(0), batch_ctx_(NULL), +template<> +int ObTempRowStoreBase::RAReader::get_row(const int64_t row_id, const ObCompactRow *&sr) +{ + int ret = OB_SUCCESS; + cur_blk_id_ = row_id; + if (OB_FAIL(next_block())) { // get the block contains cur_blk_id_ + LOG_WARN("load block failed", K(ret)); + } else if (OB_FAIL(cur_blk_->get_row(row_id, sr))) { + LOG_WARN("get row from block failed", K(ret), K(row_id), K(*cur_blk_)); + } else if (OB_ISNULL(sr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("NULL store row returned", K(ret)); + } + if (ret == OB_ITER_END) { + ret = OB_INDEX_OUT_OF_RANGE; + } + return ret; +} + +template<> +int ObTempRowStoreBase::RAReader::get_batch_rows(const ObExprPtrIArray &exprs, + ObEvalCtx &ctx, + const int64_t start_idx, + const int64_t end_idx, + int64_t &read_rows, + const ObCompactRow **stored_rows) { + int ret = OB_SUCCESS; + cur_blk_id_ = start_idx; + ret = get_next_batch(exprs, ctx, end_idx - start_idx, read_rows, stored_rows); + return ret; +} + +template <> +int ObTempRowStoreBase::RAReader::get_batch_rows(const int64_t start_idx, + const int64_t end_idx, int64_t &read_rows, + const ObCompactRow **stored_rows) +{ + int ret = OB_SUCCESS; + cur_blk_id_ = start_idx; + return get_next_batch(end_idx - start_idx, read_rows, stored_rows); +} + +template +ObTempRowStoreBase::ObTempRowStoreBase(common::ObIAllocator *alloc /* = NULL */) + : ObTempBlockStore(alloc), col_cnt_(0), batch_ctx_(NULL), row_meta_(allocator_), max_batch_size_(0) { } -ObTempRowStore::~ObTempRowStore() +template +ObTempRowStoreBase::~ObTempRowStoreBase() { destroy(); } -void ObTempRowStore::destroy() +template +void ObTempRowStoreBase::destroy() { row_meta_.reset(); reset(); } -void ObTempRowStore::reset() +template +void ObTempRowStoreBase::reset() { if (NULL != batch_ctx_) { batch_ctx_->~BatchCtx(); allocator_->free(batch_ctx_); batch_ctx_ = NULL; - cur_blk_ = NULL; } ObTempBlockStore::reset(); } -int ObTempRowStore::init(const ObExprPtrIArray &exprs, +template +int ObTempRowStoreBase::init(const ObExprPtrIArray &exprs, const int64_t max_batch_size, const lib::ObMemAttr &mem_attr, const int64_t mem_limit, bool enable_dump, uint32_t row_extra_size, + const common::ObCompressorType compressor_type, const bool reorder_fixed_expr /*true*/, - const bool enable_trunc /*false*/, - const common::ObCompressorType compressor_type /*NONE_COMPRESSOR*/) + const bool enable_trunc /*false*/) { int ret = OB_SUCCESS; mem_attr_ = mem_attr; @@ -531,27 +626,35 @@ int ObTempRowStore::init(const ObExprPtrIArray &exprs, return ret; } -int ObTempRowStore::init(const RowMeta &row_meta, +template +int ObTempRowStoreBase::init(const RowMeta &row_meta, const int64_t max_batch_size, const lib::ObMemAttr &mem_attr, const int64_t mem_limit, bool enable_dump, - const bool enable_trunc /*false*/, - const common::ObCompressorType compressor_type /*NONE_COMPRESSOR*/) + const common::ObCompressorType compressor_type, + const bool enable_trunc /*false*/) { int ret = OB_SUCCESS; mem_attr_ = mem_attr; col_cnt_ = row_meta.col_cnt_; max_batch_size_ = max_batch_size; - CK (!row_meta.fixed_expr_reordered()) - row_meta_ = row_meta; + if (!row_meta.fixed_expr_reordered()) { + row_meta_ = row_meta; + } else if (OB_ISNULL(allocator_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("allocator_ is null", K(ret)); + } else if (OB_FAIL(row_meta_.deep_copy(row_meta, allocator_))) { + LOG_WARN("deep copy row meta failed", K(ret)); + } OZ(ObTempBlockStore::init(mem_limit, enable_dump, mem_attr.tenant_id_, mem_attr.ctx_id_, mem_attr_.label_, compressor_type, enable_trunc)); inited_ = true; return ret; } -int ObTempRowStore::init_batch_ctx() +template +int ObTempRowStoreBase::init_batch_ctx() { int ret = OB_SUCCESS; const int64_t max_batch_size = max_batch_size_; @@ -593,7 +696,8 @@ int ObTempRowStore::init_batch_ctx() return ret; } -int ObTempRowStore::add_batch(const common::ObIArray &exprs, ObEvalCtx &ctx, +template +int ObTempRowStoreBase::add_batch(const common::ObIArray &exprs, ObEvalCtx &ctx, const ObBatchRows &brs, int64_t &stored_rows_count, ObCompactRow **stored_rows, const int64_t start_pos /* 0 */) @@ -624,13 +728,57 @@ int ObTempRowStore::add_batch(const common::ObIArray &exprs, ObEvalCtx } OZ (add_batch(batch_ctx_->vectors_, batch_ctx_->selector_, size, stored_rows)); } - stored_rows_count = size; + if (OB_SUCC(ret)) { + stored_rows_count = size; + } return ret; } -int ObTempRowStore::try_add_batch(const common::ObIArray &exprs, ObEvalCtx *ctx, - const int64_t batch_size, const int64_t memory_limit, - bool &batch_added) +template +int ObTempRowStoreBase::add_batch(const common::ObIArray &exprs, + ObEvalCtx &ctx, const EvalBound &bound, + const ObBitVector &skip, int64_t &stored_rows_count, + ObCompactRow **stored_rows) +{ + int ret = OB_SUCCESS; + int16_t size = 0; + if (OB_FAIL(init_batch_ctx())) { + LOG_WARN("init batch ctx failed", K(ret)); + } else { + for (int i = bound.start(); i < bound.end(); i++) { + if (skip.at(i)) { + continue; + } else { + batch_ctx_->selector_[size++] = i; + } + } + } + if (OB_SUCC(ret) && size > 0) { + for (int i = 0; OB_SUCC(ret) && i < exprs.count(); i++) { + ObExpr *e = exprs.at(i); + ObIVector *vec = nullptr; + if (OB_FAIL(e->eval_vector(ctx, skip, bound))) { + LOG_WARN("eval vector failed", K(ret)); + } else { + vec = e->get_vector(ctx); + batch_ctx_->vectors_.at(i) = vec; + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(add_batch(batch_ctx_->vectors_, batch_ctx_->selector_, size, stored_rows))) { + LOG_WARN("add batch rows failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + stored_rows_count = size; + } + return ret; +} + +template <> +int ObTempRowStoreBase::try_add_batch(const common::ObIArray &exprs, + ObEvalCtx *ctx, const int64_t batch_size, + const int64_t memory_limit, bool &batch_added) { int ret = OB_SUCCESS; int64_t rows_size = 0; @@ -694,7 +842,27 @@ int ObTempRowStore::try_add_batch(const common::ObIArray &exprs, ObEval return ret; } -int ObTempRowStore::try_add_batch(const ObCompactRow **stored_rows, +template +int ObTempRowStoreBase::add_row(const ObCompactRow *src_row, ObCompactRow *&stored_row) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(init_batch_ctx())) { + LOG_WARN("init batch ctx failed", K(ret)); + } else if (OB_ISNULL(src_row) || src_row->get_row_size() <= 0) { + } else if (OB_FAIL(ensure_write_blk(src_row->get_row_size()))) { + LOG_WARN("ensure write block failed", K(ret), K(src_row->get_row_size())); + } else if (OB_FAIL(cur_blk()->add_row(blk_buf_, src_row, stored_row))) { + LOG_WARN("fail to add row", K(ret)); + } else { + block_id_cnt_ += 1; + inc_mem_used(src_row->get_row_size()); + } + return ret; +} + + +template<> +int ObTempRowStoreBase::try_add_batch(const ObCompactRow **stored_rows, const int64_t batch_size, const int64_t memory_limit, bool &batch_added) @@ -723,45 +891,31 @@ int ObTempRowStore::try_add_batch(const ObCompactRow **stored_rows, return ret; } -int ObTempRowStore::add_row(const common::ObIArray &exprs, +template +int ObTempRowStoreBase::add_row(const common::ObIArray &exprs, ObEvalCtx &ctx, ObCompactRow *&stored_row) { int ret = OB_SUCCESS; int64_t row_size = 0; + const int64_t idx_size = RA ? ROW_INDEX_SIZE : 0; if (OB_FAIL(init_batch_ctx())) { LOG_WARN("init batch ctx failed", K(ret)); } else if (OB_FAIL(RowBlock::calc_row_size(exprs, row_meta_, ctx, row_size))) { LOG_WARN("fail to calc row size", K(ret)); - } else if (OB_FAIL(ensure_write_blk(row_size))) { - LOG_WARN("ensure write block failed", K(ret), K(row_size)); - } else if (OB_FAIL(cur_blk_->add_row(exprs, row_meta_, ctx, stored_row))) { + } else if (OB_FAIL(ensure_write_blk(row_size + idx_size))) { + LOG_WARN("ensure write block failed", K(ret), K(row_size + idx_size)); + } else if (OB_FAIL(cur_blk()->add_row(blk_buf_, exprs, row_meta_, ctx, stored_row))) { LOG_WARN("fail to add row", K(ret)); } else { block_id_cnt_ += 1; - inc_mem_used(row_size); + inc_mem_used(row_size + idx_size); } return ret; } -int ObTempRowStore::add_row(const ObCompactRow *src_row, ObCompactRow *&stored_row) -{ - int ret = OB_SUCCESS; - if (OB_FAIL(init_batch_ctx())) { - LOG_WARN("init batch ctx failed", K(ret)); - } else if (OB_ISNULL(src_row) || src_row->get_row_size() <= 0) { - } else if (OB_FAIL(ensure_write_blk(src_row->get_row_size()))) { - LOG_WARN("ensure write block failed", K(ret), K(src_row->get_row_size())); - } else if (OB_FAIL(cur_blk_->add_row(src_row, stored_row))) { - LOG_WARN("fail to add row", K(ret)); - } else { - block_id_cnt_ += 1; - inc_mem_used(src_row->get_row_size()); - } - return ret; -} - -int ObTempRowStore::add_row(const common::ObIArray &exprs, const int64_t batch_idx, +template +int ObTempRowStoreBase::add_row(const common::ObIArray &exprs, const int64_t batch_idx, ObEvalCtx &ctx, ObCompactRow *&stored_row) { int ret = OB_SUCCESS; @@ -770,7 +924,8 @@ int ObTempRowStore::add_row(const common::ObIArray &exprs, const int64 return add_row(exprs, ctx, stored_row); } -int ObTempRowStore::add_batch(const IVectorPtrs &vectors, +template +int ObTempRowStoreBase::add_batch(const IVectorPtrs &vectors, const uint16_t selector[], const int64_t size, ObCompactRow **stored_rows, @@ -791,10 +946,11 @@ int ObTempRowStore::add_batch(const IVectorPtrs &vectors, for (int64_t i = 0; i < size; i++) { batch_mem_size += batch_ctx_->row_size_array_[i]; } + batch_mem_size += size * (RA ? ROW_INDEX_SIZE : 0); if (OB_FAIL(ensure_write_blk(batch_mem_size))) { LOG_WARN("ensure write block failed", K(ret)); - } else if (OB_FAIL(cur_blk_->add_batch(vectors, row_meta_, selector, size, - batch_ctx_->row_size_array_, batch_mem_size, rows))) { + } else if (OB_FAIL(cur_blk()->add_batch(blk_buf_, vectors, row_meta_, selector, size, + batch_ctx_->row_size_array_, batch_mem_size, rows))) { LOG_WARN("fail to add batch", K(ret)); } else { block_id_cnt_ += size; @@ -804,7 +960,8 @@ int ObTempRowStore::add_batch(const IVectorPtrs &vectors, return ret; } -int ObTempRowStore::RowBlock::vector_to_nulls(const sql::RowMeta &row_meta, +template +int ObTempRowStoreBase::RowBlock::vector_to_nulls(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, const uint16_t *selector, const int64_t size, const int64_t col_idx) @@ -817,7 +974,7 @@ int ObTempRowStore::RowBlock::vector_to_nulls(const sql::RowMeta &row_meta, return ret; } -OB_DEF_SERIALIZE(ObTempRowStore) +OB_DEF_SERIALIZE(ObTempRowStoreBase, template ) { int ret = ObTempBlockStore::serialize(buf, buf_len, pos); LST_DO_CODE(OB_UNIS_ENCODE, @@ -828,7 +985,7 @@ OB_DEF_SERIALIZE(ObTempRowStore) } -OB_DEF_DESERIALIZE(ObTempRowStore) +OB_DEF_DESERIALIZE(ObTempRowStoreBase, template ) { int ret = ObTempBlockStore::deserialize(buf, data_len, pos); if (OB_SUCC(ret)) { @@ -849,7 +1006,7 @@ OB_DEF_DESERIALIZE(ObTempRowStore) return ret; } -OB_DEF_SERIALIZE_SIZE(ObTempRowStore) +OB_DEF_SERIALIZE_SIZE(ObTempRowStoreBase, template ) { int64_t len = ObTempBlockStore::get_serialize_size(); LST_DO_CODE(OB_UNIS_ADD_LEN, @@ -859,5 +1016,9 @@ OB_DEF_SERIALIZE_SIZE(ObTempRowStore) return len; } +template class ObTempRowStoreBase; +template class ObTempRowStoreBase; + +#undef ROW_BLK } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/engine/basic/ob_temp_row_store.h b/src/sql/engine/basic/ob_temp_row_store.h index 783ee6b0f4..d2705658be 100644 --- a/src/sql/engine/basic/ob_temp_row_store.h +++ b/src/sql/engine/basic/ob_temp_row_store.h @@ -24,27 +24,63 @@ namespace oceanbase namespace sql { -class ObTempRowStore : public ObTempBlockStore +template +class ObTempRowStoreBase : public ObTempBlockStore { OB_UNIS_VERSION_V(1); public: + class ReaderBase; class Iterator; - // RowBlock provides functions for writing and reading, and does not occupy memory + class RAReader; + typedef uint32_t row_idx_t; + const static int64_t ROW_INDEX_SIZE = sizeof(row_idx_t); + /* + * RowBlock provides functions for writing and reading + * + * RA = false means supports sequential access, the payload layout when RA = false is as follows: + * + * +-----------------------------------------------------------------------------------------+ + * | compact_row1(variable_length) | compact_row2 |...| + * +-----------------------------------------------------------------------------------------+ + * + * Compact rows are variable-length and are read or written from head to tail. + * + * + * RA = true means suppport Random Access, the payload layout when RA = true is as follows: + * get_buffer() ____________________________________________________________________ + * | + * V + * +-----------------------------------------------------------------------------------------+ + * | compact_row1(variable_length) | compact_row2 |...| |...| pos2 | pos1 | buf | + * +-----------------------------------------------------------------------------------------+ + * ^ ^ | | + * | |______________________________________| | + * |_____________________________________________________________________________| + * + * Compact rows are variable-length and are written from head to tail. The index area at + * the tial that records the position of each compact row is written from tail to head. + * + * The buffer is at the end of payload, the begin of buffer is the end of indexes. + * During random reading, use get_buffer() to find the end of indexes, then get the position by + * index, and finally obtain the compact row based on the position. + */ struct RowBlock : public Block { - int add_row(const common::ObIArray &exprs, + int add_row(ShrinkBuffer &buf, + const common::ObIArray &exprs, const RowMeta &row_meta, ObEvalCtx &ctx, ObCompactRow *&stored_row); - int add_row(const ObCompactRow *src_row, ObCompactRow *&stored_row); - int add_batch(const IVectorPtrs &vectors, + int add_row(ShrinkBuffer &buf, const ObCompactRow *src_row, ObCompactRow *&stored_row); + int add_batch(ShrinkBuffer &buf, + const IVectorPtrs &vectors, const RowMeta &row_meta, const uint16_t selector[], const int64_t size, const uint32_t row_size_arr[], int64_t batch_mem_size, ObCompactRow **stored_rows); - int get_next_batch(ObTempRowStore::Iterator &iter, + int get_next_batch(ObTempRowStoreBase::ReaderBase &iter, const int64_t max_rows, int64_t &read_rows, const ObCompactRow **stored_rows) const; @@ -60,34 +96,52 @@ public: int64_t &size); int32_t rows() const { return cnt_; } int get_store_row(int64_t &cur_pos, const ObCompactRow *&sr); + int get_row(const int64_t row_id, const ObCompactRow *&sr) const; private: static int vector_to_nulls(const sql::RowMeta &row_meta, sql::ObCompactRow **stored_rows, const uint16_t selector[], const int64_t size, const int64_t col_idx); + inline int post_add_row(ShrinkBuffer &buf, const int32_t row_size) + { + int ret = OB_SUCCESS; + if (!RA) { + // do nothing + } else if (OB_FAIL(buf.fill_tail(ROW_INDEX_SIZE))) { + SQL_LOG(WARN, "fill buffer tail failed", K(ret), K(buf), LITERAL_K(ROW_INDEX_SIZE)); + } else { + *reinterpret_cast(buf.tail()) = static_cast(buf.head() - payload_); + } + buf.fast_advance(row_size); + return ret; + } + int32_t get_row_location(const int64_t row_id) const; }; struct DtlRowBlock : public RowBlock { + ObTempBlockStore::ShrinkBuffer *get_buffer() { + return static_cast( + static_cast(reinterpret_cast(this) + this->buf_off_)); + } static int calc_rows_size(const IVectorPtrs &vectors, const RowMeta &row_meta, const ObBatchRows &brs, uint32_t row_size_arr[]); - }; const static int64_t BLOCK_SIZE = (64L << 10); - class Iterator : public ObTempBlockStore::BlockReader + + class ReaderBase : public ObTempBlockStore::BlockReader { public: friend struct RowBlock; - friend class ObTempRowStore; - Iterator() : ObTempBlockStore::BlockReader(), row_store_(NULL), cur_blk_(NULL), + friend class ObTempRowStoreBase; + ReaderBase() : ObTempBlockStore::BlockReader(), row_store_(NULL), cur_blk_(NULL), cur_blk_id_(0), row_idx_(0), read_pos_(0) {} - virtual ~Iterator() {} + virtual ~ReaderBase() {} - int init(ObTempRowStore *store); + int init(ObTempRowStoreBase *store); bool is_valid() { return nullptr != row_store_; } - inline bool has_next() const { return cur_blk_id_ < get_row_cnt(); } inline int64_t get_row_cnt() const { return row_store_->get_row_cnt(); } int get_next_batch(const ObExprPtrIArray &exprs, ObEvalCtx &ctx, @@ -97,17 +151,12 @@ public: int get_next_batch(const int64_t max_rows, int64_t &read_rows, const ObCompactRow **stored_rows); - static int attach_rows(const ObExprPtrIArray &exprs, - ObEvalCtx &ctx, - const RowMeta &row_meta, - const ObCompactRow **srows, - const int64_t read_rows); void reset() { cur_blk_id_ = 0; + cur_blk_ = NULL; row_idx_ = 0; read_pos_ = 0; - cur_blk_ = NULL; ObTempBlockStore::BlockReader::reset(); } inline sql::RowMeta &get_row_meta() const { return row_store_->row_meta_; } @@ -117,18 +166,51 @@ public: const int64_t max_rows, int64_t &read_rows, const ObCompactRow **stored_rows = NULL); - static int attach_rows(const RowMeta &row_meta, const ObCompactRow **srows, - const int64_t read_rows, const IVectorPtrs &vectors); int next_block(); private: - ObTempRowStore *row_store_; + ObTempRowStoreBase *row_store_; const RowBlock *cur_blk_; int64_t cur_blk_id_; // block id(row_id) for iterator, from 0 to row_cnt_ int32_t row_idx_; // current row index in reader block int32_t read_pos_; // current memory read position in reader block }; + class Iterator : public ReaderBase + { + public: + Iterator() {} + virtual ~Iterator() {} + + inline bool has_next() const { return this->cur_blk_id_ < this->get_row_cnt(); } + static int attach_rows(const ObExprPtrIArray &exprs, + ObEvalCtx &ctx, + const RowMeta &row_meta, + const ObCompactRow **srows, + const int64_t read_rows); + }; + + + class RAReader : public ReaderBase { + friend class ObTempRowStoreBase; + public: + explicit RAReader() {} + virtual ~RAReader() { this->reset(); } + int get_row(const int64_t row_id, const ObCompactRow *&sr); + + int get_batch_rows(const ObExprPtrIArray &exprs, + ObEvalCtx &ctx, + const int64_t start_idx, + const int64_t end_idx, + int64_t &read_rows, + const ObCompactRow **stored_rows); + int get_batch_rows(const int64_t start_idx, const int64_t end_idx, int64_t &read_rows, + const ObCompactRow **stored_rows); // TODO: + + private: + DISALLOW_COPY_AND_ASSIGN(RAReader); + }; + struct BatchCtx { ~BatchCtx() { @@ -145,9 +227,9 @@ public: }; public: - explicit ObTempRowStore(common::ObIAllocator *alloc = NULL); + explicit ObTempRowStoreBase(common::ObIAllocator *alloc = NULL); - virtual ~ObTempRowStore(); + virtual ~ObTempRowStoreBase(); void destroy(); void reset(); @@ -157,17 +239,17 @@ public: const int64_t mem_limit, bool enable_dump, uint32_t row_extra_size, + const common::ObCompressorType compressor_type, const bool reorder_fixed_expr = true, - const bool enable_trunc = false, - const common::ObCompressorType compressor_type = NONE_COMPRESSOR); + const bool enable_trunc = false); int init(const RowMeta &row_meta, const int64_t max_batch_size, const lib::ObMemAttr &mem_attr, const int64_t mem_limit, bool enable_dump, - const bool enable_trunc = false, - const common::ObCompressorType compressor_type = NONE_COMPRESSOR); + const common::ObCompressorType compressor_type, + const bool enable_trunc = false); int init_batch_ctx(); @@ -180,6 +262,9 @@ public: const ObBatchRows &brs, int64_t &stored_rows_count, ObCompactRow **stored_rows = NULL, const int64_t start_pos = 0); + int add_batch(const common::ObIArray &exprs, ObEvalCtx &ctx, const EvalBound &bound, + const ObBitVector &skip, int64_t &stored_rows_count, + ObCompactRow **stored_rows); // TODO int add_row(const common::ObIArray &exprs, ObEvalCtx &ctx, ObCompactRow *&stored_row); @@ -220,20 +305,31 @@ public: inline int ensure_write_blk(const int64_t mem_size) { int ret = common::OB_SUCCESS; - if (NULL == cur_blk_ || mem_size > cur_blk_->remain()) { + if (NULL == blk_ || mem_size > blk_buf_.remain()) { if (OB_FAIL(new_block(mem_size))) { SQL_ENG_LOG(WARN, "fail to new block", K(ret), K(mem_size)); - } else { - cur_blk_ = static_cast(blk_); } } return ret; } + + virtual int prepare_blk_for_switch(Block *blk) override + { + int ret = OB_SUCCESS; + if (!RA) { + // do nothing, non ra store dont need compact + } else if (OB_FAIL(blk_buf_.compact())) { + SQL_LOG(WARN, "fail to compact block", K(ret)); + } + return ret; + } + const RowMeta &get_row_meta() const { return row_meta_; } inline int64_t get_max_batch_size() const { return max_batch_size_; } inline int64_t get_row_cnt() const { return block_id_cnt_; } inline int64_t get_row_cnt_on_disk() const { return dumped_block_id_cnt_; } inline int64_t get_row_cnt_in_memory() const { return get_row_cnt() - get_row_cnt_on_disk(); } + const lib::ObMemAttr &get_mem_attr() const { return mem_attr_; } INHERIT_TO_STRING_KV("ObTempBlockStore", ObTempBlockStore, K_(mem_attr), @@ -241,16 +337,22 @@ public: K_(row_meta), K_(max_batch_size)); +private: + inline RowBlock *cur_blk() { + return reinterpret_cast(blk_); + } private: lib::ObMemAttr mem_attr_; - RowBlock *cur_blk_; int64_t col_cnt_; BatchCtx *batch_ctx_; RowMeta row_meta_; int64_t max_batch_size_; }; +using ObRATempRowStore = ObTempRowStoreBase; +using ObTempRowStore = ObTempRowStoreBase; + } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp b/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp index 3e34fad761..57f14b22ca 100644 --- a/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp +++ b/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp @@ -216,7 +216,8 @@ int ObTempTableInsertVecOp::create_interm_result_info(ObDTLIntermResultInfo *&in mem_attr, 0 /*mem_limit*/, true /*enable_dump*/, - false /*reuse_vector_array*/))) { + false /*reuse_vector_array*/, + MY_SPEC.compress_type_))) { LOG_WARN("failed to init the chunk row store.", K(ret)); } else if (OB_FAIL(all_interm_res_info_.push_back(interm_res_info))) { LOG_WARN("failed to push back row store", K(ret)); diff --git a/src/sql/engine/basic/ob_vector_result_holder.cpp b/src/sql/engine/basic/ob_vector_result_holder.cpp index 93c092b504..439f0245d5 100644 --- a/src/sql/engine/basic/ob_vector_result_holder.cpp +++ b/src/sql/engine/basic/ob_vector_result_holder.cpp @@ -256,9 +256,9 @@ int ObVectorsResultHolder::init(const common::ObIArray &exprs, ObEvalC exprs_ = &exprs; eval_ctx_ = &eval_ctx; int64_t batch_size = eval_ctx.max_batch_size_; - if (OB_ISNULL(backup_cols_ = static_cast - (eval_ctx.exec_ctx_.get_allocator().alloc(sizeof(ObColResultHolder) - * exprs.count())))) { + ObIAllocator &allocator = (tmp_alloc_ != nullptr ? *tmp_alloc_ : eval_ctx_->exec_ctx_.get_allocator()); + if (OB_ISNULL(backup_cols_ = static_cast( + allocator.alloc(sizeof(ObColResultHolder) * exprs.count())))) { ret = OB_ALLOCATE_MEMORY_FAILED; LOG_WARN("failed to alloc ptrs", K(ret)); } else { @@ -281,7 +281,7 @@ int ObVectorsResultHolder::save(const int64_t batch_size) ret = OB_NOT_INIT; } else { saved_size_ = batch_size; - ObIAllocator &alloc = eval_ctx_->exec_ctx_.get_allocator(); + ObIAllocator &alloc = (tmp_alloc_ != nullptr ? *tmp_alloc_ : eval_ctx_->exec_ctx_.get_allocator()); for (int64_t i = 0; OB_SUCC(ret) && i < exprs_->count(); ++i) { if (OB_FAIL(backup_cols_[i].header_.assign(exprs_->at(i)->get_vector_header(*eval_ctx_)))) { LOG_WARN("failed to assign vector", K(ret)); @@ -375,5 +375,82 @@ int ObVectorsResultHolder::restore() const return ret; } +#define CALC_FIXED_COL(vec_tc) \ + case (vec_tc): { \ + mem_size += sizeof(RTCType) * batch_size; \ + } break + +template +int ObVectorsResultHolder::calc_col_backup_size(ObExpr *expr, int32_t batch_size, int32_t &mem_size) +{ + int ret = OB_SUCCESS; + mem_size = 0; + if (fmt == VEC_UNIFORM || fmt == VEC_UNIFORM_CONST) { + int32_t copy_size = (fmt == VEC_UNIFORM_CONST ? 1 : batch_size); + mem_size += sizeof(ObDatum) * copy_size; + bool need_copy_rev_buf = (expr->is_fixed_length_data_ + || ObNumberTC == ob_obj_type_class(expr->datum_meta_.get_type())); + if (need_copy_rev_buf) { + mem_size += expr->res_buf_len_ * copy_size; + } + } else { + // bitmap + mem_size += sql::ObBitVector::memory_size(batch_size); + if (fmt == VEC_FIXED) { + switch(expr->get_vec_value_tc()) { + LST_DO_CODE(CALC_FIXED_COL, FIXED_VEC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected vector class", K(ret), K(expr->get_vec_value_tc())); + } + } + } else if (fmt == VEC_DISCRETE) { + mem_size += (sizeof(ObLength) + sizeof(char *)) * batch_size; + } else if (fmt == VEC_CONTINUOUS) { + mem_size += sizeof(uint32_t) * (batch_size + 1); + } + } + return ret; +} + +#define CALC_COL_BACKUP_SIZE(fmt) \ + do { \ + if (OB_FAIL(calc_col_backup_size(exprs.at(i), eval_ctx.max_batch_size_, col_mem_size))) { \ + LOG_WARN("calc col backup size failed", K(ret)); \ + } else { \ + max_col_mem_size = std::max(col_mem_size, max_col_mem_size); \ + } \ + } while (false) + +int ObVectorsResultHolder::calc_backup_size(const common::ObIArray &exprs, + ObEvalCtx &eval_ctx, int32_t &mem_size) +{ + int ret = OB_SUCCESS; + mem_size = 0; + if (OB_LIKELY(exprs.count() > 0)) { + // ObColResultHolder + mem_size += sizeof(ObColResultHolder) * exprs.count(); + // VectorHeader + mem_size += sizeof(VectorHeader) * exprs.count(); + for (int i = 0; OB_SUCC(ret) && i < exprs.count(); i++) { + int32_t max_col_mem_size = 0, col_mem_size = 0; + if (!exprs.at(i)->is_batch_result()) { + CALC_COL_BACKUP_SIZE(VEC_UNIFORM_CONST); + } else if (is_fixed_length_vec(exprs.at(i)->get_vec_value_tc())) { + LST_DO_CODE(CALC_COL_BACKUP_SIZE, VEC_FIXED, VEC_UNIFORM); + } else { + LST_DO_CODE(CALC_COL_BACKUP_SIZE, VEC_DISCRETE, VEC_CONTINUOUS, VEC_UNIFORM); + } + if (OB_SUCC(ret)) { + mem_size += max_col_mem_size; + } + } + } + return ret; +} + +#undef CALC_FIXED_COL +#undef CALC_COL_BACKUP_SIZE + } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/engine/basic/ob_vector_result_holder.h b/src/sql/engine/basic/ob_vector_result_holder.h index 92d280739a..f0fa30eb98 100644 --- a/src/sql/engine/basic/ob_vector_result_holder.h +++ b/src/sql/engine/basic/ob_vector_result_holder.h @@ -37,13 +37,18 @@ namespace sql class ObVectorsResultHolder { public: - ObVectorsResultHolder() : exprs_(nullptr), eval_ctx_(nullptr), - inited_(false), backup_cols_(nullptr), saved_(false), saved_size_(0) {} + ObVectorsResultHolder(ObIAllocator *tmp_alloc = nullptr) : + exprs_(nullptr), eval_ctx_(nullptr), inited_(false), backup_cols_(nullptr), saved_(false), + saved_size_(0), tmp_alloc_(tmp_alloc) + {} int init(const common::ObIArray &exprs, ObEvalCtx &eval_ctx); int save(const int64_t batch_size); int restore() const; void reset() { } + static int calc_backup_size(const common::ObIArray &exprs, ObEvalCtx &eval_ctx, int32_t &mem_size); private: + template + static int calc_col_backup_size(ObExpr *expr, int32_t batch_size, int32_t &mem_size); struct ObColResultHolder { ObColResultHolder(int64_t max_batch_size, const ObExpr *expr) : @@ -116,6 +121,7 @@ private: ObColResultHolder *backup_cols_; bool saved_; int64_t saved_size_; + ObIAllocator *tmp_alloc_; }; diff --git a/src/sql/engine/expr/ob_expr_eval_functions.cpp b/src/sql/engine/expr/ob_expr_eval_functions.cpp index e02aa412ba..81e0b0c9ca 100644 --- a/src/sql/engine/expr/ob_expr_eval_functions.cpp +++ b/src/sql/engine/expr/ob_expr_eval_functions.cpp @@ -375,6 +375,8 @@ #include "ob_expr_st_symdifference.h" #include "ob_expr_priv_st_asmvtgeom.h" #include "ob_expr_priv_st_makevalid.h" +#include "ob_expr_func_ceil.h" +#include "ob_expr_topn_filter.h" #include "ob_expr_sdo_relate.h" #include "ob_expr_inner_table_option_printer.h" #include "ob_expr_password.h" @@ -1180,7 +1182,7 @@ static ObExpr::EvalFunc g_expr_eval_functions[] = { NULL, // ObExprWaitUntilSQLThreadAfterGTIDs::eval_wait_until_sql_thread_after_gtids /* 689 */ ObExprLastRefreshScn::eval_last_refresh_scn, /* 690 */ ObExprDocLength::generate_doc_length, /* 691 */ - NULL, // ObExprTopNFilter::eval_topn_filter, /* 692 */ + ObExprTopNFilter::eval_topn_filter, /* 692 */ NULL, // ObExprIsEnabledRole::eval_is_enabled_role, /* 693 */ NULL, // ObExprCanAccessTrigger::can_access_trigger, /* 694 */ NULL, //ObRelationalExprOperator::eval_min_max_compare, /* 695 */ @@ -1354,7 +1356,7 @@ static ObExpr::EvalBatchFunc g_expr_eval_batch_functions[] = { ObBatchCast::explicit_batch_cast, /* 127 */ ObBatchCast::implicit_batch_cast, /* 128 */ NULL,//ObExprDecodeTraceId::calc_decode_trace_id_expr_batch, /* 129 */ - NULL,//ObExprTopNFilter::eval_topn_filter_batch, /* 130 */ + ObExprTopNFilter::eval_topn_filter_batch, /* 130 */ NULL,//ObRelationalExprOperator::eval_batch_min_max_compare, /* 131 */ NULL,//ObExprBM25::eval_batch_bm25_relevance_expr, /* 132 */ }; @@ -1472,10 +1474,10 @@ static ObExpr::EvalVectorFunc g_expr_eval_vector_functions[] = { ObExprInOrNotIn::eval_vector_in_without_row_fallback, /* 109 */ ObExprInOrNotIn::eval_vector_in_without_row, /* 110 */ NULL,//ObExprDecodeTraceId::calc_decode_trace_id_expr_vector /* 111 */ - NULL,//ObExprTopNFilter::eval_topn_filter_vector, /* 112 */ + ObExprTopNFilter::eval_topn_filter_vector, /* 112 */ NULL,//ObRelationalExprOperator::eval_vector_min_max_compare, /* 113 */ - NULL,//ObExprCeilFloor::calc_ceil_floor_vector, /* 114 */ - NULL,//ObExprRepeat::eval_repeat_vector, /* 115 */ + ObExprCeilFloor::calc_ceil_floor_vector, /* 114 */ + ObExprRepeat::eval_repeat_vector, /* 115 */ }; REG_SER_FUNC_ARRAY(OB_SFA_SQL_EXPR_EVAL, diff --git a/src/sql/engine/expr/ob_expr_func_ceil.cpp b/src/sql/engine/expr/ob_expr_func_ceil.cpp index 41dc1fb437..4781805561 100644 --- a/src/sql/engine/expr/ob_expr_func_ceil.cpp +++ b/src/sql/engine/expr/ob_expr_func_ceil.cpp @@ -427,6 +427,218 @@ int eval_batch_ceil_floor(const ObExpr &expr, return ret; } +int ObExprCeilFloor::calc_ceil_floor_vector(const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const EvalBound &bound) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(expr.args_[0]->eval_vector(ctx, skip, bound))) { + LOG_WARN("eval arg failed", K(ret), K(expr)); + } else { + VectorFormat res_format = expr.get_format(ctx); + VectorFormat left_format = expr.args_[0]->get_format(ctx); + const ObObjType arg_type = expr.args_[0]->datum_meta_.type_; + const ObObjType res_type = expr.datum_meta_.type_; + int input_type = 0; + if (ObNumberTC == ob_obj_type_class(arg_type)) { + input_type = NUMBER_TYPE; + } else if (ob_is_integer_type(arg_type)) { + input_type = INTEGER_TYPE; + } else if (ObFloatType == arg_type) { + input_type = FLOAT_TYPE; + } else if (ObDoubleType == arg_type) { + input_type = DOUBLE_TYPE; + } else if (ObDecimalIntType == arg_type) { + input_type = DECIMAL_INT_TYPE; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected arg type", K(ret), K(arg_type)); + } + VecValueTypeClass arg_tc = get_vec_value_tc(expr.args_[0]->datum_meta_.type_, expr.args_[0]->datum_meta_.scale_, + expr.args_[0]->datum_meta_.precision_); + if (OB_SUCC(ret)) { + if (res_format == VEC_FIXED && left_format == VEC_FIXED) { + if (arg_tc == VEC_TC_INTEGER || arg_tc == VEC_TC_UINTEGER) { + ret = inner_calc_ceil_floor_vector>, ObFixedLengthFormat>> + (expr, input_type, ctx, skip, bound); + } else if (arg_tc == VEC_TC_DOUBLE || arg_tc == VEC_TC_FIXED_DOUBLE) { + ret = inner_calc_ceil_floor_vector>, ObFixedLengthFormat>> + (expr, input_type, ctx, skip, bound); + } else if (ObDecimalIntType == arg_type) { + // if use ObFixedLengthFormat, don't know length of decimal_int + // use ObFixedLengthBase, don't need length + ret = inner_calc_ceil_floor_vector(expr, input_type, ctx, skip, bound); + } else { + ret = inner_calc_ceil_floor_vector(expr, input_type, ctx, skip, bound); + } + } else { + // float type input is uniform....., output is fixed format... + ret = inner_calc_ceil_floor_vector(expr, input_type, ctx, skip, bound); + } + } + } + return ret; +} + +#define CHECK_CEIL_VECTOR() \ +if (IsCheck) { \ + if (skip.at(j) || eval_flags.at(j)) { \ + continue; \ + } else if (left_vec->is_null(j)) { \ + res_vec->set_null(j); \ + eval_flags.set(j); \ + continue; \ + } \ +} + +template +static int do_eval_ceil_floor_vector(const ObExpr &expr,ObEvalCtx &ctx, + int input_type, + const ObBitVector &skip, + const EvalBound &bound) +{ + int ret = OB_SUCCESS; + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + ResVec *res_vec = static_cast(expr.get_vector(ctx)); + LeftVec *left_vec = static_cast(expr.args_[0]->get_vector(ctx)); + + switch (input_type) { + case NUMBER_TYPE: { + for (int64_t j = bound.start(); OB_SUCC(ret) && j < bound.end(); ++j) { + CHECK_CEIL_VECTOR(); + const number::ObNumber arg_nmb(left_vec->get_number(j)); + number::ObNumber res_nmb; + ObNumStackOnceAlloc tmp_alloc; + if (OB_FAIL(res_nmb.from(arg_nmb, tmp_alloc))) { + LOG_WARN("get number from arg failed", K(ret), K(arg_nmb)); + } else { + if (IS_FLOOR) { + if (OB_FAIL(res_nmb.floor(0))) { + LOG_WARN("calc floor for number failed", K(ret), K(res_nmb)); + } + } else { + if (OB_FAIL(res_nmb.ceil(0))) { + LOG_WARN("calc ceil for number failed", K(ret), K(res_nmb)); + } + } + const ObObjType res_type = expr.datum_meta_.type_; + if (OB_FAIL(ret)) { + } else if (ObNumberTC == ob_obj_type_class(res_type)) { + res_vec->set_number(j, res_nmb); + eval_flags.set(j); + } else if (ob_is_integer_type(res_type)) { + int64_t res_int = 0; + if (res_nmb.is_valid_int64(res_int)) { + res_vec->set_int(j, res_int); + eval_flags.set(j); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected res type", K(ret), K(res_type)); + } + } + } + } + break; + } + case INTEGER_TYPE: { + for (int64_t j = bound.start(); OB_SUCC(ret) && j < bound.end(); ++j) { + CHECK_CEIL_VECTOR(); + res_vec->set_int(j, left_vec->get_int(j)); + eval_flags.set(j); + } + break; + } + case FLOAT_TYPE: { + for (int64_t j = bound.start(); OB_SUCC(ret) && j < bound.end(); ++j) { + CHECK_CEIL_VECTOR(); + if (IS_FLOOR) { + res_vec->set_float(j, floorf(left_vec->get_float(j))); + } else { + res_vec->set_float(j, ceilf(left_vec->get_float(j))); + } + eval_flags.set(j); + } + break; + } + case DOUBLE_TYPE: { + for (int64_t j = bound.start(); OB_SUCC(ret) && j < bound.end(); ++j) { + CHECK_CEIL_VECTOR(); + if (IS_FLOOR) { + res_vec->set_double(j, floor(left_vec->get_double(j))); + } else { + res_vec->set_double(j, ceil(left_vec->get_double(j))); + } + eval_flags.set(j); + } + break; + } + case DECIMAL_INT_TYPE: { + const ObDatumMeta &in_meta = expr.args_[0]->datum_meta_; + const ObDatumMeta &out_meta = expr.datum_meta_; + for (int64_t j = bound.start(); OB_SUCC(ret) && j < bound.end(); ++j) { + CHECK_CEIL_VECTOR(); + ObDatum left_datum(left_vec->get_payload(j), left_vec->get_length(j), false); + ObDatum res_datum(res_vec->get_payload(j), res_vec->get_length(j), false); + if (OB_FAIL(ObExprCeilFloor::ceil_floor_decint(IS_FLOOR, &left_datum, in_meta, out_meta, res_datum))) { + LOG_WARN("do_round_decimalint failed", K(ret), K(in_meta), K(out_meta), K(left_datum), K(res_datum)); + } + res_vec->set_decimal_int(j, res_datum.get_decimal_int(), res_datum.get_int_bytes()); + eval_flags.set(j); + } + break; + } + default: + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected arg type", K(ret), K(input_type)); + break; + } + return ret; +} + +uint64_t bitwise_or(uint64_t l, uint64_t r) { + return l | r; +} + +template +static bool if_vector_need_cal_all(LeftVec *left_vec, + const ObBitVector &skip, + const ObBitVector &eval_flags, + const EvalBound &bound) +{ + bool is_need = ObBitVector::bit_op_zero(skip, eval_flags, bound, bitwise_or); + for (int64_t j = bound.start(); is_need && j < bound.end(); ++j) { + is_need = !(left_vec->is_null(j)); + } + return is_need; +} + +template +int ObExprCeilFloor::inner_calc_ceil_floor_vector(const ObExpr &expr, + int input_type, + ObEvalCtx &ctx, + const ObBitVector &skip, + const EvalBound &bound) +{ + int ret = OB_SUCCESS; + LeftVec *left_vec = static_cast(expr.args_[0]->get_vector(ctx)); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); // to do for skip + const ObObjType arg_type = expr.args_[0]->datum_meta_.type_; + const bool is_floor = (T_FUN_SYS_FLOOR == expr.type_) ? true : false; + const bool is_check = if_vector_need_cal_all(left_vec, skip, eval_flags, bound) ? false : true; + + if (is_check && is_floor) { + ret = do_eval_ceil_floor_vector(expr, ctx, input_type, skip, bound); + } else if (!is_check && is_floor) { + ret = do_eval_ceil_floor_vector(expr, ctx, input_type, skip, bound); + } else if (is_check && !is_floor) { + ret = do_eval_ceil_floor_vector(expr, ctx, input_type, skip, bound); + } else if (!is_check && !is_floor) { + ret = do_eval_ceil_floor_vector(expr, ctx, input_type, skip, bound); + } + return ret; +} + int ObExprCeilFloor::cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_expr, ObExpr &rt_expr) const { @@ -435,6 +647,7 @@ int ObExprCeilFloor::cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_expr UNUSED(raw_expr); rt_expr.eval_func_ = calc_ceil_floor; rt_expr.eval_batch_func_ = eval_batch_ceil_floor; + rt_expr.eval_vector_func_ = calc_ceil_floor_vector; return ret; } diff --git a/src/sql/engine/expr/ob_expr_func_ceil.h b/src/sql/engine/expr/ob_expr_func_ceil.h index f5179b9fe7..e572d3013c 100644 --- a/src/sql/engine/expr/ob_expr_func_ceil.h +++ b/src/sql/engine/expr/ob_expr_func_ceil.h @@ -43,6 +43,29 @@ public: const ObDatumMeta &in_meta, const ObDatumMeta &out_meta, ObDatum &res_datum); + template + static int inner_calc_ceil_floor_vector(const ObExpr &expr, + int intput_type, + ObEvalCtx &ctx, + const ObBitVector &skip, + const EvalBound &bound); + + static int calc_ceil_floor_vector(const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const EvalBound &bound); + + template + static int ceil_floor_decint_vector(const ObDatumMeta &in_meta, + const ObDatumMeta &out_meta, + LeftVec *left_vec, + ResVec *res_vec, + const int64_t &idx); + + static int inner_calc_ceil_floor_fixed_vector(const ObExpr &expr, + ObEvalCtx &ctx, + const ObBitVector &skip, + const EvalBound &bound); private: // disallow copy DISALLOW_COPY_AND_ASSIGN(ObExprCeilFloor); diff --git a/src/sql/engine/expr/ob_expr_join_filter.cpp b/src/sql/engine/expr/ob_expr_join_filter.cpp index 435cc27fe6..8666c99134 100644 --- a/src/sql/engine/expr/ob_expr_join_filter.cpp +++ b/src/sql/engine/expr/ob_expr_join_filter.cpp @@ -25,92 +25,68 @@ namespace oceanbase { namespace sql { +#define FILL_BATCH_RESULT() \ + if (OB_FAIL(ObBitVector::flip_foreach( \ + skip, batch_size, [&](int64_t idx) __attribute__((always_inline)) { \ + ++join_filter_ctx->n_times_; \ + eval_flags.set(idx); \ + results[idx].set_int(is_match); \ + ++join_filter_ctx->total_count_; \ + join_filter_ctx->collect_sample_info(!is_match, 1); \ + return OB_SUCCESS; \ + }))) {} -#define FILL_BATCH_RESULT() \ - if (OB_FAIL(ObBitVector::flip_foreach(skip, batch_size,\ - [&](int64_t idx) __attribute__((always_inline)) {\ - ++join_filter_ctx->n_times_;\ - int ret = OB_SUCCESS;\ - eval_flags.set(idx);\ - results[idx].set_int(is_match);\ - collect_sample_info(join_filter_ctx, is_match);\ - ++join_filter_ctx->total_count_;\ - return ret;\ - }))) {} -#define CHECK_MAX_WAIT_TIME() \ - int64_t cur_time = ObTimeUtility::current_time();\ - if (cur_time - join_filter_ctx->start_time_ >\ - join_filter_ctx->max_wait_time_ms_ * 1000) {\ - join_filter_ctx->need_wait_rf_ = false;\ - break;\ - } else {\ - ob_usleep(1000);\ - } +#define CHECK_MAX_WAIT_TIME() \ + int64_t cur_time = ObTimeUtility::current_time(); \ + if (cur_time - join_filter_ctx->start_time_ > join_filter_ctx->max_wait_time_ms_ * 1000) { \ + join_filter_ctx->need_wait_rf_ = false; \ + break; \ + } else { \ + ob_usleep(1000); \ + } template -static int proc_if_das(ResVec *res_vec, const ObBitVector &skip, const EvalBound &bound); +static int proc_by_pass(ResVec *res_vec, const ObBitVector &skip, const EvalBound &bound, + int64_t &valid_cnt, bool calc_valid_cnt = false); template <> -int proc_if_das(IntegerUniVec *res_vec, const ObBitVector &skip, - const EvalBound &bound) +int proc_by_pass(IntegerUniVec *res_vec, const ObBitVector &skip, + const EvalBound &bound, int64_t &valid_cnt, bool calc_valid_cnt) { int ret = OB_SUCCESS; - if (OB_FAIL(ObBitVector::flip_foreach( - skip, bound, [&](int64_t idx) __attribute__((always_inline)) { - res_vec->set_int(idx, 1); - return OB_SUCCESS; - }))) { - LOG_WARN("fail to do for each operation", K(ret)); + if (OB_LIKELY(calc_valid_cnt)) { + valid_cnt = 0; + if (OB_FAIL(ObBitVector::flip_foreach( + skip, bound, [&](int64_t idx) __attribute__((always_inline)) { + ++valid_cnt; + res_vec->set_int(idx, 1); + return OB_SUCCESS; + }))) { + LOG_WARN("fail to do for each operation", K(ret)); + } else { + } + } else { + if (OB_FAIL(ObBitVector::flip_foreach( + skip, bound, [&](int64_t idx) __attribute__((always_inline)) { + res_vec->set_int(idx, 1); + return OB_SUCCESS; + }))) { + LOG_WARN("fail to do for each operation", K(ret)); + } } return ret; } -template <> -int proc_if_das(IntegerFixedVec *res_vec, const ObBitVector &skip, - const EvalBound &bound) -{ - int ret = OB_SUCCESS; - uint64_t *data = reinterpret_cast(res_vec->get_data()); - MEMSET(data + bound.start(), 1, (bound.range_size() * res_vec->get_length(0))); - return ret; -} - -template -static int proc_by_pass(ResVec *res_vec, const ObBitVector &skip, const EvalBound &bound, - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx); - -template <> -int proc_by_pass(IntegerUniVec *res_vec, const ObBitVector &skip, - const EvalBound &bound, - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx) -{ - int ret = OB_SUCCESS; - int valid_cnt = 0; - if (OB_FAIL(ObBitVector::flip_foreach( - skip, bound, [&](int64_t idx) __attribute__((always_inline)) { - ++valid_cnt; - res_vec->set_int(idx, 1); - return OB_SUCCESS; - }))) {} - join_filter_ctx->n_times_ += valid_cnt; - join_filter_ctx->total_count_ += valid_cnt; - ObExprJoinFilter::collect_sample_info_batch(*join_filter_ctx, 0, valid_cnt); - return ret; -} - template <> int proc_by_pass(IntegerFixedVec *res_vec, const ObBitVector &skip, - const EvalBound &bound, - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx) + const EvalBound &bound, int64_t &valid_cnt, bool calc_valid_cnt) { int ret = OB_SUCCESS; uint64_t *data = reinterpret_cast(res_vec->get_data()); MEMSET(data + bound.start(), 1, (bound.range_size() * res_vec->get_length(0))); - - int64_t valid_cnt = bound.range_size() - skip.accumulate_bit_cnt(bound); - join_filter_ctx->n_times_ += valid_cnt; - join_filter_ctx->total_count_ += valid_cnt; - ObExprJoinFilter::collect_sample_info_batch(*join_filter_ctx, 0, valid_cnt); + if (OB_LIKELY(calc_valid_cnt)) { + valid_cnt = bound.range_size() - skip.accumulate_bit_cnt(bound); + } return ret; } @@ -126,27 +102,6 @@ ObExprJoinFilter::ObExprJoinFilterContext::~ObExprJoinFilterContext() cur_row_.reset(); } -void ObExprJoinFilter::ObExprJoinFilterContext::reset_monitor_info() -{ - filter_count_ = 0; - total_count_ = 0; - check_count_ = 0; - n_times_ = 0; - ready_ts_ = 0; - dynamic_disable_ = false; - is_ready_ = false; -} - -void ObExprJoinFilter::ObExprJoinFilterContext::collect_monitor_info( - const int64_t filtered_rows_count, - const int64_t check_rows_count, - const int64_t total_rows_count) -{ - filter_count_ += filtered_rows_count; - check_count_ += check_rows_count; - total_count_ += total_rows_count; -} - ObExprJoinFilter::ObExprJoinFilter(ObIAllocator& alloc) : ObExprOperator(alloc, T_OP_RUNTIME_FILTER, @@ -213,11 +168,15 @@ int ObExprJoinFilter::check_rf_ready( ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx) { int ret = OB_SUCCESS; + if (join_filter_ctx->is_first_) { + join_filter_ctx->start_time_ = ObTimeUtility::current_time(); + join_filter_ctx->is_first_ = false; + } ObP2PDatahubMsgBase *&rf_msg = join_filter_ctx->rf_msg_; - if (join_filter_ctx->is_ready()) { + if (join_filter_ctx->is_ready_) { } else if (OB_ISNULL(rf_msg)) { if (join_filter_ctx->need_wait_ready()) { - while (!join_filter_ctx->is_ready() && OB_SUCC(exec_ctx.fast_check_status())) { + while (!join_filter_ctx->is_ready_ && OB_SUCC(exec_ctx.fast_check_status())) { if (OB_NOT_NULL(rf_msg)) { #ifdef ERRSIM int ecode = EventTable::EN_PX_JOIN_FILTER_HOLD_MSG; @@ -240,7 +199,7 @@ int ObExprJoinFilter::check_rf_ready( } } } - } else if ((join_filter_ctx->n_times_ & CHECK_TIMES) == 0) { + } else if (join_filter_ctx->need_check_ready()) { if (OB_FAIL(PX_P2P_DH.atomic_get_msg(join_filter_ctx->rf_key_, rf_msg))) { if (OB_HASH_NOT_EXIST == ret) { ret = OB_SUCCESS; @@ -253,31 +212,17 @@ int ObExprJoinFilter::check_rf_ready( if (OB_NOT_NULL(rf_msg) && rf_msg->check_ready()) { join_filter_ctx->is_ready_ = true; join_filter_ctx->ready_ts_ = ObTimeUtility::current_time(); + join_filter_ctx->slide_window_.start_to_work(); } } - } else if ((join_filter_ctx->n_times_ & CHECK_TIMES) == 0 && - rf_msg->check_ready()) { + } else if (join_filter_ctx->need_check_ready() && rf_msg->check_ready()) { join_filter_ctx->ready_ts_ = ObTimeUtility::current_time(); join_filter_ctx->is_ready_ = true; + join_filter_ctx->slide_window_.start_to_work(); } return ret; } -void ObExprJoinFilter::collect_sample_info( - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx, - bool is_match) -{ - if (OB_NOT_NULL(join_filter_ctx)) { - check_need_dynamic_diable_bf(join_filter_ctx); - if (!join_filter_ctx->dynamic_disable()) { - if (!is_match) { - join_filter_ctx->partial_filter_count_++; - } - join_filter_ctx->partial_total_count_++; - } - } -} - int ObExprJoinFilter::prepare_storage_white_filter_data(const ObExpr &expr, ObDynamicFilterExecutor &dynamic_filter, ObEvalCtx &eval_ctx, @@ -293,16 +238,13 @@ int ObExprJoinFilter::prepare_storage_white_filter_data(const ObExpr &expr, if (OB_ISNULL(join_filter_ctx = static_cast( exec_ctx.get_expr_op_ctx(op_id)))) { // join filter ctx may be null in das. + is_data_prepared = true; dynamic_filter.set_filter_action(DynamicFilterAction::PASS_ALL); } else { - if (join_filter_ctx->is_first_) { - join_filter_ctx->start_time_ = ObTimeUtility::current_time(); - join_filter_ctx->is_first_ = false; - } if (OB_FAIL(check_rf_ready(exec_ctx, join_filter_ctx))) { LOG_WARN("fail to check bf ready", K(ret)); } else if (OB_ISNULL(join_filter_ctx->rf_msg_)) { - } else if (!join_filter_ctx->is_ready() || join_filter_ctx->dynamic_disable()) { + } else if (!join_filter_ctx->is_ready_ || join_filter_ctx->dynamic_disable()) { } else if (OB_FAIL(join_filter_ctx->rf_msg_->prepare_storage_white_filter_data( dynamic_filter, eval_ctx, params, is_data_prepared))) { LOG_WARN("fail to prepare_storage_white_filter_data", K(ret)); @@ -314,84 +256,6 @@ int ObExprJoinFilter::prepare_storage_white_filter_data(const ObExpr &expr, return ret; } -void ObExprJoinFilter::check_need_dynamic_diable_bf( - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx) -{ - if (OB_ISNULL(join_filter_ctx)) { - } else if (join_filter_ctx->cur_pos_ == join_filter_ctx->next_check_start_pos_) { - join_filter_ctx->partial_total_count_ = 0; - join_filter_ctx->partial_filter_count_ = 0; - if (join_filter_ctx->dynamic_disable()) { - join_filter_ctx->dynamic_disable_ = false; - } - } else if (join_filter_ctx->cur_pos_ >= - join_filter_ctx->next_check_start_pos_ + join_filter_ctx->window_size_) { - if (join_filter_ctx->partial_total_count_ - - join_filter_ctx->partial_filter_count_ < - join_filter_ctx->partial_filter_count_) { - // partial_filter_count_ / partial_total_count_ > 0.5 - // The optimizer choose the bloom filter when the filter threshold is larger than 0.6 - // 0.5 is a acceptable value - join_filter_ctx->partial_total_count_ = 0; - join_filter_ctx->partial_filter_count_ = 0; - join_filter_ctx->window_cnt_ = 0; - join_filter_ctx->next_check_start_pos_ = join_filter_ctx->cur_pos_; - } else { - join_filter_ctx->partial_total_count_ = 0; - join_filter_ctx->partial_filter_count_ = 0; - join_filter_ctx->window_cnt_++; - join_filter_ctx->next_check_start_pos_ = join_filter_ctx->cur_pos_ + - (join_filter_ctx->window_size_ * join_filter_ctx->window_cnt_); - join_filter_ctx->dynamic_disable_ = true; - } - } -} - -void ObExprJoinFilter::collect_sample_info_batch( - ObExprJoinFilter::ObExprJoinFilterContext &join_filter_ctx, - int64_t filter_count, int64_t total_count) -{ - if (!join_filter_ctx.dynamic_disable()) { - join_filter_ctx.partial_filter_count_ += filter_count; - join_filter_ctx.partial_total_count_ += total_count; - } - check_need_dynamic_diable_bf_batch(join_filter_ctx); -} - -void ObExprJoinFilter::check_need_dynamic_diable_bf_batch( - ObExprJoinFilter::ObExprJoinFilterContext &join_filter_ctx) -{ - if (join_filter_ctx.cur_pos_ >= join_filter_ctx.next_check_start_pos_ - && join_filter_ctx.need_reset_sample_info_) { - join_filter_ctx.partial_total_count_ = 0; - join_filter_ctx.partial_filter_count_ = 0; - join_filter_ctx.need_reset_sample_info_ = false; - if (join_filter_ctx.dynamic_disable()) { - join_filter_ctx.dynamic_disable_ = false; - } - } else if (join_filter_ctx.cur_pos_ >= - join_filter_ctx.next_check_start_pos_ + join_filter_ctx.window_size_) { - if (join_filter_ctx.partial_total_count_ - - join_filter_ctx.partial_filter_count_ < - join_filter_ctx.partial_filter_count_) { - // partial_filter_count_ / partial_total_count_ > 0.5 - // The optimizer choose the bloom filter when the filter threshold is larger than 0.6 - // 0.5 is a acceptable value - // if enabled, the slide window not needs to expand - join_filter_ctx.window_cnt_ = 0; - join_filter_ctx.next_check_start_pos_ = join_filter_ctx.cur_pos_; - } else { - join_filter_ctx.window_cnt_++; - join_filter_ctx.next_check_start_pos_ = join_filter_ctx.cur_pos_ + - (join_filter_ctx.window_size_ * join_filter_ctx.window_cnt_); - join_filter_ctx.dynamic_disable_ = true; - } - join_filter_ctx.partial_total_count_ = 0; - join_filter_ctx.partial_filter_count_ = 0; - join_filter_ctx.need_reset_sample_info_ = true; - } -} - int ObExprJoinFilter::eval_bloom_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res) { return eval_filter_internal(expr, ctx, res); @@ -420,21 +284,24 @@ int ObExprJoinFilter::eval_filter_internal(const ObExpr &expr, ObEvalCtx &ctx, O // join filter ctx may be null in das. res.set_int(1); } else { - if (join_filter_ctx->is_first_) { - join_filter_ctx->start_time_ = ObTimeUtility::current_time(); - join_filter_ctx->is_first_ = false; - } if (OB_FAIL(check_rf_ready(exec_ctx, join_filter_ctx))) { LOG_WARN("fail to check bf ready", K(ret)); - } else if (OB_ISNULL(join_filter_ctx->rf_msg_)) { - res.set_int(1); - } else if (!join_filter_ctx->is_ready() || join_filter_ctx->dynamic_disable()) { + } else if (OB_ISNULL(join_filter_ctx->rf_msg_) || !join_filter_ctx->is_ready_ + || join_filter_ctx->dynamic_disable()) { res.set_int(1); + // if msg not ready, add n_times_ and check ready every CHECK_TIMES + if (!join_filter_ctx->dynamic_disable()) { + if (++join_filter_ctx->n_times_ > CHECK_TIMES) { + join_filter_ctx->need_check_ready_ = true; + join_filter_ctx->n_times_ = 0; + } + } else { + (void)join_filter_ctx->collect_sample_info(0, 1); + } } else if (OB_FAIL(join_filter_ctx->rf_msg_->might_contain(expr, ctx, *join_filter_ctx, res))) { LOG_WARN("fail to check contain row", K(ret)); } if (OB_SUCC(ret)) { - join_filter_ctx->n_times_++; join_filter_ctx->total_count_++; } @@ -489,15 +356,15 @@ int ObExprJoinFilter::eval_filter_batch_internal( return OB_SUCCESS; }))) { /* do nothing*/ } } else { - if (join_filter_ctx->is_first_) { - join_filter_ctx->start_time_ = ObTimeUtility::current_time(); - join_filter_ctx->is_first_ = false; - } if (OB_FAIL(check_rf_ready(exec_ctx, join_filter_ctx))) { LOG_WARN("fail to check bf ready", K(ret)); - } else if (OB_ISNULL(join_filter_ctx->rf_msg_)) { + } else if (OB_ISNULL(join_filter_ctx->rf_msg_) || !join_filter_ctx->is_ready_) { FILL_BATCH_RESULT(); - } else if (!join_filter_ctx->is_ready() || join_filter_ctx->dynamic_disable()) { + if ((join_filter_ctx->n_times_) > CHECK_TIMES) { + join_filter_ctx->need_check_ready_ = true; + join_filter_ctx->n_times_ = 0; + } + } else if (join_filter_ctx->dynamic_disable()) { FILL_BATCH_RESULT(); } else if (OB_FAIL(join_filter_ctx->rf_msg_->might_contain_batch( expr, ctx, skip, batch_size, *join_filter_ctx))) { @@ -536,6 +403,7 @@ int ObExprJoinFilter::eval_filter_vector_internal( const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const EvalBound &bound) { int ret = OB_SUCCESS; + int64_t valid_cnt = 0; uint64_t op_id = expr.expr_ctx_id_; ObExecContext &exec_ctx = ctx.exec_ctx_; ObExprJoinFilterContext *join_filter_ctx = NULL; @@ -546,32 +414,43 @@ int ObExprJoinFilter::eval_filter_vector_internal( // join filter ctx may be null in das. if (VEC_UNIFORM == res_format) { IntegerUniVec *res_vec = static_cast(expr.get_vector(ctx)); - ret = proc_if_das(res_vec, skip, bound); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, false /* calc_valid_cnt */); } else if (VEC_FIXED == res_format) { IntegerFixedVec *res_vec = static_cast(expr.get_vector(ctx)); - ret = proc_if_das(res_vec, skip, bound); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, false /* calc_valid_cnt */); } eval_flags.set_all(true); } else { - if (join_filter_ctx->is_first_) { - join_filter_ctx->start_time_ = ObTimeUtility::current_time(); - join_filter_ctx->is_first_ = false; - } if (OB_FAIL(check_rf_ready(exec_ctx, join_filter_ctx))) { LOG_WARN("fail to check bf ready", K(ret)); - } else if (OB_ISNULL(join_filter_ctx->rf_msg_) || !join_filter_ctx->is_ready() + } else if (OB_ISNULL(join_filter_ctx->rf_msg_) || !join_filter_ctx->is_ready_ || join_filter_ctx->dynamic_disable()) { // rf_msg_ is null: no msg arrived yet // rf_msg_ not ready: not all msgs arrived // rf_msg_ dynamic_disable: disable filter when filter rate < 0.5 if (VEC_UNIFORM == res_format) { IntegerUniVec *res_vec = static_cast(expr.get_vector(ctx)); - ret = proc_by_pass(res_vec, skip, bound, join_filter_ctx); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, true /* calc_valid_cnt */); } else if (VEC_FIXED == res_format) { IntegerFixedVec *res_vec = static_cast(expr.get_vector(ctx)); - ret = proc_by_pass(res_vec, skip, bound, join_filter_ctx); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, true /* calc_valid_cnt */); + } + if (OB_FAIL(ret)) { + LOG_WARN("failed to proc_by_pass", K(res_format), K(ret)); + } else { + join_filter_ctx->total_count_ += valid_cnt; + // if msg not ready, add n_times_ and check ready every CHECK_TIMES + if (!join_filter_ctx->dynamic_disable()) { + join_filter_ctx->n_times_ += valid_cnt; + if (join_filter_ctx->n_times_ > CHECK_TIMES) { + join_filter_ctx->need_check_ready_ = true; + join_filter_ctx->n_times_ = 0; + } + } else { + (void)join_filter_ctx->collect_sample_info(0, valid_cnt); + } + eval_flags.set_all(true); } - eval_flags.set_all(true); } else if (OB_FAIL(join_filter_ctx->rf_msg_->might_contain_vector(expr, ctx, skip, bound, *join_filter_ctx))) { LOG_WARN("fail to might contain batch"); diff --git a/src/sql/engine/expr/ob_expr_join_filter.h b/src/sql/engine/expr/ob_expr_join_filter.h index 1eb003deaf..db6042400d 100644 --- a/src/sql/engine/expr/ob_expr_join_filter.h +++ b/src/sql/engine/expr/ob_expr_join_filter.h @@ -35,42 +35,66 @@ using ObRuntimeFilterParams = common::ObSEArray; class ObExprJoinFilter : public ObExprOperator { public: - class ObExprJoinFilterContext : public ObExprOperatorCtx + class ObExprJoinFilterContext final: public ObExprOperatorCtx { public: ObExprJoinFilterContext() : ObExprOperatorCtx(), rf_msg_(nullptr), rf_key_(), hash_funcs_(), cmp_funcs_(), start_time_(0), filter_count_(0), total_count_(0), check_count_(0), - n_times_(0), ready_ts_(0), next_check_start_pos_(0), - window_cnt_(0), window_size_(0), - partial_filter_count_(0), partial_total_count_(0), - cur_pos_(total_count_), need_reset_sample_info_(false), flag_(0), - cur_row_(), cur_row_with_hash_(nullptr), skip_vector_(nullptr), - is_partition_wise_jf_(false) + n_times_(0), ready_ts_(0), slide_window_(total_count_), flag_(0), + cur_row_(), cur_row_with_hash_(nullptr), skip_vector_(nullptr) { cur_row_.set_attr(ObMemAttr(MTL_ID(), "RfCurRow")); need_wait_rf_ = true; + need_check_ready_ = true; is_first_ = true; + is_partition_wise_jf_ = false; } virtual ~ObExprJoinFilterContext(); public: - bool is_ready() { return is_ready_; } - bool need_wait_ready() { return need_wait_rf_; } - bool dynamic_disable() { return dynamic_disable_; } - void collect_monitor_info( - const int64_t filtered_rows_count, - const int64_t check_rows_count, - const int64_t total_rows_count); - void inc_partial_rows_count( - const int64_t filtered_rows_count, - const int64_t total_rows_count) + inline bool need_wait_ready() { return need_wait_rf_; } + inline bool need_check_ready() { return need_check_ready_; } + inline bool dynamic_disable() override final { - if (!dynamic_disable()) { - partial_filter_count_ += filtered_rows_count; - partial_total_count_ += total_rows_count; + return slide_window_.dynamic_disable(); + } + inline bool need_reset_in_rescan() override final + { + // for runtime filter pushdown, if is partition wise join, we need to reset + // pushdown filter parameters + return is_partition_wise_jf_; + } + inline void collect_monitor_info(const int64_t filtered_rows_count, + const int64_t check_rows_count, + const int64_t total_rows_count) override final + { + filter_count_ += filtered_rows_count; + check_count_ += check_rows_count; + total_count_ += total_rows_count; + if (!is_ready_) { + n_times_ += total_rows_count; + if (n_times_ > CHECK_TIMES) { + need_check_ready_ = true; + n_times_ = 0; + } } } - void reset_monitor_info(); + inline void reset_monitor_info() + { + filter_count_ = 0; + total_count_ = 0; + check_count_ = 0; + n_times_ = 0; + ready_ts_ = 0; + is_ready_ = false; + } + + inline void collect_sample_info(const int64_t filter_count, + const int64_t total_count) override final + { + (void)slide_window_.update_slide_window_info(filter_count, total_count); + } + public: ObP2PDatahubMsgBase *rf_msg_; ObP2PDhKey rf_key_; @@ -83,23 +107,22 @@ public: int64_t n_times_; int64_t ready_ts_; - // for adaptive bloom filter - int64_t next_check_start_pos_; - int64_t window_cnt_; - int64_t window_size_; - int64_t partial_filter_count_; - int64_t partial_total_count_; - int64_t &cur_pos_; - bool need_reset_sample_info_; + ObAdaptiveFilterSlideWindow slide_window_; + union { uint64_t flag_; struct { - bool need_wait_rf_:1; bool is_ready_:1; - bool dynamic_disable_:1; bool is_first_:1; + // whether need to sync wait + bool need_wait_rf_:1; + // check ready every CHECK_TIMES + bool need_check_ready_:1; + // for runtime filter pushdown, if is partition wise join, we need to reset + // pushdown filter parameters + bool is_partition_wise_jf_ : 1; int32_t max_wait_time_ms_:32; - int32_t reserved_:28; + int32_t reserved_:27; }; }; ObTMArray cur_row_; @@ -108,10 +131,6 @@ public: ObBitVector *skip_vector_; // used in ObRFInFilterVecMsg/ObRFBloomFilterMsg, cal probe data's hash value uint64_t *right_hash_vals_; - - // for runtime filter pushdown, if is partition wise join, we need to reset - // pushdown filter parameters - bool is_partition_wise_jf_; }; ObExprJoinFilter(); explicit ObExprJoinFilter(common::ObIAllocator& alloc); @@ -161,12 +180,6 @@ public: virtual bool need_rt_ctx() const override { return true; } // hard code seed, 32 bit max prime number static const int64_t JOIN_FILTER_SEED = 4294967279; - static void collect_sample_info( - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx, - bool is_match); - static void collect_sample_info_batch( - ObExprJoinFilter::ObExprJoinFilterContext &join_filter_ctx, - int64_t filter_count, int64_t total_count); static int prepare_storage_white_filter_data( const ObExpr &expr, ObDynamicFilterExecutor &dynamic_filter, @@ -178,10 +191,6 @@ private: ObExecContext &exec_ctx, ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx); - static void check_need_dynamic_diable_bf( - ObExprJoinFilter::ObExprJoinFilterContext *join_filter_ctx); - static void check_need_dynamic_diable_bf_batch( - ObExprJoinFilter::ObExprJoinFilterContext &join_filter_ctx); private: static const int64_t CHECK_TIMES = 127; private: diff --git a/src/sql/engine/expr/ob_expr_like.cpp b/src/sql/engine/expr/ob_expr_like.cpp index e66868d9fb..6dfd138ff8 100644 --- a/src/sql/engine/expr/ob_expr_like.cpp +++ b/src/sql/engine/expr/ob_expr_like.cpp @@ -1324,17 +1324,17 @@ int ObExprLike::eval_like_expr_vector_only_text_vectorized(VECTOR_EVAL_FUNC_ARG_ VectorFormat text_format = expr.args_[0]->get_format(ctx); VectorFormat res_format = expr.get_format(ctx); if (VEC_DISCRETE == text_format && VEC_DISCRETE == res_format) { - ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_UNIFORM == text_format && VEC_DISCRETE == res_format) { - ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_CONTINUOUS == text_format && VEC_DISCRETE == res_format) { - ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_DISCRETE == text_format && VEC_UNIFORM == res_format) { - ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_UNIFORM == text_format && VEC_UNIFORM == res_format) { - ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_CONTINUOUS == text_format && VEC_UNIFORM == res_format) { - ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); } else { ret = vector_like(VECTOR_EVAL_FUNC_ARG_LIST); } diff --git a/src/sql/engine/expr/ob_expr_lower.cpp b/src/sql/engine/expr/ob_expr_lower.cpp index 753bf63187..eea89d615a 100644 --- a/src/sql/engine/expr/ob_expr_lower.cpp +++ b/src/sql/engine/expr/ob_expr_lower.cpp @@ -683,17 +683,17 @@ int ObExprLowerUpper::calc_common_vector( VectorFormat arg_format = expr.args_[0]->get_format(ctx); VectorFormat res_format = expr.get_format(ctx); if (VEC_DISCRETE == arg_format && VEC_DISCRETE == res_format) { - ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); + ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); } else if (VEC_UNIFORM == arg_format && VEC_DISCRETE == res_format) { - ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); + ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); } else if (VEC_CONTINUOUS == arg_format && VEC_DISCRETE == res_format) { - ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); + ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); } else if (VEC_DISCRETE == arg_format && VEC_UNIFORM == res_format) { - ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); + ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); } else if (VEC_UNIFORM == arg_format && VEC_UNIFORM == res_format) { - ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); + ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); } else if (VEC_CONTINUOUS == arg_format && VEC_UNIFORM == res_format) { - ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); + ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); } else { ret = vector_lower_upper(VECTOR_EVAL_FUNC_ARG_LIST, cs_type); } diff --git a/src/sql/engine/expr/ob_expr_operator.cpp b/src/sql/engine/expr/ob_expr_operator.cpp index 92a57b412a..c206404dd9 100644 --- a/src/sql/engine/expr/ob_expr_operator.cpp +++ b/src/sql/engine/expr/ob_expr_operator.cpp @@ -107,6 +107,44 @@ int get_tz_offset(const ObTimeZoneInfo *tz_info, int64_t &offset) return ret; } +int ObAdaptiveFilterSlideWindow::update_slide_window_info(const int64_t filtered_rows_count, + const int64_t total_rows_count) +{ + int ret = OB_SUCCESS; + if (!ready_to_work_) { + // if filter is not ready, do not need check whether dynamic disable + } else if (dynamic_disable()) { + // if filter is dynamic disable, not need to add any filter statistic info + // but check whether end of the punished windows, try reenable it. + // since we cannot control the size of each batch/block size, we may drop part of the + // statistic info which located in the effective window + if (cur_pos_ >= next_check_start_pos_) { + dynamic_disable_ = false; + } + } else { + // if filter is enabled, add statistic info + partial_filter_count_ += filtered_rows_count; + partial_total_count_ += total_rows_count; + + // if end of the window, check the filter rate and clear the statistic info + if (cur_pos_ >= next_check_start_pos_ + window_size_) { + if (partial_filter_count_ >= partial_total_count_ * adptive_ratio_thresheld_) { + // partial_filter_count_ / partial_total_count_ >= adptive_ratio_thresheld_ + // if enabled, the slide window not needs to expand + window_cnt_ = 0; + next_check_start_pos_ = cur_pos_; + } else { + window_cnt_++; + next_check_start_pos_ = cur_pos_ + (window_size_ * window_cnt_); + dynamic_disable_ = true; + } + partial_total_count_ = 0; + partial_filter_count_ = 0; + } + } + return ret; +} + OB_SERIALIZE_MEMBER(ObExprOperator::DatumCastExtraInfo, cmp_meta_, cm_); int ObExprOperator::DatumCastExtraInfo::deep_copy(common::ObIAllocator &allocator, diff --git a/src/sql/engine/expr/ob_expr_operator.h b/src/sql/engine/expr/ob_expr_operator.h index 5948173d76..d470715837 100644 --- a/src/sql/engine/expr/ob_expr_operator.h +++ b/src/sql/engine/expr/ob_expr_operator.h @@ -99,6 +99,67 @@ private: uint32_t flag_; }; +// Adaptive filter: +// In every slide window with window_size_, we check the realtime filter rate of the filter, +// if the realtime filter rate is too low, we disable it in the following slide window. +// +// 1.The slide window start to work once the filter is ready(see function 'start_to_work'). +// 2.When data is checked by the filter, we use 'update_slide_window_info' to update the statistic +// info and decide whether to disable it or not. +// 3.If the filter is disabled in this window, we will enable it in the next window. However, if the +// filter rate is too low for several windows continuously, we will punish it with expanding +// window_size(by adding window_cnt_), the filter will be disabled for a more long time. +class ObAdaptiveFilterSlideWindow +{ +public: + explicit ObAdaptiveFilterSlideWindow(int64_t &total_count) + : next_check_start_pos_(0), window_cnt_(0), window_size_(4096), partial_filter_count_(0), + partial_total_count_(0), adptive_ratio_thresheld_(0.5), cur_pos_(total_count), + dynamic_disable_(false), ready_to_work_(false) + {} + ~ObAdaptiveFilterSlideWindow() = default; + inline bool dynamic_disable() { return dynamic_disable_; } + inline void start_to_work() { + next_check_start_pos_ = cur_pos_; + ready_to_work_ = true; + } + int update_slide_window_info(const int64_t filtered_rows_count, const int64_t total_rows_count); + inline void reset_for_rescan() { + dynamic_disable_ = false; + next_check_start_pos_ = 0; + window_cnt_ = 0; + partial_filter_count_ = 0; + partial_total_count_ = 0; + ready_to_work_ = false; + } + + inline void set_window_size(int64_t window_size) { window_size_ = window_size; } + inline void set_adptive_ratio_thresheld(double thresheld) { adptive_ratio_thresheld_ = thresheld; } + TO_STRING_KV(K(next_check_start_pos_), K(window_cnt_), K(partial_filter_count_), + K(partial_total_count_), K(cur_pos_), K(dynamic_disable_), K(ready_to_work_)); + +private: + // the start posistion of slide window we check next + int64_t next_check_start_pos_; + // if the filter rate is too low for several slide windows continuously, we punish it with expanding + // the window_size, window_cnt_ implicts the window size now we maintain + int64_t window_cnt_; + // the original window_size, we default value = 4096 + int64_t window_size_; + // filtered data count in a slide window + int64_t partial_filter_count_; + // total data count in a slide window + int64_t partial_total_count_; + // if the realtime filter ratio is smaller than thresheld_, the filter will be disabled, + // default=0.5 + double adptive_ratio_thresheld_; + // an reference of total_count, mark the real position now + int64_t &cur_pos_; + bool dynamic_disable_; + // when the filter msg are not ready, skip the update process + bool ready_to_work_; +}; + class ObExprOperatorCtx { public: @@ -112,6 +173,23 @@ public: UNUSED(buf_len); return 0; } + virtual inline bool dynamic_disable() + { + return false; + } + // This interface is used to collect the filter statistic info + // when runtime filter is pushdown as white filter in storage layer, + // override by ObExprJoinFilterContext and ObExprTopNFilterContext now. + virtual void collect_monitor_info(const int64_t filtered_rows_count, + const int64_t check_rows_count, const int64_t total_rows_count) + {} + + virtual inline void collect_sample_info(const int64_t filter_count, const int64_t total_count) + {} + + virtual inline bool need_reset_in_rescan() { return false; } + virtual inline bool is_data_version_updated(int64_t old_version) { return false; } + }; typedef common::ObIArray RowIterIArray; diff --git a/src/sql/engine/expr/ob_expr_operator_factory.cpp b/src/sql/engine/expr/ob_expr_operator_factory.cpp index 3e2b84acb1..c7c0f828a6 100644 --- a/src/sql/engine/expr/ob_expr_operator_factory.cpp +++ b/src/sql/engine/expr/ob_expr_operator_factory.cpp @@ -452,6 +452,7 @@ #include "sql/engine/expr/ob_expr_rb_from_string.h" #include "sql/engine/expr/ob_expr_lock_func.h" +#include "sql/engine/expr/ob_expr_topn_filter.h" using namespace oceanbase::common; namespace oceanbase @@ -1074,7 +1075,7 @@ void ObExprOperatorFactory::register_expr_operators() REG_OP(ObExprTransactionId); REG_OP(ObExprInnerRowCmpVal); REG_OP(ObExprLastRefreshScn); - // REG_OP(ObExprTopNFilter); + REG_OP(ObExprTopNFilter); REG_OP(ObExprPrivSTMakeEnvelope); REG_OP(ObExprPrivSTClipByBox2D); REG_OP(ObExprPrivSTPointOnSurface); @@ -1434,6 +1435,7 @@ void ObExprOperatorFactory::register_expr_operators() REG_OP_ORCL(ObExprTransactionId); REG_OP_ORCL(ObExprInnerRowCmpVal); REG_OP_ORCL(ObExprLastRefreshScn); + REG_OP_ORCL(ObExprTopNFilter); REG_OP_ORCL(ObExprInnerTableOptionPrinter); REG_OP_ORCL(ObExprInnerTableSequenceGetter); // REG_OP_ORCL(ObExprTopNFilter); diff --git a/src/sql/engine/expr/ob_expr_regexp_replace.cpp b/src/sql/engine/expr/ob_expr_regexp_replace.cpp index a373e5decf..5b6b23896f 100644 --- a/src/sql/engine/expr/ob_expr_regexp_replace.cpp +++ b/src/sql/engine/expr/ob_expr_regexp_replace.cpp @@ -525,7 +525,7 @@ int ObExprRegexpReplace::vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_DECL) { res_vec->set_null(i); } else { // if text is lob type, res_replace only get locator - ret = vector_regexp_replace_convert( + ret = vector_regexp_replace_convert( VECTOR_EVAL_FUNC_ARG_LIST, text_vec->get_string(i), true, expr.args_[0]->datum_meta_.cs_type_, out_alloc, tmp_alloc, i); } @@ -618,7 +618,7 @@ int ObExprRegexpReplace::vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_DECL) { res_vec->set_null(i); } else { // if text is lob type, res_replace only get locator; - ret = vector_regexp_replace_convert( + ret = vector_regexp_replace_convert( VECTOR_EVAL_FUNC_ARG_LIST, text_vec->get_string(i), true, expr.args_[0]->datum_meta_.cs_type_, out_alloc, tmp_alloc, i); } @@ -661,7 +661,7 @@ int ObExprRegexpReplace::vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_DECL) { } else if (res_replace.empty() && lib::is_oracle_mode()) { res_vec->set_null(i); } else { - ret = vector_regexp_replace_convert(VECTOR_EVAL_FUNC_ARG_LIST, + ret = vector_regexp_replace_convert(VECTOR_EVAL_FUNC_ARG_LIST, res_replace, is_no_pattern_to_replace, res_coll_type, out_alloc, tmp_alloc, i); } } @@ -693,17 +693,17 @@ int ObExprRegexpReplace::eval_regexp_replace_vector(VECTOR_EVAL_FUNC_ARG_DECL) VectorFormat arg_format = expr.args_[0]->get_format(ctx); VectorFormat res_format = expr.get_format(ctx); if (VEC_DISCRETE == arg_format && VEC_DISCRETE == res_format) { - ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_UNIFORM == arg_format && VEC_DISCRETE == res_format) { - ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_CONTINUOUS == arg_format && VEC_DISCRETE == res_format) { - ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_DISCRETE == arg_format && VEC_UNIFORM == res_format) { - ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_UNIFORM == arg_format && VEC_UNIFORM == res_format) { - ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_CONTINUOUS == arg_format && VEC_UNIFORM == res_format) { - ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); } else { ret = vector_regexp_replace(VECTOR_EVAL_FUNC_ARG_LIST); } diff --git a/src/sql/engine/expr/ob_expr_repeat.cpp b/src/sql/engine/expr/ob_expr_repeat.cpp index 335f5b47ae..cfed0651a3 100644 --- a/src/sql/engine/expr/ob_expr_repeat.cpp +++ b/src/sql/engine/expr/ob_expr_repeat.cpp @@ -277,6 +277,9 @@ int ObExprRepeat::cg_expr(ObExprCGCtx &, const ObRawExpr &, ObExpr &rt_expr) con int ret = OB_SUCCESS; CK(2 == rt_expr.arg_cnt_); rt_expr.eval_func_ = eval_repeat; + if (GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_3_2_0) { + rt_expr.eval_vector_func_ = eval_repeat_vector; + } return ret; } @@ -324,6 +327,117 @@ int ObExprRepeat::eval_repeat(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &expr_ return ret; } +template +int ObExprRepeat::repeat_vector(VECTOR_EVAL_FUNC_ARG_DECL) +{ + int ret = OB_SUCCESS; + ResVec *res_vec = static_cast(expr.get_vector(ctx)); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + + int64_t max_size = 0; // used for limit size of result + + if (OB_FAIL(ctx.exec_ctx_.get_my_session()->get_max_allowed_packet(max_size))) { + LOG_WARN("get max size failed", K(ret)); + } else { + const Arg0Vec *arg0_vec = static_cast(expr.args_[0]->get_vector(ctx)); + const Arg1Vec *arg1_vec = static_cast(expr.args_[1]->get_vector(ctx)); + + // the count may be a variable number, repeat expr support the + ObEvalCtx::BatchInfoScopeGuard batch_info_guard(ctx); + batch_info_guard.set_batch_size(bound.batch_size()); + for (int64_t idx = bound.start(); OB_SUCC(ret) && idx < bound.end(); ++idx) { + if (skip.at(idx) || eval_flags.at(idx)) { + continue; + } + + batch_info_guard.set_batch_idx(idx); + if (arg0_vec->is_null(idx) || arg1_vec->is_null(idx)) { + res_vec->set_null(idx); + eval_flags.set(idx); + } else { + // prepare & init needed params + ObEvalCtx::TempAllocGuard tmp_alloc_g(ctx); + common::ObArenaAllocator &tmp_allocator = tmp_alloc_g.get_allocator(); + + int64_t count = arg1_vec->get_int(idx); + bool is_null = false; + ObString text; + ObString output; + + ObExprStrResAlloc calc_alloc(expr, ctx); + + if (OB_FAIL(ObTextStringHelper::read_real_string_data( + tmp_allocator, arg0_vec, expr.args_[0]->datum_meta_, + expr.args_[0]->obj_meta_.has_lob_header(), text, idx))) { + LOG_WARN("failed to read text", K(ret), K(text)); + } else if (!ob_is_text_tc(expr.datum_meta_.type_)) { + // 2.1 deal with string tc + ret = repeat(output, is_null, text, count, calc_alloc, max_size); + } else { + // 2.2 deal with text tc + ret = repeat_text(expr.datum_meta_.type_, + expr.obj_meta_.has_lob_header(), output, is_null, + text, count, calc_alloc, max_size); + } + if (OB_FAIL(ret)) { + LOG_WARN("do repeat in vector failed", K(ret)); + } else { + if (is_null) { + res_vec->set_null(idx); + } else { + res_vec->set_string(idx, output); + } + } + eval_flags.set(idx); + } + } + } + return ret; +} + +int ObExprRepeat::eval_repeat_vector(VECTOR_EVAL_FUNC_ARG_DECL) +{ + int ret = OB_SUCCESS; + // make sure that `repeat` operand should have 2 params. + if (OB_FAIL(expr.args_[0]->eval_vector(ctx, skip, bound)) || + OB_FAIL(expr.args_[1]->eval_vector(ctx, skip, bound))) { + LOG_WARN("failed to evaluate `repeat` parameters", K(ret)); + } else { + VectorFormat arg0_format = expr.args_[0]->get_format(ctx); + VectorFormat arg1_format = expr.args_[1]->get_format(ctx); + VectorFormat res_format = expr.get_format(ctx); + if (VEC_DISCRETE == arg0_format && VEC_UNIFORM_CONST == arg1_format && + VEC_DISCRETE == res_format) { + ret = repeat_vector( + VECTOR_EVAL_FUNC_ARG_LIST); + } else if (VEC_UNIFORM == arg0_format && VEC_UNIFORM_CONST == arg1_format && + VEC_DISCRETE == res_format) { + ret = repeat_vector( + VECTOR_EVAL_FUNC_ARG_LIST); + } else if (VEC_CONTINUOUS == arg0_format && + VEC_UNIFORM_CONST == arg1_format && VEC_DISCRETE == res_format) { + ret = repeat_vector( + VECTOR_EVAL_FUNC_ARG_LIST); + } else if (VEC_DISCRETE == arg0_format && + VEC_UNIFORM_CONST == arg1_format && VEC_UNIFORM == res_format) { + ret = repeat_vector( + VECTOR_EVAL_FUNC_ARG_LIST); + } else if (VEC_UNIFORM == arg0_format && VEC_UNIFORM_CONST == arg1_format && + VEC_UNIFORM == res_format) { + ret = repeat_vector( + VECTOR_EVAL_FUNC_ARG_LIST); + } else if (VEC_CONTINUOUS == arg0_format && + VEC_UNIFORM_CONST == arg1_format && VEC_UNIFORM == res_format) { + ret = repeat_vector( + VECTOR_EVAL_FUNC_ARG_LIST); + } else { + ret = repeat_vector( + VECTOR_EVAL_FUNC_ARG_LIST); + } + } + return ret; +} + DEF_SET_LOCAL_SESSION_VARS(ObExprRepeat, raw_expr) { int ret = OB_SUCCESS; if (is_mysql_mode()) { diff --git a/src/sql/engine/expr/ob_expr_repeat.h b/src/sql/engine/expr/ob_expr_repeat.h index f7dcc769cc..b293cbcc04 100644 --- a/src/sql/engine/expr/ob_expr_repeat.h +++ b/src/sql/engine/expr/ob_expr_repeat.h @@ -47,6 +47,8 @@ public: static int eval_repeat(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &expr_datum); + static int eval_repeat_vector(VECTOR_EVAL_FUNC_ARG_DECL); + static int repeat(common::ObString &output, bool &is_null, const common::ObString &input, @@ -63,6 +65,9 @@ public: const int64_t max_result_size); DECLARE_SET_LOCAL_SESSION_VARS; private: + template + static int repeat_vector(VECTOR_EVAL_FUNC_ARG_DECL); + static const int64_t MEM_WARN_THRESHOLD = 100 * 1024 * 1024; // 100M // disallow copy DISALLOW_COPY_AND_ASSIGN(ObExprRepeat); diff --git a/src/sql/engine/expr/ob_expr_substr.cpp b/src/sql/engine/expr/ob_expr_substr.cpp index 1e70371a57..6f2b8e8f33 100644 --- a/src/sql/engine/expr/ob_expr_substr.cpp +++ b/src/sql/engine/expr/ob_expr_substr.cpp @@ -982,17 +982,17 @@ int ObExprSubstr::eval_substr_vector(VECTOR_EVAL_FUNC_ARG_DECL) VectorFormat arg_format = expr.args_[0]->get_format(ctx); VectorFormat res_format = expr.get_format(ctx); if (VEC_DISCRETE == arg_format && VEC_DISCRETE == res_format) { - ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_UNIFORM == arg_format && VEC_DISCRETE == res_format) { - ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_CONTINUOUS == arg_format && VEC_DISCRETE == res_format) { - ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_DISCRETE == arg_format && VEC_UNIFORM == res_format) { - ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_UNIFORM == arg_format && VEC_UNIFORM == res_format) { - ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); } else if (VEC_CONTINUOUS == arg_format && VEC_UNIFORM == res_format) { - ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); + ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); } else { ret = vector_substr(VECTOR_EVAL_FUNC_ARG_LIST); } diff --git a/src/sql/engine/expr/ob_expr_topn_filter.cpp b/src/sql/engine/expr/ob_expr_topn_filter.cpp new file mode 100644 index 0000000000..b9e54e5513 --- /dev/null +++ b/src/sql/engine/expr/ob_expr_topn_filter.cpp @@ -0,0 +1,488 @@ +/** + * Copyright (c) 2024 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_EXE +#include "sql/engine/expr/ob_expr_topn_filter.h" +#include "sql/engine/ob_exec_context.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" + +namespace oceanbase +{ +namespace sql +{ +template +static inline int proc_by_pass(ResVec *res_vec, const ObBitVector &skip, const EvalBound &bound, + int64_t &valid_cnt, bool calc_valid_cnt = false); + +template <> +inline int proc_by_pass(IntegerUniVec *res_vec, const ObBitVector &skip, + const EvalBound &bound, int64_t &valid_cnt, + bool calc_valid_cnt) +{ + int ret = OB_SUCCESS; + if (OB_LIKELY(calc_valid_cnt)) { + valid_cnt = 0; + if (OB_FAIL(ObBitVector::flip_foreach( + skip, bound, [&](int64_t idx) __attribute__((always_inline)) { + ++valid_cnt; + res_vec->set_int(idx, 1); + return OB_SUCCESS; + }))) { + LOG_WARN("fail to do for each operation", K(ret)); + } + } else { + if (OB_FAIL(ObBitVector::flip_foreach( + skip, bound, [&](int64_t idx) __attribute__((always_inline)) { + res_vec->set_int(idx, 1); + return OB_SUCCESS; + }))) { + LOG_WARN("fail to do for each operation", K(ret)); + } + } + return ret; +} + +template <> +inline int proc_by_pass(IntegerFixedVec *res_vec, const ObBitVector &skip, + const EvalBound &bound, int64_t &valid_cnt, + bool calc_valid_cnt) +{ + int ret = OB_SUCCESS; + uint64_t *data = reinterpret_cast(res_vec->get_data()); + MEMSET(data + bound.start(), 1, (bound.range_size() * res_vec->get_length(0))); + if (OB_LIKELY(calc_valid_cnt)) { + valid_cnt = bound.range_size() - skip.accumulate_bit_cnt(bound); + } + return ret; +} + +const int64_t ObExprTopNFilterContext::ROW_COUNT_CHECK_INTERVAL = 255; +const int64_t ObExprTopNFilterContext::EVAL_TIME_CHECK_INTERVAL = 63; + +ObExprTopNFilterContext::~ObExprTopNFilterContext() +{ + if (OB_NOT_NULL(topn_filter_msg_)) { + // topn_filter_msg_ is got from PX_P2P_DH map + // do not destroy it, because other worker threads may not start yet + topn_filter_msg_->dec_ref_count(); + topn_filter_msg_ = nullptr; + } + cmp_funcs_.reset(); + int ret = OB_SUCCESS; + double filter_rate = filter_count_ / double(check_count_ + 1); + LOG_TRACE("[TopN Filter] print the filter rate", K(total_count_), K(check_count_), + K(filter_count_), K(filter_rate)); +} + +void ObExprTopNFilterContext::reset_for_rescan() +{ + state_ = FilterState::NOT_READY; + if (OB_NOT_NULL(topn_filter_msg_)) { + // topn_filter_msg_ is got from PX_P2P_DH map + // do not destroy it, because other worker threads may not start yet + topn_filter_msg_->dec_ref_count(); + topn_filter_msg_ = nullptr; + } + n_times_ = 0; + n_rows_ = 0; + flag_ = 0; + is_first_ = true; + slide_window_.reset_for_rescan(); +} + +// with perfect forwarding, we can write one state machine rather than +// write three state machines for eval/eval_batch/eval_vector interface +template +int ObExprTopNFilterContext::state_machine(const ObExpr &expr, ObEvalCtx &ctx, Args &&... args) +{ + // state transformation of topn filter: + // each n epoch check pass filter rate < 0.5 + // NOT_READY ------------> CHECK_READY ------------> ENABLE ------------> DYNAMIC_DISABLE + // <------------ <------------ + // check not pass after k windows + + int ret = OB_SUCCESS; + bool result_filled = false; + do { + switch (state_) { + case FilterState::NOT_READY: { + ret = bypass(expr, ctx, std::forward(args)...); + result_filled = true; + break; + } + case FilterState::CHECK_READY: { + if (OB_FAIL(check_filter_ready())) { + LOG_WARN("fail to check filter ready", K(ret)); + } + // result not filled, so while loop continues + break; + } + case FilterState::ENABLE: { + ret = do_process(expr, ctx, std::forward(args)...); + result_filled = true; + break; + } + case FilterState::DYNAMIC_DISABLE: { + ret = bypass(expr, ctx, std::forward(args)...); + result_filled = true; + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected state_"); + break; + } + } + } while (!result_filled && OB_SUCC(ret)); + return ret; +} + +// bypass interface for eval one row, for storege black filter +inline int ObExprTopNFilterContext::bypass(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res) +{ + UNUSED(expr); + UNUSED(ctx); + int ret = OB_SUCCESS; + res.set_int(1); + if (!dynamic_disable()) { + // if msg not ready, add n_times_ and check ready every ROW_COUNT_CHECK_INTERVAL + if ((++n_times_ > EVAL_TIME_CHECK_INTERVAL) || (++n_rows_ > ROW_COUNT_CHECK_INTERVAL)) { + state_ = FilterState::CHECK_READY; + } + } else { + // if msg is ready but dynamic disable, collect_sample_info so it can be enabled later + (void)collect_sample_info(0, 1); + if (!dynamic_disable()) { + state_ = FilterState::ENABLE; + } + } + return ret; +} + +// bypass interface for eval_batch rows, for storege black filter +inline int ObExprTopNFilterContext::bypass(const ObExpr &expr, ObEvalCtx &ctx, + const ObBitVector &skip, const int64_t batch_size) +{ + int ret = OB_SUCCESS; + int64_t total_count = 0; + ObDatum *results = expr.locate_batch_datums(ctx); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + if (OB_FAIL(ObBitVector::flip_foreach( + skip, batch_size, [&](int64_t idx) __attribute__((always_inline)) { + eval_flags.set(idx); + results[idx].set_int(1); + ++total_count; + return OB_SUCCESS; + }))) { + LOG_WARN("failed to flip_foreach"); + } else if (FALSE_IT(total_count_ += total_count)) { + } else if (!dynamic_disable()) { + // if msg not ready, add n_times_ and check ready every ROW_COUNT_CHECK_INTERVAL + n_rows_ += total_count; + if ((++n_times_ > EVAL_TIME_CHECK_INTERVAL) || (n_rows_ > ROW_COUNT_CHECK_INTERVAL)) { + state_ = FilterState::CHECK_READY; + } + } else { + // if msg is ready but dynamic disable, collect_sample_info so it can be enabled later + (void)collect_sample_info(0, total_count); + if (!dynamic_disable()) { + state_ = FilterState::ENABLE; + } + } + return ret; +} + +// bypass interface for eval_vector rows, for storege black filter +inline int ObExprTopNFilterContext::bypass(const ObExpr &expr, ObEvalCtx &ctx, + const ObBitVector &skip, const EvalBound &bound) +{ + int ret = OB_SUCCESS; + int64_t valid_cnt = 0; + VectorFormat res_format = expr.get_format(ctx); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + if (VEC_UNIFORM == res_format) { + IntegerUniVec *res_vec = static_cast(expr.get_vector(ctx)); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, true /* calc_valid_cnt */); + } else if (VEC_FIXED == res_format) { + IntegerFixedVec *res_vec = static_cast(expr.get_vector(ctx)); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, true /* calc_valid_cnt */); + } + if (OB_FAIL(ret)) { + LOG_WARN("failed to proc_by_pass", K(res_format), K(ret)); + } else { + total_count_ += valid_cnt; + eval_flags.set_all(true); + // if msg not ready, add n_times_ and check ready every ROW_COUNT_CHECK_INTERVAL + if (!dynamic_disable()) { + n_rows_ += valid_cnt; + if ((++n_times_ > EVAL_TIME_CHECK_INTERVAL) || (n_rows_ > ROW_COUNT_CHECK_INTERVAL)) { + state_ = FilterState::CHECK_READY; + } + } else { + // if msg is ready but dynamic disable, collect_sample_info so it can be enabled later + (void)collect_sample_info(0, valid_cnt); + if (!dynamic_disable()) { + state_ = FilterState::ENABLE; + } + } + } + return ret; +} + +// bypass interface for storege white filter +inline int ObExprTopNFilterContext::bypass(const ObExpr &expr, ObEvalCtx &ctx, + ObDynamicFilterExecutor &dynamic_filter, + ObRuntimeFilterParams ¶ms, bool &is_data_prepared) +{ + // for prepare storege white data stage + dynamic_filter.set_filter_action(DynamicFilterAction::PASS_ALL); + return OB_SUCCESS; +} + +// filter interface for eval one row, for storege black filter +inline int ObExprTopNFilterContext::do_process(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res) +{ + int ret = topn_filter_msg_->filter_out_data(expr, ctx, *this, res); + if (dynamic_disable()) { + state_ = FilterState::DYNAMIC_DISABLE; + } + return ret; +} + +// filter interface for eval_batch rows, for storege black filter +inline int ObExprTopNFilterContext::do_process(const ObExpr &expr, ObEvalCtx &ctx, + const ObBitVector &skip, const int64_t batch_size) +{ + int ret = topn_filter_msg_->filter_out_data_batch(expr, ctx, skip, batch_size, *this); + if (dynamic_disable()) { + state_ = FilterState::DYNAMIC_DISABLE; + } + return ret; +} + +// filter interface for eval_vector rows, for storege black filter +inline int ObExprTopNFilterContext::do_process(const ObExpr &expr, ObEvalCtx &ctx, + const ObBitVector &skip, const EvalBound &bound) +{ + int ret = topn_filter_msg_->filter_out_data_vector(expr, ctx, skip, bound, *this); + if (dynamic_disable()) { + state_ = FilterState::DYNAMIC_DISABLE; + } + return ret; +} + +// get filter date from topn filter msg to enable storege white filter +inline int ObExprTopNFilterContext::do_process(const ObExpr &expr, ObEvalCtx &ctx, + ObDynamicFilterExecutor &dynamic_filter, + ObRuntimeFilterParams ¶ms, + bool &is_data_prepared) +{ + // for prepare data stage + int ret = OB_SUCCESS; + if (OB_FAIL(topn_filter_msg_->prepare_storage_white_filter_data(dynamic_filter, ctx, params, + is_data_prepared))) { + LOG_WARN("fail to prepare_storage_white_filter_data", K(ret)); + } else { + if (topn_filter_msg_->is_null_first(dynamic_filter.get_col_idx())) { + dynamic_filter.cmp_func_ = + expr.args_[dynamic_filter.get_col_idx()]->basic_funcs_->null_first_cmp_; + } else { + dynamic_filter.cmp_func_ = + expr.args_[dynamic_filter.get_col_idx()]->basic_funcs_->null_last_cmp_; + } + LOG_TRACE("[TopN Filter] prepare white filter data succ", K(total_count_), K(filter_count_), + K(check_count_), K(topn_filter_msg_->is_null_first(dynamic_filter.get_col_idx()))); + } + return ret; +} + +int ObExprTopNFilterContext::check_filter_ready() +{ + int ret = OB_SUCCESS; + if (is_first_) { + start_time_ = ObTimeUtility::current_time(); + is_first_ = false; + } + // try get msg + ObP2PDatahubMsgBase *basic_msg = topn_filter_msg_; + if (OB_ISNULL(basic_msg)) { + if (OB_FAIL(PX_P2P_DH.atomic_get_msg(topn_filter_key_, basic_msg))) { + if (OB_HASH_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to get msg", K(ret)); + } + } else { + topn_filter_msg_ = static_cast(basic_msg); + } + } + if (OB_SUCC(ret) && OB_NOT_NULL(basic_msg)) { + LOG_TRACE("[TopN Filter] succ get msg from P2PDH", K(topn_filter_key_), K(basic_msg), + K(total_count_)); + } + // try check msg ready + if (OB_SUCC(ret)) { + if (OB_NOT_NULL(topn_filter_msg_) && topn_filter_msg_->check_ready()) { + ready_time_ = ObTimeUtility::current_time(); + state_ = FilterState::ENABLE; + slide_window_.start_to_work(); + } else { + state_ = FilterState::NOT_READY; + } + n_times_ = 0; + n_rows_ = 0; + } + return ret; +} + +int ObExprTopNFilter::calc_result_typeN(ObExprResType &type, ObExprResType *types_stack, + int64_t param_num, ObExprTypeCtx &type_ctx) const +{ + int ret = OB_SUCCESS; + UNUSED(type_ctx); + UNUSED(types_stack); + UNUSED(param_num); + type.set_int32(); + type.set_precision(DEFAULT_PRECISION_FOR_BOOL); + type.set_scale(DEFAULT_SCALE_FOR_INTEGER); + return ret; +} + +int ObExprTopNFilter::cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_expr, + ObExpr &rt_expr) const +{ + int ret = OB_SUCCESS; + UNUSED(expr_cg_ctx); + UNUSED(raw_expr); + rt_expr.eval_func_ = eval_topn_filter; + rt_expr.eval_batch_func_ = eval_topn_filter_batch; + rt_expr.eval_vector_func_ = eval_topn_filter_vector; + return ret; +} + +int ObExprTopNFilter::eval_topn_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res) +{ + int ret = OB_SUCCESS; + uint64_t expr_ctx_id = expr.expr_ctx_id_; + ObExecContext &exec_ctx = ctx.exec_ctx_; + ObExprTopNFilterContext *topn_filter_ctx = nullptr; + if (OB_ISNULL(topn_filter_ctx = static_cast( + exec_ctx.get_expr_op_ctx(expr_ctx_id)))) { + // topn filter ctx may be null in das. + res.set_int(1); + } else { + ret = topn_filter_ctx->state_machine(expr, ctx, res); + } + return ret; +} + +int ObExprTopNFilter::eval_topn_filter_batch(const ObExpr &expr, ObEvalCtx &ctx, + const ObBitVector &skip, const int64_t batch_size) +{ + int ret = OB_SUCCESS; + uint64_t expr_ctx_id = expr.expr_ctx_id_; + ObExecContext &exec_ctx = ctx.exec_ctx_; + ObExprTopNFilterContext *topn_filter_ctx = nullptr; + ObDatum *results = expr.locate_batch_datums(ctx); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + + if (OB_ISNULL(topn_filter_ctx = static_cast( + exec_ctx.get_expr_op_ctx(expr_ctx_id)))) { + // topn filter ctx may be null in das. + if (OB_FAIL(ObBitVector::flip_foreach( + skip, batch_size, [&](int64_t idx) __attribute__((always_inline)) { + eval_flags.set(idx); + results[idx].set_int(1); + return OB_SUCCESS; + }))) {} + } else { + ret = topn_filter_ctx->state_machine(expr, ctx, skip, batch_size); + } + return ret; +} + +int ObExprTopNFilter::eval_topn_filter_vector(const ObExpr &expr, ObEvalCtx &ctx, + const ObBitVector &skip, const EvalBound &bound) +{ + int ret = OB_SUCCESS; + ObExprTopNFilterContext *topn_filter_ctx = nullptr; + if (OB_ISNULL(topn_filter_ctx = static_cast( + ctx.exec_ctx_.get_expr_op_ctx(expr.expr_ctx_id_)))) { + // topn filter ctx may be null in das. + int64_t valid_cnt = 0; + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + VectorFormat res_format = expr.get_format(ctx); + if (VEC_UNIFORM == res_format) { + IntegerUniVec *res_vec = static_cast(expr.get_vector(ctx)); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, false /* calc_valid_cnt */); + } else if (VEC_FIXED == res_format) { + IntegerFixedVec *res_vec = static_cast(expr.get_vector(ctx)); + ret = proc_by_pass(res_vec, skip, bound, valid_cnt, false /* calc_valid_cnt */); + } + if (OB_FAIL(ret)) { + LOG_WARN("failed to proc_by_pass for das", K(res_format), K(ret)); + } else { + eval_flags.set_all(true); + } + } else { + ret = topn_filter_ctx->state_machine(expr, ctx, skip, bound); + } + return ret; +} + +int ObExprTopNFilter::prepare_storage_white_filter_data(const ObExpr &expr, + ObDynamicFilterExecutor &dynamic_filter, + ObEvalCtx &eval_ctx, + ObRuntimeFilterParams ¶ms, + bool &is_data_prepared) +{ + int ret = OB_SUCCESS; + is_data_prepared = false; + uint64_t op_id = expr.expr_ctx_id_; + ObExecContext &exec_ctx = eval_ctx.exec_ctx_; + ObExprTopNFilterContext *topn_filter_ctx = NULL; + // get expr ctx from exec ctx + if (OB_ISNULL(topn_filter_ctx = static_cast( + exec_ctx.get_expr_op_ctx(op_id)))) { + // topn filter ctx may be null in das. + is_data_prepared = true; + dynamic_filter.set_filter_action(DynamicFilterAction::PASS_ALL); + } else { + ret = topn_filter_ctx->state_machine(expr, eval_ctx, dynamic_filter, params, is_data_prepared); + } + return ret; +} + +int ObExprTopNFilter::update_storage_white_filter_data(const ObExpr &expr, + ObDynamicFilterExecutor &dynamic_filter, + ObEvalCtx &eval_ctx, + ObRuntimeFilterParams ¶ms, + bool &is_update) +{ + int ret = OB_SUCCESS; + uint64_t op_id = expr.expr_ctx_id_; + ObExecContext &exec_ctx = eval_ctx.exec_ctx_; + ObExprTopNFilterContext *topn_filter_ctx = + static_cast(exec_ctx.get_expr_op_ctx(op_id)); + if (OB_ISNULL(topn_filter_ctx)) { + // in update stage, means prepare succ, topn_filter_ctx must not null + ret = OB_ERR_UNEXPECTED; + LOG_WARN("topn_filter_ctx must not null during update stage"); + } else if (OB_FAIL(topn_filter_ctx->topn_filter_msg_->update_storage_white_filter_data( + dynamic_filter, params, is_update))) { + LOG_WARN("Failed to update_storage_white_filter_data"); + } + return ret; +} + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/expr/ob_expr_topn_filter.h b/src/sql/engine/expr/ob_expr_topn_filter.h new file mode 100644 index 0000000000..99c8f3f12c --- /dev/null +++ b/src/sql/engine/expr/ob_expr_topn_filter.h @@ -0,0 +1,193 @@ +/** + * Copyright (c) 2024 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#pragma once + +#include "sql/engine/expr/ob_expr_operator.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_share_info.h" +#include "sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObDynamicFilterExecutor; + +using ObRuntimeFilterParams = common::ObSEArray; + +class ObExprTopNFilterContext final: public ObExprOperatorCtx +{ +public: + enum class FilterState + { + NOT_READY = 0, + CHECK_READY = 1, + ENABLE = 2, + DYNAMIC_DISABLE = 3, + }; + + ObExprTopNFilterContext() + : ObExprOperatorCtx(), topn_filter_msg_(nullptr), topn_filter_key_(), cmp_funcs_(), + start_time_(0), ready_time_(0), filter_count_(0), total_count_(0), check_count_(0), + n_times_(0), n_rows_(0), slide_window_(total_count_), flag_(0), + state_(FilterState::NOT_READY), row_selector_(nullptr) + { + is_first_ = true; + } + virtual ~ObExprTopNFilterContext(); + +public: + inline bool need_reset_in_rescan() override final + { + return true; + } + void reset_for_rescan(); + inline bool dynamic_disable() override final + { + return slide_window_.dynamic_disable(); + } + inline void collect_monitor_info(const int64_t filtered_rows_count, + const int64_t check_rows_count, + const int64_t total_rows_count) override final + { + filter_count_ += filtered_rows_count; + check_count_ += check_rows_count; + total_count_ += total_rows_count; + if (FilterState::NOT_READY == state_) { + n_times_ += 1; + n_rows_ += total_rows_count; + if (n_rows_ > ROW_COUNT_CHECK_INTERVAL || n_times_ > EVAL_TIME_CHECK_INTERVAL) { + state_ = FilterState::CHECK_READY; + n_times_ = 0; + n_rows_ = 0; + } + } + } + + inline bool is_data_version_updated(int64_t old_version) override + { + bool bool_ret = false; + if (OB_NOT_NULL(topn_filter_msg_)) { + bool_ret = topn_filter_msg_->get_current_data_version() > old_version; + } + return bool_ret; + } + + inline void reset_monitor_info() { + filter_count_ = 0; + total_count_ = 0; + check_count_ = 0; + n_times_ = 0; + n_rows_ = 0; + ready_time_ = 0; + } + + inline void collect_sample_info(const int64_t filter_count, + const int64_t total_count) override final + { + (void)slide_window_.update_slide_window_info(filter_count, total_count); + } + + template + int state_machine(const ObExpr &expr, ObEvalCtx &ctx, Args &&... args); + +private: + inline int bypass(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res); + inline int bypass(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const int64_t batch_size); + inline int bypass(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const EvalBound &bound); + inline int bypass(const ObExpr &expr, ObEvalCtx &ctx,ObDynamicFilterExecutor &dynamic_filter, + ObRuntimeFilterParams ¶ms, bool &is_data_prepared); + + inline int do_process(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res); + inline int do_process(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const int64_t batch_size); + inline int do_process(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const EvalBound &bound); + inline int do_process(const ObExpr &expr, ObEvalCtx &ctx, ObDynamicFilterExecutor &dynamic_filter, + ObRuntimeFilterParams ¶ms, bool &is_data_prepared); + int check_filter_ready(); + +public: + ObPushDownTopNFilterMsg *topn_filter_msg_; + ObP2PDhKey topn_filter_key_; + // TODO XUNSI: if the sort key(from left table) is join key (of right table), + // and the type of them are different, then the cmp_func should be prepared. + // Not support yet + ObCmpFuncs cmp_funcs_; + int64_t start_time_; + int64_t ready_time_; + int64_t filter_count_; + int64_t total_count_; + int64_t check_count_; + int64_t n_times_; + int64_t n_rows_; + ObAdaptiveFilterSlideWindow slide_window_; + + union + { + uint64_t flag_; + struct + { + bool is_first_ : 1; + int64_t reserved_ : 63; + }; + }; + FilterState state_; + uint16_t *row_selector_; +public: + static const int64_t ROW_COUNT_CHECK_INTERVAL; + static const int64_t EVAL_TIME_CHECK_INTERVAL; +}; + +class ObExprTopNFilter final : public ObExprOperator +{ +public: + explicit ObExprTopNFilter(common::ObIAllocator &alloc) + : ObExprOperator(alloc, T_OP_PUSHDOWN_TOPN_FILTER, "PUSHDOWN_TOPN_FILTER", MORE_THAN_ZERO, + VALID_FOR_GENERATED_COL, NOT_ROW_DIMENSION, INTERNAL_IN_MYSQL_MODE) + {} + ~ObExprTopNFilter() + {} + int calc_result_typeN(ObExprResType &type, ObExprResType *types, int64_t param_num, + common::ObExprTypeCtx &type_ctx) const override; + int cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_expr, ObExpr &rt_expr) const override; + + inline bool need_rt_ctx() const override + { + return true; + } + +public: + static int eval_topn_filter(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res); + static int eval_topn_filter_batch(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const int64_t batch_size); + static int eval_topn_filter_vector(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const EvalBound &bound); + + static int prepare_storage_white_filter_data(const ObExpr &expr, + ObDynamicFilterExecutor &dynamic_filter, + ObEvalCtx &eval_ctx, ObRuntimeFilterParams ¶ms, + bool &is_data_prepared); + + static int update_storage_white_filter_data(const ObExpr &expr, + ObDynamicFilterExecutor &dynamic_filter, + ObEvalCtx &eval_ctx, ObRuntimeFilterParams ¶ms, + bool &is_update); +private: + DISALLOW_COPY_AND_ASSIGN(ObExprTopNFilter); +}; + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/join/hash_join/ob_hash_join_vec_op.cpp b/src/sql/engine/join/hash_join/ob_hash_join_vec_op.cpp index 1c385f90e9..76312991b1 100644 --- a/src/sql/engine/join/hash_join/ob_hash_join_vec_op.cpp +++ b/src/sql/engine/join/hash_join/ob_hash_join_vec_op.cpp @@ -1269,7 +1269,7 @@ int ObHashJoinVecOp::create_partition(bool is_left, int64_t part_id, ObHJPartiti (tmp_batch_round << 32) + part_id, is_left, part))) { LOG_WARN("fail to get partition", K(ret), K(part_level_), K(part_id), K(is_left)); } else if (OB_FAIL(part->init(is_left ? left_->get_spec().output_ : right_->get_spec().output_, - MY_SPEC.max_batch_size_))) { + MY_SPEC.max_batch_size_, MY_SPEC.compress_type_))) { LOG_WARN("fail to init batch", K(ret), K(part_level_), K(part_id), K(is_left)); } else { part->get_row_store().set_dir_id(sql_mem_processor_.get_dir_id()); diff --git a/src/sql/engine/join/hash_join/ob_hj_partition.cpp b/src/sql/engine/join/hash_join/ob_hj_partition.cpp index 08370d1371..8c86f07e4e 100644 --- a/src/sql/engine/join/hash_join/ob_hj_partition.cpp +++ b/src/sql/engine/join/hash_join/ob_hj_partition.cpp @@ -142,7 +142,8 @@ int ObHJPartition::begin_iterator() return ret; } -int ObHJPartition::init(const ObExprPtrIArray &exprs, const int64_t max_batch_size) +int ObHJPartition::init(const ObExprPtrIArray &exprs, const int64_t max_batch_size, + const ObCompressorType compressor_type) { int ret = OB_SUCCESS; ObMemAttr mem_attr(tenant_id_, common::ObModIds::OB_ARENA_HASH_JOIN, ObCtxIds::WORK_AREA); @@ -151,7 +152,8 @@ int ObHJPartition::init(const ObExprPtrIArray &exprs, const int64_t max_batch_si mem_attr, 0/*mem limit*/, true, - 8/*extra_size*/))) { + 8/*extra_size*/, + compressor_type))) { LOG_WARN("failed to init chunk row store", K(ret)); } return ret; diff --git a/src/sql/engine/join/hash_join/ob_hj_partition.h b/src/sql/engine/join/hash_join/ob_hj_partition.h index 0a444dd498..bf4d4966d1 100644 --- a/src/sql/engine/join/hash_join/ob_hj_partition.h +++ b/src/sql/engine/join/hash_join/ob_hj_partition.h @@ -68,7 +68,8 @@ public: return row_store_.add_row(reinterpret_cast(src), stored_row); } - int init(const ObExprPtrIArray &exprs, const int64_t max_batch_size); + int init(const ObExprPtrIArray &exprs, const int64_t max_batch_size, + const common::ObCompressorType compressor_type); int open(); void close(); diff --git a/src/sql/engine/join/ob_join_filter_op.cpp b/src/sql/engine/join/ob_join_filter_op.cpp index d9622ae6d4..287714444f 100644 --- a/src/sql/engine/join/ob_join_filter_op.cpp +++ b/src/sql/engine/join/ob_join_filter_op.cpp @@ -1005,7 +1005,7 @@ int ObJoinFilterOp::open_join_filter_use() ObP2PDhKey dh_key(MY_SPEC.rf_infos_.at(i).p2p_datahub_id_, px_seq_id, task_id); join_filter_ctx->rf_key_ = dh_key; int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); - join_filter_ctx->window_size_ = ADAPTIVE_BF_WINDOW_ORG_SIZE; + join_filter_ctx->slide_window_.set_window_size(ADAPTIVE_BF_WINDOW_ORG_SIZE); join_filter_ctx->max_wait_time_ms_ = filter_input->config_.runtime_filter_wait_time_ms_; join_filter_ctx->hash_funcs_.set_allocator(&ctx_.get_allocator()); join_filter_ctx->cmp_funcs_.set_allocator(&ctx_.get_allocator()); diff --git a/src/sql/engine/ob_operator.cpp b/src/sql/engine/ob_operator.cpp index a73ae36ee8..c47518a4cc 100644 --- a/src/sql/engine/ob_operator.cpp +++ b/src/sql/engine/ob_operator.cpp @@ -618,27 +618,67 @@ int ObOperator::output_expr_sanity_check() int ObOperator::output_expr_sanity_check_batch() { int ret = OB_SUCCESS; - // TODO: add sanity check for different vector formats - // for (int64_t i = 0; OB_SUCC(ret) && i < spec_.output_.count(); ++i) { - // const ObExpr *expr = spec_.output_[i]; - // if (OB_ISNULL(expr)) { - // ret = OB_ERR_UNEXPECTED; - // LOG_WARN("error unexpected, expr is nullptr", K(ret)); - // } else if (OB_FAIL(expr->eval_batch(eval_ctx_, *brs_.skip_, brs_.size_))) { - // LOG_WARN("evaluate expression failed", K(ret)); - // } else if (!expr->is_batch_result()){ - // const ObDatum &datum = expr->locate_expr_datum(eval_ctx_); - // SANITY_CHECK_RANGE(datum.ptr_, datum.len_); - // } else { - // const ObDatum *datums = expr->locate_batch_datums(eval_ctx_); - // for (int64_t j = 0; j < brs_.size_; j++) { - // if (!brs_.skip_->at(j)) { - // SANITY_CHECK_RANGE(datums[j].ptr_, datums[j].len_); - // } - // } - // } - // } + for (int64_t i = 0; OB_SUCC(ret) && i < spec_.output_.count(); ++i) { + const ObExpr *expr = spec_.output_[i]; + if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("error unexpected, expr is nullptr", K(ret)); + } else if (OB_FAIL(expr->eval_vector(eval_ctx_, brs_))) { + LOG_WARN("eval vector failed", K(ret)); + } else { + VectorFormat vec_fmt = expr->get_format(eval_ctx_); + ObIVector *ivec = expr->get_vector(eval_ctx_); + if (vec_fmt == VEC_UNIFORM || vec_fmt == VEC_UNIFORM_CONST) { + ObUniformBase *uni_data = static_cast(ivec); + if (vec_fmt == VEC_UNIFORM_CONST) { + if (brs_.skip_->accumulate_bit_cnt(brs_.size_) < brs_.size_) { + ObDatum &datum = uni_data->get_datums()[0]; + SANITY_CHECK_RANGE(datum.ptr_, datum.len_); + } + } else { + ObDatum *datums = uni_data->get_datums(); + for (int j = 0; j < brs_.size_; j++) { + if (!brs_.skip_->at(j)) { + SANITY_CHECK_RANGE(datums[j].ptr_, datums[j].len_); + } + } + } + } else if (vec_fmt == VEC_FIXED) { + ObFixedLengthBase *fixed_data = static_cast(ivec); + ObBitmapNullVectorBase *nulls = static_cast(ivec); + int32_t len = fixed_data->get_length(); + for (int j = 0; j < brs_.size_; j++) { + if (!brs_.skip_->at(j) && !nulls->is_null(j)) { + SANITY_CHECK_RANGE(fixed_data->get_data() + j * len, len); + } + } + } else if (vec_fmt == VEC_DISCRETE) { + ObDiscreteBase *dis_data = static_cast(ivec); + ObBitmapNullVectorBase *nulls = static_cast(ivec); + char **ptrs = dis_data->get_ptrs(); + ObLength *lens = dis_data->get_lens(); + for (int j = 0; j < brs_.size_; j++) { + if (!brs_.skip_->at(j) && !nulls->is_null(j)) { + SANITY_CHECK_RANGE(ptrs[j], lens[j]); + } + } + } else if (vec_fmt == VEC_CONTINUOUS) { + ObContinuousBase *cont_base = static_cast(ivec); + ObBitmapNullVectorBase *nulls = static_cast(ivec); + uint32_t *offsets = cont_base->get_offsets(); + char *data = cont_base->get_data(); + for (int j = 0; j < brs_.size_; j++) { + if (!brs_.skip_->at(j) && !nulls->is_null(j)) { + SANITY_CHECK_RANGE(data + offsets[j], offsets[j + 1] - offsets[j]); + } + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret), K(vec_fmt)); + } + } + } return ret; } @@ -1318,7 +1358,7 @@ int ObOperator::get_next_batch(const int64_t max_row_cnt, const ObBatchRows *&ba } } #ifdef ENABLE_SANITY - if (OB_SUCC(ret) && !all_filtered) { + if (OB_SUCC(ret) && spec_.use_rich_format_ && !all_filtered) { if (OB_FAIL(output_expr_sanity_check_batch())) { LOG_WARN("output expr sanity check batch failed", K(ret)); } diff --git a/src/sql/engine/ob_operator_factory.cpp b/src/sql/engine/ob_operator_factory.cpp index dcf6d856ab..285bc94fc3 100644 --- a/src/sql/engine/ob_operator_factory.cpp +++ b/src/sql/engine/ob_operator_factory.cpp @@ -93,6 +93,7 @@ #include "sql/optimizer/ob_log_stat_collector.h" #include "sql/optimizer/ob_log_optimizer_stats_gathering.h" #include "sql/engine/aggregate/ob_merge_distinct_op.h" +#include "sql/engine/aggregate/ob_merge_distinct_vec_op.h" #include "sql/engine/aggregate/ob_hash_distinct_op.h" #include "sql/engine/basic/ob_material_op.h" #include "sql/engine/basic/ob_material_vec_op.h" @@ -173,6 +174,11 @@ #include "sql/engine/basic/ob_temp_table_access_vec_op.h" #include "sql/engine/basic/ob_temp_table_transformation_vec_op.h" #include "sql/engine/sort/ob_sort_vec_op.h" +#include "sql/engine/set/ob_hash_union_vec_op.h" +#include "sql/engine/set/ob_hash_union_vec_op.h" +#include "sql/engine/set/ob_hash_intersect_vec_op.h" +#include "sql/engine/set/ob_hash_except_vec_op.h" +#include "sql/engine/window_function/ob_window_function_vec_op.h" #include "sql/optimizer/ob_log_values_table_access.h" #include "sql/engine/basic/ob_values_table_access_op.h" diff --git a/src/sql/engine/ob_operator_reg.h b/src/sql/engine/ob_operator_reg.h index 7666cf80f4..1eda5ca310 100644 --- a/src/sql/engine/ob_operator_reg.h +++ b/src/sql/engine/ob_operator_reg.h @@ -166,6 +166,12 @@ class ObMergeDistinctOp; REGISTER_OPERATOR(ObLogDistinct, PHY_MERGE_DISTINCT, ObMergeDistinctSpec, ObMergeDistinctOp, NOINPUT, VECTORIZED_OP); +class ObLogDistinct; +class ObMergeDistinctVecSpec; +class ObMergeDistinctVecOp; +REGISTER_OPERATOR(ObLogDistinct, PHY_VEC_MERGE_DISTINCT, ObMergeDistinctVecSpec, + ObMergeDistinctVecOp, NOINPUT, VECTORIZED_OP, 0, SUPPORT_RICH_FORMAT); + class ObHashDistinctSpec; class ObHashDistinctOp; REGISTER_OPERATOR(ObLogDistinct, PHY_HASH_DISTINCT, ObHashDistinctSpec, @@ -232,18 +238,39 @@ class ObHashUnionOp; REGISTER_OPERATOR(ObLogSet, PHY_HASH_UNION, ObHashUnionSpec, ObHashUnionOp, NOINPUT, VECTORIZED_OP); +class ObLogSet; +class ObHashUnionVecSpec; +class ObHashUnionVecOp; +REGISTER_OPERATOR(ObLogSet, PHY_VEC_HASH_UNION, ObHashUnionVecSpec, ObHashUnionVecOp, + NOINPUT, VECTORIZED_OP, 0 /*+version*/, + SUPPORT_RICH_FORMAT); + class ObLogSet; class ObHashIntersectSpec; class ObHashIntersectOp; REGISTER_OPERATOR(ObLogSet, PHY_HASH_INTERSECT, ObHashIntersectSpec, ObHashIntersectOp, NOINPUT, VECTORIZED_OP); +class ObLogSet; +class ObHashIntersectVecSpec; +class ObHashIntersectVecOp; +REGISTER_OPERATOR(ObLogSet, PHY_VEC_HASH_INTERSECT, ObHashIntersectVecSpec, ObHashIntersectVecOp, + NOINPUT, VECTORIZED_OP, 0 /*+version*/, + SUPPORT_RICH_FORMAT); + class ObLogSet; class ObHashExceptSpec; class ObHashExceptOp; REGISTER_OPERATOR(ObLogSet, PHY_HASH_EXCEPT, ObHashExceptSpec, ObHashExceptOp, NOINPUT, VECTORIZED_OP); +class ObLogSet; +class ObHashExceptVecSpec; +class ObHashExceptVecOp; +REGISTER_OPERATOR(ObLogSet, PHY_VEC_HASH_EXCEPT, ObHashExceptVecSpec, ObHashExceptVecOp, + NOINPUT, VECTORIZED_OP, 0 /*+version*/, + SUPPORT_RICH_FORMAT); + class ObLogSet; class ObMergeUnionSpec; class ObMergeUnionOp; @@ -507,9 +534,16 @@ class ObLogWindowFunction; class ObWindowFunctionSpec; class ObWindowFunctionOp; class ObWindowFunctionOpInput; +class ObWindowFunctionVecOpInput; +class ObWindowFunctionVecOp; +class ObWindowFunctionVecSpec; REGISTER_OPERATOR(ObLogWindowFunction, PHY_WINDOW_FUNCTION, ObWindowFunctionSpec, ObWindowFunctionOp, ObWindowFunctionOpInput, VECTORIZED_OP); +REGISTER_OPERATOR(ObLogWindowFunction, PHY_VEC_WINDOW_FUNCTION, ObWindowFunctionVecSpec, + ObWindowFunctionVecOp, ObWindowFunctionVecOpInput, VECTORIZED_OP, 0, + SUPPORT_RICH_FORMAT); + class ObLogJoin; class ObMergeJoinSpec; class ObMergeJoinOp; diff --git a/src/sql/engine/ob_phy_operator_type.h b/src/sql/engine/ob_phy_operator_type.h index 6948a80866..39e5212f70 100644 --- a/src/sql/engine/ob_phy_operator_type.h +++ b/src/sql/engine/ob_phy_operator_type.h @@ -134,6 +134,11 @@ PHY_OP_DEF(PHY_VEC_TEMP_TABLE_INSERT) PHY_OP_DEF(PHY_VEC_TEMP_TABLE_ACCESS) PHY_OP_DEF(PHY_VEC_TEMP_TABLE_TRANSFORMATION) PHY_OP_DEF(PHY_VEC_SORT) +PHY_OP_DEF(PHY_VEC_MERGE_DISTINCT) +PHY_OP_DEF(PHY_VEC_HASH_UNION) +PHY_OP_DEF(PHY_VEC_HASH_INTERSECT) +PHY_OP_DEF(PHY_VEC_HASH_EXCEPT) +PHY_OP_DEF(PHY_VEC_WINDOW_FUNCTION) PHY_OP_DEF(PHY_END) #endif /*PHY_OP_DEF*/ diff --git a/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.cpp b/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.cpp index 8cf5ed7d11..a63d3d13f9 100644 --- a/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.cpp +++ b/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.cpp @@ -15,6 +15,7 @@ #include "sql/engine/px/datahub/components/ob_dh_range_dist_wf.h" #include "sql/engine/px/ob_px_util.h" #include "sql/engine/window_function/ob_window_function_op.h" +#include "sql/engine/window_function/ob_window_function_vec_op.h" namespace oceanbase { @@ -339,5 +340,346 @@ int ObRDWFPieceMsgListener::on_message(ObRDWFPieceMsgCtx &ctx, return ret; } +// RDWinFuncPXPieceMsg/RDWinFuncPXWholeMsg + +OB_DEF_SERIALIZE_SIZE(RDWinFuncPXPartialInfo) +{ + int64_t len = 0; + OB_UNIS_ADD_LEN(sqc_id_); + OB_UNIS_ADD_LEN(thread_id_); + OB_UNIS_ADD_LEN(row_meta_); + int32_t first_row_size = 0, last_row_size = 0; + OB_UNIS_ADD_LEN(first_row_size); + if (first_row_ != nullptr) { + first_row_size = first_row_->get_row_size(); + } + len += first_row_size; + + OB_UNIS_ADD_LEN(last_row_size); + if (last_row_ != nullptr) { + last_row_size = last_row_->get_row_size(); + } + len += last_row_size; + return len; +} + +OB_DEF_SERIALIZE(RDWinFuncPXPartialInfo) +{ + int ret = OB_SUCCESS; + OB_UNIS_ENCODE(sqc_id_); + OB_UNIS_ENCODE(thread_id_); + OB_UNIS_ENCODE(row_meta_); + int32_t row_size = 0; + if (first_row_ != nullptr) { + row_size = first_row_->get_row_size(); + OB_UNIS_ENCODE(row_size); + MEMCPY(buf + pos, first_row_, first_row_->get_row_size()); + pos += first_row_->get_row_size(); + } else { + OB_UNIS_ENCODE(row_size); + } + row_size = 0; + if (last_row_ != nullptr) { + row_size = last_row_->get_row_size(); + OB_UNIS_ENCODE(row_size); + MEMCPY(buf + pos, last_row_, last_row_->get_row_size()); + pos += last_row_->get_row_size(); + } else { + OB_UNIS_ENCODE(row_size) + } + return ret; +} + +OB_DEF_DESERIALIZE(RDWinFuncPXPartialInfo) +{ + int ret = OB_SUCCESS; + LST_DO_CODE(OB_UNIS_DECODE, sqc_id_, thread_id_, row_meta_); + int64_t row_size = 0; + OB_UNIS_DECODE(row_size); + void *row_buf = nullptr; + if (row_size > 0) { + if (OB_ISNULL(row_buf = alloc_.alloc(row_size))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(row_buf, buf + pos, row_size); + first_row_ = reinterpret_cast(row_buf); + pos += row_size; + } + } else { + first_row_ = nullptr; + } + OB_UNIS_DECODE(row_size); + if (row_size > 0) { + if (OB_ISNULL(row_buf = alloc_.alloc(row_size))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(row_buf, buf + pos, row_size); + last_row_ = reinterpret_cast(row_buf); + pos += row_size; + } + } else { + last_row_ = nullptr; + } + return ret; +} + +OB_SERIALIZE_MEMBER((RDWinFuncPXPieceMsg, + ObDatahubPieceMsg), + info_); + +OB_DEF_SERIALIZE_SIZE(RDWinFuncPXWholeMsg) +{ + int64_t len = ObDatahubWholeMsg::get_serialize_size(); + int64_t cnt = infos_.count(); + OB_UNIS_ADD_LEN(cnt); + FOREACH_CNT(info, infos_) { + OB_UNIS_ADD_LEN(**info); + } + return len; +} + +OB_DEF_SERIALIZE(RDWinFuncPXWholeMsg) +{ + int ret = ObDatahubWholeMsg::serialize(buf, buf_len, pos); + if (OB_SUCC(ret)) { + int64_t cnt = infos_.count(); + OB_UNIS_ENCODE(cnt); + FOREACH_CNT_X(info, infos_, OB_SUCC(ret)) { + OB_UNIS_ENCODE(**info); + } + } + return ret; +} + +OB_DEF_DESERIALIZE(RDWinFuncPXWholeMsg) +{ + int ret = ObDatahubWholeMsg::deserialize(buf, data_len, pos); + if (OB_SUCC(ret)) { + int64_t cnt = 0; + OB_UNIS_DECODE(cnt); + for (int i = 0; OB_SUCC(ret) && i < cnt; i++) { + RDWinFuncPXPartialInfo *info = OB_NEWx(RDWinFuncPXPartialInfo, &arena_alloc_, arena_alloc_); + if (OB_ISNULL(info)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + OB_UNIS_DECODE(*info); + if (OB_SUCC(ret) && OB_FAIL(infos_.push_back(info))) { + LOG_WARN("push back element failed", K(ret)); + } + } + } + } + return ret; +} + +ObCompactRow *RDWinFuncPXPartialInfo::dup_store_row(common::ObArenaAllocator &alloc, const ObCompactRow &row) +{ + ObCompactRow *copied_row = reinterpret_cast(alloc.alloc(row.get_row_size())); + if (copied_row != nullptr) { + MEMCPY(copied_row, &row, row.get_row_size()); + } + return copied_row; +} + +RDWinFuncPXPartialInfo *RDWinFuncPXPartialInfo::dup(common::ObArenaAllocator &alloc) const +{ + int ret = OB_SUCCESS; + RDWinFuncPXPartialInfo *info = OB_NEWx(RDWinFuncPXPartialInfo, &alloc, alloc); + if (OB_ISNULL(info)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + info->sqc_id_ = sqc_id_; + info->thread_id_ = thread_id_; + if (OB_FAIL(info->row_meta_.deep_copy(row_meta_, &alloc))) { + LOG_WARN("deep copy row meta failed", K(ret)); + } else if (first_row_ != nullptr) { + info->first_row_ = dup_store_row(alloc, *first_row_); + if (OB_ISNULL(info->first_row_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (last_row_ != nullptr) { + info->last_row_ = dup_store_row(alloc, *last_row_); + if (OB_ISNULL(info->last_row_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + } + return (OB_SUCC(ret)) ? info : nullptr; +} + +int RDWinFuncPXPieceMsgCtx::alloc_piece_msg_ctx(const RDWinFuncPXPieceMsg &pkt, + ObPxCoordInfo &coord_info, ObExecContext &ctx, + int64_t task_cnt, ObPieceMsgCtx *&msg_ctx) +{ + int ret = OB_SUCCESS; + msg_ctx = nullptr; + if (OB_ISNULL(ctx.get_my_session()) || OB_ISNULL(ctx.get_physical_plan_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null arguments", K(ret), K(ctx.get_my_session()), K(ctx.get_physical_plan_ctx())); + } else { + common::ObMemAttr attr(ctx.get_my_session()->get_effective_tenant_id(), + ObModIds::OB_SQL_WINDOW_FUNC, ObCtxIds::WORK_AREA); + msg_ctx = OB_NEWx(RDWinFuncPXPieceMsgCtx, &(ctx.get_allocator()), pkt.op_id_, task_cnt, + ctx.get_physical_plan_ctx()->get_timeout_timestamp(), ctx, attr); + if (OB_ISNULL(msg_ctx)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + return ret; +} + +struct __part_info_cmp_op +{ + OB_INLINE bool operator()(RDWinFuncPXPartialInfo *l, RDWinFuncPXPartialInfo *r) + { + return std::tie(l->sqc_id_, l->thread_id_) < std::tie(r->sqc_id_, r->thread_id_); + } +}; + +struct __sqc_id_cmp_op +{ + OB_INLINE bool operator()(RDWinFuncPXPartialInfo *info, int64_t id) + { + return info->sqc_id_ < id; + } +}; + +int RDWinFuncPXPieceMsgCtx::send_whole_msg(common::ObIArray &sqcs) +{ + int ret = OB_SUCCESS; + ObOperatorKit *op_kit = exec_ctx_.get_operator_kit(op_id_); + if (NULL == op_kit || NULL == op_kit->spec_ || PHY_VEC_WINDOW_FUNCTION != op_kit->spec_->type_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("no window function operator", K(ret), K(op_kit), K(op_id_)); + } else { + ObEvalCtx eval_ctx(exec_ctx_); + const ObWindowFunctionVecSpec *wf = static_cast(op_kit->spec_); + if (OB_FAIL(wf->rd_generate_patch(*this, eval_ctx))) { + LOG_WARN("generate patch failed", K(ret)); + } else { + std::sort(infos_.begin(), infos_.end(), __part_info_cmp_op()); + } + RDWinFuncPXWholeMsg *responses = nullptr; + if (OB_SUCC(ret)) { + responses = static_cast(arena_alloc_.alloc(sizeof(RDWinFuncPXWholeMsg) * sqcs.count())); + if (OB_ISNULL(responses)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + common::ObMemAttr attr(exec_ctx_.get_my_session()->get_effective_tenant_id(), + arena_alloc_.get_label(), ObCtxIds::WORK_AREA); + for (int i = 0; i < sqcs.count(); i++) { + new (&responses[i]) RDWinFuncPXWholeMsg(attr); + } + } + } + dtl::ObDtlChannel *ch = nullptr; + for (int i = 0; OB_SUCC(ret) && i < sqcs.count(); i++) { + ObPxSqcMeta &sqc = *sqcs.at(i); + RDWinFuncPXWholeMsg &msg = responses[i]; + decltype(infos_)::iterator it = std::lower_bound(infos_.begin(), infos_.end(), sqc.get_sqc_id(), __sqc_id_cmp_op()); + if (it == infos_.end() || (*it)->sqc_id_ != sqc.get_sqc_id()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("sqc not found", K(ret), K(sqc)); + } else { + msg.op_id_ = op_id_; + while (OB_SUCC(ret) && it != infos_.end() && (*it)->sqc_id_ == sqc.get_sqc_id()) { + if (OB_FAIL(msg.infos_.push_back((*it)))) { + LOG_WARN("push back element failed", K(ret)); + } else { + it++; + } + } + if (OB_FAIL(ret)) { + } else if (OB_ISNULL(ch = sqc.get_qc_channel())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null channel", K(ret)); + } else if (OB_FAIL(ch->send(msg, timeout_ts_))) { + LOG_WARN("send msg failed", K(ret)); + } else if (OB_FAIL(ch->flush(true, false))) { + LOG_WARN("flush failed", K(ret)); + } + } + } // end for + if (OB_FAIL(ret)) { + } else if (OB_FAIL(ObPxChannelUtil::sqcs_channles_asyn_wait(sqcs))) { + LOG_WARN("async wait failed", K(ret)); + } + for (int i = 0; OB_SUCC(ret) && responses != nullptr && i < sqcs.count(); i++) { + responses[i].~RDWinFuncPXWholeMsg(); + } + } + return ret; +} + +void RDWinFuncPXPieceMsgCtx::reset_resource() +{ + received_ = 0; + infos_.reset(); + arena_alloc_.reset(); +} + +int RDWinFuncPXWholeMsg::assign(const RDWinFuncPXWholeMsg &other) +{ + int ret = OB_SUCCESS; + op_id_ = other.op_id_; + infos_.reset(); + FOREACH_CNT_X(info, other.infos_, OB_SUCC(ret)) { + if (OB_ISNULL(*info)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null info", K(ret)); + } else { + RDWinFuncPXPartialInfo *dup_info = (*info)->dup(arena_alloc_); + if (OB_ISNULL(dup_info)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(infos_.push_back(dup_info))) { + LOG_WARN("push back element failed", K(ret)); + } + } + } + return ret; +} + +int RDWinFuncPXPieceMsgListener::on_message(RDWinFuncPXPieceMsgCtx &ctx, + common::ObIArray &sqcs, + const RDWinFuncPXPieceMsg &pkt) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(pkt.op_id_ != ctx.op_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected piece msg", K(ret), K(pkt), K(ctx)); + } else if (ctx.received_ >= ctx.task_cnt_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("already got all pkts, expect no more", K(ret)); + } else { + ctx.received_ += 1; + LOG_TRACE("get range distribution window function piece msg", K(pkt.info_), K(ctx.received_), + K(ctx.task_cnt_), K(pkt.thread_id_)); + RDWinFuncPXPartialInfo *info = pkt.info_.dup(ctx.arena_alloc_); + if (OB_ISNULL(info)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(ctx.infos_.push_back(info))) { + LOG_WARN("push back element failed", K(ret)); + } else if (ctx.received_ == ctx.task_cnt_) { + if (OB_FAIL(ctx.send_whole_msg(sqcs))) { + LOG_WARN("send whole msg failed", K(ret)); + } + IGNORE_RETURN ctx.reset_resource(); + } + } + return ret; +} + } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.h b/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.h index 42e1d36f29..04304869b0 100644 --- a/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.h +++ b/src/sql/engine/px/datahub/components/ob_dh_range_dist_wf.h @@ -155,6 +155,167 @@ public: typedef ObPieceMsgP ObRDWFPieceMsgP; typedef ObWholeMsgP ObRDWFWholeMsgP; +// Piece/Whole Msg used for vectorization 2.0 +class RDWinFuncPXPieceMsgListener; +class RDWinFuncPXPieceMsgCtx; +class RDWinFuncPXPartialInfo +{ + OB_UNIS_VERSION_V(1); +public: + RDWinFuncPXPartialInfo(common::ObArenaAllocator &alloc) : + sqc_id_(0), thread_id_(0), row_meta_(&alloc), first_row_(nullptr), last_row_(nullptr), + alloc_(alloc) + {} + RDWinFuncPXPartialInfo *dup(common::ObArenaAllocator &alloc) const; + + static ObCompactRow *dup_store_row(common::ObArenaAllocator &alloc, const ObCompactRow &row); + + TO_STRING_KV(K(sqc_id_), + K(thread_id_), + K(first_row_), + "first_row_frame_offset", + NULL == first_row_ ? -1 : *reinterpret_cast(first_row_->get_extra_payload(row_meta_)), + K(last_row_), + "last_row_frame_offset", + NULL == last_row_ ? -1 : *reinterpret_cast(last_row_->get_extra_payload(row_meta_))); + void reset() + { + sqc_id_ = 0; + thread_id_ = 0; + row_meta_.reset(); + first_row_ = nullptr; + last_row_ = nullptr; + } + + int64_t &first_row_frame_offset() + { + return *reinterpret_cast(first_row_->get_extra_payload(row_meta_)); + } + + int64_t &last_row_frame_offset() + { + return *reinterpret_cast(last_row_->get_extra_payload(row_meta_)); + } + + void get_cell(int64_t cell_idx, bool first_row, const char *&payload, int32_t &len) + { + (first_row) ? (first_row_->get_cell_payload(row_meta_, cell_idx, payload, len)) : + (last_row_->get_cell_payload(row_meta_, cell_idx, payload, len)); + } + + const char* get_cell(int64_t cell_idx, bool first_row) + { + return (first_row) ? (first_row_->get_cell_payload(row_meta_, cell_idx)) : + (last_row_->get_cell_payload(row_meta_, cell_idx)); + } + + bool is_null(int64_t cell_idx, bool first_row) const + { + if (first_row && first_row_ == nullptr) { + return true; + } else if (!first_row && last_row_ == nullptr) { + return true; + } else { + return (first_row) ? (first_row_->is_null(cell_idx)) : (last_row_->is_null(cell_idx)); + } + } + +public: + int64_t sqc_id_; + int64_t thread_id_; + RowMeta row_meta_; + ObCompactRow *first_row_; + ObCompactRow *last_row_; + common::ObArenaAllocator &alloc_; +}; + +class RDWinFuncPXPieceMsg : public ObDatahubPieceMsg +{ + OB_UNIS_VERSION_V(1); + +public: + using PieceMsgListener = RDWinFuncPXPieceMsgListener; + using PieceMsgCtx = RDWinFuncPXPieceMsgCtx; + RDWinFuncPXPieceMsg(const common::ObMemAttr &mem_attr) : + arena_alloc_(mem_attr), info_(arena_alloc_) + {} + + RDWinFuncPXPieceMsg() : arena_alloc_(), info_(arena_alloc_) + {} + + void reset() + { + info_.reset(); + arena_alloc_.reset(); + } + + INHERIT_TO_STRING_KV("meta", ObDatahubPieceMsg, + K_(info)); + + common::ObArenaAllocator arena_alloc_; + RDWinFuncPXPartialInfo info_; +}; + +class RDWinFuncPXWholeMsg: public ObDatahubWholeMsg +{ + OB_UNIS_VERSION_V(1); +public: + using WholeMsgProvider = ObWholeMsgProvider; + RDWinFuncPXWholeMsg(): infos_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(arena_alloc_)) {} + + RDWinFuncPXWholeMsg(const common::ObMemAttr &attr) : + arena_alloc_(attr), infos_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(arena_alloc_)) + {} + + void reset() + { + infos_.reset(); + arena_alloc_.reset(); + } + + int assign(const RDWinFuncPXWholeMsg &other); + common::ObArenaAllocator arena_alloc_; + common::ObSEArray infos_; +}; + +class RDWinFuncPXPieceMsgCtx: public ObPieceMsgCtx +{ +public: + RDWinFuncPXPieceMsgCtx(uint64_t op_id, int64_t task_cnt, int64_t timeout_ts, + ObExecContext &exec_ctx, const common::ObMemAttr &attr) : + ObPieceMsgCtx(op_id, task_cnt, timeout_ts), + received_(0), arena_alloc_(attr), + infos_(OB_MALLOC_NORMAL_BLOCK_SIZE, ModulePageAllocator(arena_alloc_)), exec_ctx_(exec_ctx), + eval_ctx_(exec_ctx) + {} + + virtual int send_whole_msg(common::ObIArray &sqcs) override; + virtual void reset_resource() override; + static int alloc_piece_msg_ctx(const RDWinFuncPXPieceMsg &pkt, + ObPxCoordInfo &coord_info, + ObExecContext &ctx, + int64_t task_cnt, + ObPieceMsgCtx *&msg_ctx); + int64_t received_; + common::ObArenaAllocator arena_alloc_; + common::ObSEArray infos_; + ObExecContext &exec_ctx_; + ObEvalCtx eval_ctx_; +private: + DISALLOW_COPY_AND_ASSIGN(RDWinFuncPXPieceMsgCtx); +}; + +class RDWinFuncPXPieceMsgListener +{ +public: + static int on_message(RDWinFuncPXPieceMsgCtx &ctx, + common::ObIArray &sqcs, + const RDWinFuncPXPieceMsg &pkt); +}; + +typedef ObPieceMsgP ObRDWinFuncPXPieceMsgP; +typedef ObWholeMsgP ObRDWinFuncPXWholeMsgP; + } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/engine/px/datahub/components/ob_dh_sample.cpp b/src/sql/engine/px/datahub/components/ob_dh_sample.cpp index 0d6f8acbfc..3846a41260 100644 --- a/src/sql/engine/px/datahub/components/ob_dh_sample.cpp +++ b/src/sql/engine/px/datahub/components/ob_dh_sample.cpp @@ -484,8 +484,19 @@ int ObDynamicSamplePieceMsgCtx::split_range( LOG_WARN("reserve datum key failed", K(ret), K(sort_def_.exprs_->count())); } for (int64_t i = 0; OB_SUCC(ret) && i < sort_def_.exprs_->count(); ++i) { - if (OB_FAIL(copied_key.push_back(ObDatum()))) { - LOG_WARN("push back empty datum failed", K(ret), K(i)); + ObExpr *expr = sort_def_.exprs_->at(i); + if (coord_.get_spec().use_rich_format_ && + !is_uniform_format(expr->get_format(coord_.get_eval_ctx()))) { + if (OB_FAIL(expr->init_vector(coord_.get_eval_ctx(), + expr->is_const_expr() ? VEC_UNIFORM_CONST : VEC_UNIFORM, + coord_.get_eval_ctx().get_batch_size()))) { + LOG_WARN("expr init vector failed", K(ret), K(i)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(copied_key.push_back(ObDatum()))) { + LOG_WARN("push back empty datum failed", K(ret), K(i)); + } } } while (OB_SUCC(ret) && !sort_iter_end && tmp_key_count < expect_range_count) { diff --git a/src/sql/engine/px/datahub/components/ob_dh_winbuf.cpp b/src/sql/engine/px/datahub/components/ob_dh_winbuf.cpp index 7c7a8ce26f..b46bb7dd1c 100644 --- a/src/sql/engine/px/datahub/components/ob_dh_winbuf.cpp +++ b/src/sql/engine/px/datahub/components/ob_dh_winbuf.cpp @@ -347,3 +347,266 @@ int ObWinbufWholeMsg::assign(const ObWinbufWholeMsg &other, common::ObIAllocator } return ret; } + +// ================ SPWinFuncPXPieceMsg/SPWinFuncPXWholeMsg +int SPWinFuncPXPieceMsgCtx::alloc_piece_msg_ctx(const SPWinFuncPXPieceMsg &pkt, + ObPxCoordInfo &coord_info, ObExecContext &ctx, + int64_t task_cnt, ObPieceMsgCtx *&msg_ctx) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(ctx.get_my_session()) || OB_ISNULL(ctx.get_physical_plan_ctx()) + || OB_ISNULL(ctx.get_physical_plan_ctx()->get_phy_plan())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("session or physical plan ctx is null", K(ret), K(ctx.get_my_session()), + K(ctx.get_physical_plan_ctx()), K(ctx.get_physical_plan_ctx()->get_phy_plan())); + } else { + uint64_t tenant_id = ctx.get_my_session()->get_effective_tenant_id(); + common::ObMemAttr mem_attr(tenant_id, ObModIds::OB_SQL_WINDOW_LOCAL, ObCtxIds::WORK_AREA); + void *buf = ctx.get_allocator().alloc(sizeof(SPWinFuncPXPieceMsgCtx)); + if (OB_ISNULL(buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + msg_ctx = new (buf) SPWinFuncPXPieceMsgCtx( + pkt.op_id_, task_cnt, ctx.get_physical_plan_ctx()->get_timeout_timestamp(), tenant_id, + ctx.get_physical_plan_ctx()->get_phy_plan()->get_batch_size(), mem_attr); + LOG_DEBUG("init msg ctx", K(pkt.op_id_), K(task_cnt), K(tenant_id), + K(ctx.get_physical_plan_ctx()->get_phy_plan()->get_batch_size())); + } + } + return ret; +} + +void SPWinFuncPXPieceMsgCtx::reset_resource() +{ + whole_msg_.reset(); + received_ = 0; +} + +int SPWinFuncPXPieceMsgCtx::send_whole_msg(common::ObIArray &sqcs) +{ + int ret = OB_SUCCESS; + whole_msg_.op_id_ = op_id_; + whole_msg_.is_empty_ = (!whole_msg_.row_store_.is_inited()); + ARRAY_FOREACH_X(sqcs, idx, cnt, OB_SUCC(ret)) { + dtl::ObDtlChannel *ch = sqcs.at(idx)->get_qc_channel(); + if (OB_ISNULL(ch)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null channel", K(ret), K(ch)); + } else if (OB_FAIL(ch->send(whole_msg_, timeout_ts_))) { + LOG_WARN("send data failed", K(ret)); + } else if (OB_FAIL(ch->flush(true, false))) { + LOG_WARN("flush dtl data failed", K(ret)); + } else { + LOG_DEBUG("dispatched sp_winfunc_px_whole_msg", K(idx), K(cnt), K(whole_msg_), K(*ch)); + } + } + if (OB_SUCC(ret) && OB_FAIL(ObPxChannelUtil::sqcs_channles_asyn_wait(sqcs))) { + LOG_WARN("failed to wait response", K(ret)); + } + return ret; +} + +int SPWinFuncPXPieceMsgListener::on_message(SPWinFuncPXPieceMsgCtx &ctx, + common::ObIArray &sqcs, + const SPWinFuncPXPieceMsg &pkt) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(pkt.op_id_ != ctx.op_id_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected piece msg", K(ret), K(pkt), K(ctx)); + } else if (ctx.received_ >= ctx.task_cnt_) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("already got all pkts, expect no more", K(ret)); + } else if (pkt.is_empty_) { + // do nothing + } else { + if (!ctx.whole_msg_.row_store_.is_inited()) { + // init row_store + common::ObMemAttr attr(ctx.tenant_id_, ObModIds::OB_SQL_WINDOW_LOCAL, ObCtxIds::WORK_AREA); + if (OB_FAIL( + ctx.whole_msg_.row_meta_.deep_copy(pkt.row_meta_, &ctx.whole_msg_.assign_allocator_))) { + LOG_WARN("deep copy row meta failed", K(ret)); + } else if (OB_FAIL(ctx.whole_msg_.row_store_.init(ctx.whole_msg_.row_meta_, + ctx.max_batch_size_, attr, 0, false, + NONE_COMPRESSOR))) { + LOG_WARN("init row store failed", K(ret)); + } + } + ObCompactRow *sr = nullptr; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(ctx.whole_msg_.row_store_.add_row(pkt.row_, sr))) { + LOG_WARN("add row failed", K(ret)); + } else { + } + } + if (OB_SUCC(ret)) { + ctx.received_++; + LOG_TRACE("got a peice msg", "all_got", ctx.received_, "expected", ctx.task_cnt_, + K(ctx.whole_msg_), K(pkt.thread_id_)); + if (ctx.received_ == ctx.task_cnt_) { + if (OB_FAIL(ctx.send_whole_msg(sqcs))) { + LOG_WARN("send whole msg failed", K(ret)); + } else { + IGNORE_RETURN ctx.reset_resource(); + } + } + } + return ret; +} + +int SPWinFuncPXWholeMsg::assign(const SPWinFuncPXWholeMsg &other, common::ObIAllocator *allocator) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(allocator)) { + allocator = &assign_allocator_; + } + row_meta_.set_allocator(allocator); + is_empty_ = other.is_empty_; + if (OB_FAIL(row_meta_.deep_copy(other.row_meta_, allocator))) { + LOG_WARN("deep copy row meta failed", K(ret)); + } else if (!is_empty_) { + // deep copy row store by ser/deser + int64_t ser_len = 0, ser_pos = 0, des_pos = 0; + void *ser_buf = nullptr; + ser_len = other.row_store_.get_serialize_size(); + row_store_.reset(); + const int64_t constexpr read_batch = 256; + int64_t read_rows = 0; + const ObCompactRow *stored_rows[read_batch] = {nullptr}; + ObTempRowStore::Iterator other_it; + ObTempRowStore ©ing_store = const_cast(other.row_store_); + if (OB_FAIL(row_store_.init(other.row_store_.get_row_meta(), + other.row_store_.get_max_batch_size(), + other.row_store_.get_mem_attr(), 0, false, NONE_COMPRESSOR))) { + LOG_WARN("init temp row store failed", K(ret)); + } else if (OB_FAIL(other_it.init(©ing_store))) { + LOG_WARN("init row store iterator failed", K(ret)); + } + while (OB_SUCC(ret)) { + if (OB_FAIL(other_it.get_next_batch(read_batch, read_rows, stored_rows))) { + if (OB_LIKELY(ret == OB_ITER_END)) { + } else { + LOG_WARN("get batch rows failed", K(ret)); + } + } else if (OB_UNLIKELY(read_rows == 0)) { + ret = OB_ITER_END; + } + if (ret == OB_ITER_END) { + ret = OB_SUCCESS; + break; + } + ObCompactRow *sr_row = nullptr; + for (int i = 0; OB_SUCC(ret) && i < read_rows; i++) { + if (OB_FAIL(row_store_.add_row(stored_rows[i], sr_row))) { + LOG_WARN("add row failed", K(ret)); + } + } + } + } + return ret; +} + +OB_DEF_SERIALIZE(SPWinFuncPXPieceMsg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObDatahubPieceMsg::serialize(buf, buf_len, pos))) { + LOG_WARN("datahub ser failed", K(ret)); + } else { + OB_UNIS_ENCODE(is_empty_); + OB_UNIS_ENCODE(row_meta_); + if (OB_SUCC(ret) && !is_empty_) { + OB_UNIS_ENCODE(row_->get_row_size()); + if (OB_SUCC(ret)) { + MEMCPY(buf + pos, row_, row_->get_row_size()); + pos += row_->get_row_size(); + } + } + } + return ret; +} + +OB_DEF_DESERIALIZE(SPWinFuncPXPieceMsg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObDatahubPieceMsg::deserialize(buf, data_len, pos))) { + LOG_WARN("datahub deser failed", K(ret)); + } else { + OB_UNIS_DECODE(is_empty_); + OB_UNIS_DECODE(row_meta_); + if (OB_SUCC(ret) && !is_empty_) { + int64_t row_size = 0; + void *row_buf = nullptr; + OB_UNIS_DECODE(row_size); + if (OB_UNLIKELY(row_size <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected row size", K(ret)); + } else { + if (OB_ISNULL(row_buf = deserial_allocator_.alloc(row_size))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(row_buf, buf + pos, row_size); + row_ = reinterpret_cast(row_buf); + row_->set_row_size(row_size); + pos += row_size; + } + } + } + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(SPWinFuncPXPieceMsg) +{ + int64_t len = 0; + len += ObDatahubPieceMsg::get_serialize_size(); + OB_UNIS_ADD_LEN(is_empty_); + OB_UNIS_ADD_LEN(row_meta_); + if (!is_empty_) { + OB_UNIS_ADD_LEN(row_->get_row_size()); + len += row_->get_row_size(); + } + return len; +} + +OB_DEF_SERIALIZE(SPWinFuncPXWholeMsg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObDatahubWholeMsg::serialize(buf, buf_len, pos))) { + LOG_WARN("whole msg ser failed", K(ret)); + } else { + OB_UNIS_ENCODE(is_empty_); + OB_UNIS_ENCODE(row_meta_); + if (!is_empty_) { + OB_UNIS_ENCODE(row_store_); + } + } + return ret; +} + +OB_DEF_DESERIALIZE(SPWinFuncPXWholeMsg) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObDatahubWholeMsg::deserialize(buf, data_len, pos))) { + LOG_WARN("whole msg deser failed", K(ret)); + } else { + OB_UNIS_DECODE(is_empty_); + OB_UNIS_DECODE(row_meta_); + if (!is_empty_) { + OB_UNIS_DECODE(row_store_); + } + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(SPWinFuncPXWholeMsg) +{ + int64_t len = ObDatahubWholeMsg::get_serialize_size(); + OB_UNIS_ADD_LEN(is_empty_); + OB_UNIS_ADD_LEN(row_meta_); + if(!is_empty_) { + OB_UNIS_ADD_LEN(row_store_); + } + return len; +} \ No newline at end of file diff --git a/src/sql/engine/px/datahub/components/ob_dh_winbuf.h b/src/sql/engine/px/datahub/components/ob_dh_winbuf.h index 215c21d518..1d25f965dc 100644 --- a/src/sql/engine/px/datahub/components/ob_dh_winbuf.h +++ b/src/sql/engine/px/datahub/components/ob_dh_winbuf.h @@ -19,6 +19,7 @@ #include "sql/engine/px/datahub/ob_dh_msg_provider.h" #include "sql/engine/basic/ob_chunk_row_store.h" #include "sql/engine/basic/ob_chunk_datum_store.h" +#include "sql/engine/basic/ob_temp_row_store.h" namespace oceanbase { @@ -136,7 +137,108 @@ private: DISALLOW_COPY_AND_ASSIGN(ObWinbufPieceMsgListener); }; +// piece/whole msg for single part winfunc parallel execution +// used for ObWindowFunctionVecOp +// SP stands for Single Partition +class SPWinFuncPXPieceMsg; +class SPWinFuncPXWholeMsg; +typedef ObPieceMsgP ObSPWinFuncPXPieceMsgP; +typedef ObWholeMsgP ObSPWinFuncPXWholeMsgP; +class SPWinFuncPXPieceMsgListener; +class SPWinFuncPXPieceMsgCtx; + +class SPWinFuncPXPieceMsg: public ObDatahubPieceMsg +{ + OB_UNIS_VERSION_V(1); +public: + using PieceMsgListener = SPWinFuncPXPieceMsgListener; + using PieceMsgCtx = SPWinFuncPXPieceMsgCtx; +public: + SPWinFuncPXPieceMsg(const lib::ObMemAttr &mem_attr) : + deserial_allocator_(mem_attr), is_empty_(true), row_meta_(&deserial_allocator_), row_(nullptr) + {} + SPWinFuncPXPieceMsg(): + deserial_allocator_(), is_empty_(true), row_meta_(&deserial_allocator_), row_(nullptr) {} + ~SPWinFuncPXPieceMsg() = default; + + void reset() { deserial_allocator_.reset(); } + + INHERIT_TO_STRING_KV("meta", ObDatahubPieceMsg, + K_(op_id)); + +public: + common::ObArenaAllocator deserial_allocator_; + bool is_empty_; // wether this piece contains data + RowMeta row_meta_; // row meta of compact row + ObCompactRow *row_; +}; + +class SPWinFuncPXWholeMsg: public ObDatahubWholeMsg +{ + OB_UNIS_VERSION_V(1); +public: + using WholeMsgProvider = ObWholeMsgProvider; +public: + SPWinFuncPXWholeMsg(const common::ObMemAttr &mem_attr) : + assign_allocator_(mem_attr), is_empty_(true), row_meta_(&assign_allocator_), + row_store_(&assign_allocator_) + {} + + SPWinFuncPXWholeMsg(): + assign_allocator_(), is_empty_(true), row_meta_(), row_store_() {} + + int assign(const SPWinFuncPXWholeMsg &other, common::ObIAllocator *allocator = NULL); + void reset() + { + is_empty_ = true; + row_meta_.reset(); + row_store_.reset(); + } + + TO_STRING_KV(K_(is_empty), K(row_store_.get_row_cnt())); + +public: + common::ObArenaAllocator assign_allocator_; + bool is_empty_; // wether row_store_ is empty, if so, do not serialize row_store_ + RowMeta row_meta_; // row meta of stored rows + sql::ObTempRowStore row_store_; +}; + +class SPWinFuncPXPieceMsgCtx : public ObPieceMsgCtx +{ +public: + SPWinFuncPXPieceMsgCtx(uint64_t op_id, int64_t task_cnt, int64_t timeout_ts, uint64_t tenant_id, + int64_t max_batch_size, const common::ObMemAttr &mem_attr) : + ObPieceMsgCtx(op_id, task_cnt, timeout_ts), + received_(0), tenant_id_(tenant_id), max_batch_size_(max_batch_size), whole_msg_(mem_attr) + {} + + static int alloc_piece_msg_ctx(const SPWinFuncPXPieceMsg &pkt, ObPxCoordInfo &coord_info, + ObExecContext &ctx, int64_t task_cnt, ObPieceMsgCtx *&msg_ctx); + + virtual int send_whole_msg(common::ObIArray &sqcs) override; + virtual void reset_resource() override; + +public: + int64_t received_; // number of piece msgs received + uint64_t tenant_id_; + int64_t max_batch_size_; + SPWinFuncPXWholeMsg whole_msg_; + +private: + DISALLOW_COPY_AND_ASSIGN(SPWinFuncPXPieceMsgCtx); +}; + +class SPWinFuncPXPieceMsgListener +{ +public: + static int on_message(SPWinFuncPXPieceMsgCtx &ctx, common::ObIArray &sqcs, + const SPWinFuncPXPieceMsg &pkt); + +private: + DISALLOW_COPY_AND_ASSIGN(SPWinFuncPXPieceMsgListener); +}; } } #endif /* __OB_SQL_ENG_PX_DH_WINBUF_H__ */ diff --git a/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp b/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp index a9f858c353..8ba7ded347 100644 --- a/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_fifo_coord_op.cpp @@ -51,7 +51,9 @@ ObPxFifoCoordOp::ObPxFifoCoordOp(ObExecContext &exec_ctx, const ObOpSpec &spec, rd_wf_piece_msg_proc_(exec_ctx, msg_proc_), init_channel_piece_msg_proc_(exec_ctx, msg_proc_), reporting_wf_piece_msg_proc_(exec_ctx, msg_proc_), - opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_) + opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_), + sp_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_), + rd_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_) {} int ObPxFifoCoordOp::inner_open() @@ -102,6 +104,8 @@ int ObPxFifoCoordOp::setup_loop_proc() .register_processor(init_channel_piece_msg_proc_) .register_processor(reporting_wf_piece_msg_proc_) .register_processor(opt_stats_gather_piece_msg_proc_) + .register_processor(sp_winfunc_px_piece_msg_proc_) + .register_processor(rd_winfunc_px_piece_msg_proc_) .register_interrupt_processor(interrupt_proc_); return ret; } @@ -231,6 +235,8 @@ int ObPxFifoCoordOp::fetch_rows(const int64_t row_cnt) case ObDtlMsgType::DH_INIT_CHANNEL_PIECE_MSG: case ObDtlMsgType::DH_SECOND_STAGE_REPORTING_WF_PIECE_MSG: case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG: + case ObDtlMsgType::DH_SP_WINFUNC_PX_PIECE_MSG: + case ObDtlMsgType::DH_RD_WINFUNC_PX_PIECE_MSG: // all message processed in callback break; default: diff --git a/src/sql/engine/px/exchange/ob_px_fifo_coord_op.h b/src/sql/engine/px/exchange/ob_px_fifo_coord_op.h index 50c1d8935e..8c414cd0db 100644 --- a/src/sql/engine/px/exchange/ob_px_fifo_coord_op.h +++ b/src/sql/engine/px/exchange/ob_px_fifo_coord_op.h @@ -110,6 +110,8 @@ private: ObInitChannelPieceMsgP init_channel_piece_msg_proc_; ObReportingWFPieceMsgP reporting_wf_piece_msg_proc_; ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc_; + ObSPWinFuncPXPieceMsgP sp_winfunc_px_piece_msg_proc_; + ObRDWinFuncPXPieceMsgP rd_winfunc_px_piece_msg_proc_; }; } // end namespace sql diff --git a/src/sql/engine/px/exchange/ob_px_ms_coord_op.cpp b/src/sql/engine/px/exchange/ob_px_ms_coord_op.cpp index f95d2ee3e3..164c21faee 100644 --- a/src/sql/engine/px/exchange/ob_px_ms_coord_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_ms_coord_op.cpp @@ -97,6 +97,8 @@ ObPxMSCoordOp::ObPxMSCoordOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOp init_channel_piece_msg_proc_(exec_ctx, msg_proc_), reporting_wf_piece_msg_proc_(exec_ctx, msg_proc_), opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_), + sp_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_), + rd_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_), store_rows_(), last_pop_row_(nullptr), row_heap_(), @@ -175,6 +177,8 @@ int ObPxMSCoordOp::setup_loop_proc() .register_processor(init_channel_piece_msg_proc_) .register_processor(reporting_wf_piece_msg_proc_) .register_processor(opt_stats_gather_piece_msg_proc_) + .register_processor(sp_winfunc_px_piece_msg_proc_) + .register_processor(rd_winfunc_px_piece_msg_proc_) .register_interrupt_processor(interrupt_proc_); msg_loop_.set_tenant_id(ctx_.get_my_session()->get_effective_tenant_id()); return ret; @@ -424,6 +428,8 @@ int ObPxMSCoordOp::inner_get_next_row() case ObDtlMsgType::DH_INIT_CHANNEL_PIECE_MSG: case ObDtlMsgType::DH_SECOND_STAGE_REPORTING_WF_PIECE_MSG: case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG: + case ObDtlMsgType::DH_SP_WINFUNC_PX_PIECE_MSG: + case ObDtlMsgType::DH_RD_WINFUNC_PX_PIECE_MSG: // 这几种消息都在 process 回调函数里处理了 break; default: diff --git a/src/sql/engine/px/exchange/ob_px_ms_coord_op.h b/src/sql/engine/px/exchange/ob_px_ms_coord_op.h index 80b3bfd638..859d5193df 100644 --- a/src/sql/engine/px/exchange/ob_px_ms_coord_op.h +++ b/src/sql/engine/px/exchange/ob_px_ms_coord_op.h @@ -165,6 +165,8 @@ private: ObInitChannelPieceMsgP init_channel_piece_msg_proc_; ObReportingWFPieceMsgP reporting_wf_piece_msg_proc_; ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc_; + ObSPWinFuncPXPieceMsgP sp_winfunc_px_piece_msg_proc_; + ObRDWinFuncPXPieceMsgP rd_winfunc_px_piece_msg_proc_; // 存储merge sort的每一路的当前行 ObArray store_rows_; ObChunkDatumStore::LastStoredRow* last_pop_row_; diff --git a/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.cpp b/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.cpp index 8b028e767d..b2beb9b2f2 100644 --- a/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.cpp @@ -96,6 +96,8 @@ ObPxMSCoordVecOp::ObPxMSCoordVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec init_channel_piece_msg_proc_(exec_ctx, msg_proc_), reporting_wf_piece_msg_proc_(exec_ctx, msg_proc_), opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_), + rd_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_), + sp_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_), store_rows_(), last_pop_row_(nullptr), row_heap_(), @@ -150,7 +152,8 @@ int ObPxMSCoordVecOp::inner_open() ObMemAttr attr(ctx_.get_my_session()->get_effective_tenant_id(), "PxMsOutputStore", ObCtxIds::EXECUTE_CTX_ID); if (OB_FAIL(output_store_.init(MY_SPEC.all_exprs_, get_spec().max_batch_size_, - attr, 0 /*mem_limit*/, false/*enable_dump*/, 0 /*row_extra_size*/))) { + attr, 0 /*mem_limit*/, false/*enable_dump*/, + 0 /*row_extra_size*/, NONE_COMPRESSOR))) { LOG_WARN("init output store failed", K(ret)); } else if (OB_ISNULL(mem = ctx_.get_allocator().alloc( ObBitVector::memory_size(1)))) { @@ -198,6 +201,8 @@ int ObPxMSCoordVecOp::setup_loop_proc() .register_processor(init_channel_piece_msg_proc_) .register_processor(reporting_wf_piece_msg_proc_) .register_processor(opt_stats_gather_piece_msg_proc_) + .register_processor(rd_winfunc_px_piece_msg_proc_) + .register_processor(sp_winfunc_px_piece_msg_proc_) .register_interrupt_processor(interrupt_proc_); msg_loop_.set_tenant_id(ctx_.get_my_session()->get_effective_tenant_id()); return ret; @@ -473,6 +478,8 @@ int ObPxMSCoordVecOp::next_row(const bool need_store_output) case ObDtlMsgType::DH_INIT_CHANNEL_PIECE_MSG: case ObDtlMsgType::DH_SECOND_STAGE_REPORTING_WF_PIECE_MSG: case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG: + case ObDtlMsgType::DH_RD_WINFUNC_PX_PIECE_MSG: + case ObDtlMsgType::DH_SP_WINFUNC_PX_PIECE_MSG: // 这几种消息都在 process 回调函数里处理了 break; default: diff --git a/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.h b/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.h index c40fbd1dd6..0ba976c2b3 100644 --- a/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.h +++ b/src/sql/engine/px/exchange/ob_px_ms_coord_vec_op.h @@ -169,6 +169,8 @@ private: ObInitChannelPieceMsgP init_channel_piece_msg_proc_; ObReportingWFPieceMsgP reporting_wf_piece_msg_proc_; ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc_; + ObRDWinFuncPXPieceMsgP rd_winfunc_px_piece_msg_proc_; + ObSPWinFuncPXPieceMsgP sp_winfunc_px_piece_msg_proc_; // 存储merge sort的每一路的当前行 ObArray store_rows_; LastCompactRow *last_pop_row_; diff --git a/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.cpp b/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.cpp index f2915dc89e..4956f3c99d 100644 --- a/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.cpp @@ -104,6 +104,7 @@ int ObPxMSReceiveVecOp::init_merge_sort_input(int64_t n_channel) msi->alloc_ = &mem_context_->get_malloc_allocator(); msi->sql_mem_processor_ = &sql_mem_processor_; msi->io_event_observer_ = &io_event_observer_; + msi->compressor_type_ = MY_SPEC.compress_type_; if (OB_FAIL(merge_inputs_.push_back(msi))) { LOG_WARN("push back merge sort input fail", K(idx), K(ret)); } @@ -144,7 +145,7 @@ int ObPxMSReceiveVecOp::inner_open() "PxMsOutputStore", ObCtxIds::EXECUTE_CTX_ID); if (OB_FAIL(output_store_.init(MY_SPEC.all_exprs_, get_spec().max_batch_size_, attr, 0 /*mem_limit*/, false /*enable_dump*/, - 0 /*row_extra_size*/))) { + 0 /*row_extra_size*/, NONE_COMPRESSOR))) { LOG_WARN("init output store failed", K(ret)); } else if (OB_ISNULL(mem = ctx_.get_allocator().alloc( ObBitVector::memory_size(get_spec().max_batch_size_)))) { @@ -594,7 +595,8 @@ int ObPxMSReceiveVecOp::GlobalOrderInput::create_temp_row_store( const ObPxMSReceiveVecSpec &spec = ms_receive_op.my_spec(); if (OB_FAIL(row_store->init(spec.all_exprs_, spec.max_batch_size_, mem_attr, mem_limit, true /* enable_dump*/, - 0 /*row_extra_size*/))) { + 0 /*row_extra_size*/, + compressor_type_))) { row_store->~ObTempRowStore(); ctx.get_allocator().free(buf); row_store = nullptr; @@ -824,7 +826,8 @@ int ObPxMSReceiveVecOp::new_local_order_input(MergeSortInput *&out_msi) ObMemAttr mem_attr(ctx_.get_my_session()->get_effective_tenant_id(), "PxMSRecvLocal", ObCtxIds::WORK_AREA); if (OB_FAIL(local_input->row_store_.init(MY_SPEC.all_exprs_, get_spec().max_batch_size_, mem_attr, 0 /* mem_limit */, true /* enable_dump*/, - 0 /*row_extra_size*/))) { + 0 /*row_extra_size*/, + local_input->compressor_type_))) { LOG_WARN("failed to init temp row store", K(ret)); } else if (FALSE_IT(local_input->row_store_.set_dir_id(sql_mem_processor_.get_dir_id()))) { LOG_WARN("failed to allocate dir id for temp row store", K(ret)); diff --git a/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.h b/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.h index 8642b9777d..f415ce4cd8 100644 --- a/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.h +++ b/src/sql/engine/px/exchange/ob_px_ms_receive_vec_op.h @@ -78,7 +78,8 @@ private: alloc_(nullptr), sql_mem_processor_(nullptr), io_event_observer_(nullptr), - processed_cnt_(0) + processed_cnt_(0), + compressor_type_(NONE_COMPRESSOR) {} virtual ~MergeSortInput() = default; @@ -113,6 +114,7 @@ private: ObSqlMemMgrProcessor *sql_mem_processor_; ObIOEventObserver *io_event_observer_; int64_t processed_cnt_; + common::ObCompressorType compressor_type_; }; // 全局有序,表示merge sort receive的每一个channel传入的数据是有序的。只要对所有路进行归并排序即可 diff --git a/src/sql/engine/px/exchange/ob_px_ordered_coord_op.cpp b/src/sql/engine/px/exchange/ob_px_ordered_coord_op.cpp index d938ea7ac8..ba5910c67e 100644 --- a/src/sql/engine/px/exchange/ob_px_ordered_coord_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_ordered_coord_op.cpp @@ -53,6 +53,8 @@ ObPxOrderedCoordOp::ObPxOrderedCoordOp(ObExecContext &exec_ctx, const ObOpSpec & init_channel_piece_msg_proc_(exec_ctx, msg_proc_), reporting_wf_piece_msg_proc_(exec_ctx, msg_proc_), opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_), + sp_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_), + rd_winfunc_px_piece_msg_proc_(exec_ctx, msg_proc_), readers_(NULL), receive_order_(), reader_cnt_(0), @@ -135,6 +137,8 @@ int ObPxOrderedCoordOp::setup_loop_proc() .register_processor(init_channel_piece_msg_proc_) .register_processor(reporting_wf_piece_msg_proc_) .register_processor(opt_stats_gather_piece_msg_proc_) + .register_processor(sp_winfunc_px_piece_msg_proc_) + .register_processor(rd_winfunc_px_piece_msg_proc_) .register_interrupt_processor(interrupt_proc_); return ret; } @@ -243,6 +247,8 @@ int ObPxOrderedCoordOp::inner_get_next_row() case ObDtlMsgType::DH_INIT_CHANNEL_PIECE_MSG: case ObDtlMsgType::DH_SECOND_STAGE_REPORTING_WF_PIECE_MSG: case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG: + case ObDtlMsgType::DH_RD_WINFUNC_PX_PIECE_MSG: + case ObDtlMsgType::DH_SP_WINFUNC_PX_PIECE_MSG: // 这几种消息都在 process 回调函数里处理了 break; default: @@ -398,6 +404,8 @@ int ObPxOrderedCoordOp::inner_get_next_batch(const int64_t max_row_cnt) case ObDtlMsgType::DH_INIT_CHANNEL_PIECE_MSG: case ObDtlMsgType::DH_SECOND_STAGE_REPORTING_WF_PIECE_MSG: case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG: + case ObDtlMsgType::DH_RD_WINFUNC_PX_PIECE_MSG: + case ObDtlMsgType::DH_SP_WINFUNC_PX_PIECE_MSG: // 这几种消息都在 process 回调函数里处理了 break; default: diff --git a/src/sql/engine/px/exchange/ob_px_ordered_coord_op.h b/src/sql/engine/px/exchange/ob_px_ordered_coord_op.h index fdcf7852b4..0c94ab9715 100644 --- a/src/sql/engine/px/exchange/ob_px_ordered_coord_op.h +++ b/src/sql/engine/px/exchange/ob_px_ordered_coord_op.h @@ -148,6 +148,8 @@ private: ObInitChannelPieceMsgP init_channel_piece_msg_proc_; ObReportingWFPieceMsgP reporting_wf_piece_msg_proc_; ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc_; + ObSPWinFuncPXPieceMsgP sp_winfunc_px_piece_msg_proc_; + ObRDWinFuncPXPieceMsgP rd_winfunc_px_piece_msg_proc_; ObReceiveRowReader *readers_; ObOrderedReceiveFilter receive_order_; int64_t reader_cnt_; diff --git a/src/sql/engine/px/exchange/ob_px_transmit_op.cpp b/src/sql/engine/px/exchange/ob_px_transmit_op.cpp index b43e8ae5b4..dbb1b036ac 100644 --- a/src/sql/engine/px/exchange/ob_px_transmit_op.cpp +++ b/src/sql/engine/px/exchange/ob_px_transmit_op.cpp @@ -611,13 +611,34 @@ int ObPxTransmitOp::set_wf_hybrid_slice_id_calc_type(ObSliceIdxCalc &slice_calc) const ObPxTransmitSpec &spec = static_cast(get_spec()); if (spec.is_wf_hybrid_) { - ObDatum &wf_hybrid_aggr_status = - MY_SPEC.wf_hybrid_aggr_status_expr_->locate_expr_datum(eval_ctx_); - if (OB_ISNULL(wf_hybrid_aggr_status.ptr().int_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("wf_hybrid_aggr_status_expr_expr_ is null ptr", K(ret)); + int64_t aggr_status = INT64_MIN; + if (spec.use_rich_format_) { + VectorFormat fmt = MY_SPEC.wf_hybrid_aggr_status_expr_->get_format(eval_ctx_); + if (OB_UNLIKELY(VEC_FIXED != fmt && fmt != VEC_UNIFORM)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret), K(fmt)); + } else if (OB_LIKELY(fmt == VEC_FIXED)) { + ObIVector *data = MY_SPEC.wf_hybrid_aggr_status_expr_->get_vector(eval_ctx_); + int64_t batch_idx = eval_ctx_.get_batch_idx(); + aggr_status = *reinterpret_cast( + static_cast *>(data)->get_payload(batch_idx)); + } else { + ObIVector *data = MY_SPEC.wf_hybrid_aggr_status_expr_->get_vector(eval_ctx_); + int64_t batch_idx = eval_ctx_.get_batch_idx(); + aggr_status = *reinterpret_cast( + static_cast *>(data)->get_payload(batch_idx)); + } } else { - int64_t aggr_status = wf_hybrid_aggr_status.get_int(); + ObDatum &wf_hybrid_aggr_status = + MY_SPEC.wf_hybrid_aggr_status_expr_->locate_expr_datum(eval_ctx_); + if (OB_ISNULL(wf_hybrid_aggr_status.ptr().int_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("wf_hybrid_aggr_status_expr_expr_ is null ptr", K(ret)); + } else { + aggr_status = wf_hybrid_aggr_status.get_int(); + } + } + if (OB_SUCC(ret)) { ObWfHybridDistSliceIdCalc &wf_hybrid_slice_calc = static_cast< ObWfHybridDistSliceIdCalc &>(slice_calc); // distribute method is calculate by aggr_status diff --git a/src/sql/engine/px/ob_px_coord_msg_proc.cpp b/src/sql/engine/px/ob_px_coord_msg_proc.cpp index 0ad38cf491..1e44b6eabe 100644 --- a/src/sql/engine/px/ob_px_coord_msg_proc.cpp +++ b/src/sql/engine/px/ob_px_coord_msg_proc.cpp @@ -147,3 +147,15 @@ int ObPxSubCoordMsgProc::on_whole_msg( ObDhWholeeMsgProc proc; return proc.on_whole_msg(sqc_ctx_, dtl::DH_OPT_STATS_GATHER_WHOLE_MSG, pkt); } + +int ObPxSubCoordMsgProc::on_whole_msg(const SPWinFuncPXWholeMsg &pkt) const +{ + ObDhWholeeMsgProc proc; + return proc.on_whole_msg(sqc_ctx_, dtl::DH_SP_WINFUNC_PX_WHOLE_MSG, pkt); +} + +int ObPxSubCoordMsgProc::on_whole_msg(const RDWinFuncPXWholeMsg &pkt) const +{ + ObDhWholeeMsgProc proc; + return proc.on_whole_msg(sqc_ctx_, dtl::DH_RD_WINFUNC_PX_WHOLE_MSG, pkt); +} \ No newline at end of file diff --git a/src/sql/engine/px/ob_px_coord_msg_proc.h b/src/sql/engine/px/ob_px_coord_msg_proc.h index 3c7901818c..6376545db0 100644 --- a/src/sql/engine/px/ob_px_coord_msg_proc.h +++ b/src/sql/engine/px/ob_px_coord_msg_proc.h @@ -37,6 +37,10 @@ class ObReportingWFPieceMsg; class ObReportingWFWholeMsg; class ObOptStatsGatherPieceMsg; class ObOptStatsGatherWholeMsg; +class SPWinFuncPXPieceMsg; +class SPWinFuncPXWholeMsg; +class RDWinFuncPXPieceMsg; +class RDWinFuncPXWholeMsg; // 抽象出本接口类的目的是为了 MsgProc 和 ObPxCoord 解耦 class ObIPxCoordMsgProc { @@ -55,6 +59,8 @@ public: virtual int on_piece_msg(ObExecContext &ctx, const ObInitChannelPieceMsg &pkt) = 0; virtual int on_piece_msg(ObExecContext &ctx, const ObReportingWFPieceMsg &pkt) = 0; virtual int on_piece_msg(ObExecContext &ctx, const ObOptStatsGatherPieceMsg &pkt) = 0; + virtual int on_piece_msg(ObExecContext &ctx, const SPWinFuncPXPieceMsg &pkt) = 0; + virtual int on_piece_msg(ObExecContext &ctx, const RDWinFuncPXPieceMsg &pkt) = 0; }; class ObIPxSubCoordMsgProc @@ -84,6 +90,10 @@ public: const ObReportingWFWholeMsg &pkt) const = 0; virtual int on_whole_msg( const ObOptStatsGatherWholeMsg &pkt) const = 0; + virtual int on_whole_msg( + const SPWinFuncPXWholeMsg &pkt) const = 0; + virtual int on_whole_msg( + const RDWinFuncPXWholeMsg &pkt) const = 0; // SQC 被中断 virtual int on_interrupted(const ObInterruptCode &ic) const = 0; }; @@ -120,8 +130,12 @@ public: const ObReportingWFWholeMsg &pkt) const; virtual int on_whole_msg( const ObOptStatsGatherWholeMsg &pkt) const; -private: - ObSqcCtx &sqc_ctx_; + virtual int on_whole_msg( + const SPWinFuncPXWholeMsg &pkt) const; + virtual int on_whole_msg( + const RDWinFuncPXWholeMsg &pkt) const; + private: + ObSqcCtx &sqc_ctx_; }; } diff --git a/src/sql/engine/px/ob_px_coord_op.cpp b/src/sql/engine/px/ob_px_coord_op.cpp index 11053f4caf..8a3e914171 100644 --- a/src/sql/engine/px/ob_px_coord_op.cpp +++ b/src/sql/engine/px/ob_px_coord_op.cpp @@ -682,7 +682,8 @@ int ObPxCoordOp::wait_all_running_dfos_exit() dtl::ObDtlPacketEmptyProc init_channel_piece_msg_proc; dtl::ObDtlPacketEmptyProc reporting_wf_piece_msg_proc; dtl::ObDtlPacketEmptyProc opt_stats_gather_piece_msg_proc; - + dtl::ObDtlPacketEmptyProc sp_winfunc_px_piece_msg_proc; + dtl::ObDtlPacketEmptyProc rd_winfunc_px_piece_msg_proc; // 这个注册会替换掉旧的proc. (void)msg_loop_.clear_all_proc(); (void)msg_loop_ @@ -697,7 +698,9 @@ int ObPxCoordOp::wait_all_running_dfos_exit() .register_processor(rd_wf_piece_msg_proc) .register_processor(init_channel_piece_msg_proc) .register_processor(reporting_wf_piece_msg_proc) - .register_processor(opt_stats_gather_piece_msg_proc); + .register_processor(opt_stats_gather_piece_msg_proc) + .register_processor(sp_winfunc_px_piece_msg_proc) + .register_processor(rd_winfunc_px_piece_msg_proc); loop.ignore_interrupt(); ObPxControlChannelProc control_channels; @@ -758,6 +761,8 @@ int ObPxCoordOp::wait_all_running_dfos_exit() case ObDtlMsgType::DH_INIT_CHANNEL_PIECE_MSG: case ObDtlMsgType::DH_SECOND_STAGE_REPORTING_WF_PIECE_MSG: case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG: + case ObDtlMsgType::DH_SP_WINFUNC_PX_PIECE_MSG: + case ObDtlMsgType::DH_RD_WINFUNC_PX_PIECE_MSG: break; default: ret = OB_ERR_UNEXPECTED; diff --git a/src/sql/engine/px/ob_px_scheduler.cpp b/src/sql/engine/px/ob_px_scheduler.cpp index 9365faaae5..a0b49cbf7c 100644 --- a/src/sql/engine/px/ob_px_scheduler.cpp +++ b/src/sql/engine/px/ob_px_scheduler.cpp @@ -518,6 +518,18 @@ int ObPxMsgProc::on_piece_msg( return proc.on_piece_msg(coord_info_, ctx, pkt); } +int ObPxMsgProc::on_piece_msg(ObExecContext &ctx, const SPWinFuncPXPieceMsg &pkt) +{ + ObDhPieceMsgProc proc; + return proc.on_piece_msg(coord_info_, ctx, pkt); +} + +int ObPxMsgProc::on_piece_msg(ObExecContext &ctx, const RDWinFuncPXPieceMsg &pkt) +{ + ObDhPieceMsgProc proc; + return proc.on_piece_msg(coord_info_, ctx, pkt); +} + int ObPxMsgProc::on_eof_row(ObExecContext &ctx) { int ret = OB_SUCCESS; diff --git a/src/sql/engine/px/ob_px_scheduler.h b/src/sql/engine/px/ob_px_scheduler.h index be5cb9c674..d8ff7c7953 100644 --- a/src/sql/engine/px/ob_px_scheduler.h +++ b/src/sql/engine/px/ob_px_scheduler.h @@ -203,6 +203,8 @@ public: int on_piece_msg(ObExecContext &ctx, const ObInitChannelPieceMsg &pkt) { UNUSED(ctx); UNUSED(pkt); return common::OB_NOT_SUPPORTED; } int on_piece_msg(ObExecContext &ctx, const ObReportingWFPieceMsg &pkt) { UNUSED(ctx); UNUSED(pkt); return common::OB_NOT_SUPPORTED; } int on_piece_msg(ObExecContext &ctx, const ObOptStatsGatherPieceMsg &pkt) { UNUSED(ctx); UNUSED(pkt); return common::OB_NOT_SUPPORTED; } + int on_piece_msg(ObExecContext &ctx, const SPWinFuncPXPieceMsg &pkt) { UNUSED(ctx); UNUSED(pkt); return common::OB_NOT_SUPPORTED; } + int on_piece_msg(ObExecContext &ctx, const RDWinFuncPXPieceMsg &pkt) { UNUSED(ctx); UNUSED(pkt); return common::OB_NOT_SUPPORTED; } // End Datahub processing ObPxCoordInfo &coord_info_; ObIPxCoordEventListener &listener_; @@ -241,6 +243,8 @@ public: int on_piece_msg(ObExecContext &ctx, const ObInitChannelPieceMsg &pkt); int on_piece_msg(ObExecContext &ctx, const ObReportingWFPieceMsg &pkt); int on_piece_msg(ObExecContext &ctx, const ObOptStatsGatherPieceMsg &pkt); + int on_piece_msg(ObExecContext &ctx, const SPWinFuncPXPieceMsg &pkt); + int on_piece_msg(ObExecContext &ctx, const RDWinFuncPXPieceMsg &pkt); void clean_dtl_interm_result(ObExecContext &ctx); // end DATAHUB msg processing void log_warn_sqc_fail(int ret, const ObPxFinishSqcResultMsg &pkt, ObPxSqcMeta *sqc); diff --git a/src/sql/engine/px/ob_px_sqc_proxy.cpp b/src/sql/engine/px/ob_px_sqc_proxy.cpp index 4f2147ffdb..fb83b388f4 100644 --- a/src/sql/engine/px/ob_px_sqc_proxy.cpp +++ b/src/sql/engine/px/ob_px_sqc_proxy.cpp @@ -164,6 +164,8 @@ int ObPxSQCProxy::setup_loop_proc(ObSqcCtx &sqc_ctx) .register_processor(sqc_ctx.init_channel_whole_msg_proc_) .register_processor(sqc_ctx.reporting_wf_piece_msg_proc_) .register_processor(sqc_ctx.opt_stats_gather_whole_msg_proc_) + .register_processor(sqc_ctx.sp_winfunc_whole_msg_proc_) + .register_processor(sqc_ctx.rd_winfunc_whole_msg_proc_) .register_interrupt_processor(sqc_ctx.interrupt_proc_); } return ret; diff --git a/src/sql/engine/px/ob_px_sub_coord.cpp b/src/sql/engine/px/ob_px_sub_coord.cpp index 41fcc69927..8f4caf29e9 100644 --- a/src/sql/engine/px/ob_px_sub_coord.cpp +++ b/src/sql/engine/px/ob_px_sub_coord.cpp @@ -46,6 +46,7 @@ #include "sql/engine/px/ob_granule_pump.h" #include "sql/das/ob_das_utils.h" #include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "sql/engine/window_function/ob_window_function_vec_op.h" using namespace oceanbase::common; using namespace oceanbase::sql; @@ -535,6 +536,27 @@ int ObPxSubCoord::setup_op_input(ObExecContext &ctx, LOG_DEBUG("debug wf input", K(wf_spec->role_type_), K(sqc.get_task_count()), K(sqc.get_total_task_count())); } + } else if (root.get_type() == PHY_VEC_SORT) { + // TODO XUNSI: if shared topn filter, init the shared topn msg here + } else if (root.get_type() == PHY_VEC_WINDOW_FUNCTION) { + ObPxSqcMeta &sqc = sqc_arg_.sqc_; + ObWindowFunctionVecOpInput *wf_input = NULL; + ObOperatorKit *kit = ctx.get_operator_kit(root.id_); + ObWindowFunctionVecSpec *wf_spec = reinterpret_cast(&root); + if (OB_ISNULL(kit) || OB_ISNULL(kit->input_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("operator is null", K(ret), K(kit)); + } else if (FALSE_IT(wf_input = static_cast(kit->input_))) { + } else if (wf_spec->is_participator()) { + wf_input->set_local_task_count(sqc.get_task_count()); + wf_input->set_total_task_count(sqc.get_total_task_count()); + if (OB_FAIL(wf_input->init_wf_participator_shared_info(ctx.get_allocator(), + sqc.get_task_count()))) { + LOG_WARN("init wf participator shared info failed", K(ret)); + } + LOG_DEBUG("debug wf input", K(wf_spec->role_type_), K(sqc.get_task_count()), + K(sqc.get_total_task_count())); + } } if (OB_SUCC(ret)) { if (OB_FAIL(root.register_to_datahub(ctx))) { diff --git a/src/sql/engine/px/ob_px_task_process.cpp b/src/sql/engine/px/ob_px_task_process.cpp index db57323384..18367566dc 100644 --- a/src/sql/engine/px/ob_px_task_process.cpp +++ b/src/sql/engine/px/ob_px_task_process.cpp @@ -40,6 +40,7 @@ #include "sql/engine/basic/ob_select_into_op.h" #include "observer/mysql/obmp_base.h" #include "lib/alloc/ob_malloc_callback.h" +#include "sql/engine/window_function/ob_window_function_vec_op.h" using namespace oceanbase::common; using namespace oceanbase::share; @@ -866,6 +867,23 @@ int ObPxTaskProcess::OpPostparation::apply(ObExecContext &ctx, ObOpSpec &op) LOG_TRACE("debug post apply info", K(ret_)); } } + } else if (PHY_VEC_WINDOW_FUNCTION == op.type_) { + if (OB_ISNULL(kit->input_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("operator is null", K(ret), K(kit)); + } else { + ObWindowFunctionVecSpec &wf_spec = static_cast(op); + ObWindowFunctionOpInput *input = static_cast(kit->input_); + if (OB_ISNULL(input)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("input is null", K(ret)); + } else if (wf_spec.is_participator() && OB_SUCCESS != ret_) { + input->set_error_code(ret_); + LOG_TRACE("debug post apply info", K(ret_)); + } else { + LOG_TRACE("debug post apply info", K(ret_)); + } + } } else if (PHY_PX_MULTI_PART_INSERT == op.get_type()) { if (OB_ISNULL(kit->input_)) { ret = OB_ERR_UNEXPECTED; diff --git a/src/sql/engine/px/ob_sqc_ctx.cpp b/src/sql/engine/px/ob_sqc_ctx.cpp index 9525342ecd..c7dede9f7d 100644 --- a/src/sql/engine/px/ob_sqc_ctx.cpp +++ b/src/sql/engine/px/ob_sqc_ctx.cpp @@ -36,7 +36,9 @@ ObSqcCtx::ObSqcCtx(ObPxRpcInitSqcArgs &sqc_arg) : msg_loop_(), interrupted_(false), bf_ch_provider_(sqc_proxy_.get_msg_ready_cond()), px_bloom_filter_msg_proc_(msg_proc_), - opt_stats_gather_whole_msg_proc_(msg_proc_){} + opt_stats_gather_whole_msg_proc_(msg_proc_), + sp_winfunc_whole_msg_proc_(msg_proc_), + rd_winfunc_whole_msg_proc_(msg_proc_) {} int ObSqcCtx::add_whole_msg_provider(uint64_t op_id, dtl::ObDtlMsgType msg_type, ObPxDatahubDataProvider &provider) { diff --git a/src/sql/engine/px/ob_sqc_ctx.h b/src/sql/engine/px/ob_sqc_ctx.h index 619da663fa..4ce376ab69 100644 --- a/src/sql/engine/px/ob_sqc_ctx.h +++ b/src/sql/engine/px/ob_sqc_ctx.h @@ -110,6 +110,8 @@ public: // 超过一个算子会使用 datahub,所以大小默认为 1 即可 common::ObSEArray whole_msg_provider_list_; common::ObSEArray, 1> init_channel_msg_cnts_; // + ObSPWinFuncPXWholeMsgP sp_winfunc_whole_msg_proc_; + ObRDWinFuncPXWholeMsgP rd_winfunc_whole_msg_proc_; private: DISALLOW_COPY_AND_ASSIGN(ObSqcCtx); }; diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp index 67a3f73b4a..ef30b18103 100644 --- a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.cpp @@ -15,6 +15,7 @@ #include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" #include "sql/engine/px/p2p_datahub/ob_runtime_filter_msg.h" #include "sql/engine/px/p2p_datahub/ob_runtime_filter_vec_msg.h" +#include "sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h" #include "lib/rc/context.h" #include "sql/engine/px/ob_px_sqc_proxy.h" #include "share/ob_rpc_share.h" @@ -103,59 +104,27 @@ int ObP2PDatahubManager::alloc_msg( ObP2PDatahubMsgBase::ObP2PDatahubMsgType type, ObP2PDatahubMsgBase *&msg_ptr) { +#define ALLOC_MSG_HELPER(msg_type, detail_class, label) \ + case ObP2PDatahubMsgBase::msg_type: { \ + detail_class *new_msg = nullptr; \ + ObMemAttr attr(ob_get_tenant_id(), label); \ + if (OB_FAIL(alloc_msg(allocator, new_msg, attr))) { \ + LOG_WARN("fail to alloc msg", K(ret)); \ + } else { \ + msg_ptr = new_msg; \ + } \ + break; \ + } + int ret = OB_SUCCESS; switch(type) { - case ObP2PDatahubMsgBase::BLOOM_FILTER_VEC_MSG: - case ObP2PDatahubMsgBase::BLOOM_FILTER_MSG: { - ObRFBloomFilterMsg *bf_ptr = nullptr; - ObMemAttr attr(ob_get_tenant_id(), "PxBfMsg"); - if (OB_FAIL(alloc_msg(allocator, bf_ptr, attr))) { - LOG_WARN("fail to alloc msg", K(ret)); - } else { - msg_ptr = bf_ptr; - } - break; - } - case ObP2PDatahubMsgBase::RANGE_FILTER_MSG: { - ObRFRangeFilterMsg *range_ptr = nullptr; - ObMemAttr attr(ob_get_tenant_id(), "PxRangeMsg"); - if (OB_FAIL(alloc_msg(allocator, range_ptr, attr))) { - LOG_WARN("fail to alloc msg", K(ret)); - } else { - msg_ptr = range_ptr; - } - break; - } - case ObP2PDatahubMsgBase::RANGE_FILTER_VEC_MSG: { - ObRFRangeFilterVecMsg *range_ptr = nullptr; - ObMemAttr attr(ob_get_tenant_id(), "PxRangeVecMsg"); - if (OB_FAIL(alloc_msg(allocator, range_ptr, attr))) { - LOG_WARN("fail to alloc msg", K(ret)); - } else { - msg_ptr = range_ptr; - } - break; - } - case ObP2PDatahubMsgBase::IN_FILTER_MSG: { - ObRFInFilterMsg *in_ptr = nullptr; - ObMemAttr attr(ob_get_tenant_id(), "PxInMsg"); - if (OB_FAIL(alloc_msg(allocator, in_ptr, attr))) { - LOG_WARN("fail to alloc msg", K(ret)); - } else { - msg_ptr = in_ptr; - } - break; - } - case ObP2PDatahubMsgBase::IN_FILTER_VEC_MSG: { - ObRFInFilterVecMsg *in_ptr = nullptr; - ObMemAttr attr(ob_get_tenant_id(), "PxInVecMsg"); - if (OB_FAIL(alloc_msg(allocator, in_ptr, attr))) { - LOG_WARN("fail to alloc msg", K(ret)); - } else { - msg_ptr = in_ptr; - } - break; - } + ALLOC_MSG_HELPER(BLOOM_FILTER_MSG, ObRFBloomFilterMsg, "PxBfMsg") + ALLOC_MSG_HELPER(BLOOM_FILTER_VEC_MSG, ObRFBloomFilterMsg, "PxBfVecMsg") + ALLOC_MSG_HELPER(RANGE_FILTER_MSG, ObRFRangeFilterMsg, "PxRangeMsg") + ALLOC_MSG_HELPER(RANGE_FILTER_VEC_MSG, ObRFRangeFilterVecMsg, "PxRangeVecMsg") + ALLOC_MSG_HELPER(IN_FILTER_MSG, ObRFInFilterMsg, "PxInMsg") + ALLOC_MSG_HELPER(IN_FILTER_VEC_MSG, ObRFInFilterVecMsg, "PxInVecMsg") + ALLOC_MSG_HELPER(PD_TOPN_FILTER_MSG, ObPushDownTopNFilterMsg, "PxTopNMsg") default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected type", K(type), K(ret)); diff --git a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h index 5955806d56..06f2e361b5 100644 --- a/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h +++ b/src/sql/engine/px/p2p_datahub/ob_p2p_dh_msg.h @@ -39,6 +39,7 @@ public: ACT(BLOOM_FILTER_VEC_MSG, ) \ ACT(RANGE_FILTER_VEC_MSG, ) \ ACT(IN_FILTER_VEC_MSG, ) \ + ACT(PD_TOPN_FILTER_MSG, ) \ ACT(MAX_TYPE, ) DECLARE_ENUM(ObP2PDatahubMsgType, p2p_datahub_msg_type, P2P_DATAHUB_MSG_TYPE, static); diff --git a/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.cpp b/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.cpp index 2d4af78414..b430a50234 100644 --- a/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.cpp +++ b/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.cpp @@ -13,16 +13,833 @@ #define USING_LOG_PREFIX SQL_ENG #include "sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "sql/engine/sort/ob_sort_vec_op.h" +#include "sql/engine/basic/ob_compact_row.h" +#include "sql/engine/sort/ob_sort_vec_op_context.h" +#include "sql/engine/expr/ob_expr.h" +#include "sql/engine/expr/ob_expr_topn_filter.h" +#include "sql/engine/px/ob_px_sqc_handler.h" + namespace oceanbase { namespace sql { +class InitSelecterOP +{ +public: + InitSelecterOP(uint16_t *selector, int64_t &selector_cnt) + : selector_(selector), selector_cnt_(selector_cnt) + {} + OB_INLINE int operator() (int64_t batch_i) { + selector_[selector_cnt_++] = batch_i; + return OB_SUCCESS; + } +private: + uint16_t *selector_; + int64_t &selector_cnt_; +}; + + OB_SERIALIZE_MEMBER(ObTopNFilterCmpMeta, ser_cmp_func_, obj_meta_); OB_SERIALIZE_MEMBER(ObTopNFilterCompare, build_meta_, filter_meta_, is_ascending_, null_pos_); OB_SERIALIZE_MEMBER(ObPushDownTopNFilterInfo, enabled_, p2p_dh_id_, effective_sk_cnt_, total_sk_cnt_, cmp_metas_, dh_msg_type_, expr_ctx_id_, is_shared_, is_shuffle_, max_batch_size_, adaptive_filter_ratio_); + +int ObPushDownTopNFilterInfo::init(int64_t p2p_dh_id, int64_t effective_sk_cnt, + int64_t total_sk_cnt, + const ObIArray &cmp_metas, + ObP2PDatahubMsgBase::ObP2PDatahubMsgType dh_msg_type, + uint32_t expr_ctx_id, bool is_shared, bool is_shuffle, + int64_t max_batch_size, double adaptive_filter_ratio) +{ + int ret = OB_SUCCESS; + p2p_dh_id_ = p2p_dh_id; + effective_sk_cnt_ = effective_sk_cnt; + total_sk_cnt_ = total_sk_cnt; + dh_msg_type_ = dh_msg_type; + expr_ctx_id_ = expr_ctx_id; + is_shared_ = is_shared; + is_shuffle_ = is_shuffle; + max_batch_size_ = max_batch_size; + adaptive_filter_ratio_ = adaptive_filter_ratio; + if (OB_FAIL(cmp_metas_.assign(cmp_metas))) { + LOG_WARN("failed to assign cmp_metas"); + } else { + enabled_ = true; + } + return ret; +} + +int ObPushDownTopNFilterInfo::assign(const ObPushDownTopNFilterInfo &src) +{ + int ret = OB_SUCCESS; + p2p_dh_id_ = src.p2p_dh_id_; + effective_sk_cnt_ = src.effective_sk_cnt_; + total_sk_cnt_ = src.total_sk_cnt_; + dh_msg_type_ = src.dh_msg_type_; + expr_ctx_id_ = src.expr_ctx_id_; + is_shared_ = src.is_shared_; + is_shuffle_ = src.is_shuffle_; + max_batch_size_ = src.max_batch_size_; + adaptive_filter_ratio_ = src.adaptive_filter_ratio_; + enabled_ = src.enabled_; + if (OB_FAIL(cmp_metas_.assign(src.cmp_metas_))) { + LOG_WARN("failed to assign cmp_metas"); + } + return ret; +} + +OB_DEF_SERIALIZE(ObPushDownTopNFilterMsg) +{ + int ret = OB_SUCCESS; + BASE_SER((ObPushDownTopNFilterMsg, ObP2PDatahubMsgBase)); + LST_DO_CODE(OB_UNIS_ENCODE, total_sk_cnt_, compares_, heap_top_datums_, cells_size_, + data_version_); + return ret; +} + +OB_DEF_DESERIALIZE(ObPushDownTopNFilterMsg) +{ + int ret = OB_SUCCESS; + BASE_DESER((ObPushDownTopNFilterMsg, ObP2PDatahubMsgBase)); + LST_DO_CODE(OB_UNIS_DECODE, total_sk_cnt_, compares_, heap_top_datums_, cells_size_, + data_version_); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(adjust_cell_size())) { + LOG_WARN("fail do adjust cell size", K(ret)); + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObPushDownTopNFilterMsg) +{ + int64_t len = 0; + BASE_ADD_LEN((ObPushDownTopNFilterMsg, ObP2PDatahubMsgBase)); + LST_DO_CODE(OB_UNIS_ADD_LEN, total_sk_cnt_, compares_, heap_top_datums_, cells_size_, + data_version_); + return len; +} +ObPushDownTopNFilterMsg::ObPushDownTopNFilterMsg() + : ObP2PDatahubMsgBase(), total_sk_cnt_(0), compares_(allocator_), heap_top_datums_(allocator_), + cells_size_(allocator_), data_version_(0) +{} + +int ObPushDownTopNFilterMsg::init(const ObPushDownTopNFilterInfo *pd_topn_filter_info, + uint64_t tenant_id, + const ObIArray *sort_collations, + ObExecContext *exec_ctx, + int64_t px_seq_id) +{ + int ret = OB_SUCCESS; + int64_t timeout_ts = GET_PHY_PLAN_CTX(*exec_ctx)->get_timeout_timestamp(); + int64_t effective_sk_cnt = pd_topn_filter_info->effective_sk_cnt_; + ObPxSqcHandler *sqc_handler = exec_ctx->get_sqc_handler(); + int64_t task_id = 0; + common::ObRegisterDmInfo register_dm_info; + if (nullptr == sqc_handler) { + // none px plan + } else { + task_id = exec_ctx->get_px_task_id(); + ObPxSqcMeta &sqc = sqc_handler->get_sqc_init_arg().sqc_; + register_dm_info.detectable_id_ = sqc.get_px_detectable_ids().qc_detectable_id_; + register_dm_info.addr_ = sqc.get_qc_addr(); + } + + if (OB_FAIL(ObP2PDatahubMsgBase::init( + pd_topn_filter_info->p2p_dh_id_, px_seq_id, task_id, + tenant_id, timeout_ts, register_dm_info))) { + LOG_WARN("fail to init basic p2p msg", K(ret)); + } else if (FALSE_IT(total_sk_cnt_ = pd_topn_filter_info->total_sk_cnt_)) { + } else if (OB_FAIL(heap_top_datums_.prepare_allocate(effective_sk_cnt))) { + LOG_WARN("fail to prepare allocate heap_top_datums_", K(ret)); + } else if (OB_FAIL(cells_size_.prepare_allocate(effective_sk_cnt))) { + LOG_WARN("fail to prepare allocate cells_size_", K(ret)); + } else if (OB_FAIL(compares_.prepare_allocate(effective_sk_cnt))) { + LOG_WARN("fail to prepare allocate compares_", K(ret)); + } else { + for (int64_t i = 0; i < effective_sk_cnt && OB_SUCC(ret); ++i) { + // TODO XUNSI: in join scene, if the sort key is the join key of the right table + // the build_meta_ and filter_meta_ may different, preprae it + compares_.at(i).build_meta_.cmp_func_ = pd_topn_filter_info->cmp_metas_.at(i).cmp_func_; + compares_.at(i).build_meta_.obj_meta_.set_meta(pd_topn_filter_info->cmp_metas_.at(i).obj_meta_); + compares_.at(i).filter_meta_.cmp_func_ = pd_topn_filter_info->cmp_metas_.at(i).cmp_func_; + compares_.at(i).filter_meta_.obj_meta_.set_meta(pd_topn_filter_info->cmp_metas_.at(i).obj_meta_); + cells_size_.at(i) = 0; + compares_.at(i).is_ascending_ = sort_collations->at(i).is_ascending_; + compares_.at(i).null_pos_ = sort_collations->at(i).null_pos_; + } + set_msg_expect_cnt(1); // TODO fix me in shared msg + set_msg_cur_cnt(1); + } + LOG_TRACE("[TopN Filter] init ObPushDownTopNFilterMsg", K(ret), K(effective_sk_cnt)); + return ret; +} + +int ObPushDownTopNFilterMsg::destroy() +{ + int ret = OB_SUCCESS; + compares_.reset(); + heap_top_datums_.reset(); + cells_size_.reset(); + allocator_.reset(); + LOG_DEBUG("[TopN Filter] destroy ObPushDownTopNFilterMsg", K(common::lbt())); + return OB_SUCCESS; +} + +int ObPushDownTopNFilterMsg::assign(const ObP2PDatahubMsgBase &src_msg) +{ + int ret = OB_SUCCESS; + const ObPushDownTopNFilterMsg &src_topn_msg = + static_cast(src_msg); + if (OB_FAIL(ObP2PDatahubMsgBase::assign(src_msg))) { + LOG_WARN("failed to assign base data", K(ret)); + } else if (FALSE_IT(total_sk_cnt_ = src_topn_msg.total_sk_cnt_)) { + } else if (OB_FAIL(compares_.assign(src_topn_msg.compares_))) { + LOG_WARN("fail to assign compares_", K(ret)); + } else if (OB_FAIL(heap_top_datums_.assign(src_topn_msg.heap_top_datums_))) { + LOG_WARN("fail to assign heap top datums", K(ret)); + } else if (OB_FAIL(cells_size_.assign(src_topn_msg.cells_size_))) { + LOG_WARN("failed to assign cell size", K(ret)); + } else if (OB_FAIL(adjust_cell_size())) { + LOG_WARN("fail to adjust cell size", K(ret)); + } else if (FALSE_IT(data_version_ = src_topn_msg.data_version_)) { + } else { + // deep copy datum memory + for (int i = 0; i < src_topn_msg.heap_top_datums_.count() && OB_SUCC(ret); ++i) { + const ObDatum &src_datum = src_topn_msg.heap_top_datums_.at(i); + if (OB_FAIL(heap_top_datums_.at(i).deep_copy(src_datum, allocator_))) { + LOG_WARN("fail to deep copy heap top datum", K(ret)); + } + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::deep_copy_msg(ObP2PDatahubMsgBase *&dest_msg) +{ + int ret = OB_SUCCESS; + ObPushDownTopNFilterMsg *new_topn_msg = nullptr; + ObMemAttr attr(tenant_id_, "TOPNVECMSG"); + if (OB_FAIL(PX_P2P_DH.alloc_msg(attr, new_topn_msg))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else if (OB_FAIL(new_topn_msg->assign(*this))) { + LOG_WARN("fail to assign msg", K(ret)); + } else { + dest_msg = new_topn_msg; + } + return ret; +} + +int ObPushDownTopNFilterMsg::merge(ObP2PDatahubMsgBase &msg) +{ + int ret = OB_SUCCESS; + ObPushDownTopNFilterMsg &incomming_topn_msg = static_cast(msg); + if (incomming_topn_msg.heap_top_datums_.count() != heap_top_datums_.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected heap top datum count", K(incomming_topn_msg.heap_top_datums_.count()), + K(heap_top_datums_.count())); + } else if (incomming_topn_msg.is_empty_) { + /*do nothing*/ + } else { + ObSpinLockGuard guard(lock_); + if (OB_FAIL(merge_heap_top_datums(incomming_topn_msg.heap_top_datums_))) { + LOG_WARN("fail to merge heap top datums", K(ret)); + } else if (is_empty_) { + is_empty_ = false; + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::filter_out_data(const ObExpr &expr, ObEvalCtx &ctx, + ObExprTopNFilterContext &filter_ctx, ObDatum &res) +{ + int ret = OB_SUCCESS; + ObDatum *datum = nullptr; + int cmp_res = 0; + bool is_filtered = false; + filter_ctx.total_count_++; + if (OB_UNLIKELY(is_empty_)) { + res.set_int(0); + filter_ctx.filter_count_++; + filter_ctx.check_count_++; + } else { + filter_ctx.check_count_++; + for (int i = 0; OB_SUCC(ret) && i < expr.arg_cnt_; ++i) { + if (OB_FAIL(expr.args_[i]->eval(ctx, datum))) { + LOG_WARN("failed to eval datum", K(ret)); + } else { + if (OB_FAIL(get_compare_result(i, *datum, cmp_res))) { + LOG_WARN("fail to compare", K(ret)); + } else if (cmp_res > 0) { + is_filtered = true; + break; + } else if (cmp_res < 0) { + // the data less than head top data is selected + break; + } else { + // only if the data of the previous column is equal, we need compare the next column + } + } + } + if (OB_SUCC(ret)) { + if (!is_filtered) { + res.set_int(1); + } else { + filter_ctx.filter_count_++; + res.set_int(0); + } + filter_ctx.collect_sample_info(is_filtered, 1); + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::filter_out_data_batch( + const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const int64_t batch_size, + ObExprTopNFilterContext &filter_ctx) +{ + int ret = OB_SUCCESS; + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + if (OB_UNLIKELY(is_empty_)) { + ObDatum *results = expr.locate_batch_datums(ctx); + for (int64_t i = 0; i < batch_size; i++) { + results[i].set_int(0); + } + filter_ctx.total_count_ += batch_size; + filter_ctx.check_count_ += batch_size; + filter_ctx.filter_count_ += batch_size; + } else if (OB_FAIL(do_filter_out_data_batch(expr, ctx, skip, batch_size, filter_ctx))) { + LOG_WARN("failed to do_filter_out_data_batch"); + } + if (OB_SUCC(ret)) { + eval_flags.set_all(batch_size); + } + return ret; +} + +int ObPushDownTopNFilterMsg::filter_out_data_vector( + const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const EvalBound &bound, + ObExprTopNFilterContext &filter_ctx) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_empty_)) { + int64_t total_count = 0; + int64_t filter_count = 0; + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + VectorFormat res_format = expr.get_format(ctx); + if (VEC_UNIFORM == res_format) { + IntegerUniVec *res_vec = static_cast(expr.get_vector(ctx)); + ret = proc_filter_empty(res_vec, skip, bound, total_count, filter_count); + } else if (VEC_FIXED == res_format) { + IntegerFixedVec *res_vec = static_cast(expr.get_vector(ctx)); + ret = proc_filter_empty(res_vec, skip, bound, total_count, filter_count); + } + if (OB_SUCC(ret)) { + eval_flags.set_all(true); + filter_ctx.filter_count_ += filter_count; + filter_ctx.check_count_ += total_count; + filter_ctx.total_count_ += total_count; + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < expr.arg_cnt_; ++i) { + ObExpr *e = expr.args_[i]; + if (OB_FAIL(e->eval_vector(ctx, skip, bound))) { + LOG_WARN("evaluate vector failed", K(ret), K(*e)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(do_filter_out_data_vector(expr, ctx, skip, bound, filter_ctx))) { + LOG_WARN("fail to do filter out data vector"); + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::update_filter_data(ObCompactRow *compact_row, const RowMeta *row_meta, + bool &is_updated) +{ + int ret = OB_SUCCESS; + is_updated = false; + + // TODO XUNSI: update data for shared topn msg, be care of thread safe + if (check_has_null(compact_row)) { + // do nothing, null will not be updated into filter + } else if (OB_FAIL(copy_heap_top_datums_from(compact_row, row_meta))) { + LOG_WARN("failed to copy"); + } else { + is_updated = true; + int64_t v = ATOMIC_AAF(&data_version_, 1); + LOG_TRACE("[TopN Filter] update_filter_data", K(v)); + } + if (OB_SUCC(ret) && is_updated && OB_UNLIKELY(is_empty_)) { + is_empty_ = false; + LOG_TRACE("[TopN Filter] first update filter data"); + } + return ret; +} + +int ObPushDownTopNFilterMsg::update_filter_data(ObChunkDatumStore::StoredRow *store_row, + bool &is_updated) +{ + int ret = OB_SUCCESS; + is_updated = false; + // TODO XUNSI: update data for shared topn msg, be care of thread safe + if (check_has_null(store_row)) { + // do nothing, null will not be updated into filter + } else if (OB_FAIL(copy_heap_top_datums_from(store_row))) { + LOG_WARN("failed to copy"); + } else { + is_updated = true; + int64_t v = ATOMIC_AAF(&data_version_, 1); + LOG_TRACE("[TopN Filter] update_filter_data", K(v)); + } + if (OB_SUCC(ret) && is_updated && OB_UNLIKELY(is_empty_)) { + is_empty_ = false; + LOG_TRACE("[TopN Filter] first update filter data"); + } + return ret; +} + +bool ObPushDownTopNFilterMsg::check_has_null(ObCompactRow *compact_row) +{ + int ret = OB_SUCCESS; + bool has_null = false; + for (int64_t i = 0; i < heap_top_datums_.count() && OB_SUCC(ret); ++i) { + if (compact_row->is_null(i)) { + has_null = true; + break; + } + } + return has_null; +} + +bool ObPushDownTopNFilterMsg::check_has_null(ObChunkDatumStore::StoredRow *store_row) +{ + int ret = OB_SUCCESS; + bool has_null = false; + const common::ObDatum *incomming_datums = store_row->cells(); + for (int64_t i = 0; i < heap_top_datums_.count() && OB_SUCC(ret); ++i) { + if (incomming_datums[i].is_null()) { + has_null = true; + break; + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::prepare_storage_white_filter_data( + ObDynamicFilterExecutor &dynamic_filter, ObEvalCtx &eval_ctx, ObRuntimeFilterParams ¶ms, + bool &is_data_prepared) +{ + int ret = OB_SUCCESS; + int col_idx = dynamic_filter.get_col_idx(); + if (is_empty_) { + dynamic_filter.set_filter_action(DynamicFilterAction::FILTER_ALL); + is_data_prepared = true; + } else if (heap_top_datums_.at(col_idx).is_null()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("expect no null in topn runtime filter"); + } else if (OB_FAIL(params.push_back(heap_top_datums_.at(col_idx)))) { + LOG_WARN("failed to push back bound data"); + } else { + int64_t now_data_version = ATOMIC_LOAD(&data_version_); + dynamic_filter.set_filter_action(DynamicFilterAction::DO_FILTER); + dynamic_filter.set_filter_val_meta(compares_.at(col_idx).build_meta_.obj_meta_); + dynamic_filter.set_stored_data_version(now_data_version); + // caution, see bool ObOpRawExpr::is_white_runtime_filter_expr() const + // now only one column topn filter can be pushdown as white filter, + // if these column is the last sort key, means we can filter the data which the compare reuslt is equal. + ObWhiteFilterOperatorType op_type; + if (col_idx == total_sk_cnt_ - 1) { + op_type = compares_.at(col_idx).is_ascending_ ? WHITE_OP_LT : WHITE_OP_GT; + } else { + op_type = compares_.at(col_idx).is_ascending_ ? WHITE_OP_LE : WHITE_OP_GE; + } + dynamic_filter.get_filter_node().set_op_type(op_type); + is_data_prepared = true; + } + return ret; +} + +int ObPushDownTopNFilterMsg::update_storage_white_filter_data( + ObDynamicFilterExecutor &dynamic_filter, ObRuntimeFilterParams ¶ms, bool &is_update) +{ + int ret = OB_SUCCESS; + int64_t now_data_version = ATOMIC_LOAD(&data_version_); + int col_idx = dynamic_filter.get_col_idx(); + if (heap_top_datums_.at(col_idx).is_null()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("expect no null in topn runtime filter"); + } else if (OB_FAIL(params.push_back(heap_top_datums_.at(col_idx)))) { + LOG_WARN("failed to push back bound data"); + } else { + dynamic_filter.set_stored_data_version(now_data_version); + is_update = true; + } + return ret; +} + +// private interface +int ObPushDownTopNFilterMsg::merge_heap_top_datums(ObIArray &incomming_datums) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(is_empty_)) { + // if self is empty, directly copy data from the incomming msg + if (OB_FAIL(copy_heap_top_datums_from(incomming_datums))) { + LOG_WARN("failed to copy"); + } else { + is_empty_ = false; + } + } else { + // compare in vector format + int cmp_res = 0; + for (int i = 0; i < incomming_datums.count() && OB_SUCC(ret); ++i) { + const ObTopNFilterCompare &compare = compares_.at(i); + const ObDatum &incomming_datum = incomming_datums.at(i); + ObDatum &origin_datum = heap_top_datums_.at(i); + cmp_res = 0; + if (OB_FAIL(compare.compare_for_build(incomming_datum, origin_datum, cmp_res))) { + LOG_WARN("fail to compare_for_build", K(ret)); + } else if (cmp_res < 0) { + break; + } + } + // the new incomming_datums is less than self, we need copy it. + if (OB_SUCC(ret) && cmp_res < 0) { + if (OB_FAIL(copy_heap_top_datums_from(incomming_datums))) { + LOG_WARN("failed to copy"); + } + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::copy_heap_top_datums_from(ObIArray &incomming_datums) +{ + int ret = OB_SUCCESS; + for (int i = 0; i < heap_top_datums_.count() && OB_SUCC(ret); ++i) { + const ObDatum &incomming_datum = incomming_datums.at(i); + ObDatum &origin_datum = heap_top_datums_.at(i); + int64_t &cell_size = cells_size_.at(i); + if (OB_FAIL(dynamic_copy_cell(incomming_datum, origin_datum, cell_size))) { + LOG_WARN("fail to deep copy datum"); + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::copy_heap_top_datums_from(ObCompactRow *compact_row, + const RowMeta *row_meta) +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; i < heap_top_datums_.count() && OB_SUCC(ret); ++i) { + const ObDatum &incomming_datum = compact_row->get_datum(*row_meta, i); + ObDatum &origin_datum = heap_top_datums_.at(i); + int64_t &cell_size = cells_size_.at(i); + if (OB_FAIL(dynamic_copy_cell(incomming_datum, origin_datum, cell_size))) { + LOG_WARN("fail to deep copy datum"); + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::copy_heap_top_datums_from(ObChunkDatumStore::StoredRow *store_row) +{ + int ret = OB_SUCCESS; + const common::ObDatum *incomming_datums = store_row->cells(); + for (int64_t i = 0; i < heap_top_datums_.count() && OB_SUCC(ret); ++i) { + const ObDatum &incomming_datum = incomming_datums[i]; + ObDatum &origin_datum = heap_top_datums_.at(i); + int64_t &cell_size = cells_size_.at(i); + if (OB_FAIL(dynamic_copy_cell(incomming_datum, origin_datum, cell_size))) { + LOG_WARN("fail to deep copy datum"); + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::dynamic_copy_cell(const ObDatum &src, ObDatum &target, + int64_t &cell_size) +{ + int ret = OB_SUCCESS; + int64_t need_size = src.len_; + if (src.is_null()) { + target.null_ = 1; + } else { + if (need_size > cell_size) { + need_size = need_size * 2; + char *buff_ptr = NULL; + if (OB_ISNULL(buff_ptr = static_cast(allocator_.alloc(need_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + SQL_LOG(WARN, "fall to alloc buff", K(need_size), K(ret)); + } else { + memcpy(buff_ptr, src.ptr_, src.len_); + target.pack_ = src.pack_; + target.ptr_ = buff_ptr; + cell_size = need_size; + } + } else { + memcpy(const_cast(target.ptr_), src.ptr_, src.len_); + target.pack_ = src.pack_; + } + } + return ret; +} + +int ObPushDownTopNFilterMsg::adjust_cell_size() +{ + int ret = OB_SUCCESS; + CK(cells_size_.count() == heap_top_datums_.count()); + for (int i = 0; OB_SUCC(ret) && i < cells_size_.count(); ++i) { + cells_size_.at(i) = std::min(cells_size_.at(i), (int64_t)heap_top_datums_.at(i).len_); + } + return ret; +} + +int ObPushDownTopNFilterMsg::do_filter_out_data_batch( + const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const int64_t batch_size, + ObExprTopNFilterContext &filter_ctx) +{ + int ret = OB_SUCCESS; + int64_t filter_count = 0; + int64_t total_count = 0; + ObDatum *results = expr.locate_batch_datums(ctx); + ObEvalCtx::BatchInfoScopeGuard batch_info_guard(ctx); + batch_info_guard.set_batch_size(batch_size); + for (int idx = 0; OB_SUCC(ret) && idx < expr.arg_cnt_; ++idx) { + if (OB_FAIL(expr.args_[idx]->eval_batch(ctx, skip, batch_size))) { + LOG_WARN("eval_batch failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + int cmp_res = 0; + ObDatum *datum = nullptr; + bool is_filtered = false; + for (int64_t batch_i = 0; OB_SUCC(ret) && batch_i < batch_size; ++batch_i) { + if (skip.at(batch_i)) { + continue; + } + cmp_res = 0; + is_filtered = false; + total_count++; + batch_info_guard.set_batch_idx(batch_i); + for (int arg_i = 0; OB_SUCC(ret) && arg_i < expr.arg_cnt_; ++arg_i) { + datum = &expr.args_[arg_i]->locate_expr_datum(ctx, batch_i); + if (OB_FAIL(get_compare_result(arg_i, *datum, cmp_res))) { + LOG_WARN("fail to compare", K(ret)); + } else if (cmp_res > 0) { + // the data bigger than head top data should be filterd out. + filter_count++; + is_filtered = true; + break; + } else if (cmp_res < 0) { + // the data less than head top data is selected + break; + } else { + // only if the data of the previous column is equal, we need compare the next column + } + } + results[batch_i].set_int(is_filtered ? 0 : 1); + } + } + if (OB_SUCC(ret)) { + filter_ctx.filter_count_ += filter_count; + filter_ctx.total_count_ += total_count; + filter_ctx.check_count_ += total_count; + filter_ctx.collect_sample_info(filter_count, total_count); + } + return ret; +} + +#define MULTI_COL_COMPARE_DISPATCH_VECTOR_RES_FORMAT(func_name, arg_format, res_format, ...) \ + if (res_format == VEC_FIXED) { \ + MULTI_COL_COMPARE_DISPATCH_VECTOR_ARG_FORMAT(func_name, arg_format, IntegerFixedVec, \ + __VA_ARGS__); \ + } else { \ + MULTI_COL_COMPARE_DISPATCH_VECTOR_ARG_FORMAT(func_name, arg_format, IntegerUniVec, \ + __VA_ARGS__); \ + } + +#define MULTI_COL_COMPARE_DISPATCH_VECTOR_ARG_FORMAT(func_name, arg_format, result_format, ...) \ + switch (arg_format) { \ + case VEC_FIXED: { \ + ret = func_name(__VA_ARGS__); \ + break; \ + } \ + case VEC_DISCRETE: { \ + ret = func_name(__VA_ARGS__); \ + break; \ + } \ + case VEC_CONTINUOUS: { \ + ret = func_name(__VA_ARGS__); \ + break; \ + } \ + case VEC_UNIFORM: { \ + ret = func_name, result_format>(__VA_ARGS__); \ + break; \ + } \ + case VEC_UNIFORM_CONST: { \ + ret = func_name, result_format>(__VA_ARGS__); \ + break; \ + } \ + default: { \ + ret = func_name(__VA_ARGS__); \ + } \ + } + +template +int ObPushDownTopNFilterMsg::process_multi_columns(int64_t arg_idx, const ObExpr &expr, + ObEvalCtx &ctx, uint16_t *selector, + int64_t old_row_selector_cnt, + int64_t &new_row_selector_cnt, + int64_t &filter_count) +{ + int ret = OB_SUCCESS; + ArgVec *arg_vec = static_cast(expr.args_[arg_idx]->get_vector(ctx)); + ResVec *res_vec = static_cast(expr.get_vector(ctx)); + const char *fixed_base_arg_payload = nullptr; + bool arg_hash_null = arg_vec->has_null(); + ObLength arg_len = 0; + ObDatum datum; + int cmp_res = 0; + const int64_t not_filtered_payload = 1; // for VEC_FIXED set set_payload, always 1 + new_row_selector_cnt = 0; + +#define FILL_MUL_COLUMN_RESULT \ + if (OB_FAIL(get_compare_result(arg_idx, datum, cmp_res))) { \ + LOG_WARN("fail to get_compare_result", K(ret)); \ + } else { \ + if (std::is_same::value) { \ + if (cmp_res > 0) { \ + filter_count += 1; \ + } else if (cmp_res < 0) { \ + res_vec->set_payload(batch_idx, ¬_filtered_payload, sizeof(int64_t)); \ + } else { \ + selector[new_row_selector_cnt++] = batch_idx; \ + } \ + } else { \ + if (cmp_res > 0) { \ + filter_count += 1; \ + res_vec->set_int(batch_idx, 0); \ + } else if (cmp_res < 0) { \ + res_vec->set_int(batch_idx, 1); \ + } else { \ + selector[new_row_selector_cnt++] = batch_idx; \ + } \ + } \ + } + + if (std::is_same::value) { + fixed_base_arg_payload = (reinterpret_cast(arg_vec))->get_data(); + ObLength arg_len = (reinterpret_cast(arg_vec))->get_length(); + datum.len_ = arg_len; + if (!arg_hash_null) { + datum.null_ = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < old_row_selector_cnt; ++i) { + int64_t batch_idx = selector[i]; + datum.ptr_ = fixed_base_arg_payload + arg_len * batch_idx; + FILL_MUL_COLUMN_RESULT + } + } else { + ObBitmapNullVectorBase *bn_vec_base = reinterpret_cast(arg_vec); + for (int64_t i = 0; OB_SUCC(ret) && i < old_row_selector_cnt; ++i) { + int64_t batch_idx = selector[i]; + datum.ptr_ = fixed_base_arg_payload + arg_len * batch_idx; + datum.null_ = bn_vec_base->is_null(batch_idx); + FILL_MUL_COLUMN_RESULT + } + } + } else { + if (!arg_hash_null) { + datum.null_ = 0; + for (int64_t i = 0; OB_SUCC(ret) && i < old_row_selector_cnt; ++i) { + int64_t batch_idx = selector[i]; + arg_vec->get_payload(batch_idx, datum.ptr_, arg_len); + datum.len_ = arg_len; + FILL_MUL_COLUMN_RESULT + } + } else { + if (!std::is_same>::value + && !std::is_same>::value) { + ObBitmapNullVectorBase *bn_vec_base = reinterpret_cast(arg_vec); + for (int64_t i = 0; OB_SUCC(ret) && i < old_row_selector_cnt; ++i) { + int64_t batch_idx = selector[i]; + arg_vec->get_payload(batch_idx, datum.ptr_, arg_len); + datum.len_ = arg_len; + datum.null_ = bn_vec_base->is_null(batch_idx); + FILL_MUL_COLUMN_RESULT + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < old_row_selector_cnt; ++i) { + int64_t batch_idx = selector[i]; + arg_vec->get_payload(batch_idx, datum.ptr_, arg_len); + datum.len_ = arg_len; + datum.null_ = arg_vec->is_null(batch_idx); + FILL_MUL_COLUMN_RESULT + } + } + } + } +#undef FILL_MUL_COLUMN_RESULT + return ret; +} + +int ObPushDownTopNFilterMsg::do_filter_out_data_vector( + const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, const EvalBound &bound, + ObExprTopNFilterContext &filter_ctx) +{ + int ret = OB_SUCCESS; + int64_t total_count = 0; + int64_t filter_count = 0; + int64_t batch_size = bound.batch_size(); + ObBitVector &eval_flags = expr.get_evaluated_flags(ctx); + VectorFormat res_format = expr.get_format(ctx); + + if (VEC_FIXED == res_format) { + IntegerFixedVec *res_vec = static_cast(expr.get_vector(ctx)); + if (OB_FAIL(preset_not_match(res_vec, bound))) { + LOG_WARN("failed to preset_not_match", K(ret)); + } + } + + if (OB_FAIL(ret)) { + } else { + // init selector + int64_t old_row_selector_cnt = 0; + int64_t new_row_selector_cnt = 0; + if (bound.get_all_rows_active()) { + for (int64_t batch_i = bound.start(); batch_i < bound.end(); ++batch_i) { + filter_ctx.row_selector_[old_row_selector_cnt++] = batch_i; + } + } else { + InitSelecterOP init_select_op(filter_ctx.row_selector_, old_row_selector_cnt); + (void)ObBitVector::flip_foreach(skip, bound, init_select_op); + } + total_count += old_row_selector_cnt; + // compare by column + for (int arg_idx = 0; OB_SUCC(ret) && arg_idx < expr.arg_cnt_; ++arg_idx) { + const ObExpr &arg_expr = *expr.args_[arg_idx]; + VectorFormat arg_format = arg_expr.get_format(ctx); + MULTI_COL_COMPARE_DISPATCH_VECTOR_RES_FORMAT( + process_multi_columns, arg_format, res_format, arg_idx, expr, ctx, + filter_ctx.row_selector_, old_row_selector_cnt, new_row_selector_cnt, filter_count); + old_row_selector_cnt = new_row_selector_cnt; + new_row_selector_cnt = 0; + } + if (OB_SUCC(ret)) { + filter_ctx.total_count_ += total_count; + filter_ctx.check_count_ += total_count; + filter_ctx.filter_count_ += filter_count; + filter_ctx.collect_sample_info(filter_count, total_count); + } + } + return ret; +} + } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h b/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h index c472d56889..73d8200089 100644 --- a/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h +++ b/src/sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h @@ -22,6 +22,10 @@ namespace oceanbase namespace sql { +struct ObCompactRow; +class ObExprTopNFilterContext; +struct ObSortFieldCollation; + struct ObTopNFilterCmpMeta final { OB_UNIS_VERSION_V(1); @@ -38,6 +42,28 @@ public: struct ObTopNFilterCompare final { OB_UNIS_VERSION_V(1); +public: + inline int compare_for_build(const ObDatum &l, const ObDatum &r, int &cmp_res) const { + int ret = build_meta_.cmp_func_(build_meta_.obj_meta_, build_meta_.obj_meta_, + l.ptr_, l.len_, l.is_null(), + r.ptr_, r.len_, r.is_null(), + cmp_res); + // when compare new comming data with origin data, we always want maintan the smaller one. + if (!is_ascending_) { + cmp_res = -cmp_res; + } + return ret; + } + inline int compare_for_filter(const ObDatum &l, const ObDatum &r, int &cmp_res) const { + int ret = filter_meta_.cmp_func_(filter_meta_.obj_meta_, build_meta_.obj_meta_, + l.ptr_, l.len_, l.is_null(), + r.ptr_, r.len_, r.is_null(), + cmp_res); + if (!is_ascending_) { + cmp_res = -cmp_res; + } + return ret; + } public: // in join scene, the join cond is T1.a=T2.b, sometimes a and b are not same type but not need to // cast if the sql with order by T1.a, the topn filter can pushdown to T2.b, but the compare info @@ -61,6 +87,12 @@ public: expr_ctx_id_(UINT32_MAX /*INVALID_EXP_CTX_ID*/), is_shared_(false), is_shuffle_(false), max_batch_size_(0), adaptive_filter_ratio_(0.5) {} + int init(int64_t p2p_dh_id, int64_t effective_sk_cnt, int64_t total_sk_cnt, + const ObIArray &cmp_metas, + ObP2PDatahubMsgBase::ObP2PDatahubMsgType dh_msg_type, uint32_t expr_ctx_id, + bool is_shared, bool is_shuffle, int64_t max_batch_size, double adaptive_filter_ratio); + int assign(const ObPushDownTopNFilterInfo &src); + public: bool enabled_; int64_t p2p_dh_id_; @@ -77,5 +109,104 @@ public: K(is_shuffle_), K(max_batch_size_), K(adaptive_filter_ratio_)); }; +class ObPushDownTopNFilterMsg final : public ObP2PDatahubMsgBase +{ + OB_UNIS_VERSION_V(1); + +public: + ObPushDownTopNFilterMsg(); + ~ObPushDownTopNFilterMsg() + { + destroy(); + } + int init(const ObPushDownTopNFilterInfo *pd_topn_filter_info, uint64_t tenant_id, + const ObIArray *sort_collations, ObExecContext *exec_ctx, + int64_t px_seq_id); + int destroy(); + int assign(const ObP2PDatahubMsgBase &src_msg) override; + int deep_copy_msg(ObP2PDatahubMsgBase *&dest_msg) override; + int merge(ObP2PDatahubMsgBase &) override; + + int filter_out_data(const ObExpr &expr, ObEvalCtx &ctx, ObExprTopNFilterContext &filter_ctx, + ObDatum &res); + int filter_out_data_batch(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const int64_t batch_size, + ObExprTopNFilterContext &filter_ctx); + int filter_out_data_vector(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const EvalBound &bound, + ObExprTopNFilterContext &filter_ctx); + int update_filter_data(ObCompactRow *compact_row, const RowMeta *row_meta_, bool &is_updated); + int update_filter_data(ObChunkDatumStore::StoredRow *store_row, bool &is_updated); + + int prepare_storage_white_filter_data(ObDynamicFilterExecutor &dynamic_filter, + ObEvalCtx &eval_ctx, + ObRuntimeFilterParams ¶ms, + bool &is_data_prepared) override final; + + int update_storage_white_filter_data(ObDynamicFilterExecutor &dynamic_filter, + ObRuntimeFilterParams ¶ms, bool &is_update); + + inline int64_t get_current_data_version() { return ATOMIC_LOAD_ACQ(&data_version_); } + inline bool is_null_first(int64_t col_idx) { + return (ObCmpNullPos::NULL_FIRST == compares_.at(col_idx).null_pos_); + } + +private: + bool check_has_null(ObCompactRow *compact_row); + bool check_has_null(ObChunkDatumStore::StoredRow *store_row); + // for merge p2p msg in consumer + int merge_heap_top_datums(ObIArray &incomming_datums); + int copy_heap_top_datums_from(ObIArray &incomming_datums); + + // for update from local thread in producer + int copy_heap_top_datums_from(ObCompactRow *compact_row, const RowMeta *row_meta_); + int copy_heap_top_datums_from(ObChunkDatumStore::StoredRow *store_row); + + // for filter out data + inline int compare(int64_t col_idx, ObDatum &prob_datum, int &cmp_res) + { + return compares_.at(col_idx).compare_for_filter(prob_datum, heap_top_datums_.at(col_idx), + cmp_res); + } + + inline int get_compare_result(int64_t col_idx, ObDatum &datum, int &cmp_res) { + int ret = OB_SUCCESS; + if (OB_FAIL(compare(col_idx, datum, cmp_res))) { + SQL_LOG(WARN, "fail to compare", K(ret)); + } else if (cmp_res == 0) { + if (col_idx == total_sk_cnt_ - 1) { + // this arg is the last one of sort key, we can directly filter + cmp_res = 1; + } else if (col_idx == heap_top_datums_.count() - 1) { + // last sort key pushdown, we need to output the data + cmp_res = -1; + } + } + return ret; + } + + int dynamic_copy_cell(const ObDatum &src, ObDatum &target, int64_t &cell_size); + int adjust_cell_size(); + + int do_filter_out_data_batch(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const int64_t batch_size, + ObExprTopNFilterContext &filter_ctx); + int do_filter_out_data_vector(const ObExpr &expr, ObEvalCtx &ctx, const ObBitVector &skip, + const EvalBound &bound, ObExprTopNFilterContext &filter_ctx); + + template + int process_multi_columns(int64_t arg_idx, const ObExpr &expr, ObEvalCtx &ctx, uint16_t *selector, + int64_t old_row_selector_cnt, int64_t &new_row_selector_cnt, + int64_t &filter_count); + +private: + // total sort key count in topn sort operator, total_sk_cnt_ >= heap_top_datums_.count() + int64_t total_sk_cnt_; + ObTopNFilterCompares compares_; + ObFixedArray heap_top_datums_; + ObFixedArray cells_size_; + int64_t data_version_; +}; + } // end namespace sql } // end namespace oceanbase diff --git a/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.cpp b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.cpp index 071bddc0b8..57f1288be0 100644 --- a/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.cpp +++ b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_msg.cpp @@ -581,7 +581,7 @@ int ObRFBloomFilterMsg::might_contain(const ObExpr &expr, } filter_ctx.check_count_++; res.set_int(is_match ? 1 : 0); - ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + filter_ctx.collect_sample_info(!is_match, 1); } } } @@ -656,7 +656,7 @@ int ObRFBloomFilterMsg::might_contain_batch( filter_ctx.filter_count_ += filter_count; filter_ctx.check_count_ += total_count; filter_ctx.total_count_ += total_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } } return ret; @@ -841,7 +841,7 @@ int ObRFBloomFilterMsg::do_might_contain_vector( filter_ctx.check_count_ += total_count; filter_ctx.total_count_ += total_count; filter_ctx.filter_count_ += filter_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } return ret; } @@ -1553,7 +1553,7 @@ int ObRFRangeFilterMsg::might_contain(const ObExpr &expr, } filter_ctx.check_count_++; res.set_int(is_match ? 1 : 0); - ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + filter_ctx.collect_sample_info(!is_match, 1); } } return ret; @@ -1609,7 +1609,7 @@ int ObRFRangeFilterMsg::do_might_contain_batch(const ObExpr &expr, filter_ctx.filter_count_ += filter_count; filter_ctx.total_count_ += total_count; filter_ctx.check_count_ += total_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } return ret; } @@ -1972,7 +1972,7 @@ int ObRFInFilterMsg::might_contain(const ObExpr &expr, } filter_ctx.check_count_++; res.set_int(is_match ? 1 : 0); - ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + filter_ctx.collect_sample_info(!is_match, 1); } } return ret; @@ -2052,7 +2052,7 @@ int ObRFInFilterMsg::do_might_contain_batch(const ObExpr &expr, filter_ctx.filter_count_ += filter_count; filter_ctx.total_count_ += total_count; filter_ctx.check_count_ += total_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } return ret; } diff --git a/src/sql/engine/px/p2p_datahub/ob_runtime_filter_vec_msg.cpp b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_vec_msg.cpp index a27dea059a..c7714afeed 100644 --- a/src/sql/engine/px/p2p_datahub/ob_runtime_filter_vec_msg.cpp +++ b/src/sql/engine/px/p2p_datahub/ob_runtime_filter_vec_msg.cpp @@ -667,7 +667,7 @@ int ObRFRangeFilterVecMsg::might_contain(const ObExpr &expr, } filter_ctx.check_count_++; res.set_int(is_match ? 1 : 0); - ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + filter_ctx.collect_sample_info(!is_match, 1); } } return ret; @@ -726,7 +726,7 @@ int ObRFRangeFilterVecMsg::do_might_contain_batch(const ObExpr &expr, filter_ctx.filter_count_ += filter_count; filter_ctx.total_count_ += total_count; filter_ctx.check_count_ += total_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } return ret; } @@ -838,7 +838,7 @@ int ObRFRangeFilterVecMsg::do_might_contain_vector( filter_ctx.total_count_ += total_count; filter_ctx.check_count_ += total_count; filter_ctx.filter_count_ += filter_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } } return ret; @@ -1494,7 +1494,7 @@ int ObRFInFilterVecMsg::might_contain(const ObExpr &expr, } filter_ctx.check_count_++; res.set_int(is_match ? 1 : 0); - ObExprJoinFilter::collect_sample_info(&filter_ctx, is_match); + filter_ctx.collect_sample_info(!is_match, 1); } } return ret; @@ -1557,7 +1557,7 @@ int ObRFInFilterVecMsg::do_might_contain_batch(const ObExpr &expr, filter_ctx.filter_count_ += filter_count; filter_ctx.total_count_ += total_count; filter_ctx.check_count_ += total_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } return ret; } @@ -1683,7 +1683,7 @@ int ObRFInFilterVecMsg::do_might_contain_vector( filter_ctx.total_count_ += total_count; filter_ctx.check_count_ += total_count; filter_ctx.filter_count_ += filter_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } } return ret; @@ -1762,7 +1762,7 @@ int ObRFInFilterVecMsg::do_might_contain_vector_impl( filter_ctx.total_count_ += total_count; filter_ctx.check_count_ += total_count; filter_ctx.filter_count_ += filter_count; - ObExprJoinFilter::collect_sample_info_batch(filter_ctx, filter_count, total_count); + filter_ctx.collect_sample_info(filter_count, total_count); } } #undef IN_FILTER_PROBE_HELPER diff --git a/src/sql/engine/set/ob_hash_except_vec_op.cpp b/src/sql/engine/set/ob_hash_except_vec_op.cpp new file mode 100644 index 0000000000..293b08e4bd --- /dev/null +++ b/src/sql/engine/set/ob_hash_except_vec_op.cpp @@ -0,0 +1,234 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "sql/engine/set/ob_hash_except_vec_op.h" +#include "sql/engine/px/ob_px_util.h" + +namespace oceanbase +{ +using namespace common; +namespace sql +{ + +ObHashExceptVecSpec::ObHashExceptVecSpec(ObIAllocator &alloc, const ObPhyOperatorType type) + : ObHashSetVecSpec(alloc, type) +{ +} + +OB_SERIALIZE_MEMBER((ObHashExceptVecSpec, ObHashSetVecSpec)); + +ObHashExceptVecOp::ObHashExceptVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input) + : ObHashSetVecOp(exec_ctx, spec, input), get_row_from_hash_table_(false) +{ +} + +int ObHashExceptVecOp::inner_open() +{ + return ObHashSetVecOp::inner_open(); +} + +int ObHashExceptVecOp::inner_close() +{ + return ObHashSetVecOp::inner_close(); +} + +int ObHashExceptVecOp::inner_rescan() +{ + return ObHashSetVecOp::inner_rescan(); +} + +void ObHashExceptVecOp::destroy() +{ + return ObHashSetVecOp::destroy(); +} + +int ObHashExceptVecOp::build_hash_table_by_part(int64_t batch_size) +{ + int ret= OB_SUCCESS; + bool found = false; + batch_size = batch_size == 0 ? 1 : batch_size; + while (OB_SUCC(ret) && !found) { + if (OB_FAIL(hp_infras_.get_next_pair_partition(InputSide::LEFT))) { + LOG_WARN("failed to get next partition", K(ret)); + } else if (!hp_infras_.has_cur_part(InputSide::LEFT)) { + //If there is no part on the left, there is no data to return. + ret = OB_ITER_END; + } else if (OB_FAIL(build_hash_table_from_left_batch(false, batch_size))) { + LOG_WARN("failed to build hash table batch", K(ret)); + } else if (!hp_infras_.has_cur_part(InputSide::RIGHT)) { + //The right side is null, and the data is taken directly from the hash table on the left. + get_row_from_hash_table_ = true; + found = true; + } else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::RIGHT))) { + LOG_WARN("failed to open cur part"); + } else { + found = true; + hp_infras_.switch_right();//dump logic falls on the right + } + } + return ret; +} + +int ObHashExceptVecOp::inner_get_next_row() +{ + return OB_NOT_IMPLEMENT; +} + +int ObHashExceptVecOp::batch_process_right_vectorize(const int64_t batch_size) +{ + int ret = OB_SUCCESS; + const common::ObIArray *cur_exprs = nullptr; + const ObBatchRows *right_brs = nullptr; + int64_t read_rows = 0; + brs_.skip_->reset(batch_size); + while (OB_SUCC(ret)) { + if (!has_got_part_) { + if (OB_FAIL(right_->get_next_batch(batch_size, right_brs))) { + LOG_WARN("failed to get next batch", K(ret)); + } else if (OB_FAIL(convert_vector(right_->get_spec().output_, + MY_SPEC.set_exprs_, + right_brs))) { + } else if (right_brs->end_ && 0 == right_brs->size_) { + ret = OB_ITER_END; + } else { + cur_exprs = &MY_SPEC.set_exprs_; + read_rows = right_brs->size_; + } + } else if (OB_FAIL(hp_infras_.get_right_next_batch(MY_SPEC.set_exprs_, + batch_size, + read_rows))) { + LOG_WARN("failed to get next batch from dumped partition", K(ret), K(read_rows)); + } else { + cur_exprs = &MY_SPEC.set_exprs_; + brs_.size_ = read_rows; + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(try_check_status())) { + LOG_WARN("failed to check status", K(ret)); + } else if (OB_FAIL(hp_infras_.exists_batch(*cur_exprs, + has_got_part_ ? brs_ : *right_brs, + brs_.skip_, + hash_values_for_batch_))) { + LOG_WARN("failed to exists batch", K(ret)); + } else { + //for except, do not need set skip vector in exists_batch, all rows are from hash table. + brs_.skip_->reset(batch_size); + } + } + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + } + return ret; +} + +int ObHashExceptVecOp::get_next_batch_from_hashtable(const int64_t batch_size) +{ + int ret = OB_SUCCESS; + bool got_batch = false; + int64_t read_rows = 0; + const ObCompactRow *store_rows[batch_size]; + while (OB_SUCC(ret) && !got_batch) { + if (!get_row_from_hash_table_) { + if (OB_FAIL(hp_infras_.finish_insert_row())) { + LOG_WARN("failed to finish insert row", K(ret)); + } else if (OB_FAIL(hp_infras_.end_round())) { + LOG_WARN("faild to end round", K(ret)); + } else if (OB_FAIL(hp_infras_.start_round())) { + LOG_WARN("failed to start round", K(ret)); + } else if (OB_FAIL(build_hash_table_by_part(batch_size))) { + if (OB_ITER_END != ret) { + LOG_WARN("failed to build hash table by part", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (get_row_from_hash_table_) { + } else if (OB_FAIL(batch_process_right_vectorize(batch_size))) { + LOG_WARN("failed to process right vec", K(ret)); + } else if (OB_FAIL(hp_infras_.close_cur_part(InputSide::RIGHT))) { + LOG_WARN("failed to close right part", K(ret)); + } else { + get_row_from_hash_table_ = true; + } + if (OB_SUCC(ret) && OB_FAIL(hp_infras_.open_hash_table_part())) { + LOG_WARN("failed to open hashtable part", K(ret)); + } + } else if (OB_FAIL(hp_infras_.get_next_hash_table_batch(MY_SPEC.set_exprs_, + batch_size, + read_rows, + &store_rows[0]))) { + if (OB_ITER_END != ret) { + LOG_WARN("failed to get next hash table batch", K(ret)); + } else { + get_row_from_hash_table_ = false; + ret = OB_SUCCESS; + } + } else { + brs_.size_ = read_rows; + for (int64_t i = 0; i < read_rows; ++i) { + const ObHashPartItem *sr = static_cast(store_rows[i]); + if (sr->is_match(hp_infras_.get_hash_store_row_meta())) { + brs_.skip_->set(i); + } + } + got_batch = true; + } + } + return ret; +} + +int ObHashExceptVecOp::inner_get_next_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + clear_evaluated_flag(); + if (first_get_left_) { + // Use a larger size before outputting. + const ObBatchRows *child_brs = nullptr; + if (OB_FAIL(left_->get_next_batch(MY_SPEC.max_batch_size_, child_brs))) { + LOG_WARN("failed get left batch", K(ret)); + } else if (FALSE_IT(left_brs_ = child_brs)) { + } else if (child_brs->end_ && 0 == child_brs->size_) { + ret = OB_ITER_END; + } else if (OB_FAIL(init_hash_partition_infras_for_batch())) { + LOG_WARN("failed to init hash partition infras", K(ret)); + } else if (OB_FAIL(build_hash_table_from_left_batch(true, MY_SPEC.max_batch_size_))) { + LOG_WARN("failed to build hash table", K(ret)); + } else { + hp_infras_.switch_right(); + if (OB_FAIL(batch_process_right_vectorize(MY_SPEC.max_batch_size_))) { + LOG_WARN("failed to batch process right", K(ret)); + } else if (OB_FAIL(hp_infras_.open_hash_table_part())) { + LOG_WARN("failed to open hash table part", K(ret)); + } else { + get_row_from_hash_table_ = true; + has_got_part_ = true; + } + } + } + + const int64_t batch_size = std::min(max_row_cnt, MY_SPEC.max_batch_size_); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(get_next_batch_from_hashtable(batch_size))) { + if (OB_ITER_END != ret) { + LOG_WARN("failed to get next row from hash table", K(ret)); + } + } + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + brs_.end_ = true; + } + return ret; +} + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/set/ob_hash_except_vec_op.h b/src/sql/engine/set/ob_hash_except_vec_op.h new file mode 100644 index 0000000000..87da4ef90d --- /dev/null +++ b/src/sql/engine/set/ob_hash_except_vec_op.h @@ -0,0 +1,55 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_EXCEPT_VEC_OP_H_ +#define OCEANBASE_BASIC_OB_SET_OB_HASH_EXCEPT_VEC_OP_H_ + +#include "sql/engine/set/ob_hash_set_vec_op.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObHashExceptVecSpec : public ObHashSetVecSpec +{ +OB_UNIS_VERSION_V(1); +public: + ObHashExceptVecSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type); +}; + +class ObHashExceptVecOp : public ObHashSetVecOp +{ +public: + ObHashExceptVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input); + ~ObHashExceptVecOp() {} + + virtual int inner_open() override; + virtual int inner_get_next_row() override; + virtual int inner_get_next_batch(const int64_t max_row_cnt) override; + virtual int inner_close() override; + virtual int inner_rescan() override; + virtual void destroy() override; + +private: + int build_hash_table_by_part(int64_t batch_size); + //Get the processed data from the hash table + int get_next_batch_from_hashtable(const int64_t batch_size); + //Continue processing the row on the right or return the row directly from the hash table + bool get_row_from_hash_table_; + int batch_process_right_vectorize(const int64_t batch_size); +}; + +} // end namespace sql +} // end namespace oceanbase + +#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_EXCEPT_VEC_OP_H_ diff --git a/src/sql/engine/set/ob_hash_intersect_vec_op.cpp b/src/sql/engine/set/ob_hash_intersect_vec_op.cpp new file mode 100644 index 0000000000..9b0041b0ec --- /dev/null +++ b/src/sql/engine/set/ob_hash_intersect_vec_op.cpp @@ -0,0 +1,183 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "sql/engine/set/ob_hash_intersect_vec_op.h" +#include "sql/engine/px/ob_px_util.h" + +namespace oceanbase +{ +using namespace common; +namespace sql +{ + +ObHashIntersectVecSpec::ObHashIntersectVecSpec(ObIAllocator &alloc, const ObPhyOperatorType type) + : ObHashSetVecSpec(alloc, type) +{ +} + +OB_SERIALIZE_MEMBER((ObHashIntersectVecSpec, ObHashSetVecSpec)); + +ObHashIntersectVecOp::ObHashIntersectVecOp( + ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input) + : ObHashSetVecOp(exec_ctx, spec, input) +{ +} + +int ObHashIntersectVecOp::inner_open() +{ + return ObHashSetVecOp::inner_open(); +} + +int ObHashIntersectVecOp::inner_close() +{ + return ObHashSetVecOp::inner_close(); +} + +int ObHashIntersectVecOp::inner_rescan() +{ + return ObHashSetVecOp::inner_rescan(); +} + +void ObHashIntersectVecOp::destroy() +{ + return ObHashSetVecOp::destroy(); +} + +int ObHashIntersectVecOp::build_hash_table_by_part(int64_t batch_size) +{ + int ret = OB_SUCCESS; + bool found = false; + batch_size = batch_size == 0 ? 1 :batch_size; + while (OB_SUCC(ret) && !found) { + if (OB_FAIL(hp_infras_.get_next_pair_partition(InputSide::LEFT))) { + LOG_WARN("failed to get next pair partitions", K(ret)); + } else if (!hp_infras_.has_cur_part(InputSide::LEFT)) { + ret = OB_ITER_END; + } else if (!hp_infras_.has_cur_part(InputSide::RIGHT)) { + // left part has no matched right part + if (OB_FAIL(hp_infras_.close_cur_part(InputSide::LEFT))) { + LOG_WARN("failed to close cur part", K(ret)); + } + } else if (OB_FAIL(build_hash_table_from_left_batch(false, batch_size))) { + LOG_WARN("failed to build hash table batch", K(ret)); + } else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::RIGHT))) { + LOG_WARN("failed to open cur part"); + } else { + found = true; + hp_infras_.switch_right(); + } + } + return ret; +} + +int ObHashIntersectVecOp::inner_get_next_row() +{ + return OB_NOT_IMPLEMENT; +} + +int ObHashIntersectVecOp::inner_get_next_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + clear_evaluated_flag(); + const int64_t batch_size = std::min(max_row_cnt, MY_SPEC.max_batch_size_); + brs_.skip_->reset(batch_size); + if (first_get_left_) { + const ObBatchRows *child_brs = nullptr; + if (OB_FAIL(left_->get_next_batch(batch_size, child_brs))) { + LOG_WARN("failed to get next batch", K(ret)); + } else if (FALSE_IT(left_brs_ = child_brs)) { + } else if (child_brs->end_ && 0 == child_brs->size_) { + ret = OB_ITER_END; + } else if (OB_FAIL(init_hash_partition_infras_for_batch())) { + LOG_WARN("failed to init hash partition for batch", K(ret)); + } else if (OB_FAIL(build_hash_table_from_left_batch(true, batch_size))) { + LOG_WARN("failed to build hash table for batch", K(ret)); + } else { + hp_infras_.switch_right(); + } + } + + bool got_batch = false; + const ObBatchRows *right_brs = nullptr; + const common::ObIArray *cur_exprs = nullptr; + int64_t read_rows = 0; + while(OB_SUCC(ret) && !got_batch) { + if (!has_got_part_) { + if (OB_FAIL(right_->get_next_batch(batch_size, right_brs))) { + LOG_WARN("failed to get next batch", K(ret)); + } else if (OB_FAIL(convert_vector(right_->get_spec().output_, + MY_SPEC.set_exprs_, + right_brs))) { + } else if (right_brs->end_ && 0 == right_brs->size_) { + ret = OB_ITER_END; + } else { + cur_exprs = &MY_SPEC.set_exprs_; + read_rows = right_brs->size_; + brs_.size_ = read_rows; + } + } else if (OB_FAIL(hp_infras_.get_right_next_batch(MY_SPEC.set_exprs_, + batch_size, + read_rows))) { + if (OB_ITER_END != ret) { + LOG_WARN("failed to get next batch", K(ret)); + } + } else { + cur_exprs = &MY_SPEC.set_exprs_; + brs_.size_ = read_rows; + } + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + // get next dumped partition + if (OB_FAIL(hp_infras_.finish_insert_row())) { + LOG_WARN("failed to finish to insert row", K(ret)); + } else if (!has_got_part_) { + has_got_part_ = true; + } else { + if (OB_FAIL(hp_infras_.close_cur_part(InputSide::RIGHT))) { + LOG_WARN("failed to close cur part", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(hp_infras_.end_round())) { + LOG_WARN("failed to end round", K(ret)); + } else if (OB_FAIL(hp_infras_.start_round())) { + LOG_WARN("failed to open round", K(ret)); + } else if (OB_FAIL(build_hash_table_by_part(read_rows))) { + if (OB_ITER_END != ret) { + LOG_WARN("failed to build hash table", K(ret)); + } + } + } else if (OB_FAIL(ret)) { + } else if (OB_FAIL(try_check_status())) { + LOG_WARN("failed to check status", K(ret)); + } else if (OB_FAIL(hp_infras_.exists_batch(*cur_exprs, + has_got_part_ ? brs_ : *right_brs, + brs_.skip_, + hash_values_for_batch_))) { + LOG_WARN("failed to exist batch", K(ret)); + } else { + got_batch = true; + } + } + + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + brs_.end_ = true; + brs_.size_ = 0; + } + return ret; +} + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/set/ob_hash_intersect_vec_op.h b/src/sql/engine/set/ob_hash_intersect_vec_op.h new file mode 100644 index 0000000000..7b316b40e7 --- /dev/null +++ b/src/sql/engine/set/ob_hash_intersect_vec_op.h @@ -0,0 +1,50 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_INTERSECT_VEC_OP_H_ +#define OCEANBASE_BASIC_OB_SET_OB_HASH_INTERSECT_VEC_OP_H_ + +#include "sql/engine/set/ob_hash_set_vec_op.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObHashIntersectVecSpec : public ObHashSetVecSpec +{ +OB_UNIS_VERSION_V(1); +public: + ObHashIntersectVecSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type); +}; + +class ObHashIntersectVecOp : public ObHashSetVecOp +{ +public: + ObHashIntersectVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input); + ~ObHashIntersectVecOp() {} + + virtual int inner_open() override; + virtual int inner_get_next_row() override; + virtual int inner_get_next_batch(const int64_t max_row_cnt) override; + virtual int inner_close() override; + virtual int inner_rescan() override; + virtual void destroy() override; + +private: + int build_hash_table_by_part(int64_t batch_size); +}; + +} // end namespace sql +} // end namespace oceanbase + +#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_INTERSECT_VEC_OP_H_ diff --git a/src/sql/engine/set/ob_hash_set_vec_op.cpp b/src/sql/engine/set/ob_hash_set_vec_op.cpp new file mode 100644 index 0000000000..73e76f1c58 --- /dev/null +++ b/src/sql/engine/set/ob_hash_set_vec_op.cpp @@ -0,0 +1,307 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "sql/engine/set/ob_hash_set_vec_op.h" +#include "sql/engine/px/ob_px_util.h" +#include "sql/engine/basic/ob_hp_infras_vec_op.h" + +namespace oceanbase +{ +using namespace common; +namespace sql +{ + +ObHashSetVecSpec::ObHashSetVecSpec(ObIAllocator &alloc, const ObPhyOperatorType type) + : ObOpSpec(alloc, type), + set_exprs_(alloc), + sort_collations_(alloc) +{ +} + +OB_SERIALIZE_MEMBER((ObHashSetVecSpec, ObOpSpec), set_exprs_, sort_collations_); + +ObHashSetVecOp::ObHashSetVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input) + : ObOperator(exec_ctx, spec, input), + first_get_left_(true), + has_got_part_(false), + profile_(ObSqlWorkAreaType::HASH_WORK_AREA), + sql_mem_processor_(profile_, op_monitor_info_), + hp_infras_(), + hash_values_for_batch_(nullptr), + need_init_(true), + left_brs_(nullptr), + mem_context_(nullptr) +{ +} + +int ObHashSetVecOp::inner_open() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(left_) || OB_ISNULL(right_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: left or right is null", K(ret), K(left_), K(right_)); + } else if (OB_FAIL(ObOperator::inner_open())) { + LOG_WARN("failed to inner open", K(ret)); + } else if (OB_FAIL(init_mem_context())) { + LOG_WARN("failed to init mem context", K(ret)); + } + return ret; +} + +void ObHashSetVecOp::reset() +{ + first_get_left_ = true; + has_got_part_ = false; + left_brs_ = nullptr; + hp_infras_.reset(); +} + +int ObHashSetVecOp::inner_close() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObOperator::inner_close())) { + LOG_WARN("failed to inner close", K(ret)); + } else { + reset(); + } + sql_mem_processor_.unregister_profile(); + return ret; +} + +int ObHashSetVecOp::inner_rescan() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObOperator::inner_rescan())) { + LOG_WARN("failed to rescan", K(ret)); + } else { + reset(); + } + return ret; +} + +void ObHashSetVecOp::destroy() +{ + sql_mem_processor_.unregister_profile_if_necessary(); + hp_infras_.destroy(); + if (OB_LIKELY(NULL != mem_context_)) { + DESTROY_CONTEXT(mem_context_); + mem_context_ = NULL; + } + ObOperator::destroy(); +} + +int ObHashSetVecOp::get_left_batch(const int64_t batch_size, const ObBatchRows *&child_brs) +{ + int ret = OB_SUCCESS; + if (first_get_left_) { + CK(OB_NOT_NULL(left_brs_)); + child_brs = left_brs_; + first_get_left_ = false; + } else { + if (OB_FAIL(left_->get_next_batch(batch_size, child_brs))) { + LOG_WARN("failed to get batch from child", K(ret)); + } + } + return ret; +} +// use for HashExcept, HashIntersect +int ObHashSetVecOp::build_hash_table_from_left_batch(bool from_child, const int64_t batch_size) +{ + int ret = OB_SUCCESS; + if (!from_child) { + if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) { + LOG_WARN("failed to open cur part", K(ret)); + } else if (OB_FAIL(hp_infras_.resize( + hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) { + LOG_WARN("failed to init hash table", K(ret)); + } else if (OB_FAIL(sql_mem_processor_.init( + &mem_context_->get_malloc_allocator(), + ctx_.get_my_session()->get_effective_tenant_id(), + hp_infras_.get_cur_part_file_size(InputSide::LEFT), + spec_.type_, + spec_.id_, + &ctx_))) { + LOG_WARN("failed to init sql mem processor", K(ret)); + } + } + hp_infras_.switch_left(); + ObBitVector *output_vec = nullptr; + while (OB_SUCC(ret)) { + if (from_child) { + const ObBatchRows *left_brs = nullptr; + if (OB_FAIL(get_left_batch(batch_size, left_brs))) { + LOG_WARN("failed to get left batch", K(ret)); + } else if (left_brs->end_ && 0 == left_brs->size_) { + ret = OB_ITER_END; + } else if (OB_FAIL(convert_vector(left_->get_spec().output_, + static_cast + (get_spec()).set_exprs_, + left_brs))) { + LOG_WARN("failed to convert vector", K(ret)); + } else if (OB_FAIL(hp_infras_.calc_hash_value_for_batch( + static_cast + (get_spec()).set_exprs_, + *left_brs, + hash_values_for_batch_))) { + LOG_WARN("failed to calc hash value for batch", K(ret)); + } else if (OB_FAIL(hp_infras_.insert_row_for_batch(static_cast + (get_spec()).set_exprs_, hash_values_for_batch_, + left_brs->size_, left_brs->skip_, output_vec))) { + LOG_WARN("failed to insert row for batch", K(ret)); + } + } else { + int64_t read_rows = 0; + if (OB_FAIL(hp_infras_.get_left_next_batch(static_cast + (get_spec()).set_exprs_, batch_size, read_rows, hash_values_for_batch_))) { + LOG_WARN("failed to get left next batch", K(ret)); + } else if (OB_FAIL(hp_infras_.insert_row_for_batch(static_cast + (get_spec()).set_exprs_, + hash_values_for_batch_, + read_rows, nullptr, output_vec))) { + LOG_WARN("failed to insert row", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(try_check_status())) { + LOG_WARN("check status exit", K(ret)); + } + } //end of while + if (OB_ITER_END == ret) { + if (OB_FAIL(hp_infras_.finish_insert_row())) { + LOG_WARN("failed to finish insert", K(ret)); + } else if (!from_child && OB_FAIL(hp_infras_.close_cur_part(InputSide::LEFT))) { + LOG_WARN("failed to close cur part", K(ret)); + } + } + return ret; +} + +int ObHashSetVecOp::init_hash_partition_infras() +{ + int ret = OB_SUCCESS; + int64_t est_rows = get_spec().rows_; + if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size( + &ctx_, get_spec().px_est_size_factor_, est_rows, est_rows))) { + LOG_WARN("failed to get px size", K(ret)); + } else if (OB_FAIL(sql_mem_processor_.init( + &mem_context_->get_malloc_allocator(), + ctx_.get_my_session()->get_effective_tenant_id(), + est_rows * get_spec().width_, + get_spec().type_, + get_spec().id_, + &ctx_))) { + LOG_WARN("failed to init sql mem processor", K(ret)); + } else if (OB_FAIL(hp_infras_.init(ctx_.get_my_session()->get_effective_tenant_id(), + GCONF.is_sql_operator_dump_enabled(), + true, true, 1, (get_spec()).max_batch_size_, + static_cast(get_spec()).set_exprs_, + &sql_mem_processor_, (get_spec()).compress_type_))) { + LOG_WARN("failed to init hash partition infrastructure", K(ret)); + } else { + const ObHashSetVecSpec &spec = static_cast(get_spec()); + int64_t est_bucket_num = hp_infras_.est_bucket_count(est_rows, get_spec().width_); + hp_infras_.set_io_event_observer(&io_event_observer_); + if (OB_FAIL(hp_infras_.set_funcs(&spec.sort_collations_, &eval_ctx_))) { + LOG_WARN("failed to set funcs", K(ret)); + } else if (OB_FAIL(hp_infras_.start_round())) { + LOG_WARN("failed to start round", K(ret)); + } else if (OB_FAIL(hp_infras_.init_hash_table(est_bucket_num))) { + LOG_WARN("failed to init hash table", K(ret)); + } + } + return ret; +} + +int ObHashSetVecOp::init_hash_partition_infras_for_batch() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(init_hash_partition_infras())) { + LOG_WARN("failed to init hash partition infra", K(ret)); + } else if (need_init_) { + need_init_ = false; + int64_t batch_size = get_spec().max_batch_size_; + if (OB_FAIL(hp_infras_.init_my_skip(batch_size))) { + LOG_WARN("failed to init my_skip", K(ret)); + } else if (OB_ISNULL(hash_values_for_batch_ + = static_cast (ctx_.get_allocator().alloc(batch_size * sizeof(uint64_t))))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to init hash values for batch", K(ret), K(batch_size)); + } + } + return ret; +} + +int ObHashSetVecOp::init_mem_context() +{ + int ret = OB_SUCCESS; + if (NULL == mem_context_) { + lib::ContextParam param; + param.set_mem_attr(ctx_.get_my_session()->get_effective_tenant_id(), + "ObHashSetRows", + ObCtxIds::WORK_AREA); + if (OB_FAIL(CURRENT_CONTEXT->CREATE_CONTEXT(mem_context_, param))) { + LOG_WARN("memory entity create failed", K(ret)); + } + } + return ret; +} + +int ObHashSetVecOp::convert_vector(const common::ObIArray &src_exprs, + const common::ObIArray &dst_exprs, + const ObBatchRows *&child_brs) +{ + int ret = OB_SUCCESS; + if (child_brs->end_ && 0 == child_brs->size_) { + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < src_exprs.count(); i++) { + ObExpr *from = src_exprs.at(i); + ObExpr *to = dst_exprs.at(i); + if (OB_FAIL(from->eval_vector(eval_ctx_, *child_brs))) { + LOG_WARN("eval batch failed", K(ret)); + } else { + VectorHeader &from_vec_header = from->get_vector_header(eval_ctx_); + VectorHeader &to_vec_header = to->get_vector_header(eval_ctx_); + if (from_vec_header.format_ == VEC_UNIFORM_CONST) { + ObDatum *from_datum = + static_cast(from->get_vector(eval_ctx_))->get_datums(); + OZ(to->init_vector(eval_ctx_, VEC_UNIFORM, child_brs->size_)); + ObUniformBase *to_vec = static_cast(to->get_vector(eval_ctx_)); + ObDatum *to_datums = to_vec->get_datums(); + for (int64_t j = 0; j < child_brs->size_ && OB_SUCC(ret); j++) { + to_datums[j] = *from_datum; + } + } else if (from_vec_header.format_ == VEC_UNIFORM) { + ObUniformBase *uni_vec = static_cast(from->get_vector(eval_ctx_)); + ObDatum *src = uni_vec->get_datums(); + ObDatum *dst = to->locate_batch_datums(eval_ctx_); + if (src != dst) { + MEMCPY(dst, src, child_brs->size_ * sizeof(ObDatum)); + } + OZ(to->init_vector(eval_ctx_, VEC_UNIFORM, child_brs->size_)); + } else { + to_vec_header = from_vec_header; + } + // init eval info + if (OB_SUCC(ret)) { + to->get_eval_info(eval_ctx_).cnt_ = child_brs->size_; + to->set_evaluated_projected(eval_ctx_); + } + } + } + } + return ret; +} + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/set/ob_hash_set_vec_op.h b/src/sql/engine/set/ob_hash_set_vec_op.h new file mode 100644 index 0000000000..601b205396 --- /dev/null +++ b/src/sql/engine/set/ob_hash_set_vec_op.h @@ -0,0 +1,76 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_SET_VEC_OP_H_ +#define OCEANBASE_BASIC_OB_SET_OB_HASH_SET_VEC_OP_H_ + +#include "share/datum/ob_datum_funcs.h" +#include "sql/engine/basic/ob_chunk_datum_store.h" +#include "sql/engine/basic/ob_hp_infras_vec_op.h" +#include "sql/engine/ob_operator.h" +#include "sql/engine/sort/ob_sort_basic_info.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObHashSetVecSpec : public ObOpSpec +{ +OB_UNIS_VERSION_V(1); +public: + ObHashSetVecSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type); + + INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(sort_collations)); + ExprFixedArray set_exprs_; + ObSortCollations sort_collations_;}; + +class ObHashSetVecOp : public ObOperator +{ +public: + ObHashSetVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input); + ~ObHashSetVecOp() {} + + virtual int inner_open() override; + virtual int inner_close() override; + virtual int inner_rescan() override; + virtual void destroy() override; + +protected: + void reset(); + int get_left_batch(const int64_t batch_size, const ObBatchRows *&child_brs); + int build_hash_table_from_left_batch(bool from_child, const int64_t batch_size); + int init_hash_partition_infras(); + int init_hash_partition_infras_for_batch(); + int init_mem_context(); + int convert_vector(const common::ObIArray &src_exprs, + const common::ObIArray &dst_exprs, + const ObBatchRows *&child_brs); + +protected: + //used by intersect and except + bool first_get_left_; + bool has_got_part_; + ObSqlWorkAreaProfile profile_; + ObSqlMemMgrProcessor sql_mem_processor_; + ObHashPartInfrastructureVecImpl hp_infras_; + uint64_t *hash_values_for_batch_; + //for batch array init, not reset in rescan + bool need_init_; + const ObBatchRows *left_brs_; + lib::MemoryContext mem_context_; +}; + +} // end namespace sql +} // end namespace oceanbase + +#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_SET_VEC_OP_H_ diff --git a/src/sql/engine/set/ob_hash_union_vec_op.cpp b/src/sql/engine/set/ob_hash_union_vec_op.cpp new file mode 100644 index 0000000000..e8e79a3f63 --- /dev/null +++ b/src/sql/engine/set/ob_hash_union_vec_op.cpp @@ -0,0 +1,194 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "sql/engine/set/ob_hash_union_vec_op.h" + +namespace oceanbase +{ +using namespace common; +namespace sql +{ + +ObHashUnionVecSpec::ObHashUnionVecSpec(ObIAllocator &alloc, const ObPhyOperatorType type) + : ObHashSetVecSpec(alloc, type) +{ +} + +OB_SERIALIZE_MEMBER((ObHashUnionVecSpec, ObHashSetVecSpec)); + +ObHashUnionVecOp::ObHashUnionVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input) + : ObHashSetVecOp(exec_ctx, spec, input), + cur_child_op_(nullptr) +{} + +int ObHashUnionVecOp::inner_open() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObHashSetVecOp::inner_open())) { + LOG_WARN("failed to inner open", K(ret)); + } else { + cur_child_op_ = left_; + } + return ret; +} + +int ObHashUnionVecOp::inner_close() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObHashSetVecOp::inner_close())) { + LOG_WARN("failed to inner close", K(ret)); + } + return ret; +} + +int ObHashUnionVecOp::inner_rescan() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObHashSetVecOp::inner_rescan())) { + LOG_WARN("failed to rescan child operator", K(ret)); + } else { + cur_child_op_ = left_; + } + return ret; +} + +void ObHashUnionVecOp::destroy() +{ + ObHashSetVecOp::destroy(); +} + +int ObHashUnionVecOp::get_child_next_batch(const int64_t batch_size, const ObBatchRows *&child_brs) +{ + int ret = cur_child_op_->get_next_batch(batch_size, child_brs); + if (OB_SUCC(ret) && 0 == child_brs->size_ && child_brs->end_) { + if (cur_child_op_ == left_) { + cur_child_op_ = right_; + ret = cur_child_op_->get_next_batch(batch_size, child_brs); + } + } + return ret; +} + +int ObHashUnionVecOp::inner_get_next_row() +{ + return OB_NOT_IMPLEMENT; +} + +int ObHashUnionVecOp::inner_get_next_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + const int64_t batch_size = std::min(max_row_cnt, MY_SPEC.max_batch_size_); + bool child_op_end = false; + bool end_to_process = false; + int64_t read_rows = -1; + clear_evaluated_flag(); + if (OB_ISNULL(cur_child_op_)) { + ret = OB_NOT_INIT; + LOG_WARN("cur_child_op is null", K(ret)); + } else if (first_get_left_) { + if (OB_FAIL(init_hash_partition_infras_for_batch())) { + LOG_WARN("failed to init hash partition infra batch", K(ret)); + } + first_get_left_ = false; + } + bool got_batch = false; + ObBitVector *output_vec = nullptr; + + const ObBatchRows *child_brs = nullptr; + if (!has_got_part_) { + if (child_op_end) { + end_to_process = true; + } else if (OB_FAIL(get_child_next_batch(batch_size, child_brs))) { + LOG_WARN("failed to get child next batch", K(ret)); + } else if (OB_FAIL(convert_vector(cur_child_op_->get_spec().output_, + MY_SPEC.set_exprs_, + child_brs))) { + LOG_WARN("copy current row failed", K(ret)); + } else if (OB_FAIL(hp_infras_.calc_hash_value_for_batch(MY_SPEC.set_exprs_, + *child_brs, + hash_values_for_batch_))) { + LOG_WARN("failed to calc hash value for batch", K(ret)); + } else { + child_op_end = cur_child_op_ == right_ && child_brs->end_ && 0 != child_brs->size_; + end_to_process = cur_child_op_ == right_ && child_brs->end_ && 0 == child_brs->size_; + read_rows = child_brs->size_; + } + } else if (OB_FAIL(hp_infras_.get_left_next_batch(MY_SPEC.set_exprs_, + batch_size, + read_rows, + hash_values_for_batch_))) { + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + end_to_process = true; + } else { + LOG_WARN("failed to get batch from infra", K(ret)); + } + } + if (OB_SUCC(ret) && end_to_process) { + end_to_process = false; + if (OB_FAIL(hp_infras_.finish_insert_row())) { + LOG_WARN("failed to finish insert row", K(ret)); + } else if (!has_got_part_) { + has_got_part_ = true; + } else if (OB_FAIL(hp_infras_.close_cur_part(InputSide::LEFT))) { + LOG_WARN("failed to close cur part", K(ret)); + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(hp_infras_.end_round())) { + LOG_WARN("failed to end round", K(ret)); + } else if (OB_FAIL(try_check_status())) { + LOG_WARN("failed to check status", K(ret)); + } else if (OB_FAIL(hp_infras_.start_round())) { + LOG_WARN("failed to start round", K(ret)); + } else if (OB_FAIL(hp_infras_.get_next_partition(InputSide::LEFT))) { + if (OB_ITER_END != ret) { + LOG_WARN("failed to get next dumped partition", K(ret)); + } + } else if (OB_FAIL(hp_infras_.open_cur_part(InputSide::LEFT))) { + LOG_WARN("failed to open cur part", K(ret)); + } else if (OB_FAIL(hp_infras_.resize(hp_infras_.get_cur_part_row_cnt(InputSide::LEFT)))) { + LOG_WARN("failed to resize cur part", K(ret)); + } + } else if (OB_FAIL(ret)) { + } else if (has_got_part_ && OB_FAIL(hp_infras_.insert_row_for_batch(MY_SPEC.set_exprs_, + hash_values_for_batch_, + read_rows, + nullptr, + output_vec))) { + LOG_WARN("failed to insert batch for dump", K(ret)); + } else if (!has_got_part_ && OB_FAIL(hp_infras_.insert_row_for_batch(MY_SPEC.set_exprs_, + hash_values_for_batch_, + read_rows, + child_brs->skip_, + output_vec))) { + LOG_WARN("failed to insert batch for no dump", K(ret)); + } else if (OB_ISNULL(output_vec)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get output vec", K(ret)); + } else { + brs_.size_ = read_rows; + brs_.skip_->deep_copy(*output_vec, read_rows); + } + + + if (OB_ITER_END == ret) { + ret = OB_SUCCESS; + brs_.size_ = 0; + brs_.end_ = true; + } + return ret; +} + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/set/ob_hash_union_vec_op.h b/src/sql/engine/set/ob_hash_union_vec_op.h new file mode 100644 index 0000000000..bd45ec94cd --- /dev/null +++ b/src/sql/engine/set/ob_hash_union_vec_op.h @@ -0,0 +1,52 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_BASIC_OB_SET_OB_HASH_UNION_VEC_OP_H_ +#define OCEANBASE_BASIC_OB_SET_OB_HASH_UNION_VEC_OP_H_ + +#include "sql/engine/set/ob_hash_set_vec_op.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObHashUnionVecSpec : public ObHashSetVecSpec +{ +OB_UNIS_VERSION_V(1); +public: + ObHashUnionVecSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type); +}; + +class ObHashUnionVecOp : public ObHashSetVecOp +{ +public: + ObHashUnionVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input); + ~ObHashUnionVecOp() {} + + virtual int inner_open() override; + virtual int inner_get_next_batch(const int64_t max_row_cnt) override; + virtual int inner_close() override; + virtual int inner_rescan() override; + virtual int inner_get_next_row() override; + virtual void destroy() override; + +private: + int get_child_next_batch(const int64_t batch_size, const ObBatchRows *&child_brs); +private: + ObOperator *cur_child_op_; +}; + +} // end namespace sql +} // end namespace oceanbase + +#endif // OCEANBASE_BASIC_OB_SET_OB_HASH_UNION_VEC_OP_H_ diff --git a/src/sql/engine/sort/ob_pd_topn_sort_filter.cpp b/src/sql/engine/sort/ob_pd_topn_sort_filter.cpp new file mode 100644 index 0000000000..b7a80a6978 --- /dev/null +++ b/src/sql/engine/sort/ob_pd_topn_sort_filter.cpp @@ -0,0 +1,212 @@ +/** + * Copyright (c) 2024 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "sql/engine/sort/ob_pd_topn_sort_filter.h" +#include "sql/engine/sort/ob_sort_vec_op.h" +#include "sql/engine/sort/ob_sort_vec_op_context.h" +#include "sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "sql/engine/basic/ob_compact_row.h" +#include "share/rc/ob_context.h" +#include "sql/engine/expr/ob_expr_topn_filter.h" +#include "sql/engine/px/ob_px_sqc_handler.h" +#include "sql/engine/ob_exec_context.h" +#include "observer/ob_server_struct.h" +#include "src/observer/ob_server.h" + +namespace oceanbase +{ +namespace sql +{ + +ObPushDownTopNFilter::~ObPushDownTopNFilter() +{ + destroy(); +} + +void ObPushDownTopNFilter::destroy() +{ + if (pd_topn_filter_msg_ != nullptr) { + ObP2PDhKey dh_key(pd_topn_filter_msg_->get_p2p_datahub_id(), + pd_topn_filter_msg_->get_px_seq_id(), + pd_topn_filter_msg_->get_task_id()); + ObP2PDatahubMsgBase *msg = nullptr; + PX_P2P_DH.erase_msg(dh_key, msg); + pd_topn_filter_msg_->destroy(); + mem_context_->get_malloc_allocator().free(pd_topn_filter_msg_); + pd_topn_filter_msg_ = nullptr; + } + enabled_ = false; + need_update_ = false; + msg_set_ = false; + pd_topn_filter_info_ = nullptr; + topn_filter_ctx_ = nullptr; +} + +inline int ObPushDownTopNFilter::init(const ObSortVecOpContext &ctx, + lib::MemoryContext &mem_context) +{ + return init(ctx.pd_topn_filter_info_, ctx.tenant_id_, ctx.sk_collations_, ctx.exec_ctx_, + mem_context, true /*use_rich_format*/); +} + +int ObPushDownTopNFilter::init(const ObPushDownTopNFilterInfo *pd_topn_filter_info, + uint64_t tenant_id, + const ObIArray *sort_collations, + ObExecContext *exec_ctx, lib::MemoryContext &mem_context, + bool use_rich_format /*=false*/) +{ + int ret = OB_SUCCESS; + ObP2PDatahubMsgBase *p2p_msg = nullptr; + ObPushDownTopNFilterMsg *pd_topn_filter_msg = nullptr; + mem_context_ = mem_context; + pd_topn_filter_info_ = pd_topn_filter_info; + + ObPxSqcHandler *sqc_handler = exec_ctx->get_sqc_handler(); + int64_t px_seq_id = 0; + if (nullptr == sqc_handler) { + // For local plans that do not involve a PX coordinator, it's necessary to simulate a px seq id. + // Failing to do so means that different SQL queries could access the same plan cache, resulting + // in identical ObP2PDhKey values. Consequently, these queries would receive the same TOP-N + // pushdown runtime filter message from a single SQL query, which is a hazardous behavior. + px_seq_id = GCTX.sql_engine_->get_px_sequence_id(); + } else { + ObPxSqcMeta &sqc = sqc_handler->get_sqc_init_arg().sqc_; + px_seq_id = sqc.get_interrupt_id().px_interrupt_id_.first_; + } + + if (OB_FAIL(PX_P2P_DH.alloc_msg(mem_context_->get_malloc_allocator(), + pd_topn_filter_info_->dh_msg_type_, p2p_msg))) { + LOG_WARN("fail to alloc msg", K(ret)); + } else if (FALSE_IT(pd_topn_filter_msg = static_cast(p2p_msg))) { + } else if (OB_FAIL(pd_topn_filter_msg->init(pd_topn_filter_info, tenant_id, sort_collations, + exec_ctx, px_seq_id))) { + LOG_WARN("failed to init pushdown topn filter msg"); + } else if (!pd_topn_filter_info_->is_shuffle_ + && OB_FAIL(create_pd_topn_filter_ctx(pd_topn_filter_info, exec_ctx, use_rich_format, + px_seq_id))) { + // for local topn filter pushdown, we directly create filter_ctx here; + // for global topn filter pushdown, we need add a topn filter use operator and + // create filter_ctx in topn filter use operator; + LOG_WARN("failed to create_pd_topn_filter_ctx"); + } else { + pd_topn_filter_msg_ = pd_topn_filter_msg; + enabled_ = true; + } + + LOG_TRACE("[TopN Filter] init topn filter msg"); + return ret; +} + +int ObPushDownTopNFilter::update_filter_data(ObCompactRow *compact_row, const RowMeta *row_meta_) +{ + int ret = OB_SUCCESS; + bool is_updated = false; + if (OB_FAIL(pd_topn_filter_msg_->update_filter_data(compact_row, row_meta_, is_updated))) { + LOG_WARN("failed to update filter data", K(ret)); + } else if (FALSE_IT(set_need_update(false))) { + } else if (is_updated && OB_FAIL(publish_topn_msg())) { + LOG_WARN("failed to publish topn msg"); + } + return ret; +} + +int ObPushDownTopNFilter::update_filter_data(ObChunkDatumStore::StoredRow *store_row) +{ + int ret = OB_SUCCESS; + bool is_updated = false; + if (OB_FAIL(pd_topn_filter_msg_->update_filter_data(store_row, is_updated))) { + LOG_WARN("failed to update filter data", K(ret)); + } else if (FALSE_IT(set_need_update(false))) { + } else if (is_updated && OB_FAIL(publish_topn_msg())) { + LOG_WARN("failed to publish topn msg"); + } + return ret; +} + +int ObPushDownTopNFilter::create_pd_topn_filter_ctx( + const ObPushDownTopNFilterInfo *pd_topn_filter_info, ObExecContext *exec_ctx, + bool use_rich_format, int64_t px_seq_id) +{ + int ret = OB_SUCCESS; + // TODO XUNSI: if pushdown to the right table with join key but not sort key not from right table, + // prepare the compare info for it + uint32_t expr_ctx_id = pd_topn_filter_info->expr_ctx_id_; + ObExprTopNFilterContext *topn_filter_ctx = + static_cast(exec_ctx->get_expr_op_ctx(expr_ctx_id)); + ObPxSqcHandler *sqc_handler = exec_ctx->get_sqc_handler(); + int64_t task_id = 0; + if (nullptr == sqc_handler) { + } else { + // PX plan, but non shared topn filter, each thread has its own id + task_id = exec_ctx->get_px_task_id(); + } + + if (nullptr == topn_filter_ctx) { + if (OB_FAIL(exec_ctx->create_expr_op_ctx(expr_ctx_id, topn_filter_ctx))) { + LOG_WARN("failed to create operator ctx", K(ret), K(expr_ctx_id)); + } else { + topn_filter_ctx_ = topn_filter_ctx; + ObP2PDhKey dh_key(pd_topn_filter_info->p2p_dh_id_, px_seq_id, task_id); + topn_filter_ctx->topn_filter_key_ = dh_key; + topn_filter_ctx->slide_window_.set_adptive_ratio_thresheld( + pd_topn_filter_info->adaptive_filter_ratio_); + if (use_rich_format) { + int64_t max_batch_size = pd_topn_filter_info->max_batch_size_; + void *buf = nullptr; + if (OB_ISNULL(buf = exec_ctx->get_allocator().alloc((sizeof(uint16_t) * max_batch_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate selector_", K(max_batch_size)); + } else { + topn_filter_ctx->row_selector_ = static_cast(buf); + } + } + } + } else { + // topn_filter_ctx already exists means it is in rescan process + topn_filter_ctx_ = topn_filter_ctx; + if (nullptr == sqc_handler) { + // for none px plan rescan, the px_seq_id will be regenerated + // so the dh_key should be updated + ObP2PDhKey dh_key(pd_topn_filter_info->p2p_dh_id_, px_seq_id, task_id); + topn_filter_ctx_->topn_filter_key_ = dh_key; + } + } + return ret; +} + +int ObPushDownTopNFilter::publish_topn_msg() +{ + int ret = OB_SUCCESS; + if (pd_topn_filter_info_->is_shuffle_) { + // shuffled, not need set to local p2pmap. + // TODO XUNSI: impl global topn filter + } else if (!msg_set_) { + pd_topn_filter_msg_->set_is_ready(true); + if (OB_FAIL(PX_P2P_DH.send_local_p2p_msg(*pd_topn_filter_msg_))) { + LOG_WARN("fail to send local p2p msg", K(ret)); + } else { + msg_set_ = true; + if (topn_filter_ctx_) { + LOG_TRACE("[TopN Filter] success to set msg to local p2p datahub", K(pd_topn_filter_msg_), + K(topn_filter_ctx_->topn_filter_key_), K(topn_filter_ctx_->total_count_), + K(topn_filter_ctx_->check_count_), K(topn_filter_ctx_->filter_count_)); + } + } + } + return ret; +} + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/sort/ob_pd_topn_sort_filter.h b/src/sql/engine/sort/ob_pd_topn_sort_filter.h new file mode 100644 index 0000000000..4ef5a2d601 --- /dev/null +++ b/src/sql/engine/sort/ob_pd_topn_sort_filter.h @@ -0,0 +1,81 @@ +/** + * Copyright (c) 2024 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#pragma once + +#include "lib/ob_define.h" +#include "sql/engine/basic/ob_chunk_datum_store.h" + +namespace oceanbase +{ +namespace lib +{ + class MemoryContext; +} + +namespace sql +{ +struct ObSortVecOpContext; +struct ObCompactRow; +struct RowMeta; +class ObPushDownTopNFilterMsg; +struct ObPushDownTopNFilterInfo; +class ObExecContext; +struct ObSortFieldCollation; +class ObExprTopNFilterContext; + +// ob_sort_vec_op_impl.ipp is an ipp with variable template +// move the topn code to the ob_pd_topn_sort_filter.cpp to accelerate the compile speed +class ObPushDownTopNFilter +{ +public: + ObPushDownTopNFilter() + : enabled_(false), need_update_(false), msg_set_(false), mem_context_(nullptr), + pd_topn_filter_info_(nullptr), pd_topn_filter_msg_(nullptr), topn_filter_ctx_(nullptr) + {} + ~ObPushDownTopNFilter(); + void destroy(); + + // for vec2.0 + inline int init(const ObSortVecOpContext &ctx, lib::MemoryContext &mem_context); + int init(const ObPushDownTopNFilterInfo *pd_topn_filter_info, uint64_t tenant_id, + const ObIArray *sort_collations, ObExecContext *exec_ctx, + lib::MemoryContext &mem_context, bool use_rich_format = false); + + int update_filter_data(ObCompactRow *compact_row, const RowMeta *row_meta_); + int update_filter_data(ObChunkDatumStore::StoredRow *store_row); + + inline bool enabled() { return enabled_; } + inline void set_need_update(bool flag) { need_update_ = flag; } + inline bool need_update() { return need_update_; } + +private: + int create_pd_topn_filter_ctx(const ObPushDownTopNFilterInfo *pd_topn_filter_info, + ObExecContext *exec_ctx, bool use_rich_format, + int64_t px_seq_id); + // publish topn msg to consumer + int publish_topn_msg(); + +private: + bool enabled_; + bool need_update_; + bool msg_set_; + lib::MemoryContext mem_context_; + const ObPushDownTopNFilterInfo *pd_topn_filter_info_; + ObPushDownTopNFilterMsg *pd_topn_filter_msg_; + // for local topn filter, topn_filter_ctx_ is not null + // for global topn fitler, topn_filter_ctx_ is null + ObExprTopNFilterContext *topn_filter_ctx_; +}; + +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/sort/ob_sort_op.cpp b/src/sql/engine/sort/ob_sort_op.cpp index cab8c6dd20..619ade2694 100644 --- a/src/sql/engine/sort/ob_sort_op.cpp +++ b/src/sql/engine/sort/ob_sort_op.cpp @@ -18,6 +18,7 @@ #include "sql/engine/window_function/ob_window_function_op.h" #include "sql/engine/aggregate/ob_hash_groupby_vec_op.h" #include "share/ob_rpc_struct.h" +#include "sql/engine/expr/ob_expr_topn_filter.h" namespace oceanbase { @@ -85,11 +86,25 @@ int ObSortOp::inner_open() int ObSortOp::inner_rescan() { + if (MY_SPEC.enable_pd_topn_filter() && !MY_SPEC.pd_topn_filter_info_.is_shuffle_) { + // for local topn runtime filter, rescan topn filter expression context + reset_pd_topn_filter_expr_ctx(); + } reset(); iter_end_ = false; return ObOperator::inner_rescan(); } +void ObSortOp::reset_pd_topn_filter_expr_ctx() +{ + uint32_t expr_ctx_id = MY_SPEC.pd_topn_filter_info_.expr_ctx_id_; + ObExprTopNFilterContext *topn_filter_ctx = + static_cast(ctx_.get_expr_op_ctx(expr_ctx_id)); + if (nullptr != topn_filter_ctx) { + topn_filter_ctx->reset_for_rescan(); + } +} + void ObSortOp::reset() { sort_impl_.reset(); @@ -439,10 +454,12 @@ int ObSortOp::init_sort(int64_t tenant_id, &ctx_, MY_SPEC.px_est_size_factor_, est_rows, est_rows))) { LOG_WARN("failed to get px size", K(ret)); } - OZ(sort_impl_.init(tenant_id, &MY_SPEC.sort_collations_, &MY_SPEC.sort_cmp_funs_, - &eval_ctx_, &ctx_, MY_SPEC.enable_encode_sortkey_opt_, MY_SPEC.is_local_merge_sort_, - false /* need_rewind */, MY_SPEC.part_cnt_, topn_cnt, MY_SPEC.is_fetch_with_ties_, - ObChunkDatumStore::BLOCK_SIZE, MY_SPEC.compress_type_, &MY_SPEC.all_exprs_, est_rows, MY_SPEC.prescan_enabled_)); + OZ(sort_impl_.init(tenant_id, &MY_SPEC.sort_collations_, &MY_SPEC.sort_cmp_funs_, &eval_ctx_, + &ctx_, MY_SPEC.enable_encode_sortkey_opt_, MY_SPEC.is_local_merge_sort_, + false /* need_rewind */, MY_SPEC.part_cnt_, topn_cnt, + MY_SPEC.is_fetch_with_ties_, ObChunkDatumStore::BLOCK_SIZE, + MY_SPEC.compress_type_, &MY_SPEC.all_exprs_, est_rows, + MY_SPEC.prescan_enabled_, &MY_SPEC.pd_topn_filter_info_)); if (is_batch) { read_batch_func_ = &ObSortOp::sort_impl_next_batch; } else { diff --git a/src/sql/engine/sort/ob_sort_op.h b/src/sql/engine/sort/ob_sort_op.h index 54a8ecb1c9..743a751ae5 100644 --- a/src/sql/engine/sort/ob_sort_op.h +++ b/src/sql/engine/sort/ob_sort_op.h @@ -37,7 +37,9 @@ public: K_(topn_expr), K_(topk_limit_expr), K_(topk_offset_expr), K_(prefix_pos), K_(minimum_row_count), K_(topk_precision), K_(prefix_pos), K_(is_local_merge_sort), K_(prescan_enabled), K_(enable_encode_sortkey_opt), K_(part_cnt), - K_(compress_type)); + K_(compress_type), K_(pd_topn_filter_info)); + + inline bool enable_pd_topn_filter() const { return pd_topn_filter_info_.enabled_; } public: ObExpr *topn_expr_; ObExpr *topk_limit_expr_; @@ -132,6 +134,7 @@ private: int64_t row_count, bool is_batch, int64_t topn_cnt = INT64_MAX); + void reset_pd_topn_filter_expr_ctx(); private: ObSortOpImpl sort_impl_; ObPrefixSortImpl prefix_sort_impl_; diff --git a/src/sql/engine/sort/ob_sort_op_impl.cpp b/src/sql/engine/sort/ob_sort_op_impl.cpp index 12d0aec8ae..b1010e0d05 100644 --- a/src/sql/engine/sort/ob_sort_op_impl.cpp +++ b/src/sql/engine/sort/ob_sort_op_impl.cpp @@ -17,6 +17,7 @@ #include "sql/engine/ob_tenant_sql_memory_manager.h" #include "storage/blocksstable/encoding/ob_encoding_query_util.h" #include "lib/container/ob_iarray.h" +#include "sql/engine/px/p2p_datahub/ob_pushdown_topn_filter_msg.h" namespace oceanbase { @@ -715,7 +716,8 @@ int ObSortOpImpl::init( const ObCompressorType compress_type /* = NONE_COMPRESS */, const ExprFixedArray *exprs /* =nullptr */, const int64_t est_rows /* = 0 */, - const bool use_compact_format /* =false */) + const bool use_compact_format /* =false */, + const ObPushDownTopNFilterInfo *pd_topn_filter_info /* =nullptr */) { int ret = OB_SUCCESS; if (is_inited()) { @@ -767,6 +769,10 @@ int ObSortOpImpl::init( LOG_WARN("init row store failed", K(ret)); } else if (use_heap_sort_ && OB_FAIL(init_topn())) { LOG_WARN("init topn failed", K(ret)); + } else if (use_heap_sort_ && nullptr != pd_topn_filter_info && pd_topn_filter_info->enabled_ + && OB_FAIL(pd_topn_filter_.init(pd_topn_filter_info, tenant_id, sort_collations, + exec_ctx, mem_context_))) { + LOG_WARN("failed to init pd_topn_filter_"); } else if (use_partition_topn_sort_ && OB_FAIL(init_partition_topn(est_rows))) { LOG_WARN("init partition topn failed", K(ret)); } else if (batch_size > 0 @@ -932,6 +938,9 @@ void ObSortOpImpl::reset() } inited_ = false; io_event_observer_ = nullptr; + if (pd_topn_filter_.enabled()) { + pd_topn_filter_.destroy(); + } } template @@ -2011,6 +2020,11 @@ int ObSortOpImpl::sort_inmem_data() op_monitor_info_->otherstat_1_value_ += 1; prev = &rows_->at(i); } + if (OB_FAIL(ret)) { + } else if (pd_topn_filter_.enabled() + && OB_FAIL(pd_topn_filter_.update_filter_data(*imms_heap_->top()))) { + LOG_WARN("failed to update filter data", K(ret)); + } heap_iter_begin_ = false; } } @@ -2404,6 +2418,12 @@ int ObSortOpImpl::add_heap_sort_row(const common::ObIArray &exprs, store_row = new_row; LOG_DEBUG("in memory topn sort check add row", KPC(new_row)); } + if (OB_SUCC(ret)) { + // the first time reach heap capacity, set_need_update to update topn filter data; + if (pd_topn_filter_.enabled()) { + pd_topn_filter_.set_need_update(true); + } + } } return ret; @@ -2430,6 +2450,11 @@ int ObSortOpImpl::add_heap_sort_batch(const common::ObIArray &exprs, } row_count++; } + if (OB_SUCC(ret) && pd_topn_filter_.need_update()) { + if (OB_FAIL(pd_topn_filter_.update_filter_data(topn_heap_->heap_.top()))) { + LOG_WARN("failed to update filter data", K(ret)); + } + } if (OB_NOT_NULL(append_row_count)) { *append_row_count = row_count; } @@ -2544,6 +2569,9 @@ int ObSortOpImpl::adjust_topn_heap(const common::ObIArray &exprs, LOG_WARN("failed to replace top", K(ret)); } else { store_row = new_row; + if (pd_topn_filter_.enabled()) { + pd_topn_filter_.set_need_update(true); + } } } else { ret = comp_.ret_; diff --git a/src/sql/engine/sort/ob_sort_op_impl.h b/src/sql/engine/sort/ob_sort_op_impl.h index ef0ca9f99a..91d861cf9d 100644 --- a/src/sql/engine/sort/ob_sort_op_impl.h +++ b/src/sql/engine/sort/ob_sort_op_impl.h @@ -21,12 +21,15 @@ #include "share/ob_rpc_struct.h" #include "sql/engine/basic/chunk_store/ob_compact_store.h" #include "sql/engine/basic/ob_chunk_datum_store.h" +#include "sql/engine/sort/ob_pd_topn_sort_filter.h" namespace oceanbase { namespace sql { +struct ObPushDownTopNFilterInfo; + struct ObChunkStoreWrapper { public: @@ -265,7 +268,8 @@ public: const common::ObCompressorType compressor_type = common::NONE_COMPRESSOR, const ExprFixedArray *exprs = nullptr, const int64_t est_rows = 0, - const bool use_compact_format = false); + const bool use_compact_format = false, + const ObPushDownTopNFilterInfo *pd_topn_filter_info = nullptr); virtual int64_t get_prefix_pos() const { return 0; } // keep initialized, can sort same rows (same cell type, cell count, projector) after reuse. @@ -869,6 +873,7 @@ protected: const ExprFixedArray *sort_exprs_; common::ObCompressorType compress_type_; bool use_compact_format_; + ObPushDownTopNFilter pd_topn_filter_; }; class ObInMemoryTopnSortImpl; diff --git a/src/sql/engine/sort/ob_sort_vec_op.cpp b/src/sql/engine/sort/ob_sort_vec_op.cpp index b86a53e368..3cc4d0ef29 100644 --- a/src/sql/engine/sort/ob_sort_vec_op.cpp +++ b/src/sql/engine/sort/ob_sort_vec_op.cpp @@ -18,6 +18,7 @@ #include "sql/engine/basic/ob_temp_row_store.h" #include "sql/engine/px/ob_px_util.h" #include "sql/engine/window_function/ob_window_function_op.h" +#include "sql/engine/expr/ob_expr_topn_filter.h" namespace oceanbase { @@ -45,10 +46,24 @@ ObSortVecOp::ObSortVecOp(ObExecContext &ctx_, const ObOpSpec &spec, ObOpInput *i int ObSortVecOp::inner_rescan() { + if (MY_SPEC.enable_pd_topn_filter() && !MY_SPEC.pd_topn_filter_info_.is_shuffle_) { + // for local topn runtime filter, rescan topn filter expression context + reset_pd_topn_filter_expr_ctx(); + } reset(); return ObOperator::inner_rescan(); } +void ObSortVecOp::reset_pd_topn_filter_expr_ctx() +{ + uint32_t expr_ctx_id = MY_SPEC.pd_topn_filter_info_.expr_ctx_id_; + ObExprTopNFilterContext *topn_filter_ctx = + static_cast(ctx_.get_expr_op_ctx(expr_ctx_id)); + if (nullptr != topn_filter_ctx) { + topn_filter_ctx->reset_for_rescan(); + } +} + void ObSortVecOp::reset() { sort_row_count_ = 0; @@ -75,6 +90,11 @@ int ObSortVecOp::inner_close() { sort_op_provider_.collect_memory_dump_info(op_monitor_info_); sort_op_provider_.unregister_profile(); + if (MY_SPEC.enable_pd_topn_filter()) { + // TODO XUNSI: update plan monitor info of pushdown topn filter + // but all the others_values of the plan_monitor_node is used, + // add an extra string in json format is a considered way + } return OB_SUCCESS; } @@ -201,7 +221,7 @@ int ObSortVecOp::init_temp_row_store(const common::ObIArray &exprs, // do nothing } else if (OB_FAIL(row_store.init(exprs, batch_size, mem_attr, 2 * 1024 * 1024, true, sort_op_provider_.get_extra_size(is_sort_key) /* row_extra_size */, - reorder_fixed_expr, enable_trunc, compress_type))) { + compress_type, reorder_fixed_expr, enable_trunc))) { LOG_WARN("init row store failed", K(ret)); } else if (OB_FAIL(row_store.alloc_dir_id())) { LOG_WARN("failed to alloc dir id", K(ret)); @@ -389,9 +409,11 @@ int ObSortVecOp::init_sort(int64_t tenant_id, int64_t row_count, int64_t topn_cn context.is_fetch_with_ties_ = MY_SPEC.is_fetch_with_ties_; context.has_addon_ = MY_SPEC.has_addon_; context.compress_type_ = MY_SPEC.compress_type_; + context.enable_pd_topn_filter_ = MY_SPEC.enable_pd_topn_filter(); + context.pd_topn_filter_info_ = &MY_SPEC.pd_topn_filter_info_; + context.op_ = this; if (MY_SPEC.prefix_pos_ > 0) { context.prefix_pos_ = MY_SPEC.prefix_pos_; - context.op_ = this; context.sort_row_cnt_ = &sort_row_count_; } else { context.in_local_order_ = MY_SPEC.is_local_merge_sort_; diff --git a/src/sql/engine/sort/ob_sort_vec_op.h b/src/sql/engine/sort/ob_sort_vec_op.h index e1909a5cd5..8a694abae3 100644 --- a/src/sql/engine/sort/ob_sort_vec_op.h +++ b/src/sql/engine/sort/ob_sort_vec_op.h @@ -25,6 +25,7 @@ namespace oceanbase { namespace sql { + class ObSortVecSpec : public ObOpSpec { OB_UNIS_VERSION_V(1); @@ -32,11 +33,12 @@ class ObSortVecSpec : public ObOpSpec public: ObSortVecSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type); + inline bool enable_pd_topn_filter() const { return pd_topn_filter_info_.enabled_; } INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(topn_expr), K_(topk_limit_expr), K_(topk_offset_expr), K_(prefix_pos), K_(minimum_row_count), K_(topk_precision), K_(prefix_pos), K_(is_local_merge_sort), K_(prescan_enabled), K_(enable_encode_sortkey_opt), K_(has_addon), - K_(part_cnt), K_(compress_type)); + K_(part_cnt), K_(compress_type), K_(pd_topn_filter_info)); public: ObExpr *topn_expr_; @@ -98,6 +100,7 @@ private: int init_temp_row_store(const common::ObIArray &exprs, const int64_t batch_size, const ObMemAttr &mem_attr, const bool is_sort_key, ObCompressorType compress_type, ObTempRowStore &row_store); + void reset_pd_topn_filter_expr_ctx(); private: ObSortVecOpProvider sort_op_provider_; diff --git a/src/sql/engine/sort/ob_sort_vec_op_context.h b/src/sql/engine/sort/ob_sort_vec_op_context.h index 7a3267c6e8..01eaeb7be1 100644 --- a/src/sql/engine/sort/ob_sort_vec_op_context.h +++ b/src/sql/engine/sort/ob_sort_vec_op_context.h @@ -18,6 +18,7 @@ namespace oceanbase { namespace sql { +struct ObPushDownTopNFilterInfo; struct ObSortVecOpContext { ObSortVecOpContext() : @@ -52,11 +53,13 @@ struct ObSortVecOpContext uint32_t need_rewind_ : 1; uint32_t is_fetch_with_ties_ : 1; uint32_t has_addon_ : 1; - uint32_t reserved_ : 27; + uint32_t enable_pd_topn_filter_ : 1; + uint32_t reserved_ : 26; }; uint32_t flag_; }; ObCompressorType compress_type_; + const ObPushDownTopNFilterInfo *pd_topn_filter_info_; }; } // end namespace sql diff --git a/src/sql/engine/sort/ob_sort_vec_op_impl.h b/src/sql/engine/sort/ob_sort_vec_op_impl.h index 4610801e53..86c34d88be 100644 --- a/src/sql/engine/sort/ob_sort_vec_op_impl.h +++ b/src/sql/engine/sort/ob_sort_vec_op_impl.h @@ -26,6 +26,7 @@ #include "sql/engine/sort/ob_sort_vec_op_eager_filter.h" #include "sql/engine/sort/ob_sort_vec_op_store_row_factory.h" #include "observer/omt/ob_tenant_config_mgr.h" +#include "sql/engine/sort/ob_pd_topn_sort_filter.h" namespace oceanbase { namespace sql { @@ -87,6 +88,7 @@ public: { reset(); } + int init_pd_topn_filter_msg(ObSortVecOpContext &ctx); int init_vec_ptrs(const common::ObIArray &exprs, common::ObFixedArray &vec_ptrs, ObEvalCtx *eval_ctx); @@ -444,6 +446,7 @@ protected: ObSortVecOpEagerFilter *topn_filter_; bool is_topn_filter_enabled_; ObCompressorType compress_type_; + ObPushDownTopNFilter pd_topn_filter_; }; } // end namespace sql diff --git a/src/sql/engine/sort/ob_sort_vec_op_impl.ipp b/src/sql/engine/sort/ob_sort_vec_op_impl.ipp index a8ee1af08a..dc6c02a112 100644 --- a/src/sql/engine/sort/ob_sort_vec_op_impl.ipp +++ b/src/sql/engine/sort/ob_sort_vec_op_impl.ipp @@ -176,7 +176,8 @@ int ObSortVecOpImpl::init_temp_row_store( const bool reorder_fixed_expr = true; ObMemAttr mem_attr(tenant_id_, ObModIds::OB_SQL_SORT_ROW, ObCtxIds::WORK_AREA); if (OB_FAIL(row_store.init(exprs, batch_size, mem_attr, mem_limit, enable_dump, - extra_size /* row_extra_size */, reorder_fixed_expr, enable_trunc, compress_type))) { + extra_size /* row_extra_size */, compress_type, reorder_fixed_expr, + enable_trunc))) { SQL_ENG_LOG(WARN, "init row store failed", K(ret)); } else { row_store.set_dir_id(sql_mem_processor_.get_dir_id()); @@ -299,6 +300,9 @@ int ObSortVecOpImpl::init(ObSortVecOpContext &ctx comp_, &mem_context_->get_malloc_allocator()))) { ret = OB_ALLOCATE_MEMORY_FAILED; SQL_ENG_LOG(WARN, "allocate memory failed", K(ret)); + } else if (is_topn_sort() && ctx.enable_pd_topn_filter_ + && OB_FAIL(pd_topn_filter_.init(ctx, mem_context_))) { + SQL_ENG_LOG(WARN, "failed to init pushdown topn filter", K(ret)); } else if (batch_size > 0 && OB_ISNULL(sk_rows_ = SK_DOWNCAST_PP(mem_context_->get_malloc_allocator().alloc( sizeof(*sk_rows_) * batch_size)))) { @@ -591,6 +595,9 @@ int ObSortVecOpImpl::adjust_topn_heap(const Store SQL_ENG_LOG(WARN, "failed to replace top", K(ret)); } else { store_row = new_row; + if (pd_topn_filter_.enabled()) { + pd_topn_filter_.set_need_update(true); + } } } else { ret = comp_.ret_; @@ -932,6 +939,12 @@ int ObSortVecOpImpl::add_heap_sort_row(const Stor store_row = new_sk_row; SQL_ENG_LOG(DEBUG, "in memory topn sort check add row", KPC(new_sk_row)); } + if (OB_SUCC(ret) && topn_heap_->count() == topn_cnt_) { + // the first time reach heap capacity, set_need_update to update topn filter data; + if (pd_topn_filter_.enabled()) { + pd_topn_filter_.set_need_update(true); + } + } } return ret; } @@ -989,6 +1002,11 @@ int ObSortVecOpImpl::add_heap_sort_batch( } row_count++; } + if (OB_SUCC(ret) && pd_topn_filter_.need_update()) { + if (OB_FAIL(pd_topn_filter_.update_filter_data(topn_heap_->top(), sk_row_meta_))) { + LOG_WARN("failed to update filter data", K(ret)); + } + } if (OB_NOT_NULL(append_row_count)) { *append_row_count = row_count; } @@ -1028,6 +1046,11 @@ int ObSortVecOpImpl::add_heap_sort_batch( SQL_ENG_LOG(WARN, "failed to add heap sort batch", K(ret)); } } + if (OB_SUCC(ret) && pd_topn_filter_.need_update()) { + if (OB_FAIL(pd_topn_filter_.update_filter_data(topn_heap_->top(), sk_row_meta_))) { + LOG_WARN("failed to update filter data", K(ret)); + } + } return ret; } @@ -1489,6 +1512,11 @@ int ObSortVecOpImpl::sort_inmem_data() op_monitor_info_.otherstat_1_value_ += 1; prev = &rows_->at(i); } + if (OB_FAIL(ret)) { + } else if (pd_topn_filter_.enabled() + && OB_FAIL(pd_topn_filter_.update_filter_data(*imms_heap_->top(), sk_row_meta_))) { + LOG_WARN("failed to update filter data", K(ret)); + } heap_iter_begin_ = false; } } diff --git a/src/sql/engine/window_function/ob_window_function_vec_op.cpp b/src/sql/engine/window_function/ob_window_function_vec_op.cpp new file mode 100644 index 0000000000..90e88134cd --- /dev/null +++ b/src/sql/engine/window_function/ob_window_function_vec_op.cpp @@ -0,0 +1,4022 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#define USING_LOG_PREFIX SQL_ENG + +#include "lib/utility/ob_print_utils.h" +#include "ob_window_function_vec_op.h" +#include "share/aggregate/iaggregate.h" +#include "share/aggregate/processor.h" +#include "sql/engine/px/ob_px_sqc_handler.h" +#include "sql/engine/px/datahub/components/ob_dh_winbuf.h" + +#define SWAP_STORES(left, right) \ + do { \ + input_stores_.swap_##left##_##right(); \ + for (WinFuncColExpr *it = wf_list_.get_first(); it != wf_list_.get_header(); \ + it = it->get_next()) { \ + it->res_->swap_##left##_##right(); \ + } \ + } while (0) + +#define FOREACH_WINCOL(end) \ + for (WinFuncColExpr *it = wf_list_.get_first(); OB_SUCC(ret) && it != (end); it = it->get_next()) + +#define END_WF (wf_list_.get_header()) + +namespace oceanbase +{ +namespace sql +{ +OB_SERIALIZE_MEMBER((ObWindowFunctionVecSpec, ObWindowFunctionSpec)); + +OB_SERIALIZE_MEMBER(ObWindowFunctionVecOpInput, local_task_count_, total_task_count_, + wf_participator_shared_info_); + + +// if agg_func == T_INVALID, use wf_info.func_type_ to dispatch merge function dynamicly +template +struct __PartialResult +{ + __PartialResult(ObEvalCtx &ctx, ObIAllocator &allocator) : eval_ctx_(ctx), merge_alloc_(allocator) + {} + template + int merge(const WinFuncInfo &wf_info, const bool src_isnull, const char *src, int32_t src_len) + { + int ret = OB_SUCCESS; + ResFmt *res_data = static_cast(wf_info.expr_->get_vector(eval_ctx_)); + VecValueTypeClass out_tc = wf_info.expr_->get_vec_value_tc(); + int64_t output_idx = eval_ctx_.get_batch_idx(); + if (agg_func == T_FUN_MIN || agg_func == T_FUN_MAX + || wf_info.func_type_ == T_FUN_MIN || wf_info.func_type_ == T_FUN_MAX) { + bool is_min_fn = (agg_func == T_FUN_MIN || wf_info.func_type_ == T_FUN_MIN); + NullSafeRowCmpFunc cmp_fn = (is_min_fn ? wf_info.expr_->basic_funcs_->row_null_last_cmp_ : + wf_info.expr_->basic_funcs_->row_null_first_cmp_); + ObObjMeta &obj_meta = wf_info.expr_->obj_meta_; + const char *payload = nullptr; + int32_t len = 0; + int cmp_ret =0; + bool cur_isnull = false; + res_data->get_payload(output_idx, cur_isnull, payload, len); + if (cur_isnull && !src_isnull) { + if (OB_FAIL(set_payload(res_data, out_tc, output_idx, src, src_len))) { + LOG_WARN("set payload failed", K(ret)); + } + } else if (src_isnull) { + // do nothing + } else if (OB_FAIL(cmp_fn(obj_meta, obj_meta, payload, len, cur_isnull, src, src_len, + src_isnull, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if ((cmp_ret > 0 && (agg_func == T_FUN_MIN || wf_info.func_type_ == T_FUN_MIN)) + || (cmp_ret < 0 && (agg_func == T_FUN_MAX || wf_info.func_type_ == T_FUN_MAX))) { + if (OB_FAIL(set_payload(res_data, out_tc, output_idx, src, src_len))) { + LOG_WARN("set payload failed", K(ret)); + } + } + } else if (agg_func == T_FUN_COUNT + || agg_func == T_WIN_FUN_RANK + || agg_func == T_WIN_FUN_DENSE_RANK + || wf_info.func_type_ == T_FUN_COUNT + || wf_info.func_type_ == T_WIN_FUN_RANK + || wf_info.func_type_ == T_WIN_FUN_DENSE_RANK) { + // same as COUNT_SUM + const char *res_buf = nullptr; + bool res_isnull = false; + int32_t res_len = 0; + res_data->get_payload(output_idx, res_isnull, res_buf, res_len); + bool is_rank_like = !(agg_func == T_FUN_COUNT || wf_info.func_type_ == T_FUN_COUNT); + if (src_isnull) { + } else if (lib::is_oracle_mode()) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (is_rank_like) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (!is_rank_like ) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } + if (OB_FAIL(ret)) { + LOG_WARN("merge result failed", K(ret), K(agg_func), K(wf_info.func_type_)); + } + } else if (agg_func == T_FUN_SUM || wf_info.func_type_ == T_FUN_SUM) { + const char *res_buf; + bool res_isnull = false; + int32_t res_len = 0; + res_data->get_payload(output_idx, res_isnull, res_buf, res_len); + VecValueTypeClass res_tc = wf_info.expr_->get_vec_value_tc(); + if (src_isnull) { // do nothing + } else if (std::is_same::value || res_tc == VEC_TC_NUMBER) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (std::is_same>::value || res_tc == VEC_TC_DEC_INT32) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (std::is_same>::value || res_tc == VEC_TC_DEC_INT64) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (std::is_same>::value || res_tc == VEC_TC_DEC_INT128) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (std::is_same>::value || res_tc == VEC_TC_DEC_INT256) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (std::is_same>::value || res_tc == VEC_TC_DEC_INT512) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (std::is_same>::value || res_tc == VEC_TC_FLOAT) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } else if (std::is_same>::value || res_tc == VEC_TC_DOUBLE + || res_tc == VEC_TC_FIXED_DOUBLE) { + ret = sum_merge(res_buf, src, res_isnull, src_isnull, res_data, output_idx); + } + if (OB_FAIL(ret)) { + LOG_WARN("merge sum partial results failed", K(ret), K(res_tc), K(wf_info), K(agg_func)); + } + } + return ret; + } + + template + int add_rank(const WinFuncInfo &wf_info, const bool src_isnull, const char *src, int32_t src_len, + int64_t rank_val) + { + int ret = OB_SUCCESS; + ResFmt *res_data = static_cast(wf_info.expr_->get_vector(eval_ctx_)); + int64_t output_idx = eval_ctx_.get_batch_idx(); + if (std::is_same::value + || wf_info.expr_->datum_meta_.type_ == ObNumberType) { + ObNumStackAllocator<4> tmp_alloc; + number::ObNumber extra_nmb; + number::ObNumber src2_nmb(res_data->get_number(output_idx)); + number::ObNumber res_nmb; + if (OB_FAIL(extra_nmb.from(rank_val, tmp_alloc))) { + LOG_WARN("from number failed", K(ret)); + } else if (OB_FAIL(res_nmb.add(extra_nmb, res_nmb, tmp_alloc))) { + LOG_WARN("add number failed", K(ret)); + } else if (!src_isnull) { + number::ObNumber src_nmb(*reinterpret_cast(src)); + if (OB_FAIL(res_nmb.add(src_nmb, res_nmb, tmp_alloc))) { + LOG_WARN("add number failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (!res_data->is_null(output_idx)) { + number::ObNumber src_nmb(res_data->get_number(output_idx)); + if (OB_FAIL(res_nmb.add(src_nmb, res_nmb, tmp_alloc))) { + LOG_WARN("add number failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else { + int32_t output_len = res_nmb.get_desc().len_ * sizeof(uint32_t) + sizeof(ObNumberDesc); + void *out_buf = merge_alloc_.alloc(output_len); + if (OB_ISNULL(out_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + number::ObCompactNumber *res_cnum = reinterpret_cast(out_buf); + res_cnum->desc_ = res_nmb.get_desc(); + MEMCPY(&(res_cnum->digits_[0]), res_nmb.get_digits(), res_nmb.get_desc().len_ * sizeof(uint32_t)); + res_data->set_number_shallow(output_idx, *res_cnum); + } + } + } else if (std::is_same>::value + || wf_info.expr_->datum_meta_.type_ == ObUInt64Type) { + int64_t patch_val = rank_val; + patch_val = patch_val + (src_isnull ? 0 : static_cast(*reinterpret_cast(src))); + patch_val = patch_val + (res_data->is_null(output_idx) ? 0 : static_cast(res_data->get_uint64(output_idx))); + res_data->set_uint(output_idx, static_cast(patch_val)); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected ranking function", K(ret), K(wf_info)); + } + return ret; + } + +private: + template + int add_result(char *res_buf, const char *left_ptr, const char *right_ptr, bool l_isnull, bool r_isnull) + { + + int ret = OB_SUCCESS; + if (l_isnull || r_isnull) { + if (l_isnull && !r_isnull) { + MEMCPY(res_buf, right_ptr, sizeof(T)); + } else if (!l_isnull && r_isnull) { + MEMCPY(res_buf, left_ptr, sizeof(T)); + } + } else { + const T *left = reinterpret_cast(left_ptr); + const T *right = reinterpret_cast(right_ptr); + T &res = *reinterpret_cast(res_buf); + if (std::is_same::value) { + ret = aggregate::add_overflow(*left, *right, res_buf, sizeof(res_buf)); + } else { + res = *left + *right; + } + } + return ret; + } + + template <> + int add_result(char *res_buf, + const char *left_ptr, + const char *right_ptr, + bool l_isnull, bool r_isnull) + { + int ret = OB_SUCCESS; + ObNumStackOnceAlloc tmp_alloc; + number::ObNumber res_nmb; + const number::ObCompactNumber *left = reinterpret_cast(left_ptr); + const number::ObCompactNumber *right = reinterpret_cast(right_ptr); + if (l_isnull || r_isnull) { + if (l_isnull && !r_isnull) { + MEMCPY(res_buf, right, sizeof(uint32_t) + right->desc_.len_ * sizeof(uint32_t)); + } else if (!l_isnull && r_isnull) { + MEMCPY(res_buf, left, sizeof(uint32_t) + left->desc_.len_ * sizeof(uint32_t)); + } + } else { + number::ObNumber l(*left), r(*right); + if (OB_FAIL(l.add(r, res_nmb, tmp_alloc))) { + LOG_WARN("add_v3 failed", K(ret)); + } else { + number::ObCompactNumber *res_cnum = reinterpret_cast(res_buf); + res_cnum->desc_ = res_nmb.d_; + MEMCPY(&(res_cnum->digits_[0]), res_nmb.get_digits(), res_nmb.d_.len_ * sizeof(uint32_t)); + } + } + return ret; + } + + template + int sum_merge(const char *left, const char *right, bool l_isnull, bool r_isnull, ResFmt *res_data, + const int64_t output_idx) + { + int ret = OB_SUCCESS; + const int32_t constexpr tmp_res_size = + (std::is_same::value ? number::ObNumber::MAX_CALC_BYTE_LEN : + sizeof(T)); + char tmp_res[tmp_res_size] = {0}; + ret = add_result(tmp_res, left, right, l_isnull, r_isnull); + if (OB_FAIL(ret)) { + LOG_WARN("add result failed", K(ret)); + } else if (std::is_same::value) { + number::ObCompactNumber *res_cnum = reinterpret_cast(tmp_res); + int32_t output_len = res_cnum->desc_.len_ * sizeof(uint32_t) + sizeof(ObNumberDesc); + char *out_buf = (char *)merge_alloc_.alloc(output_len); + if (OB_ISNULL(out_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(out_buf, tmp_res, output_len); + res_data->set_payload_shallow(output_idx, out_buf, output_len); + } + } else { + res_data->set_payload(output_idx, tmp_res, sizeof(T)); + } + return ret; + } + + + template + int set_payload(ResFmt *res_data, VecValueTypeClass out_tc, int64_t output_idx, const char *src, + int32_t src_len) + { + int ret = OB_SUCCESS; + if (std::is_same::value || is_discrete_vec(out_tc)) { + res_data->set_payload_shallow(output_idx, src, src_len); + } else { + res_data->set_payload(output_idx, src, src_len); + } + return ret; + } + ObEvalCtx &eval_ctx_; + ObIAllocator &merge_alloc_; +}; + +int ObWindowFunctionVecOp::inner_open() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObOperator::inner_open())) { + LOG_WARN("inner ope child operator failed", K(ret)); + } else if (OB_ISNULL(ctx_.get_my_session())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid null session ptr", K(ret)); + } else if (OB_FAIL(init())) { + LOG_WARN("init window function failed", K(ret)); + } else if (OB_FAIL(reset_for_scan(ctx_.get_my_session()->get_effective_tenant_id()))) { + LOG_WARN("reset for scan failed", K(ret)); + } + LOG_TRACE("window function inner open", K(MY_SPEC), K(MY_SPEC.single_part_parallel_), K(MY_SPEC.range_dist_parallel_)); + return ret; +} + +int ObWindowFunctionVecOp::inner_close() +{ + int ret = OB_SUCCESS; + sql_mem_processor_.unregister_profile(); + destroy_stores(); + all_expr_vector_copy_.reset(); + FOREACH_WINCOL(END_WF) { + it->destroy(); + } + wf_list_.reset(); + pby_expr_cnt_idx_array_.reset(); + for (int i = 0; i < pby_hash_values_.count(); i++) { + pby_hash_values_.at(i)->reset(); + } + pby_hash_values_.reset(); + for (int i = 0; i < participator_whole_msg_array_.count(); i++) { + participator_whole_msg_array_.at(i)->reset(); + } + participator_whole_msg_array_.reset(); + for (int i = 0; i < pby_hash_values_sets_.count(); i++) { + pby_hash_values_sets_.at(i)->destroy(); + } + pby_hash_values_sets_.reset(); + input_row_meta_.reset(); + batch_ctx_.reset(); + if (MY_SPEC.single_part_parallel_ && all_wf_res_row_meta_ != nullptr) { + all_wf_res_row_meta_->reset(); + } + if (MY_SPEC.range_dist_parallel_ && rd_coord_row_meta_ != nullptr) { + rd_coord_row_meta_->reset(); + } + all_part_exprs_.reset(); + return ObOperator::inner_close(); +} + +int ObWindowFunctionVecOp::inner_rescan() +{ + int ret = OB_SUCCESS; + // FIXME: add rescan logic + if (OB_FAIL(ObOperator::inner_rescan())) { + LOG_WARN("inner_open child operator failed", K(ret)); + } else if (OB_ISNULL(ctx_.get_my_session())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("null session", K(ret)); + } else if (OB_FAIL(reset_for_scan(ctx_.get_my_session()->get_effective_tenant_id()))) { + LOG_WARN("reset for scan failed", K(ret)); + } else { + FOREACH_WINCOL(END_WF) { + it->reset_for_scan(); + } + } + if (OB_FAIL(ret)) { + } else { + stat_ = ProcessStatus::PARTIAL; + iter_end_ = false; + first_part_saved_ = false; + last_part_saved_ = false; + rescan_alloc_.reset(); + + patch_alloc_.reset(); + first_part_outputed_ = false; + patch_first_ = false; + patch_last_ = false; + last_computed_part_rows_ = 0; + last_aggr_status_ = 0; + next_wf_pby_expr_cnt_to_transmit_ = + const_cast(&MY_SPEC.wf_infos_)->at(0).partition_exprs_.count(); + for (int64_t i = 0; OB_SUCC(ret) && i < pby_hash_values_.count(); ++i) { + if (OB_ISNULL(pby_hash_values_.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("NULL ptr", K(ret), K(pby_hash_values_.count()), K(i)); + } else { + pby_hash_values_.at(i)->reuse(); + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < participator_whole_msg_array_.count(); ++i) { + if (OB_ISNULL(participator_whole_msg_array_.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("NULL ptr", K(ret), K(participator_whole_msg_array_.count()), K(i)); + } else { + participator_whole_msg_array_.at(i)->reset(); + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < pby_hash_values_sets_.count(); ++i) { + if (OB_ISNULL(pby_hash_values_sets_.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("NULL ptr", K(ret), K(pby_hash_values_sets_.count()), K(i)); + } else { + pby_hash_values_sets_.at(i)->reuse(); + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::inner_get_next_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + do { + switch(stat_) { + case ProcessStatus::PARTIAL: { + if (OB_FAIL(partial_next_batch(max_row_cnt))) { + LOG_WARN("partial next batch failed", K(ret)); + } else if (brs_.end_) { + if (MY_SPEC.single_part_parallel_ || MY_SPEC.range_dist_parallel_) { + stat_ = ProcessStatus::COORDINATE; + brs_.end_ = false; + iter_end_ = false; + } + } + break; + } + case ProcessStatus::COORDINATE: { + brs_.size_ = 0; + brs_.end_ = false; + if (OB_FAIL(coordinate())) { + LOG_WARN("coordinate failed", K(ret)); + } else { + stat_ = ProcessStatus::FINAL; + } + break; + } + case ProcessStatus::FINAL: { + if (OB_FAIL(final_next_batch(max_row_cnt))) { + LOG_WARN("get next batch failed", K(ret)); + } + break; + } + } + } while (OB_SUCC(ret) && !(brs_.end_ || brs_.size_ > 0)); + + // Window function expr use batch_size as 1 for evaluation. When it is shared expr and eval again + // as output expr, it will cause core dump because batch_size in output expr is set to a value + // greater than 1. So we clear evaluated flag here to make shared expr in output can evaluated + // normally. + clear_evaluated_flag(); + return ret; +} + +void ObWindowFunctionVecOp::destroy() +{ + sql_mem_processor_.unregister_profile_if_necessary(); + input_stores_.destroy(); + for (WinFuncColExpr *it = wf_list_.get_first(); it != wf_list_.get_header(); + it = it->get_next()) { + it->res_->destroy(); + it->res_ = nullptr; + } + wf_list_.~WinFuncColExprList(); + rescan_alloc_.~ObArenaAllocator(); + patch_alloc_.~ObArenaAllocator(); + destroy_mem_context(); + local_allocator_ = nullptr; + ObOperator::destroy(); +} + +int ObWindowFunctionVecOp::reset_for_scan(const int64_t tenant_id) +{ + int ret = OB_SUCCESS; + last_output_row_idx_ = OB_INVALID_INDEX; + child_iter_end_ = false; + reset_stores(); + if (sp_merged_row_ != nullptr) { + sp_merged_row_->reset(); + sp_merged_row_ = nullptr; + } + batch_ctx_.reset(); + last_computed_part_rows_ = 0; + if (rd_patch_ != nullptr) { + rd_patch_->reset(); + rd_patch_ = nullptr; + } + backuped_size_ = 0; + return ret; +} + +int ObWindowFunctionVecOp::create_stores(const int64_t tenant_id) +{ + int ret = OB_SUCCESS; + ObIAllocator *store_alloc = &mem_context_->get_malloc_allocator(); + input_stores_.processed_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, input_stores_); + input_stores_.cur_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, input_stores_); + input_stores_.set_operator(this); + if (OB_ISNULL(input_stores_.processed_) || OB_ISNULL(input_stores_.cur_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (MY_SPEC.range_dist_parallel_) { + input_stores_.first_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, input_stores_); + input_stores_.last_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, input_stores_); + if (OB_ISNULL(input_stores_.first_) || OB_ISNULL(input_stores_.last_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + ObIArray &all_exprs = get_all_expr(); + lib::ObMemAttr stored_mem_attr(tenant_id, ObModIds::OB_SQL_WINDOW_ROW_STORE, ObCtxIds::WORK_AREA); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(input_stores_.init(MY_SPEC.max_batch_size_, input_row_meta_, stored_mem_attr, + INT64_MAX, true))) { + LOG_WARN("init input stores failed", K(ret)); + } + FOREACH_WINCOL(END_WF) { + it->res_ = OB_NEWx(winfunc::RowStores, local_allocator_); + it->res_->set_operator(this); + if (OB_ISNULL(it->res_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + it->res_->processed_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, *it->res_); + it->res_->cur_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, *it->res_); + if (OB_ISNULL(it->res_->processed_) || OB_ISNULL(it->res_->cur_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (MY_SPEC.range_dist_parallel_) { + it->res_->first_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, *it->res_); + it->res_->last_ = OB_NEWx(winfunc::RowStore, local_allocator_, tenant_id, store_alloc, + local_allocator_, *it->res_); + if (OB_ISNULL(it->res_->first_) || OB_ISNULL(it->res_->last_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(it->res_->init(MY_SPEC.max_batch_size_, it->wf_res_row_meta_, + stored_mem_attr, INT64_MAX, true))) { + LOG_WARN("init row stores failed", K(ret)); + } + } + } + return ret; +} + +void ObWindowFunctionVecOp::reset_stores() +{ + input_stores_.reset(); + for (WinFuncColExpr *it = wf_list_.get_first(); it != END_WF; it = it->get_next()) { + if (it->res_ != nullptr) { it->res_->reset(); } + } +} + +void ObWindowFunctionVecOp::destroy_stores() +{ + input_stores_.destroy(); + for (WinFuncColExpr *it = wf_list_.get_first(); it != END_WF; it = it->get_next()) { + if (it->res_ != nullptr) { + it->res_->destroy(); + } + } +} + +int ObWindowFunctionVecOp::build_pby_hash_values_for_transmit() +{ + int ret = OB_SUCCESS; + for (int i = 0; OB_SUCC(ret) && i < pby_set_count_; i++) { + PbyHashValueArray *arr = OB_NEWx(PbyHashValueArray, local_allocator_); + if (OB_ISNULL(arr)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(pby_hash_values_.push_back(arr))) { + LOG_WARN("push back elements failed", K(ret)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::build_participator_whole_msg_array() +{ + int ret = OB_SUCCESS; + for (int i = 0; OB_SUCC(ret) && i < pby_set_count_; i++) { + ObReportingWFWholeMsg *msg = OB_NEWx(ObReportingWFWholeMsg, local_allocator_); + if (OB_ISNULL(msg)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(participator_whole_msg_array_.push_back(msg))) { + LOG_WARN("push back element failed", K(ret)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::setup_participator_pby_hash_sets(WFInfoFixedArray &wf_infos, + ObWindowFunctionVecOpInput *op_input) +{ + int ret = OB_SUCCESS; + int64_t prev_pushdown_pby_col_count = -1; + int64_t idx = -1; + if (OB_ISNULL(op_input)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid operator input", K(ret)); + } + for (int i = 0; OB_SUCC(ret) && i < wf_infos.count(); i++) { + WinFuncInfo &wf_info = wf_infos.at(i); + if (!wf_info.can_push_down_) { + if (OB_FAIL(pby_expr_cnt_idx_array_.push_back(OB_INVALID_ID))) { + LOG_WARN("push back element failed", K(ret)); + } + } else { + if (wf_info.partition_exprs_.count() == prev_pushdown_pby_col_count) { + if (OB_FAIL(pby_expr_cnt_idx_array_.push_back(idx))) { + LOG_WARN("push back element failed", K(ret)); + } + } else { + prev_pushdown_pby_col_count = wf_info.partition_exprs_.count(); + if (OB_FAIL(pby_expr_cnt_idx_array_.push_back(++idx))) { + LOG_WARN("push back element failed", K(ret)); + } else { + ReportingWFHashSet *hash_set = OB_NEWx(ReportingWFHashSet, local_allocator_); + if (OB_ISNULL(hash_set)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(hash_set->create(op_input->get_total_task_count() + * op_input->get_total_task_count()))) { // dop * dop + LOG_WARN("init hash sets failed", K(ret)); + } else if (OB_FAIL(pby_hash_values_sets_.push_back(hash_set))) { + LOG_WARN("push back element failed", K(ret)); + } + } + } + } + } + return ret; +} + +template +static int alloc_expr(ObIAllocator &allocator, WinExpr *&expr) +{ + int ret = OB_SUCCESS; + void *expr_buf = nullptr; + if (OB_ISNULL(expr_buf = allocator.alloc(sizeof(WinExpr)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + expr = new (expr_buf) WinExpr(); + } + return ret; +} +int ObWindowFunctionVecOp::init() +{ + int ret = OB_SUCCESS; + ObWindowFunctionVecOpInput *op_input = static_cast(input_); + if (OB_UNLIKELY(!wf_list_.is_empty())) { + ret = OB_INIT_TWICE; + LOG_WARN("init twice", K(ret)); + } else if (OB_ISNULL(ctx_.get_my_session())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null session ptr", K(ret)); + } else if (OB_FAIL(init_mem_context())) { + LOG_WARN("init memory context failed", K(ret)); + } else { + int64_t est_rows = MY_SPEC.rows_; + if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(&ctx_, MY_SPEC.px_est_size_factor_, est_rows, est_rows))) { + LOG_WARN("failed to get px size", K(ret)); + } else if (OB_FAIL(sql_mem_processor_.init( + &mem_context_->get_malloc_allocator(), + ctx_.get_my_session()->get_effective_tenant_id(), + (est_rows * MY_SPEC.width_ / MY_SPEC.estimated_part_cnt_), + MY_SPEC.type_, MY_SPEC.id_, &ctx_))) { + LOG_WARN("init sql mem processor failed", K(ret)); + } else { + LOG_TRACE("show some est values", K(ret), K(MY_SPEC.rows_), K(est_rows), K(MY_SPEC.width_), + K(MY_SPEC.estimated_part_cnt_), K(MY_SPEC.input_rows_mem_bound_ratio_)); + } + } + + if (OB_FAIL(ret)) { + } else { + const int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); + local_allocator_ = &(mem_context_->get_arena_allocator()); + rescan_alloc_.set_tenant_id(tenant_id); + rescan_alloc_.set_label("WfRescanAlloc"); + patch_alloc_.set_tenant_id(tenant_id); + patch_alloc_.set_label("WfPatchAlloc"); + ObMemAttr attr(tenant_id, "WfArray"); + participator_whole_msg_array_.set_attr(attr); + pby_hash_values_.set_attr(attr); + pby_hash_values_sets_.set_attr(attr); + pby_expr_cnt_idx_array_.set_attr(attr); + all_part_exprs_.set_attr(attr); + int prev_pushdown_pby_col_count = -1; + WFInfoFixedArray &wf_infos = const_cast(MY_SPEC.wf_infos_); + if (OB_FAIL(ObChunkStoreUtil::alloc_dir_id(dir_id_))) { + LOG_WARN("failed to alloc dir id", K(ret)); + } else if (MY_SPEC.max_batch_size_ > 0) { + if (OB_FAIL(all_expr_vector_copy_.init(child_->get_spec().output_, eval_ctx_))) { + LOG_WARN("init vector holder failed", K(ret)); + } else { + LOG_DEBUG("init expr vector holder", K(get_all_expr())); + } + } + // init wf row meta & create stores + if (OB_SUCC(ret)) { + // setup input_row_meta_ + input_row_meta_.set_allocator(local_allocator_); + if (OB_FAIL(input_row_meta_.init(get_all_expr(), 0))) { + LOG_WARN("init row meta failed", K(ret)); + } + } + // create aggr rows + for (int wf_idx = 1; OB_SUCC(ret) && wf_idx <= wf_infos.count(); wf_idx++) { + WinFuncInfo &wf_info = wf_infos.at(wf_idx - 1); + for (int j = 0; OB_SUCC(ret) && j < wf_info.partition_exprs_.count(); j++) { + if (OB_FAIL(add_var_to_array_no_dup(all_part_exprs_, wf_info.partition_exprs_.at(j)))) { + LOG_WARN("add element failed", K(ret)); + } + } + void *win_col_buf = nullptr, *pby_row_mapped_value_buf = nullptr; + WinFuncColExpr *win_col = nullptr; + int64_t agg_col_id = wf_idx - 1; + if (OB_ISNULL(win_col_buf = local_allocator_->alloc(sizeof(WinFuncColExpr)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + win_col = new (win_col_buf) WinFuncColExpr(wf_info, *this, wf_idx); + win_col->pby_row_mapped_idxes_ = reinterpret_cast(pby_row_mapped_value_buf); + switch (wf_info.func_type_) { + case T_FUN_SUM: + case T_FUN_MAX: + case T_FUN_MIN: + case T_FUN_COUNT: + case T_FUN_AVG: + case T_FUN_MEDIAN: + case T_FUN_GROUP_PERCENTILE_CONT: + case T_FUN_GROUP_PERCENTILE_DISC: + case T_FUN_STDDEV: + case T_FUN_STDDEV_SAMP: + case T_FUN_VARIANCE: + case T_FUN_STDDEV_POP: + case T_FUN_APPROX_COUNT_DISTINCT: + case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS: + case T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE: + case T_FUN_GROUP_CONCAT: + case T_FUN_CORR: + case T_FUN_COVAR_POP: + case T_FUN_COVAR_SAMP: + case T_FUN_VAR_POP: + case T_FUN_VAR_SAMP: + case T_FUN_REGR_SLOPE: + case T_FUN_REGR_INTERCEPT: + case T_FUN_REGR_COUNT: + case T_FUN_REGR_R2: + case T_FUN_REGR_AVGX: + case T_FUN_REGR_AVGY: + case T_FUN_REGR_SXX: + case T_FUN_REGR_SYY: + case T_FUN_REGR_SXY: + case T_FUN_SYS_BIT_AND: + case T_FUN_SYS_BIT_OR: + case T_FUN_SYS_BIT_XOR: + case T_FUN_KEEP_MAX: + case T_FUN_KEEP_MIN: + case T_FUN_KEEP_SUM: + case T_FUN_KEEP_COUNT: + case T_FUN_KEEP_WM_CONCAT: + case T_FUN_WM_CONCAT: + case T_FUN_TOP_FRE_HIST: + case T_FUN_PL_AGG_UDF: + case T_FUN_JSON_ARRAYAGG: + case T_FUN_JSON_OBJECTAGG: + case T_FUN_ORA_JSON_ARRAYAGG: + case T_FUN_ORA_JSON_OBJECTAGG: + case T_FUN_ORA_XMLAGG: { + aggregate::IAggregate *agg_func = nullptr; + winfunc::AggrExpr *aggr_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, aggr_expr))) { + LOG_WARN("allocate aggr expr failed", K(ret)); + } else { + win_col->wf_expr_ = aggr_expr; + } + break; + } + case T_WIN_FUN_RANK: { + using ranklike_expr = winfunc::RankLikeExpr; + ranklike_expr *rank_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, rank_expr))) { + LOG_WARN("allocate rank expr failed", K(ret)); + } else { + win_col->wf_expr_ = rank_expr; + } + break; + } + case T_WIN_FUN_DENSE_RANK: { + using ranklike_expr = winfunc::RankLikeExpr; + ranklike_expr *rank_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, rank_expr))) { + LOG_WARN("allocate rank expr failed", K(ret)); + } else { + win_col->wf_expr_ = rank_expr; + } + break; + } + case T_WIN_FUN_PERCENT_RANK: { + using ranklike_expr = winfunc::RankLikeExpr; + ranklike_expr *rank_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, rank_expr))) { + LOG_WARN("allocate rank expr failed", K(ret)); + } else { + win_col->wf_expr_ = rank_expr; + } + break; + } + case T_WIN_FUN_CUME_DIST: { + winfunc::CumeDist *cume_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, cume_expr))) { + LOG_WARN("allocate cume dist expr failed", K(ret)); + } else { + win_col->wf_expr_ = cume_expr; + } + break; + } + case T_WIN_FUN_ROW_NUMBER: { + winfunc::RowNumber *row_nmb = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, row_nmb))) { + LOG_WARN("allocate row number expr failed", K(ret)); + } else { + win_col->wf_expr_ = row_nmb; + } + break; + } + // first_value && last_value has been converted to nth_value when resolving + // case T_WIN_FUN_FIRST_VALUE: + // case T_WIN_FUN_LAST_VALUE: + case T_WIN_FUN_NTH_VALUE: { + winfunc::NthValue *nth_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, nth_expr))) { + LOG_WARN("allocate nth value expr failed", K(ret)); + } else { + win_col->wf_expr_ = nth_expr; + } + break; + } + case T_WIN_FUN_LAG: + case T_WIN_FUN_LEAD: { + winfunc::LeadOrLag *lead_lag_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, lead_lag_expr))) { + LOG_WARN("allocate lead_or_lag expr failed", K(ret)); + } else { + win_col->wf_expr_ = lead_lag_expr; + } + break; + } + case T_WIN_FUN_NTILE: { + winfunc::Ntile *ntile_expr = nullptr; + if (OB_FAIL(alloc_expr(*local_allocator_, ntile_expr))) { + LOG_WARN("allocate ntile expr failed", K(ret)); + } else { + win_col->wf_expr_ = ntile_expr; + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected window function type", K(ret), K(wf_info.func_type_)); + } + } + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_FAIL(win_col->init_res_rows(tenant_id))) { + LOG_WARN("init result compact rows failed", K(ret)); + } else if (win_col->wf_expr_->is_aggregate_expr() + && OB_FAIL(win_col->init_aggregate_ctx(tenant_id))) { + LOG_WARN("init aggr ctx and rows failed", K(ret)); + } else if (!win_col->wf_expr_->is_aggregate_expr() + && OB_FAIL(win_col->init_non_aggregate_ctx())) { + LOG_WARN("init non-aggr ctx failed", K(ret)); + } else { + if (OB_UNLIKELY(!wf_list_.add_last(win_col))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("add window function col failed", K(ret)); + } else { + LOG_DEBUG("added wf", K(*win_col)); + } + } + if (OB_SUCC(ret) && MY_SPEC.is_participator()) { + if (wf_info.can_push_down_) { + if (common::OB_INVALID_COUNT == next_wf_pby_expr_cnt_to_transmit_) { + // next_wf_pby_expr_cnt_to_transmit_ is for pushdown transmit to datahub + next_wf_pby_expr_cnt_to_transmit_ = wf_info.partition_exprs_.count(); + } + if (wf_info.partition_exprs_.count() != prev_pushdown_pby_col_count) { + pby_set_count_++; + prev_pushdown_pby_col_count = wf_info.partition_exprs_.count(); + } + } + } + } + } // end for + if (OB_SUCC(ret)) { + max_pby_col_cnt_ = all_part_exprs_.count(); + } + + if (OB_SUCC(ret) && MY_SPEC.is_participator()) { + if (OB_FAIL(build_pby_hash_values_for_transmit())) { + LOG_WARN("build transimitting hash values failed", K(ret)); + } else if (OB_FAIL(build_participator_whole_msg_array())) { + LOG_WARN("build participator whole msg array failed", K(ret)); + } else if (OB_FAIL(setup_participator_pby_hash_sets(wf_infos, op_input))) { + LOG_WARN("setup oby hash sets failed", K(ret)); + } + } + if (OB_SUCC(ret) && max_pby_col_cnt_ > 0) { + // init pby row mapped idx array + int32_t arr_buf_size = max_pby_col_cnt_ * sizeof(int32_t) * (MY_SPEC.max_batch_size_ + 1); + void *arr_buf = local_allocator_->alloc(arr_buf_size); + int32_t last_row_idx_arr_offset = max_pby_col_cnt_ * sizeof(int32_t) * MY_SPEC.max_batch_size_; + if (OB_ISNULL(arr_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMSET(arr_buf, -1, arr_buf_size); + pby_row_mapped_idx_arr_ = reinterpret_cast(arr_buf); + last_row_idx_arr_ = reinterpret_cast((char *)arr_buf + last_row_idx_arr_offset); + } + FOREACH_WINCOL(END_WF) { + it->pby_row_mapped_idxes_ = (int32_t *)local_allocator_->alloc(max_pby_col_cnt_ * sizeof(int32_t)); + if (OB_ISNULL(it->pby_row_mapped_idxes_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMSET(it->pby_row_mapped_idxes_, -1, sizeof(int32_t) * max_pby_col_cnt_); + } + // we may have wf info part exprs like: + // win_expr(T_WIN_FUN_RANK()), partition_by([testwn1.c], [testwn1.a], [testwn1.b]), + // win_expr(T_WIN_FUN_RANK()), partition_by([testwn1.b], [testwn1.a]) + // if so, we need a idx array to correctly compare partition exprs + bool same_part_order = true; + for (int i = 0; OB_SUCC(ret) && i < it->wf_info_.partition_exprs_.count() && same_part_order; i++) { + same_part_order = (it->wf_info_.partition_exprs_.at(i) == all_part_exprs_.at(i)); + } + if (OB_UNLIKELY(!same_part_order)) { + LOG_TRACE("orders of partition exprs are different", K(it->wf_info_), + K(it->wf_info_.partition_exprs_), K(all_part_exprs_)); + const ObExprPtrIArray &part_exprs = it->wf_info_.partition_exprs_; + + it->reordered_pby_row_idx_ = (int32_t *)local_allocator_->alloc(sizeof(int32_t) * part_exprs.count()); + if (OB_ISNULL(it->reordered_pby_row_idx_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + for (int i = 0; OB_SUCC(ret) && i < part_exprs.count(); i++) { + int32_t idx = -1; + for (int j = 0; idx == -1 && j < all_part_exprs_.count(); j++) { + if (all_part_exprs_.at(j) == part_exprs.at(i)) { + idx = j; + } + } + if (OB_UNLIKELY(idx == -1)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid part expr idx", K(ret)); + } else { + it->reordered_pby_row_idx_[i] = idx; + } + } + } + } else { + it->reordered_pby_row_idx_ = nullptr; + } + } + } + + if (OB_SUCC(ret)) {// init batch_ctx_ + if (OB_FAIL(init_batch_ctx())) { + LOG_WARN("init batch context failed", K(ret)); + } + } + // init sing partition parallel execution members + if (OB_SUCC(ret)) { + ObSEArray all_wf_exprs; + void *row_meta_buf = nullptr; + if (MY_SPEC.single_part_parallel_) { + FOREACH_WINCOL(END_WF) { + if (OB_FAIL(all_wf_exprs.push_back(it->wf_info_.expr_))) { + LOG_WARN("push back element failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_ISNULL(row_meta_buf = local_allocator_->alloc(sizeof(RowMeta)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + } else { + all_wf_res_row_meta_ = new(row_meta_buf)RowMeta(local_allocator_); + all_wf_res_row_meta_->set_allocator(local_allocator_); + if (OB_FAIL(all_wf_res_row_meta_->init(all_wf_exprs, 0, false))) { + LOG_WARN("init wf results row meta failed", K(ret)); + } + } + } + } else { + all_wf_res_row_meta_ = nullptr; + sp_merged_row_ = nullptr; + } + } + if (OB_SUCC(ret) && MY_SPEC.range_dist_parallel_) { + // init rd_coord_row_meta_; + rd_coord_row_meta_ = OB_NEWx(RowMeta, local_allocator_, local_allocator_); + if (OB_ISNULL(rd_coord_row_meta_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + // rd_coord_exprs + frame_offset + } else if (OB_FAIL(rd_coord_row_meta_->init(MY_SPEC.rd_coord_exprs_, sizeof(int64_t), false))) { + LOG_WARN("init rd_coord_row_meta failed", K(ret)); + } + } + // create stores + if (OB_FAIL(ret)) { + } else if (OB_FAIL(create_stores(tenant_id))) { + LOG_WARN("create stores failed", K(ret)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::init_batch_ctx() +{ + int ret = OB_SUCCESS; + int32_t bitmap_sz = ObBitVector::word_count(MY_SPEC.max_batch_size_) * ObBitVector::BYTES_PER_WORD; + int32_t tmp_buf_sz = MY_SPEC.max_batch_size_ * sizeof(ObCompactRow *) + bitmap_sz * 4 + + sizeof(int64_t) * MY_SPEC.max_batch_size_ * 2; + char *tmp_buf = nullptr; + int32_t offset = 0; + if (OB_ISNULL(tmp_buf = (char *)local_allocator_->alloc(tmp_buf_sz))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMSET(tmp_buf, 0, tmp_buf_sz); + batch_ctx_.stored_rows_ = reinterpret_cast(tmp_buf + offset); + offset += sizeof(const ObCompactRow *) * MY_SPEC.max_batch_size_; + batch_ctx_.nullres_skip_ = to_bit_vector(tmp_buf + offset); + batch_ctx_.pushdown_skip_ = to_bit_vector((char *)tmp_buf + offset + bitmap_sz); + batch_ctx_.calc_wf_skip_ = to_bit_vector((char *)tmp_buf + offset + bitmap_sz * 2); + batch_ctx_.bound_eval_skip_ = to_bit_vector((char *)tmp_buf + offset + bitmap_sz * 3); + offset += bitmap_sz * 4; + batch_ctx_.upper_pos_arr_ = reinterpret_cast(tmp_buf + offset); + offset += MY_SPEC.max_batch_size_ * sizeof(int64_t); + batch_ctx_.lower_pos_arr_ = reinterpret_cast(tmp_buf + offset); + offset += MY_SPEC.max_batch_size_ * sizeof(int64_t); + } + if (OB_SUCC(ret) && MY_SPEC.single_part_parallel_) { + batch_ctx_.tmp_wf_res_row_ = OB_NEWx(LastCompactRow, local_allocator_, *local_allocator_); + if (OB_ISNULL(batch_ctx_.tmp_wf_res_row_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + if (OB_SUCC(ret) && MY_SPEC.is_participator()) { + batch_ctx_.tmp_input_row_ = OB_NEWx(LastCompactRow, local_allocator_, *local_allocator_); + if (OB_ISNULL(batch_ctx_.tmp_input_row_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + if (OB_SUCC(ret) + && OB_FAIL(ObVectorsResultHolder::calc_backup_size(get_all_expr(), eval_ctx_, + batch_ctx_.all_exprs_backup_buf_len_))) { + LOG_WARN("calculate all exprs backup size failed", K(ret)); + } else if (batch_ctx_.all_exprs_backup_buf_len_ > 0 + && OB_ISNULL(batch_ctx_.all_exprs_backup_buf_ = + local_allocator_->alloc(batch_ctx_.all_exprs_backup_buf_len_))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + return ret; +} + +int ObWindowFunctionVecOp::get_next_batch_from_child(int64_t batch_size, + const ObBatchRows *&child_brs) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(restore_child_vectors())) { + LOG_WARN("restore all expr datas failed", K(ret)); + } else { + bool found = false; + while (!found && OB_SUCC(ret)) { + clear_evaluated_flag(); + if (OB_FAIL(child_->get_next_batch(batch_size, child_brs))) { + LOG_WARN("get child next batch failed", K(ret)); + } else if (child_brs->end_) { + found = true; + } else { + int64_t size = child_brs->size_; + found = (child_brs->skip_->accumulate_bit_cnt(size) != size); + } + } + LOG_TRACE("get next batch from child", K(found), K(child_brs->size_)); + // max rows of overwrote is MY_SPEC.max_batch_size_ + if (OB_SUCC(ret) && found && OB_FAIL(backup_child_vectors(MY_SPEC.max_batch_size_))) { + LOG_WARN("save expr data failed", K(ret)); + } + if (OB_SUCC(ret) && found && MY_SPEC.is_participator()) { + // for each iteration of child input, init agg_status for later calculating + ObExpr *agg_status = MY_SPEC.wf_aggr_status_expr_; + if (OB_FAIL(agg_status->init_vector_for_write(eval_ctx_, agg_status->get_default_res_format(), + MY_SPEC.max_batch_size_))) { + LOG_WARN("init vector failed", K(ret)); + } + } + // mapping pby row to idx array + const ObCompactRow *last_row = nullptr; + if (OB_FAIL(ret)) { + } else if (!found) { // do nothing + } else if (OB_FAIL(get_last_input_row_of_prev_batch(last_row))) { + LOG_WARN("get last row failed", K(ret)); + } else if (OB_SUCC(ret) && OB_FAIL(mapping_pby_row_to_idx_arr(*child_brs, last_row))) { + LOG_WARN("mapping pby row to idx array failed", K(ret)); + } else { + } + } + return ret; +} + +template +int ObWindowFunctionVecOp::mapping_pby_col_to_idx_arr(int32_t col_id, const ObExpr &part_expr, + const ObBatchRows &brs, + const cell_info *last_part_res) +{ + int ret = OB_SUCCESS; + int32_t val_idx = col_id, step = max_pby_col_cnt_; + const char *prev_data = nullptr, *cur_data = nullptr; + int32_t prev_len = 0, cur_len = 0; + int32_t prev = -1; + int cmp_ret = 0; + bool prev_is_null = false, cur_is_null = false; + ColumnFmt *column = static_cast(part_expr.get_vector(eval_ctx_)); + ObExprPtrIArray &all_exprs = get_all_expr(); + for (int i = 0; OB_SUCC(ret) && i < brs.size_; i++, val_idx +=step) { + if (brs.skip_->at(i)) { + continue; + } else if (OB_LIKELY(prev != -1)) { + column->get_payload(i, cur_is_null, cur_data, cur_len); + if (OB_FAIL(part_expr.basic_funcs_->row_null_first_cmp_( + part_expr.obj_meta_, part_expr.obj_meta_, + prev_data, prev_len, prev_is_null, + cur_data, cur_len, cur_is_null, + cmp_ret))) { + LOG_WARN("null first cmp failed", K(ret)); + } else if (cmp_ret == 0) { + pby_row_mapped_idx_arr_[val_idx] = prev; + } else { + int32_t new_idx = prev + 1; + pby_row_mapped_idx_arr_[val_idx] = new_idx; + prev = new_idx; + prev_data = cur_data; + prev_len = cur_len; + prev_is_null = cur_is_null; + } + } else if (last_part_res != nullptr) { + column->get_payload(i, cur_is_null, cur_data, cur_len); + prev_is_null = last_part_res->is_null_; + prev_len = last_part_res->len_; + prev_data = last_part_res->payload_; + if (OB_FAIL(part_expr.basic_funcs_->row_null_first_cmp_( + part_expr.obj_meta_, part_expr.obj_meta_, + prev_data, prev_len, prev_is_null, + cur_data, cur_len, cur_is_null, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret == 0) { + prev = last_row_idx_arr_[col_id]; + pby_row_mapped_idx_arr_[val_idx] = prev; + } else { + prev = last_row_idx_arr_[col_id] + 1; + pby_row_mapped_idx_arr_[val_idx] = prev; + prev_data = cur_data; + prev_len = cur_len; + prev_is_null = cur_is_null; + } + } else { + pby_row_mapped_idx_arr_[val_idx] = i; + column->get_payload(i, prev_is_null, prev_data, prev_len); + prev = i; + } + } + return ret; +} + +int ObWindowFunctionVecOp::eval_prev_part_exprs(const ObCompactRow *last_row, ObIAllocator &alloc, + const ObExprPtrIArray &part_exprs, + common::ObIArray &last_part_infos) +{ + int ret = OB_SUCCESS; + ObExprPtrIArray &all_exprs = get_all_expr(); + bool backuped_child_vector = false; + ObDataBuffer backup_alloc((char *)batch_ctx_.all_exprs_backup_buf_, batch_ctx_.all_exprs_backup_buf_len_); + ObVectorsResultHolder tmp_holder(&backup_alloc); + for (int i = 0; OB_SUCC(ret) && last_row != nullptr && i < part_exprs.count(); i++) { + ObExpr *part_expr = part_exprs.at(i); + int part_expr_field_idx = -1; + for (int j = 0; j < all_exprs.count() && part_expr_field_idx == -1; j++) { + if (part_expr == all_exprs.at(j)) { + part_expr_field_idx = j; + } + } + if (part_expr_field_idx != -1) { // partition expr is child input + const char *payload = nullptr; + bool is_null = false; + int32_t len = 0; + last_row->get_cell_payload(input_row_meta_, part_expr_field_idx, payload, len); + is_null = last_row->is_null(part_expr_field_idx); + if (OB_FAIL(last_part_infos.push_back(cell_info(is_null, len, payload)))) { + LOG_WARN("push back element failed", K(ret)); + } + } else { + if (backuped_child_vector) { + } else if (OB_FAIL(tmp_holder.init(all_exprs, eval_ctx_))) { + LOG_WARN("init result holder failed", K(ret)); + } else if (OB_FAIL(tmp_holder.save(1))) { + LOG_WARN("save vector results failed", K(ret)); + } else { + backuped_child_vector = true; + if (OB_FAIL(attach_row_to_output(last_row))) { + LOG_WARN("attach row failed", K(ret)); + } + } + int64_t mock_skip_data = 0; + ObBitVector *mock_skip = to_bit_vector(&mock_skip_data); + EvalBound tmp_bound(1, true); + char *part_res_buf = nullptr; + const char *payload = nullptr; + int32_t len = 0; + bool is_null = false; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(part_expr->eval_vector(eval_ctx_, *mock_skip, tmp_bound))) { + LOG_WARN("eval vector failed", K(ret)); + } else { + ObIVector *part_res_vec = part_expr->get_vector(eval_ctx_); + part_res_vec->get_payload(0, is_null, payload, len); + if (OB_ISNULL(part_res_buf = (char *)alloc.alloc(len))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(part_res_buf, payload, len); + if (OB_FAIL(last_part_infos.push_back(cell_info(is_null, len, part_res_buf)))) { + LOG_WARN("push back element failed", K(ret)); + } + } + } + } + } + if (OB_FAIL(ret)) { + } else if (backuped_child_vector && OB_FAIL(tmp_holder.restore())) { + LOG_WARN("restore vector results failed", K(ret)); + } + return ret; +} + +int ObWindowFunctionVecOp::get_last_input_row_of_prev_batch(const ObCompactRow *&last_row) +{ + int ret = OB_SUCCESS; + last_row = nullptr; + // search order: current->processed->first + winfunc::RowStore *fetch_store = nullptr; + if (!input_stores_.cur_->is_empty()) { + fetch_store = input_stores_.cur_; + } else if (!input_stores_.processed_->is_empty()) { + fetch_store = input_stores_.processed_; + } else if (MY_SPEC.range_dist_parallel_ && !input_stores_.first_->is_empty()) { + fetch_store = input_stores_.first_; + } + if (OB_ISNULL(fetch_store)) { + // do nothing + } else if (OB_FAIL(fetch_store->get_row(fetch_store->stored_row_cnt_ - 1, last_row))) { + LOG_WARN("get row failed", K(ret)); + } else { + } + return ret; +} + +// In vectorization 2.0, data accessing of expr is specified by `VectorFormat`, +// there's no easy way to access a complete partition row with multiple columns. +// In order to easily calculate partition, we map pby expr to idx array here. +// For each partition expr, pby_mapped_idx_arr[row_idx] = (get_payload(row_idx) == get_payload(row_idx - 1) ? +// pby_mapped_idx_arr[row_idx-1] +// : row_idx); +// For example, suppose partition exprs are `` and inputs are: +// 1, 2, 3 +// 1, 2, 3 +// 1, 2, 4 +// 1, 2, 4 +// 2, 3, 1 +// 2, 3, 1 +// mapped idx arrays are: +// 0, 0, 0 +// 0, 0, 0 +// 0, 0, 2 +// 0, 0, 2 +// 4, 4, 4 +// 4, 4, 4 +int ObWindowFunctionVecOp::mapping_pby_row_to_idx_arr(const ObBatchRows &child_brs, + const ObCompactRow *last_row) +{ +#define MAP_FIXED_COL_CASE(vec_tc) \ + case (vec_tc): { \ + ret = mapping_pby_col_to_idx_arr>>( \ + i, *part_exprs.at(i), child_brs, last_part_cell); \ + } break + + int ret = OB_SUCCESS; + ObIArray &part_exprs = all_part_exprs_; + ObArenaAllocator tmp_mem_alloc(ObModIds::OB_SQL_WINDOW_LOCAL, OB_MALLOC_NORMAL_BLOCK_SIZE, + ctx_.get_my_session()->get_effective_tenant_id(), + ObCtxIds::WORK_AREA); + ObSEArray part_cell_infos; + // calculate last part expr of previous batch first + // memset all idx to -1 + if (max_pby_col_cnt_ > 0) { + MEMSET(pby_row_mapped_idx_arr_, -1, child_brs.size_ * sizeof(int32_t) * max_pby_col_cnt_); + if (OB_FAIL(eval_prev_part_exprs(last_row, tmp_mem_alloc, part_exprs, part_cell_infos))) { + LOG_WARN("eval last partition exprs of last row failed", K(ret)); + } + bool prev_row_not_null = (last_row != nullptr); + for (int i = 0; OB_SUCC(ret) && i < part_exprs.count(); i++) { + const cell_info *last_part_cell = (prev_row_not_null ? &(part_cell_infos.at(i)) : nullptr); + if (OB_ISNULL(part_exprs.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr", K(ret)); + } else if (OB_FAIL(part_exprs.at(i)->eval_vector(eval_ctx_, child_brs))) { + LOG_WARN("eval vector failed", K(ret)); + } else { + VectorFormat fmt = part_exprs.at(i)->get_format(eval_ctx_); + VecValueTypeClass tc = part_exprs.at(i)->get_vec_value_tc(); + switch (fmt) { + case VEC_FIXED: { + switch (tc) { + LST_DO_CODE(MAP_FIXED_COL_CASE, FIXED_VEC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected vector type class", K(tc)); + } + } + break; + } + case VEC_UNIFORM: { + ret = mapping_pby_col_to_idx_arr>(i, *part_exprs.at(i), child_brs, + last_part_cell); + break; + } + case VEC_UNIFORM_CONST: { + ret = mapping_pby_col_to_idx_arr>(i, *part_exprs.at(i), child_brs, + last_part_cell); + break; + } + case VEC_DISCRETE: { + ret = mapping_pby_col_to_idx_arr(i, *part_exprs.at(i), child_brs, + last_part_cell); + break; + } + case VEC_CONTINUOUS: { + ret = mapping_pby_col_to_idx_arr(i, *part_exprs.at(i), child_brs, + last_part_cell); + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret), K(fmt), K(tc)); + } + } + if (OB_FAIL(ret)) { + LOG_WARN("mapping pby col to idx array failed", K(ret), K(i), K(*part_exprs.at(i))); + } + } + } + + // record last_row_idx_arr_ + for (int i = child_brs.size_ - 1; i >= 0; i--) { + if (child_brs.skip_->at(i)) { + } else { + MEMCPY(last_row_idx_arr_, &(pby_row_mapped_idx_arr_[i * max_pby_col_cnt_]), + max_pby_col_cnt_ * sizeof(int32_t)); + break; + } + } + } + return ret; +#undef MAP_FIXED_COL_CASE +} + +int ObWindowFunctionVecOp::partial_next_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + bool do_output = false; + while(OB_SUCC(ret) && !do_output) { + if (OB_FAIL(do_partial_next_batch(max_row_cnt, do_output))) { + LOG_WARN("do partial next batch failed", K(ret)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::do_partial_next_batch(const int64_t max_row_cnt, bool &do_output) +{ + int ret = OB_SUCCESS; + clear_evaluated_flag(); + int64_t check_times = 0; + int64_t output_row_cnt = MIN(max_row_cnt, MY_SPEC.max_batch_size_); + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + guard.set_batch_idx(0); + if (OB_UNLIKELY(iter_end_)) { + brs_.size_ = 0; + brs_.end_ = true; + } else if (OB_FAIL(ctx_.check_status())) { + LOG_WARN("check physical plan status failed", K(ret)); + } else if (OB_ISNULL(ctx_.get_my_session())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid null session", K(ret)); + } else { + const int64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id(); + if (input_stores_.processed_->to_output_rows() == 0 && input_stores_.processed_->count() > 0) { + // all processed rows are outputed, reset processed + input_stores_.processed_->reset(); + FOREACH_WINCOL(END_WF) { + it->res_->processed_->reset(); + } + } + // step.1 vec compute + WinFuncColExpr *first = wf_list_.get_first(); + WinFuncColExpr *end = wf_list_.get_header(); + int64_t rows_output_cnt = input_stores_.cur_->to_output_rows() + input_stores_.processed_->to_output_rows(); + while (OB_SUCC(ret) && rows_output_cnt < output_row_cnt) { + // reset to `row_cnt_` to `stored_row_cnt_` + // we may get extra rows of different big partition in one batch, those rows are added to `current` store but are not computed. + // Hence, in the beginning of next partition iteration, resetting is necessary for computing next partition values. + input_stores_.cur_->row_cnt_ = input_stores_.cur_->stored_row_cnt_; + if (OB_FAIL(get_next_partition(check_times))) { + LOG_WARN("get next partition failed", K(ret)); + } else { + rows_output_cnt = + input_stores_.processed_->to_output_rows() + input_stores_.cur_->to_output_rows(); + } + if (OB_SUCC(ret) && child_iter_end_) { + break; + } + } + // step.2 vec output + if (OB_SUCC(ret)) { + if (MY_SPEC.single_part_parallel_) { + brs_.end_ = true; + brs_.size_ = 0; + do_output = true; + } else if (MY_SPEC.range_dist_parallel_ + && child_iter_end_ + && input_stores_.cur_->to_compute_rows() == 0 + && !last_part_saved_) { + last_part_saved_ = true; + if (!first_part_saved_) { + // only one partition + first_part_saved_ = true; + SWAP_STORES(first, cur); + } else { + SWAP_STORES(last, cur); + } + // Rows stored in `processed_` neither in the first partition nor in the last partition + if (input_stores_.processed_->to_output_rows() > 0) { + if (OB_FAIL(output_batch_rows(input_stores_.processed_->to_output_rows()))) { + LOG_WARN("output batch rows failed", K(ret)); + } + } else { + brs_.size_ = 0; + brs_.end_ = true; + } + do_output = true; + } else if (OB_FAIL(output_batch_rows(output_row_cnt))) { + LOG_WARN("output batch rows failed", K(ret)); + } else { + do_output = true; + if (OB_SUCC(ret) && MY_SPEC.is_participator() && brs_.end_) { + // to make sure to send piece data to datahub even though no row fetched + // for some pushdown wf expr, it's partition count may not exceed DOP, hence no piece data + // is sent to datahub. When iteration is done, send empty piece data anyway. + if (OB_FAIL(rwf_send_empty_piece_data())) { + LOG_WARN("send empty piece data failed", K(ret)); + } + } + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::get_next_partition(int64_t &check_times) +{ + int ret = OB_SUCCESS; + int64_t batch_size = MY_SPEC.max_batch_size_; + bool found_next_part = false; + const ObBatchRows *child_brs = nullptr; + winfunc::RowStore ¤t = *input_stores_.cur_; + bool first_batch = current.count() == 0; + int64_t row_idx = -1; + WinFuncColExpr *first = wf_list_.get_first(); + WinFuncColExpr *end = wf_list_.get_header(); + ObExpr *agg_code_expr = MY_SPEC.wf_aggr_status_expr_; + if (child_iter_end_) { + if (!first_batch) { + // wf value haven't computed, why add row into input_stores_.cur_ first? + // aggr_res_row is same as last row of current partition except aggr_status_expr + // iteration in `compute_wf_values` will encounter aggr_res_row, and do wf computing. + // while aggr_res_row has same frame as before, computing will be replaced as copying results. + if (OB_FAIL(add_aggr_res_row_for_participator(end, current))) { + LOG_WARN("add aggr result row for last partition failed", K(ret)); + } else if (OB_FAIL(compute_wf_values(end, check_times))) { + LOG_WARN("compute wf values failed", K(ret)); + } + } + } else if (OB_FAIL(get_next_batch_from_child(batch_size, child_brs))) { + LOG_WARN("get next batch from child failed", K(ret)); + } else if ((child_brs->end_ && 0 == child_brs->size_) + || (child_brs->size_ == (row_idx = next_nonskip_row_index(row_idx, *child_brs)))) { + child_iter_end_ = true; + if (!first_batch) { + if (OB_FAIL(add_aggr_res_row_for_participator(end, current))) { + LOG_WARN("add aggr result row for last_partition failed", K(ret)); + } else if (OB_FAIL(compute_wf_values(end, check_times))) { + LOG_WARN("compute wf values failed", K(ret)); + } + } + } else { + if (child_brs->end_) { + child_iter_end_ = true; + } + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + int64_t part_start_idx = row_idx; + if (first_batch && child_brs->size_ > 0) { + // new partition + // 1. save pby row + // 2. update first partition row idx + // 3. detect and send aggr status if participator + if (OB_FAIL(save_pby_row_for_wf(end, row_idx))) { + LOG_WARN("save pby row failed", K(ret)); + } else if (OB_FAIL(update_part_first_row_idx(end))) { + LOG_WARN("update first row idx of partition failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(process_child_batch(row_idx, child_brs, check_times))) { + LOG_WARN("process child batch rows failed", K(ret)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::process_child_batch(const int64_t batch_idx, + const ObBatchRows *child_brs, int64_t &check_times) +{ + int ret = OB_SUCCESS; + int64_t batch_size = MY_SPEC.max_batch_size_; + int64_t row_idx = batch_idx; + WinFuncColExpr *first = wf_list_.get_first(); + WinFuncColExpr *end = wf_list_.get_header(); + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + guard.set_batch_size(child_brs->size_); + guard.set_batch_idx(row_idx); + // found next part means finding a big partition + // wf part exprs array are organized as , , + // if `` changed, found_next_part = true + bool found_next_part = false; + if (OB_FAIL(check_stack_overflow())) { + LOG_WARN("stack overflow", K(ret)); + } + while (OB_SUCC(ret) && !found_next_part) { // find a big partition + bool need_loop_util_child_brs_end = true; + while (OB_SUCC(ret) && need_loop_util_child_brs_end) { + need_loop_util_child_brs_end = false; + found_next_part = false; + winfunc::RowStore ¤t = *input_stores_.cur_; + winfunc::RowStore &processed = *input_stores_.processed_; + // winfunc::RowStore &remain = processed.is_empty() ? processed : current; + bool need_swap_store = processed.is_empty(); + int64_t part_start_idx = row_idx; + while (OB_SUCC(ret) && row_idx < child_brs->size_ && !found_next_part) { + bool same_part = true; + guard.set_batch_idx(row_idx); + if (child_brs->skip_->at(row_idx)) { + row_idx++; + continue; + } + if (OB_FAIL(check_same_partition(*first, same_part))) { + LOG_WARN("check same partition failed", K(ret)); + } else if (OB_UNLIKELY(!same_part)) { + // find same partition of any other window function + if (OB_FAIL(find_same_partition_of_wf(end))) { + LOG_WARN("find same partition of wf failed", K(ret)); + } else { + // new big partition or not + found_next_part = (end == wf_list_.get_header()); + // 1. save pby row + // 2. set values for aggr status expr if possible + // 3. detect and report aggr status + // 4. add batch rows into current store + // 5. add aggr res row into current store + // update part first row idx after computing wf values + if (OB_FAIL(save_pby_row_for_wf(end, eval_ctx_.get_batch_idx()))) { + LOG_WARN("save partition groupby row failed", K(ret)); + } else if (OB_FAIL(detect_and_report_aggr_status(*child_brs, part_start_idx, row_idx))) { + LOG_WARN("detect and report aggr status failed", K(ret)); + } else if (OB_FAIL(current.add_batch_rows( + get_all_expr(), input_row_meta_, eval_ctx_, + EvalBound(child_brs->size_, part_start_idx, row_idx, false), + *child_brs->skip_, true, nullptr, true))) { + LOG_WARN("add batch rows failed", K(ret)); + } else if (OB_FAIL(add_aggr_res_row_for_participator(end, *input_stores_.cur_))) { + // new partition found, add aggr result row + LOG_WARN("add aggregate result row for participator failed", K(ret)); + } else { + LOG_TRACE("found new partition", K(found_next_part), K(part_start_idx), K(row_idx), + K(child_brs), K(need_swap_store)); + } + } + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_FAIL(compute_wf_values(end, check_times))) { + LOG_WARN("compute wf values failed", K(ret)); + } else { + part_start_idx = row_idx; + } + } else { + row_idx++; + } + } + if (OB_SUCC(ret)) { + if (found_next_part) { + if (need_swap_store) { + // switch %cur_ and %processed_ row store if necessary + // %cur_ always be rows store we are computing. + // And rows in %processed_ row store are all computed and ready to output. + // swap $cur_ and $processed_ row store + + SWAP_STORES(cur, processed); + if (MY_SPEC.range_dist_parallel_ && !first_part_saved_) { + // save first partition by swapping + first_part_saved_ = true; + SWAP_STORES(first, processed); + } + } + // store maybe swapped, update partitions' first row idxes. + if (need_swap_store && OB_FAIL(update_part_first_row_idx(END_WF))) { + LOG_WARN("save partition first row idx failed", K(ret)); + } else { + need_loop_util_child_brs_end = true; + } + } else if (!child_iter_end_) { + // if part_start_id < row_idx, add rest of rows into current store + // note that, if need_swap_store == true, it means we haven't got a complete big partition yet + // and iteration needs going on. + // If need_swap_store == false, rest of rows are not included in big partition + // and should not count as computed rows (haven't compute wf values yet). We just add those rows + // and set `current.row_cnt_` to `wf_list_.get_last()->part_first_row_idx_`. + // `current.row_cnt_` will be adjusted back to `current.store_row_cnt_` in the beginning `do_partial_next_batch` + if (part_start_idx < row_idx) { + if (OB_FAIL(detect_and_report_aggr_status(*child_brs, part_start_idx, row_idx))) { + LOG_WARN("detect and report aggr status failed", K(ret)); + } else if (OB_FAIL(current.add_batch_rows( + get_all_expr(), input_row_meta_, eval_ctx_, + EvalBound(child_brs->size_, part_start_idx, row_idx, + child_brs->all_rows_active_), + *child_brs->skip_, need_swap_store, nullptr, true))) { + LOG_WARN("add batch rows failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (need_swap_store) { // this means we haven't got a complete big partition + if (OB_FAIL(get_next_batch_from_child(batch_size, child_brs))) { + LOG_WARN("get next batch from child failed", K(ret)); + } else { + child_iter_end_ = child_brs->end_ + || (child_brs->size_ == child_brs->skip_->accumulate_bit_cnt(child_brs->size_)); + if (child_iter_end_) { + // all child rows are iterated + // add aggr result row and compute wf values + if (OB_FAIL( + add_aggr_res_row_for_participator(END_WF, *input_stores_.cur_))) { + LOG_WARN("add aggregate result row failed", K(ret)); + } else if (OB_FAIL(compute_wf_values(END_WF, check_times))) { + LOG_WARN("compute wf values failed", K(ret)); + } else { + found_next_part = true; + } + } else { // continue to read rows from child batch + row_idx = 0; + guard.set_batch_size(child_brs->size_); + } + } + } else { + found_next_part = true; + current.row_cnt_ = wf_list_.get_last()->part_first_row_idx_; + } + } else { + // all child rows are iterated + found_next_part = true; + if (part_start_idx < row_idx) { + if (OB_FAIL(detect_and_report_aggr_status(*child_brs, part_start_idx, row_idx))) { + LOG_WARN("detect and report aggr status failed", K(ret)); + } else if (OB_FAIL(current.add_batch_rows( + get_all_expr(), input_row_meta_, eval_ctx_, + EvalBound(child_brs->size_, part_start_idx, row_idx, false), + *child_brs->skip_, true, nullptr, true))) { + LOG_WARN("add batch rows failed", K(ret)); + } else if (OB_FAIL(add_aggr_res_row_for_participator(wf_list_.get_header(), + *input_stores_.cur_))) { + LOG_WARN("add aggregate result row failed", K(ret)); + } else if (OB_FAIL(compute_wf_values(END_WF, check_times))) { + LOG_WARN("compute wf values failed", K(ret)); + } + } + } + } + } + } + return ret; +} + +int64_t ObWindowFunctionVecOp::next_nonskip_row_index(int64_t cur_idx, const ObBatchRows &brs) +{ + int64_t res = cur_idx + 1; + for (; res < brs.size_ && brs.skip_->at(res); res++) { + } + return res; +} + +int ObWindowFunctionVecOp::save_pby_row_for_wf(WinFuncColExpr *end_wf, const int64_t batch_idx) +{ + int ret = OB_SUCCESS; + if (max_pby_col_cnt_ > 0) { + int32_t offset = batch_idx * max_pby_col_cnt_; + int32_t *pby_row_idxes = &(pby_row_mapped_idx_arr_[offset]); + FOREACH_WINCOL(end_wf) + { + MEMCPY(it->pby_row_mapped_idxes_, pby_row_idxes, sizeof(int32_t) * max_pby_col_cnt_); + } + } + return ret; +} + +int ObWindowFunctionVecOp::update_part_first_row_idx(WinFuncColExpr *end) +{ + int ret = OB_SUCCESS; + FOREACH_WINCOL(end) + { + it->part_first_row_idx_ = it->res_->cur_->count(); + } + return ret; +} + +int ObWindowFunctionVecOp::check_same_partition(WinFuncColExpr &wf_col, bool &same) +{ + int ret = OB_SUCCESS; + same = (wf_col.wf_info_.partition_exprs_.count() <= 0); + if (!same) { + int64_t part_cnt = wf_col.wf_info_.partition_exprs_.count(); + int64_t row_idx = eval_ctx_.get_batch_idx(); + int32_t offset = max_pby_col_cnt_ * row_idx; + int32_t *pby_row_idxes = &(pby_row_mapped_idx_arr_[offset]); + if (OB_UNLIKELY(wf_col.reordered_pby_row_idx_ == nullptr)) { + same = (MEMCMP(pby_row_idxes, wf_col.pby_row_mapped_idxes_, sizeof(int32_t) * part_cnt) == 0); + } else { + same = true; + for (int i = 0; i < part_cnt && same; i++) { + int32_t idx = wf_col.reordered_pby_row_idx_[i]; + same = (wf_col.pby_row_mapped_idxes_[idx] == pby_row_idxes[idx]); + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::find_same_partition_of_wf(WinFuncColExpr *&end_wf) +{ + int ret = OB_SUCCESS; + end_wf = wf_list_.get_header(); + bool same = false; + FOREACH_WINCOL(END_WF) { + if (OB_FAIL(check_same_partition(*it, same))) { + LOG_WARN("check same partition failed", K(ret)); + } else if (same) { + end_wf = it; + break; + } + } + return ret; +} + +int ObWindowFunctionVecOp::coordinate() +{ + int ret = OB_SUCCESS; + if (MY_SPEC.single_part_parallel_) { + if (OB_FAIL(collect_sp_partial_results())) { + LOG_WARN("collect single partition partial results failed", K(ret)); + } + } else if (MY_SPEC.range_dist_parallel_) { + if (OB_FAIL(rd_fetch_patch())) { + LOG_WARN("fetch patch info from PX COORD failed", K(ret)); + } else { + LOG_DEBUG("fetch patch", K(*rd_patch_)); + last_output_row_idx_ = OB_INVALID_INDEX; + SWAP_STORES(first, cur); + patch_first_ = true; + if (input_stores_.last_->count() <= 0) { + // only one partition + patch_last_ = true; + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::final_next_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + if (MY_SPEC.range_dist_parallel_) { + if (OB_FAIL(rd_output_final_batch(std::min(max_row_cnt, MY_SPEC.max_batch_size_)))) { + LOG_WARN("output first part failed", K(ret)); + } else { + if (brs_.end_ && brs_.size_ == 0 && !first_part_outputed_) { + first_part_outputed_ = true; + // now output last part if possible + if (input_stores_.last_->to_output_rows() > 0) { + brs_.end_ = false; + iter_end_ = false; + SWAP_STORES(first, cur); + SWAP_STORES(last, cur); + patch_first_ = false; + patch_last_ = true; + ret = rd_output_final_batch(std::min(max_row_cnt, MY_SPEC.max_batch_size_)); + } + } + } + } else { + // ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + // guard.set_batch_idx(0); + if (OB_UNLIKELY(iter_end_)) { + brs_.size_ = 0; + brs_.end_ = true; + } else if (OB_FAIL(output_batch_rows(std::min(max_row_cnt, MY_SPEC.max_batch_size_)))) { + LOG_WARN("output batch rows failed", K(ret)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::output_batch_rows(const int64_t output_row_cnt) +{ + int ret = OB_SUCCESS; + winfunc::RowStore &processed = *input_stores_.processed_; + winfunc::RowStore ¤t = *input_stores_.cur_; + if (OB_UNLIKELY(processed.row_cnt_ != processed.stored_row_cnt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rows in processed row store should be all computed", K(ret), K(processed)); + } else { + FOREACH_WINCOL(END_WF) + { + if (OB_UNLIKELY(it->res_->processed_->row_cnt_ != processed.row_cnt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rows number in input stores and result stores should be same", K(ret), + K(*it->res_->processed_), K(processed)); + } else if (OB_UNLIKELY(it->res_->processed_->row_cnt_ + != it->res_->processed_->stored_row_cnt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rows in processed row store should be all computed", K(ret), + K(*it->res_->processed_)); + } + // only last wf contains exact row_cnt_ for output. + if (it == wf_list_.get_last() && OB_UNLIKELY(it->res_->cur_->row_cnt_ != current.row_cnt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rows number in input stores and result stores should be same", K(ret), + K(*it->res_->cur_), K(current), K(it->wf_idx_)); + } + } + if (OB_SUCC(ret)) { + int64_t rows_cnt_processed = std::min(processed.to_output_rows(), output_row_cnt); + int64_t rows_cnt_current = std::min(current.to_output_rows(), output_row_cnt - rows_cnt_processed); + int64_t word_cnt = ObBitVector::word_count(rows_cnt_processed + rows_cnt_current); + + if (backuped_size_ < rows_cnt_processed + rows_cnt_current) { + if (OB_FAIL(restore_child_vectors())) { + LOG_WARN("restore child vector results failed", K(ret)); + } else if (OB_FAIL(backup_child_vectors(rows_cnt_processed + rows_cnt_current))) { + LOG_WARN("backup child vector results failed", K(ret)); + } + } + MEMSET(brs_.skip_, 0, word_cnt * ObBitVector::BYTES_PER_WORD); + brs_.all_rows_active_ = true; + + int64_t outputed_cnt = 0, outputed_wf_res_cnt = 0; + // `output_stored_rows` gets batch rows from row store and write row into exprs + // if rows of current batch are distributed in different blocks, all blocks are loaded must be valid + winfunc::StoreGuard store_guard(*this); + if (OB_FAIL(output_stored_rows(rows_cnt_processed, rows_cnt_current, input_stores_, outputed_cnt))) { + LOG_WARN("output processed store rows failed", K(ret)); + } else { + FOREACH_WINCOL(END_WF) { + if (OB_FAIL(output_stored_rows(rows_cnt_processed, rows_cnt_current, *it, outputed_wf_res_cnt))) { + LOG_WARN("output stored rows failed", K(ret)); + } + } + } + if (OB_FAIL(ret)) { + } else if (OB_UNLIKELY(outputed_cnt != outputed_wf_res_cnt)){ + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected output rows", K(outputed_cnt), K(outputed_wf_res_cnt)); + } else if (MY_SPEC.is_consolidator()) { + // partial aggregate row with aggr_status < 0 can't be outputed + VectorFormat agg_status_fmt = MY_SPEC.wf_aggr_status_expr_->get_format(eval_ctx_); + if (agg_status_fmt == VEC_FIXED) { + ObFixedLengthFormat *aggr_status_data = + static_cast *>(MY_SPEC.wf_aggr_status_expr_->get_vector(eval_ctx_)); + for (int i = 0; i < outputed_cnt; i++) { + if (*reinterpret_cast(aggr_status_data->get_payload(i)) < 0) { + brs_.skip_->set(i); + brs_.all_rows_active_ = false; + } + } + } else if (agg_status_fmt == VEC_UNIFORM) { + ObUniformFormat *aggr_status_data = + static_cast *>(MY_SPEC.wf_aggr_status_expr_->get_vector(eval_ctx_)); + for (int i = 0; i < outputed_cnt; i++) { + if (*reinterpret_cast(aggr_status_data->get_payload(i)) < 0) { + brs_.skip_->set(i); + brs_.all_rows_active_ = false; + } + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected agg status format", K(ret), K(agg_status_fmt)); + } + } + if (OB_FAIL(ret)) { + } else { + if (outputed_cnt > 0) { + FOREACH_WINCOL(END_WF) + { + it->wf_info_.expr_->get_eval_info(eval_ctx_).cnt_ = eval_ctx_.get_batch_idx(); + it->wf_info_.expr_->set_evaluated_projected(eval_ctx_); + } + ObIArray &all_exprs = get_all_expr(); + for (int i = 0; i < all_exprs.count(); i++) { // aggr status is the last expr + all_exprs.at(i)->get_eval_info(eval_ctx_).cnt_ = eval_ctx_.get_batch_idx(); + all_exprs.at(i)->set_evaluated_projected(eval_ctx_); + } + } + brs_.size_ = outputed_cnt; + if (MY_SPEC.is_consolidator()) { + WinFuncColExpr *last_wf = wf_list_.get_last(); + winfunc::RowStore ¤t_res = *(last_wf->res_->cur_); + brs_.end_ = (current_res.to_output_rows() == 0 && current.to_compute_rows() == 0); + } else { + brs_.end_ = (current.to_output_rows() == 0 && current.to_compute_rows() == 0); + } + iter_end_ = brs_.end_; + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::output_stored_rows(const int64_t out_processed_cnt, + const int64_t out_cur_cnt, winfunc::RowStores &store, + int64_t &output_cnt) +{ + int ret = OB_SUCCESS; + OB_ASSERT(out_processed_cnt + out_cur_cnt <= MY_SPEC.max_batch_size_); + const ObCompactRow **input_stored_rows = batch_ctx_.stored_rows_; + MEMSET(input_stored_rows, 0, sizeof(ObCompactRow *) * (out_processed_cnt + out_cur_cnt)); + int64_t out_start = store.processed_->output_row_idx_; + // increase row store memory iteration age to keep row store memory + // get processed_ first + if (OB_FAIL(store.processed_->get_batch_rows( + store.processed_->output_row_idx_, store.processed_->output_row_idx_ + out_processed_cnt, + input_stored_rows))) { + LOG_WARN("get batch rows failed", K(ret)); + } else if (OB_FAIL(store.cur_->get_batch_rows(store.cur_->output_row_idx_, + store.cur_->output_row_idx_ + out_cur_cnt, + &input_stored_rows[out_processed_cnt]))) { + LOG_WARN("get batch rows failed", K(ret)); + } else { + int64_t out_batch = out_processed_cnt + out_cur_cnt; + if (OB_FAIL(attach_rows_to_output(input_stored_rows, out_batch))) { + LOG_WARN("attach rows failed", K(ret)); + } else { + store.processed_->output_row_idx_ += out_processed_cnt; + store.cur_->output_row_idx_ += out_cur_cnt; + output_cnt += out_processed_cnt + out_cur_cnt; + } + } + return ret; +} + +int ObWindowFunctionVecOp::attach_row_to_output(const ObCompactRow *row) +{ + int ret = OB_SUCCESS; + const ObCompactRow **input_stored_rows = batch_ctx_.stored_rows_; + input_stored_rows[0] = row; + return attach_rows_to_output(input_stored_rows, 1); +} + +int ObWindowFunctionVecOp::attach_rows_to_output(const ObCompactRow **rows, int64_t row_cnt) +{ + int ret = OB_SUCCESS; + ObExprPtrIArray &all_exprs = get_all_expr(); + for (int i = 0; OB_SUCC(ret) && i < all_exprs.count(); i++) { // do not project const expr!!! + if (all_exprs.at(i)->is_const_expr()) {// do nothing + } else if (OB_FAIL(all_exprs.at(i)->init_vector_for_write( + eval_ctx_, all_exprs.at(i)->get_default_res_format(), row_cnt))) { + LOG_WARN("init vector failed", K(ret)); + } else if (OB_FAIL(all_exprs.at(i)->get_vector(eval_ctx_)->from_rows(input_row_meta_, rows, row_cnt, i))) { + LOG_WARN("from rows failed", K(ret)); + } else { + all_exprs.at(i)->set_evaluated_projected(eval_ctx_); + } + } + return ret; +} + +#define SET_SP_RES(fmt) \ + for (int i = 0; i < out_batch; i++) { \ + if (is_null) { \ + i_data->set_null(i); \ + } else { \ + static_cast(i_data)->set_payload_shallow(i, res, res_len); \ + } \ + } + +#define SET_SP_FIXED_RES_CASE(vec_tc) \ + case (vec_tc): { \ + SET_SP_RES(ObFixedLengthFormat>); \ + } break + +int ObWindowFunctionVecOp::output_stored_rows(const int64_t out_processed_cnt, + const int64_t out_cur_cnt, WinFuncColExpr &wf_col, + int64_t &outputed_cnt) +{ + int ret = OB_SUCCESS; + OB_ASSERT(out_processed_cnt + out_cur_cnt <= MY_SPEC.max_batch_size_); + int64_t out_batch = out_processed_cnt + out_cur_cnt; + if (OB_FAIL(wf_col.wf_info_.expr_->init_vector_for_write( + eval_ctx_, wf_col.wf_info_.expr_->get_default_res_format(), out_batch))) { + LOG_WARN("init vector for write failed", K(ret)); + } else if (MY_SPEC.single_part_parallel_) { + int64_t col_idx = wf_col.wf_idx_ - 1; + if (OB_ISNULL(sp_merged_row_)) { + // empty merged row, no results for current worker + outputed_cnt = 0; + } else if (OB_ISNULL(all_wf_res_row_meta_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null merged row", K(ret), K(all_wf_res_row_meta_)); + } else { + const char *res = nullptr; + int32_t res_len = 0; + ObCompactRow *merged_row = sp_merged_row_->compact_row_; + bool is_null = merged_row->is_null(col_idx); + merged_row->get_cell_payload(*all_wf_res_row_meta_, col_idx, res, res_len); + VecValueTypeClass vec_tc = wf_col.wf_info_.expr_->get_vec_value_tc(); + VectorFormat fmt = wf_col.wf_info_.expr_->get_format(eval_ctx_); + ObIVector *i_data = wf_col.wf_info_.expr_->get_vector(eval_ctx_); + switch (fmt) { + case common::VEC_UNIFORM: { + SET_SP_RES(ObUniformFormat); + break; + } + case common::VEC_DISCRETE: { + SET_SP_RES(ObDiscreteFormat); + break; + } + case common::VEC_CONTINUOUS: { + SET_SP_RES(ObContinuousFormat); + break; + } + case common::VEC_FIXED: { + switch (vec_tc) { + LST_DO_CODE(SET_SP_FIXED_RES_CASE, + VEC_TC_INTEGER, + VEC_TC_UINTEGER, + VEC_TC_FLOAT, + VEC_TC_DOUBLE, + VEC_TC_FIXED_DOUBLE, + VEC_TC_DATETIME, + VEC_TC_DATE, + VEC_TC_TIME, + VEC_TC_YEAR, + VEC_TC_BIT, + VEC_TC_ENUM_SET, + VEC_TC_TIMESTAMP_TZ, + VEC_TC_TIMESTAMP_TINY, + VEC_TC_INTERVAL_YM, + VEC_TC_INTERVAL_DS, + VEC_TC_DEC_INT32, + VEC_TC_DEC_INT64, + VEC_TC_DEC_INT128, + VEC_TC_DEC_INT256, + VEC_TC_DEC_INT512); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected vector tc", K(ret), K(vec_tc)); + } + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret), K(fmt)); + } + } + if (OB_SUCC(ret)) { outputed_cnt = out_batch; } + } + } else { + MEMSET(wf_col.res_rows_, 0, sizeof(ObCompactRow *) * out_batch); + if (OB_FAIL(wf_col.res_->processed_->get_batch_rows( + wf_col.res_->processed_->output_row_idx_, + wf_col.res_->processed_->output_row_idx_ + out_processed_cnt, wf_col.res_rows_))) { + LOG_WARN("get batch rows failed", K(ret)); + } else if (OB_FAIL( + wf_col.res_->cur_->get_batch_rows(wf_col.res_->cur_->output_row_idx_, + wf_col.res_->cur_->output_row_idx_ + out_cur_cnt, + &wf_col.res_rows_[out_processed_cnt]))) { + LOG_WARN("get batch rows failed", K(ret)); + } else if (OB_FAIL(wf_col.wf_info_.expr_->get_vector(eval_ctx_)->from_rows( + wf_col.wf_res_row_meta_, wf_col.res_rows_, out_batch, 0))) { + LOG_WARN("from rows failed", K(ret)); + } else { + wf_col.res_->processed_->output_row_idx_ += out_processed_cnt; + wf_col.res_->cur_->output_row_idx_ += out_cur_cnt; + outputed_cnt = out_processed_cnt + out_cur_cnt; + } + } + return ret; +} +#undef SET_SP_RES +#undef SET_SP_FIXED_RES_CASE + +int ObWindowFunctionVecOp::compute_wf_values(WinFuncColExpr *end, int64_t &check_times) +{ + int ret = OB_SUCCESS; + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + char *tmp_buf = nullptr; + // Partition size maybe exceed MY_SPEC.batch_size_, if so, calculation is divided into several + // batches of size MY_SPEC.max_batch_size. + ObBitVector *nullres_skip = batch_ctx_.nullres_skip_, + *pushdown_skip = batch_ctx_.pushdown_skip_, + *wf_skip = batch_ctx_.calc_wf_skip_; + ObDataBuffer backup_alloc((char *)batch_ctx_.all_exprs_backup_buf_, batch_ctx_.all_exprs_backup_buf_len_); + ObVectorsResultHolder tmp_holder(&backup_alloc); + int64_t saved_batch_size = 0; + FOREACH_WINCOL(end) { + saved_batch_size = std::max(input_stores_.cur_->count() - it->part_first_row_idx_, saved_batch_size); + } + saved_batch_size = std::min(saved_batch_size, MY_SPEC.max_batch_size_); + if (OB_FAIL(tmp_holder.init(get_all_expr(), eval_ctx_))) { + LOG_WARN("init tmp result holder failed", K(ret)); + } else if (OB_FAIL(tmp_holder.save(saved_batch_size))) { + LOG_WARN("save vector resule failed", K(ret)); + } + FOREACH_WINCOL(end) { + if (it == wf_list_.get_last()) { + const int v = input_stores_.cur_->count() - it->part_first_row_idx_; + if (v > 0) { + last_computed_part_rows_ = v; + } + } + int64_t total_size = input_stores_.cur_->count() - it->part_first_row_idx_; + winfunc::WinExprEvalCtx win_expr_ctx(*input_stores_.cur_, *it, + ctx_.get_my_session()->get_effective_tenant_id()); + int64_t start_idx = it->part_first_row_idx_; + if (it->wf_expr_->is_aggregate_expr()) { + // if aggregate expr, reset last_valid_frame & last_valid_row before process partition + winfunc::AggrExpr *agg_expr = static_cast(it->wf_expr_); + agg_expr->last_valid_frame_.reset(); + agg_expr->last_aggr_row_ = nullptr; + } + while (OB_SUCC(ret) && total_size > 0) { + clear_evaluated_flag(); + if (0 == ++check_times % CHECK_STATUS_INTERVAL) { // check per-batch + if (OB_FAIL(ctx_.check_status())) { break; } + } + int64_t batch_size = std::min(total_size, MY_SPEC.max_batch_size_); + if OB_SUCC(ret) { + guard.set_batch_size(batch_size); + guard.set_batch_idx(0); + wf_skip->unset_all(0, batch_size); + // add store guard to make sure read blocks are valid + winfunc::StoreGuard store_guard(*this); + if (OB_FAIL(it->wf_info_.expr_->init_vector_for_write( + eval_ctx_, it->wf_info_.expr_->get_default_res_format(), batch_size))) { + LOG_WARN("init vector for write failed", K(ret)); + } else if (OB_FAIL(input_stores_.cur_->attach_rows(get_all_expr(), input_row_meta_, + eval_ctx_, start_idx, + start_idx + batch_size, false))) { + // step.1: attach rows + LOG_WARN("attach rows failed", K(ret), K(start_idx), K(batch_size), + K(*input_stores_.cur_)); + } else if (OB_FAIL(it->reset_for_partition(batch_size, *wf_skip))) { + LOG_WARN("reset for partition failed", K(ret)); + } else if (it->wf_info_.can_push_down_ && MY_SPEC.is_push_down()) { + if (OB_FAIL( + detect_nullres_or_pushdown_rows(*it, *nullres_skip, *pushdown_skip, *wf_skip))) { + // step.2 find nullres rows and bypass-pushdown rows + LOG_WARN("find null result rows or bypass-pushdown rows failed", K(ret)); + } else if (OB_FAIL(calc_bypass_pushdown_rows_of_wf(*it, batch_size, *pushdown_skip))) { + // by pass collect will reset null bitmap by calling `init_vector_for_write` + // must be called before `set_null_results_of_wf` + // step.4 calculate bypass-pushdown rows + LOG_WARN("calculate pushdown rows failed", K(ret)); + } else if (OB_FAIL(set_null_results_of_wf(*it, batch_size, *nullres_skip))) { + // step.3 set null results + LOG_WARN("set null results failed", K(ret)); + } else { + } + } + } + if (OB_SUCC(ret) && it->wf_expr_->is_aggregate_expr()) { + // enable removal optimization + it->agg_ctx_->removal_info_.enable_removal_opt_ = + !(MY_SPEC.single_part_parallel_) && it->wf_info_.remove_type_ != common::REMOVE_INVALID; + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(it->wf_expr_->process_partition(win_expr_ctx, it->part_first_row_idx_, + input_stores_.cur_->count(), start_idx, + start_idx + batch_size, *wf_skip))) { + // step.5 calculate window function results for rest rows + LOG_WARN("process partition failed", K(ret)); + } else { + ObSEArray tmp_exprs; + // reset skip to add all rows + wf_skip->unset_all(0, batch_size); + if (OB_FAIL(tmp_exprs.push_back(it->wf_info_.expr_))) { + LOG_WARN("push back element failed", K(ret)); + } else if (OB_FAIL(it->res_->cur_->add_batch_rows( + tmp_exprs, it->wf_res_row_meta_, eval_ctx_, + EvalBound(batch_size, 0, batch_size, true), *wf_skip, true))) { + LOG_WARN("add batch rows failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else { + start_idx += batch_size; + total_size -= batch_size; + } + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(tmp_holder.restore())) { + LOG_WARN("restore results failed", K(ret)); + } else if (OB_FAIL(update_part_first_row_idx(end))) { + LOG_WARN("update part first row idx failed", K(ret)); + } + // Row project flag is set when read row from RAStore, but the remain rows in batch + // are not evaluated, need reset the flag here. + clear_evaluated_flag(); + return ret; +} + +int ObWindowFunctionVecOp::set_null_results_of_wf(WinFuncColExpr &wf, const int64_t batch_size, + const ObBitVector &nullres_skip) +{ + int ret = OB_SUCCESS; + ObExpr *wf_expr = wf.wf_info_.expr_; + VectorFormat fmt = wf_expr->get_format(eval_ctx_); + switch (fmt) { + case common::VEC_FIXED: + case common::VEC_DISCRETE: + case common::VEC_CONTINUOUS: { + ObBitmapNullVectorBase *data = static_cast(wf_expr->get_vector(eval_ctx_)); + data->get_nulls()->bit_not(nullres_skip, batch_size); + break; + } + case common::VEC_UNIFORM: { + ObUniformFormat *data = static_cast *>(wf_expr->get_vector(eval_ctx_)); + for (int i = 0; i < batch_size; i++) { + if (nullres_skip.at(i)) { + } else { + data->set_null(i); + } + } + break; + } + case common::VEC_UNIFORM_CONST: { + ObUniformFormat *data = static_cast *>(wf_expr->get_vector(eval_ctx_)); + for (int i = 0; i < batch_size; i++) { + if (nullres_skip.at(i)) { + } else { + data->set_null(i); + break; + } + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret), K(fmt)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::calc_bypass_pushdown_rows_of_wf(WinFuncColExpr &wf, + const int64_t batch_size, + const ObBitVector &pushdown_skip) +{ + int ret = OB_SUCCESS; + winfunc::AggrExpr *agg_expr = static_cast(wf.wf_expr_); + if (pushdown_skip.accumulate_bit_cnt(batch_size) == batch_size) { + // do nothing + } else if (OB_FAIL(agg_expr->aggr_processor_->init_fast_single_row_aggs())) { + LOG_WARN("init fast single row aggregate failed", K(ret)); + } else if (OB_FAIL(agg_expr->aggr_processor_->single_row_agg_batch(wf.aggr_rows_, batch_size, + eval_ctx_, pushdown_skip))) { + LOG_WARN("bypass calculation failed", K(ret)); + } + return ret; +} + +int ObWindowFunctionVecOp::collect_sp_partial_results() +{ + int ret = OB_SUCCESS; + WinFuncColExpr &wf_col = *wf_list_.get_last(); + common::ObMemAttr attr(ctx_.get_my_session()->get_effective_tenant_id(), + ObModIds::OB_SQL_WINDOW_LOCAL, ObCtxIds::WORK_AREA); + SPWinFuncPXWholeMsg whole_msg(attr); + ObVectorsResultHolder tmp_holder; + if (wf_col.res_->cur_->count() <= 0) { + // current rows is empty, no need to compute wf values, hency just send empty msg and return. + if (OB_FAIL(sp_get_whole_msg(true, whole_msg, nullptr))) { + LOG_WARN("sp_get_whole_msg failed", K(ret)); + } + } else { + // In order to construct a compact row with wf_expr list, following steps are needed: + // 1. backup first row of exprs + // 2. attach first row wf_res_result to wf_expr seperately + // 3. calculate row size of needed compact row and allocate one row + // 4. call `ObIVector::to_rows` to get compact row + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + guard.set_batch_idx(0); + guard.set_batch_size(1); + if (OB_FAIL(tmp_holder.init(get_all_expr(), eval_ctx_))) { + LOG_WARN("init results holder failed", K(ret)); + } else if (OB_FAIL(tmp_holder.save(1))) { + LOG_WARN("save vector data failed", K(ret)); + } else { + ObCompactRow *wf_res_row = nullptr; + ObSEArray all_wf_exprs; + FOREACH_WINCOL(END_WF) { // only one row, no need to set row store mem guard + if (OB_FAIL(it->res_->cur_->attach_rows(it->wf_info_.expr_, it->wf_res_row_meta_, eval_ctx_, + 0, 1, false))) { + LOG_WARN("attach rows failed", K(ret)); + } else if (OB_FAIL(all_wf_exprs.push_back(it->wf_info_.expr_))) { + LOG_WARN("push back element failed", K(ret)); + } + } + int64_t mocked_skip = 0; + ObBatchRows brs; + brs.size_ = 1; + brs.end_ = false; + brs.all_rows_active_ = true; + brs.skip_ = to_bit_vector(&mocked_skip); + if (OB_FAIL(ret)) { + } else if (OB_ISNULL(batch_ctx_.tmp_wf_res_row_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null tmp row", K(ret)); + } else if (OB_FAIL(batch_ctx_.tmp_wf_res_row_->save_store_row(all_wf_exprs, brs, eval_ctx_, 0, false))) { + LOG_WARN("save wf res row failed", K(ret)); + } else if (OB_FAIL(sp_get_whole_msg(false, whole_msg, batch_ctx_.tmp_wf_res_row_->compact_row_))) { // all done ready to send piece msg and wait for whole + LOG_WARN("sp_get_whole msg failed", K(ret)); + } else if (OB_FAIL(sp_merge_partial_results(whole_msg))) { + LOG_WARN("merge partial results failed", K(ret)); + } else if (OB_FAIL(tmp_holder.restore())) { + LOG_WARN("restore failed", K(ret)); + } else { // do nothing + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::sp_get_whole_msg(bool is_empty, SPWinFuncPXWholeMsg &msg, + ObCompactRow *sending_row) +{ + int ret = OB_SUCCESS; + ObPxSqcHandler *handler = ctx_.get_sqc_handler(); + const SPWinFuncPXWholeMsg *temp_whole_msg = nullptr; + if (OB_ISNULL(handler)) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("parallel winbuf only supported in parallel execution mode", K(MY_SPEC.single_part_parallel_)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "parallel winbuf in non-px mode"); + } else { + ObPxSQCProxy &proxy = handler->get_sqc_proxy(); + common::ObMemAttr attr(ctx_.get_my_session()->get_effective_tenant_id(), + ObModIds::OB_SQL_WINDOW_LOCAL, ObCtxIds::WORK_AREA); + SPWinFuncPXPieceMsg piece(attr); + piece.op_id_ = MY_SPEC.get_id(); + piece.thread_id_ = GETTID(); + piece.source_dfo_id_ = proxy.get_dfo_id(); + piece.target_dfo_id_ = proxy.get_dfo_id(); + piece.is_empty_ = is_empty; + if (!is_empty) { + if (OB_ISNULL(sending_row)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null sending row", K(ret)); + } else if (OB_FAIL( + piece.row_meta_.deep_copy(*all_wf_res_row_meta_, &piece.deserial_allocator_))) { + LOG_WARN("deep copy row meta failed", K(ret)); + } else { + piece.row_ = sending_row; + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(proxy.get_dh_msg_sync( + MY_SPEC.get_id(), dtl::DH_SP_WINFUNC_PX_WHOLE_MSG, piece, temp_whole_msg, + ctx_.get_physical_plan_ctx()->get_timeout_timestamp()))) { + LOG_WARN("get whole msg failed", K(ret)); + } else if (OB_FAIL(msg.assign(*temp_whole_msg))) { + LOG_WARN("assign whole msg failed", K(ret)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::sp_merge_partial_results(SPWinFuncPXWholeMsg &msg) +{ + int ret = OB_SUCCESS; + const ObCompactRow *a_row = nullptr; + const char *res = nullptr, *cur = nullptr; + int32_t res_len = 0, cur_len = 0; + bool res_isnull = true, cur_isnull = false; + ObTempRowStore::Iterator store_iter; + ObSEArray all_wf_exprs; + int cmp_ret = 0; + if (OB_UNLIKELY(msg.is_empty_)) { + } else if (OB_FAIL(store_iter.init(&msg.row_store_))) { + LOG_WARN("init row store iteration failed", K(ret)); + } else { + int64_t batch_size = std::min(msg.row_store_.get_row_cnt(), MY_SPEC.max_batch_size_); + int64_t read_rows = 0; + MEMSET(batch_ctx_.stored_rows_, 0, sizeof(const ObCompactRow *) * batch_size); + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + guard.set_batch_idx(0); + guard.set_batch_size(1); + FOREACH_WINCOL(END_WF) { + if (OB_FAIL(all_wf_exprs.push_back(it->wf_info_.expr_))) { + LOG_WARN("push back element failed", K(ret)); + } else if (OB_FAIL(it->wf_info_.expr_->init_vector_default(eval_ctx_, 1))) { + LOG_WARN("init vector for write failed", K(ret)); + } else { + it->wf_info_.expr_->get_vector(eval_ctx_)->set_null(0); // initialize to null + } + } + __PartialResult part_res(eval_ctx_, msg.assign_allocator_); + while (OB_SUCC(ret)) { + if (OB_FAIL(store_iter.get_next_batch(batch_size, read_rows, batch_ctx_.stored_rows_))) { + if (ret == OB_ITER_END) { + } else { + LOG_WARN("get batch from store failed", K(ret)); + } + } else if (OB_UNLIKELY(read_rows == 0)) { + ret = OB_ITER_END; + } + FOREACH_WINCOL(END_WF) + { + int col_idx = it->wf_idx_ - 1; + for (int i = 0; OB_SUCC(ret) && i < read_rows; i++) { + a_row = batch_ctx_.stored_rows_[i]; + cur_isnull = a_row->is_null(col_idx); + a_row->get_cell_payload(msg.row_meta_, col_idx, cur, cur_len); + if (OB_FAIL(part_res.merge(it->wf_info_, cur_isnull, cur, cur_len))) { + LOG_WARN("merge result failed", K(ret)); + } + } + } // end wf for loop + } + if (OB_FAIL(ret)) { + if (ret == OB_ITER_END) { ret = OB_SUCCESS; } + } + if (OB_SUCC(ret)) { + if (sp_merged_row_ == nullptr) { + sp_merged_row_ = OB_NEWx(LastCompactRow, local_allocator_, *local_allocator_); + if (OB_ISNULL(sp_merged_row_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } + } + int64_t mocked_skip = 0; + ObBatchRows brs; + brs.size_ = 1; + brs.end_ = false; + brs.all_rows_active_ = true; + brs.skip_ = to_bit_vector(&mocked_skip); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(sp_merged_row_->save_store_row(all_wf_exprs, brs, eval_ctx_, 0, false))) { + LOG_WARN("save wf res row failed", K(ret)); + } else { + } + } + } + return ret; +} +#undef MERGE_AGG_RES + +int ObWindowFunctionVecOp::add_aggr_res_row_for_participator(WinFuncColExpr *end, + winfunc::RowStore &row_store) +{ + int ret = OB_SUCCESS; + if (MY_SPEC.is_participator()) { + // found a new part, add extra output row with aggr_status < 0 for partial agg results if possible + // timeline of wf computing (suppose dop = 2) + // pby_row (wf_idx = 1) (wf_idx = 2) + // firs_part(last_aggr_stats = 0, no bypass) + // second_part(last_aggr_status = 0, no bypass) + // third_part(last_aggr_status = 1, bypass) + // first_part(last_aggr_status=1, no bypass) + // second_part(last_aggr_status=1, no bypass) + // third_part(last_aggr_status = 2, bypass) + if (last_aggr_status_ < wf_list_.get_last()->wf_idx_) { + int64_t no_skip_data = 0; + ObBitVector *mock_skip = to_bit_vector(&no_skip_data); + EvalBound tmp_bound(1, true); + ObDataBuffer vec_res_alloc((char *)batch_ctx_.all_exprs_backup_buf_, + batch_ctx_.all_exprs_backup_buf_len_); + ObVectorsResultHolder tmp_holder(&vec_res_alloc); + if (OB_FAIL(tmp_holder.init(get_all_expr(), eval_ctx_))) { + LOG_WARN("init result holder failed", K(ret)); + } else if (OB_FAIL(tmp_holder.save(1))) { + LOG_WARN("save result failed", K(ret)); + } + FOREACH_WINCOL(end) + { + if (it->part_first_row_idx_ >= row_store.count()) { + // all partial results are calculated, no need calculating + break; + } + // `add_batch_rows` may invalidate memory of current block + // `attach_rows` before adding row is necessary to avoid invalid memory. + if (OB_FAIL(row_store.attach_rows(get_all_expr(), input_row_meta_, eval_ctx_, + row_store.stored_row_cnt_ - 1, row_store.stored_row_cnt_, + false))) { + LOG_WARN("attach rows failed", K(ret)); + } else { + ObExpr *agg_status_code = get_all_expr().at(get_all_expr().count() - 1); + if (last_aggr_status_ < it->wf_idx_) { + int64_t agg_status = -(it->wf_idx_); + agg_status_code->get_vector(eval_ctx_)->set_int(0, agg_status); + if (OB_FAIL(row_store.add_batch_rows(get_all_expr(), input_row_meta_, eval_ctx_, + tmp_bound, *mock_skip, true, nullptr, true))) { + LOG_WARN("add batch rows failed", K(ret)); + } + } + } + } + if (OB_SUCC(ret) && OB_FAIL(tmp_holder.restore())) { + LOG_WARN("restore result failed", K(ret)); + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::detect_and_report_aggr_status(const ObBatchRows &child_brs, + const int64_t start_idx, + const int64_t end_idx) +{ + int ret = OB_SUCCESS; + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + if (MY_SPEC.is_participator()) { + // find first nonskip idx; + int64_t batch_idx = next_nonskip_row_index(start_idx - 1, child_brs); + if (batch_idx >= end_idx) { + // maybe got a new batch within different partition. + // aggr status already detected in previous batch, do nothing + LOG_DEBUG("all rows are skipped", K(ret), K(start_idx), K(end_idx)); + } else { + guard.set_batch_idx(batch_idx); + last_aggr_status_ = 0; + int64_t total_task_cnt = 0; + int64_t prev_pushdown_wf_pby_expr_count = -1; // prev_wf_pby_expr_count transmit to datahub + FOREACH_WINCOL(END_WF) + { + bool is_pushdown_bypass = !it->wf_info_.can_push_down_; + if (OB_SUCC(ret) && it->wf_info_.can_push_down_) { + int64_t pushdown_wf_idx = pby_expr_cnt_idx_array_.at(it->wf_idx_ - 1); + if (it->wf_info_.partition_exprs_.count() > next_wf_pby_expr_cnt_to_transmit_) { + // already sent participator's hash set to datahub and got whole msg + ReportingWFHashSet *pushdown_pby_hash_values_set = + pby_hash_values_sets_.at(pushdown_wf_idx); + uint64_t hash_value = 0; + if (OB_ISNULL(pushdown_pby_hash_values_set)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null hash value set", K(ret)); + } else if (OB_FAIL(rwf_calc_pby_row_hash(child_brs, it->wf_info_.partition_exprs_, + hash_value))) { + LOG_WARN("calculate hash value failed", K(ret)); + } else if (FALSE_IT(ret = pushdown_pby_hash_values_set->exist_refactored(hash_value))) { + } else if (ret == OB_HASH_NOT_EXIST) { + is_pushdown_bypass = true; + ret = OB_SUCCESS; + } else if (ret == OB_HASH_EXIST) { + is_pushdown_bypass = false; + ret = OB_SUCCESS; + } else { + LOG_WARN("exist refactored failed", K(ret)); + } + } + total_task_cnt = + static_cast(input_)->get_total_task_count(); + if (OB_SUCC(ret) + && it->wf_info_.partition_exprs_.count() != prev_pushdown_wf_pby_expr_count + && it->wf_info_.partition_exprs_.count() <= next_wf_pby_expr_cnt_to_transmit_) { + if (pby_hash_values_.at(pushdown_wf_idx)->count() < total_task_cnt) { + // has not send data to datahub yet. + uint64_t hash_value = 0; + if (OB_FAIL( + rwf_calc_pby_row_hash(child_brs, it->wf_info_.partition_exprs_, hash_value))) { + LOG_WARN("calc hash value failed", K(ret)); + } + bool exists = false; + for (int i = 0; + OB_SUCC(ret) && !exists && i < pby_hash_values_.at(pushdown_wf_idx)->count(); + i++) { + exists = (hash_value == pby_hash_values_.at(pushdown_wf_idx)->at(i)); + } + if (OB_FAIL(ret)) { + } else if (!exists + && OB_FAIL(pby_hash_values_.at(pushdown_wf_idx)->push_back(hash_value))) { + LOG_WARN("pushback element failed", K(ret)); + } + } + if (OB_SUCC(ret) + && (pby_hash_values_.at(pushdown_wf_idx)->count() == total_task_cnt + || child_iter_end_)) { + if (OB_FAIL(rwf_participator_coordinate(pushdown_wf_idx))) { + LOG_WARN("participator coordinate failed", K(ret)); + } else { + next_wf_pby_expr_cnt_to_transmit_ = it->wf_info_.partition_exprs_.count() - 1; + } + } + } + if (OB_SUCC(ret)) { + prev_pushdown_wf_pby_expr_count = it->wf_info_.partition_exprs_.count(); + } + } + if (OB_SUCC(ret) && is_pushdown_bypass) { last_aggr_status_ = it->wf_idx_; } + LOG_TRACE("detect and report aggr status", K(next_wf_pby_expr_cnt_to_transmit_), + K(last_aggr_status_), K(total_task_cnt), K(child_iter_end_), K(it->wf_idx_)); + } + // update result value in expr + if (OB_SUCC(ret)) { + if (OB_FAIL(rwf_update_aggr_status_code(start_idx, end_idx))) { + LOG_WARN("update aggr_status code failed", K(ret)); + } + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::rwf_update_aggr_status_code(const int64_t start_idx, const int64_t end_idx) +{ + int ret = OB_SUCCESS; + if (MY_SPEC.is_participator()) { + if (OB_UNLIKELY(start_idx >= end_idx || end_idx > eval_ctx_.get_batch_size())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected row idx", K(start_idx), K(end_idx), K(eval_ctx_.get_batch_size())); + } else { + ObFixedLengthFormat *data = static_cast *>( + MY_SPEC.wf_aggr_status_expr_->get_vector(eval_ctx_)); + VectorFormat fmt = MY_SPEC.wf_aggr_status_expr_->get_format(eval_ctx_); + VecValueTypeClass vec_tc = MY_SPEC.wf_aggr_status_expr_->get_vec_value_tc(); + if (OB_UNLIKELY(!(fmt == VEC_FIXED && vec_tc == VEC_TC_INTEGER))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format for agg status expr", K(ret), K(fmt), K(vec_tc)); + } + for (int i = start_idx; OB_SUCC(ret) && i < end_idx; i++) { + data->set_payload(i, &last_aggr_status_, sizeof(int64_t)); + } + if (OB_SUCC(ret)) { + MY_SPEC.wf_aggr_status_expr_->set_evaluated_projected(eval_ctx_); + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::rwf_calc_pby_row_hash(const ObBatchRows &child_brs, + const ObIArray &pby_exprs, + uint64_t &hash_value) +{ + int ret = OB_SUCCESS; + int64_t batch_idx = eval_ctx_.get_batch_idx(); + hash_value = 99194853094755497L; // `copy from ObWindowFunctionOp::calc_part_exprs_hash` + for (int i = 0; OB_SUCC(ret) && i < pby_exprs.count(); i++) { + if (OB_FAIL(pby_exprs.at(i)->eval_vector( + eval_ctx_, *child_brs.skip_, + EvalBound(child_brs.size_, batch_idx, batch_idx + 1, true)))) { + LOG_WARN("eval vector failed", K(ret)); + } else { + ObIVector *data = pby_exprs.at(i)->get_vector(eval_ctx_); + if (OB_FAIL(data->murmur_hash_v3_for_one_row(*pby_exprs.at(i), hash_value, batch_idx, + eval_ctx_.get_batch_size(), hash_value))) { + LOG_WARN("murmur hash failed", K(ret)); + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::rwf_participator_coordinate(const int64_t pushdown_wf_idx) +{ + int ret = OB_SUCCESS; + ObReportingWFWholeMsg *whole_msg = participator_whole_msg_array_.at(pushdown_wf_idx); + const PbyHashValueArray *pby_hash_value_array = pby_hash_values_.at(pushdown_wf_idx); + ReportingWFHashSet *pushdown_pby_hash_set = pby_hash_values_sets_.at(pushdown_wf_idx); + if (OB_ISNULL(whole_msg) || OB_ISNULL(pby_hash_value_array) || OB_ISNULL(pushdown_pby_hash_set)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null arguments", K(ret), K(whole_msg), K(pby_hash_value_array), + K(pushdown_pby_hash_set)); + } else if (OB_FAIL(rwf_get_whole_msg(pby_hash_value_array, *whole_msg))) { + LOG_WARN("get whole msg failed", K(ret)); + } else if (0 == pby_hash_value_array->count()) { + // empty input, do nothing + } else { + for (int i = 0; OB_SUCC(ret) && i < whole_msg->pby_hash_value_array_.count(); i++) { + if (OB_FAIL(pushdown_pby_hash_set->set_refactored_1(whole_msg->pby_hash_value_array_.at(i), true))) { + LOG_WARN("insert hash set failed", K(ret)); + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::rwf_get_whole_msg(const PbyHashValueArray *hash_value_arr, + ObReportingWFWholeMsg &whole_msg) +{ + int ret = OB_SUCCESS; + const ObReportingWFWholeMsg *temp_whole_msg = nullptr; + ObPxSqcHandler *handler = ctx_.get_sqc_handler(); + if (OB_ISNULL(handler)) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("reporting window function only supported in parallel execution mode", + K(MY_SPEC.is_participator())); + } else { + ObPxSQCProxy &proxy = handler->get_sqc_proxy(); + ObReportingWFPieceMsg piece; + piece.op_id_ = MY_SPEC.get_id(); + piece.thread_id_ = GETTID(); + piece.source_dfo_id_ = proxy.get_dfo_id(); + piece.target_dfo_id_ = proxy.get_dfo_id(); + piece.pby_hash_value_array_ = *hash_value_arr; + if (OB_FAIL(proxy.get_dh_msg_sync(MY_SPEC.get_id(), dtl::DH_SECOND_STAGE_REPORTING_WF_WHOLE_MSG, + piece, temp_whole_msg, + ctx_.get_physical_plan_ctx()->get_timeout_timestamp()))) { + LOG_WARN("get reporting window function whole msg failed", K(ret)); + } else if (OB_ISNULL(temp_whole_msg)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null whole msg", K(ret)); + } else if (OB_FAIL(whole_msg.assign(*temp_whole_msg))) { + LOG_WARN("assign whole msg failed", K(ret)); + } else { + ObWindowFunctionVecOpInput *op_input = static_cast(input_); + ObPxDatahubDataProvider *provider = nullptr; + ObReportingWFWholeMsg::WholeMsgProvider *msg_provider = nullptr; + if (OB_FAIL(proxy.sqc_ctx_.get_whole_msg_provider( + MY_SPEC.get_id(), dtl::DH_SECOND_STAGE_REPORTING_WF_WHOLE_MSG, provider))) { + LOG_WARN("get whole msg provider failed", K(ret)); + } else { + // `sync_wait` will wait for all workers to receive whole msg and then reset datahub's resource. + // for reporting window function parallel excution, multiple pushdown wf exprs maybe exists in operator + // pushdown exprs will send pieces and get whole msg in order + // if we do not do synchronous waiting, following pushdown wf expr may get previous wf_expr's whole msg + // and cause unexpected error. + msg_provider = static_cast(provider); + if (OB_FAIL(op_input->sync_wait(ctx_, msg_provider))) { + LOG_WARN("sync wait failed", K(ret)); + } + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::detect_nullres_or_pushdown_rows(WinFuncColExpr &wf, + ObBitVector &nullres_skip, + ObBitVector &pushdown_skip, + ObBitVector &wf_skip) +{ + int ret = OB_SUCCESS; + int64_t word_cnt = ObBitVector::word_count(eval_ctx_.get_batch_size()); + nullres_skip.set_all(eval_ctx_.get_batch_size()); + pushdown_skip.set_all(eval_ctx_.get_batch_size()); + MEMSET(wf_skip.data_, 0, word_cnt); + if (MY_SPEC.is_participator()) { + if (OB_UNLIKELY(MY_SPEC.wf_aggr_status_expr_->get_format(eval_ctx_) != VEC_FIXED)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret), + K(MY_SPEC.wf_aggr_status_expr_->get_format(eval_ctx_))); + } else { + ObFixedLengthFormat *agg_status_data = static_cast *>( + MY_SPEC.wf_aggr_status_expr_->get_vector(eval_ctx_)); + for (int i = 0; i < eval_ctx_.get_batch_size(); i++) { + int64_t aggr_status = *reinterpret_cast(agg_status_data->get_payload(i)); + if (aggr_status >= 0) { + if (aggr_status < wf.wf_idx_) { // original rows participating in partial aggregation + // set aggr results to null + nullres_skip.unset(i); + } else { // original rows using bypass-pushdown output + pushdown_skip.unset(i); + } + wf_skip.set(i); + } else { // extra added row with partial aggregation results + if (aggr_status != -wf.wf_idx_) { + // if aggr_status == -wf.wf_idx_, this row needs calculation to get partial aggregation result. + nullres_skip.unset(i); + wf_skip.set(i); + } + } + } + } + } else if (MY_SPEC.is_consolidator()) { + VectorFormat status_fmt = MY_SPEC.wf_aggr_status_expr_->get_format(eval_ctx_); + if (status_fmt == VEC_FIXED) { + ObFixedLengthFormat *agg_status_data = static_cast *>( + MY_SPEC.wf_aggr_status_expr_->get_vector(eval_ctx_)); + for (int i = 0; i < eval_ctx_.get_batch_size(); i++) { + int64_t aggr_status = *reinterpret_cast(agg_status_data->get_payload(i)); + if (aggr_status < 0) { + // row with partial result, do not calculation wf value + wf_skip.set(i); + nullres_skip.unset(i); + } + } + } else if (status_fmt == VEC_UNIFORM) { + ObUniformFormat *agg_status_data = static_cast *>( + MY_SPEC.wf_aggr_status_expr_->get_vector(eval_ctx_)); + for (int i = 0; i < eval_ctx_.get_batch_size(); i++) { + int64_t aggr_status = *reinterpret_cast(agg_status_data->get_payload(i)); + if (aggr_status < 0) { + // row with partial result, do not calculation wf value + wf_skip.set(i); + nullres_skip.unset(i); + } + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret), K(status_fmt)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::rwf_send_empty_piece_data() +{ + int ret = OB_SUCCESS; + FOREACH_WINCOL(END_WF) { + if (it->wf_info_.can_push_down_ && it->wf_info_.partition_exprs_.count() <= next_wf_pby_expr_cnt_to_transmit_) { + int64_t pushdown_wf_idx = pby_expr_cnt_idx_array_.at(it->wf_idx_ - 1); + if (OB_FAIL(rwf_participator_coordinate(pushdown_wf_idx))) { + LOG_WARN("participator coordinating failed", K(ret)); + } else { + next_wf_pby_expr_cnt_to_transmit_ = it->wf_info_.partition_exprs_.count() - 1; + } + } + } + return ret; +} + +struct __tid_cmp +{ + OB_INLINE bool operator()(RDWinFuncPXPartialInfo *it, int64_t tid) + { + return it->thread_id_ < tid; + } +}; + +int ObWindowFunctionVecOp::rd_fetch_patch() +{ + int ret = OB_SUCCESS; + ObPxSqcHandler *handler = ctx_.get_sqc_handler(); + if (OB_ISNULL(handler) || OB_ISNULL(rd_coord_row_meta_) || OB_ISNULL(ctx_.get_my_session())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null arguments", K(ret), K(handler), K(rd_coord_row_meta_), + K(ctx_.get_my_session())); + } else { + common::ObMemAttr attr(ctx_.get_my_session()->get_effective_tenant_id(), + ObModIds::OB_SQL_WINDOW_FUNC, ObCtxIds::WORK_AREA); + RDWinFuncPXPieceMsg piece_msg(attr); + piece_msg.op_id_ = MY_SPEC.get_id(); + piece_msg.thread_id_ = GETTID(); + piece_msg.source_dfo_id_ = handler->get_sqc_proxy().get_dfo_id(); + piece_msg.target_dfo_id_ = handler->get_sqc_proxy().get_dfo_id(); + + piece_msg.info_.sqc_id_ = handler->get_sqc_proxy().get_sqc_id(); + piece_msg.info_.thread_id_ = GETTID(); + if (OB_FAIL(piece_msg.info_.row_meta_.deep_copy(*rd_coord_row_meta_, &piece_msg.arena_alloc_))) { + LOG_WARN("deep copy row meta failed", K(ret)); + } + if (OB_SUCC(ret) && input_stores_.first_->count() > 0) { + if (OB_FAIL(rd_build_partial_info_row(0, true, piece_msg.arena_alloc_, + piece_msg.info_.first_row_))) { + LOG_WARN("build first row failed", K(ret)); + } else if (input_stores_.last_->count() <= 0) { + // get first partition's last row if only exists one partition + if (OB_FAIL(rd_build_partial_info_row(input_stores_.first_->count() - 1, + true, piece_msg.arena_alloc_, + piece_msg.info_.last_row_))) { + LOG_WARN("build last row failed", K(ret)); + } + } + } + if (OB_SUCC(ret) && input_stores_.last_->count() > 0) { + if (OB_FAIL(rd_build_partial_info_row(input_stores_.last_->count() - 1, + false, piece_msg.arena_alloc_, + piece_msg.info_.last_row_))) { + LOG_WARN("build last row failed", K(ret)); + } else { + *reinterpret_cast(piece_msg.info_.last_row_->get_extra_payload( + *rd_coord_row_meta_)) = last_computed_part_rows_ - 1; + } + } + const RDWinFuncPXWholeMsg *whole_msg = nullptr; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(handler->get_sqc_proxy().get_dh_msg_sync( + MY_SPEC.get_id(), dtl::DH_RD_WINFUNC_PX_WHOLE_MSG, piece_msg, whole_msg, + ctx_.get_physical_plan_ctx()->get_timeout_timestamp()))) { + LOG_WARN("get range distributed winfunc msg failed", K(ret)); + } else if (OB_ISNULL(whole_msg)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null msg", K(ret)); + } else { + // find patch of this worker + // `RDWinFuncWholeMsg::infos_` already sorted by thread_id in QC + int64_t tid = GETTID(); + RDWinFuncPXWholeMsg *m = const_cast(whole_msg); + decltype(m->infos_)::iterator info = std::lower_bound(m->infos_.begin(), m->infos_.end(), tid, __tid_cmp()); + if (info == m->infos_.end() || (*info)->thread_id_ != tid) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("worker's response msg not found in whole msg", K(ret), K(tid)); + } else if (OB_ISNULL(rd_patch_ = (*info)->dup(rescan_alloc_))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("duplicate patch info failed", K(ret)); + } + } + } + return ret; +} + +int ObWindowFunctionVecOp::rd_build_partial_info_row(int64_t idx, bool is_first_part, + ObIAllocator &alloc, ObCompactRow *&build_row) +{ + int ret = OB_SUCCESS; + // no need to save first row of all_exprs + clear_evaluated_flag(); + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + guard.set_batch_size(1); + guard.set_batch_idx(0); + ObVectorsResultHolder tmp_holder; + const ObCompactRow *input_row = nullptr, *wf_res_row = nullptr; + winfunc::RowStore *input = (is_first_part ? input_stores_.first_ : input_stores_.last_); + if (OB_FAIL(tmp_holder.init(get_all_expr(), eval_ctx_))) { + LOG_WARN("init tmp result holder failed", K(ret)); + } else if (OB_FAIL(tmp_holder.save(1))) { + LOG_WARN("save batch results failed", K(ret)); + } else if (OB_FAIL(input->attach_rows(get_all_expr(), input_row_meta_, eval_ctx_, idx, idx + 1, + false))) { // first attach input row + LOG_WARN("attach row failed", K(ret)); + } + + // second: attach wf res row + FOREACH_WINCOL(END_WF) { + ObExpr *expr = it->wf_info_.expr_; + winfunc::RowStore *wf_res = (is_first_part ? it->res_->first_ : it->res_->last_); + if (OB_FAIL(wf_res->attach_rows(it->wf_info_.expr_, it->wf_res_row_meta_, eval_ctx_, idx, + idx + 1, false))) { + LOG_WARN("attach row failed", K(ret)); + } + } + // third: build rd_coord_res_row_ + if (OB_SUCC(ret)) { + LastCompactRow *rd_coord_row = OB_NEWx(LastCompactRow, &alloc, alloc); + int64_t tmp_skip = 0; + ObBatchRows tmp_brs; + tmp_brs.size_ = eval_ctx_.get_batch_size(); + tmp_brs.skip_ = to_bit_vector(&tmp_skip); + tmp_brs.end_ = false; + tmp_brs.all_rows_active_ = true; + if (OB_ISNULL(rd_coord_row)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(rd_coord_row->save_store_row(MY_SPEC.rd_coord_exprs_, tmp_brs, eval_ctx_, + sizeof(int64_t), false))) { + LOG_WARN("save store row failed", K(ret)); + } else { + build_row = rd_coord_row->compact_row_; + *reinterpret_cast(build_row->get_extra_payload(*rd_coord_row_meta_)) = idx; + } + } + if (OB_SUCC(ret) && OB_FAIL(tmp_holder.restore())) { + LOG_WARN("restore batch results failed", K(ret)); + } + return ret; +} + +int ObWindowFunctionVecOp::rd_output_final_batch(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + if (input_stores_.cur_->to_output_rows() <= 0) { + brs_.end_ = true; + brs_.size_ = 0; + iter_end_ = true; + } else { + if (patch_alloc_.used() >= OB_MALLOC_MIDDLE_BLOCK_SIZE) { + patch_alloc_.reset_remain_one_page(); + } + int64_t cnt = std::min(max_row_cnt, input_stores_.cur_->to_output_rows()); + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + guard.set_batch_size(cnt); + int64_t store_start_idx = input_stores_.cur_->output_row_idx_; + int64_t store_end_idx = input_stores_.cur_->output_row_idx_ + cnt; + LOG_DEBUG("rd output final batch", K(store_start_idx), K(store_end_idx), K(cnt), + K(patch_first_), K(patch_last_)); + winfunc::StoreGuard store_guard(*this); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(input_stores_.cur_->attach_rows(get_all_expr(), input_row_meta_, eval_ctx_, + store_start_idx, store_end_idx, true))) { + LOG_WARN("attach rows failed", K(ret)); + } + FOREACH_WINCOL(END_WF) { + if (OB_FAIL(it->res_->cur_->attach_rows(it->wf_info_.expr_, it->wf_res_row_meta_, eval_ctx_, + store_start_idx, store_end_idx, true))) { + LOG_WARN("attach rows failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(rd_apply_patches(cnt))) { + LOG_WARN("apply patches failed", K(ret)); + } else { + brs_.size_ = cnt; + brs_.all_rows_active_ = true; + input_stores_.cur_->output_row_idx_ += cnt; + FOREACH_WINCOL(END_WF) { + it->res_->cur_->output_row_idx_ += cnt; + } + } + } + return ret; +} + +template +int ObWindowFunctionVecOp::rd_merge_result(PartialMerge &part_res, WinFuncInfo &info, + int64_t rd_col_id, + int64_t first_row_same_order_upper_bound, + int64_t last_row_same_order_lower_bound) +{ + int ret = OB_SUCCESS; + int64_t batch_size = eval_ctx_.get_batch_size(); + const char *payload = nullptr; + int32_t len = 0; + bool null_payload = false; + ObExpr *wf_expr = info.expr_; + const bool constexpr is_rank = std::is_same>::value; + ResFmt *res_data = static_cast(wf_expr->get_vector(eval_ctx_)); + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx_); + if (patch_first_) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + guard.set_batch_idx(i); + null_payload = rd_patch_->first_row_->is_null(rd_col_id); + rd_patch_->first_row_->get_cell_payload(*rd_coord_row_meta_, rd_col_id, payload, len); + if (is_rank && i >= first_row_same_order_upper_bound) { + int64_t rank_patch = rd_patch_->first_row_frame_offset(); + if (OB_FAIL(part_res.template add_rank(info, true, nullptr, 0, rank_patch))) { + LOG_WARN("add rank failed", K(ret)); + } + } else if (OB_FAIL(part_res.template merge(info, null_payload, payload, len))) { + LOG_WARN("merge result failed", K(ret)); + } + } + } + if (OB_SUCC(ret) && patch_last_) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + guard.set_batch_idx(i); + if (i >= last_row_same_order_lower_bound) { + null_payload = rd_patch_->last_row_->is_null(rd_col_id); + rd_patch_->last_row_->get_cell_payload(*rd_coord_row_meta_, rd_col_id, payload, len); + if (OB_FAIL(part_res.template merge(info, null_payload, payload, len))) { + LOG_WARN("merge last row's patch failed", K(ret)); + } + } + } + } + return ret; +} + +#define MERGE_FIXED_LEN_RES(vec_tc) \ + case (vec_tc): { \ + ret = rd_merge_result>>( \ + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); \ + } break + +#define MERGE_RES_COMMON() \ + do { \ + if (fmt == VEC_DISCRETE) { \ + ret = rd_merge_result( \ + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); \ + } else if (fmt == VEC_CONTINUOUS) { \ + ret = rd_merge_result( \ + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); \ + } else if (fmt == VEC_FIXED) { \ + switch (vec_tc) { \ + LST_DO_CODE(MERGE_FIXED_LEN_RES, FIXED_VEC_LIST); \ + default: { \ + ret = OB_ERR_UNEXPECTED; \ + LOG_WARN("unexpected vector type class", K(ret), K(vec_tc)); \ + } \ + } \ + } else { \ + ret = OB_ERR_UNEXPECTED; \ + LOG_WARN("unexpected format", K(ret), K(fmt)); \ + } \ + } while (false) + +int ObWindowFunctionVecOp::rd_apply_patches(const int64_t max_row_cnt) +{ + int ret = OB_SUCCESS; + int64_t first_row_same_order_upper = 0, last_row_same_order_lower = max_row_cnt - 1; + if (patch_first_ && OB_FAIL(rd_find_first_row_upper_bound(max_row_cnt, first_row_same_order_upper))) { + LOG_WARN("find first row same order lower bound idx failed", K(ret)); + } else if (patch_last_ && OB_FAIL(rd_find_last_row_lower_bound(max_row_cnt, last_row_same_order_lower))) { + LOG_WARN("find last row same order upper bound idx failed", K(ret)); + } + for (int i = 0; OB_SUCC(ret) && i < MY_SPEC.rd_wfs_.count(); i++) { + WinFuncInfo &info = const_cast(MY_SPEC.wf_infos_.at(MY_SPEC.rd_wfs_.at(i))); + VectorFormat fmt = info.expr_->get_format(eval_ctx_); + VecValueTypeClass vec_tc = info.expr_->get_vec_value_tc(); + int64_t col_idx = i + MY_SPEC.rd_sort_collations_.count(); + switch(info.func_type_) { + case T_FUN_MIN: { + __PartialResult part_res(eval_ctx_, patch_alloc_); + MERGE_RES_COMMON(); + break; + } + case T_FUN_MAX: { + __PartialResult part_res(eval_ctx_, patch_alloc_); + MERGE_RES_COMMON(); + break; + } + case T_FUN_COUNT: { + __PartialResult part_res(eval_ctx_, patch_alloc_); + MERGE_RES_COMMON(); + break; + } + case T_FUN_SUM: { + __PartialResult part_res(eval_ctx_, patch_alloc_); + if (fmt == VEC_DISCRETE) { + ret = rd_merge_result( + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); + } else if (fmt == VEC_FIXED) { + switch(vec_tc) { + LST_DO_CODE(MERGE_FIXED_LEN_RES, VEC_TC_FLOAT, VEC_TC_DOUBLE, VEC_TC_DEC_INT32, + VEC_TC_DEC_INT64, VEC_TC_DEC_INT128, VEC_TC_DEC_INT256, VEC_TC_DEC_INT512); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected type class", K(ret)); + } + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret), K(fmt)); + } + break; + } + case T_WIN_FUN_RANK: { + __PartialResult part_res(eval_ctx_, patch_alloc_); + if (fmt == VEC_DISCRETE) { + ret = rd_merge_result( + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); + } else if (fmt == VEC_FIXED) { + if (vec_tc != VEC_TC_UINTEGER) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected vec tc", K(vec_tc)); + } else { + ret = rd_merge_result>( + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret), K(fmt)); + } + break; + } + case T_WIN_FUN_DENSE_RANK: { + __PartialResult part_res(eval_ctx_, patch_alloc_); + if (fmt == VEC_DISCRETE) { + ret = rd_merge_result( + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); + } else if (fmt == VEC_FIXED) { + if (vec_tc != VEC_TC_UINTEGER) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected vec tc", K(vec_tc)); + } else { + ret = rd_merge_result>( + part_res, info, col_idx, first_row_same_order_upper, last_row_same_order_lower); + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret), K(fmt)); + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not supported func type", K(ret), K(info.func_type_)); + } + } + if (OB_FAIL(ret)) { + LOG_WARN("merge result failed", K(info), K(fmt), K(vec_tc), K(col_idx)); + } + } + return ret; +} + +int ObWindowFunctionVecOp::rd_find_first_row_upper_bound(int64_t batch_size, int64_t &upper_bound) +{ + int ret = OB_SUCCESS; + upper_bound = batch_size; + const char *val = nullptr; + int32_t val_len = 0; + bool val_isnull = false; + int32_t word_cnt = ObBitVector::word_count(batch_size); + MEMSET(batch_ctx_.bound_eval_skip_, 0, word_cnt * ObBitVector::BYTES_PER_WORD); + EvalBound bound(batch_size, 0, batch_size, true); + if (OB_ISNULL(rd_patch_->first_row_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null first patch row", K(ret)); + } + for (int i = 0; OB_SUCC(ret) && i < MY_SPEC.rd_sort_collations_.count(); i++) { + int64_t tmp_bound = -1; + ObExpr *sort_expr = MY_SPEC.rd_coord_exprs_.at(i); + NullSafeRowCmpFunc cmp_fn = sort_expr->basic_funcs_->row_null_first_cmp_; + rd_patch_->first_row_->get_cell_payload(rd_patch_->row_meta_, i, val, val_len); + val_isnull = rd_patch_->first_row_->is_null(i); + VectorRangeUtil::NullSafeCmp cmp_op(sort_expr->obj_meta_, cmp_fn, val, val_len, val_isnull, + MY_SPEC.rd_sort_collations_.at(i).is_ascending_); + if (OB_FAIL(VectorRangeUtil::upper_bound(sort_expr, eval_ctx_, bound, + *batch_ctx_.bound_eval_skip_, cmp_op, tmp_bound))) { + LOG_WARN("find upper bound failed", K(ret)); + } else if (OB_UNLIKELY(tmp_bound == -1)) { + // no value is larger than first_row + // upper_bound = batch_size; + // do nothing + } else { + upper_bound = std::min(tmp_bound, upper_bound); + } + } + return ret; +} + +int ObWindowFunctionVecOp::rd_find_last_row_lower_bound(int64_t batch_size, int64_t &lower_bound) +{ + int ret = OB_SUCCESS; + lower_bound = 0; + const char *val = nullptr; + int32_t val_len = 0; + bool val_isnull = false; + int32_t word_cnt = ObBitVector::word_count(batch_size); + MEMSET(batch_ctx_.bound_eval_skip_, 0, word_cnt * ObBitVector::BYTES_PER_WORD); + EvalBound bound(batch_size, 0, batch_size, true); + if (OB_ISNULL(rd_patch_->last_row_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null last patch row", K(ret)); + } + for (int i = 0; OB_SUCC(ret) && i < MY_SPEC.rd_sort_collations_.count(); i++) { + int64_t tmp_bound = -1; + int64_t field_idx = MY_SPEC.rd_sort_collations_.at(i).field_idx_; + ObExpr *sort_expr = MY_SPEC.all_expr_.at(field_idx); + NullSafeRowCmpFunc cmp_fn = sort_expr->basic_funcs_->row_null_first_cmp_; + rd_patch_->last_row_->get_cell_payload(rd_patch_->row_meta_, i, val, val_len); + val_isnull = rd_patch_->last_row_->is_null(i); + VectorRangeUtil::NullSafeCmp cmp_op(sort_expr->obj_meta_, cmp_fn, val, val_len, val_isnull, + MY_SPEC.rd_sort_collations_.at(i).is_ascending_); + if (OB_FAIL(VectorRangeUtil::lower_bound(sort_expr, eval_ctx_, bound, + *batch_ctx_.bound_eval_skip_, cmp_op, tmp_bound))) { + LOG_WARN("find lower bound failed", K(ret)); + } else if (OB_UNLIKELY(tmp_bound == -1)) { + lower_bound = batch_size; + break; + } else { + lower_bound = std::max(lower_bound, tmp_bound); + } + } + return ret; +} + +bool ObWindowFunctionVecOp::all_supported_winfuncs(const ObIArray &win_exprs) +{ + bool ret = true; + for (int i = 0; ret && i < win_exprs.count(); i++) { + ObWinFunRawExpr *win_expr = win_exprs.at(i); + if (win_expr->get_agg_expr() != nullptr) { + ret = aggregate::supported_aggregate_function(win_expr->get_func_type()) + && !win_expr->get_agg_expr()->is_param_distinct(); + } + } + return ret; +} + +int ObWindowFunctionVecOp::backup_child_vectors(int64_t batch_size) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(all_expr_vector_copy_.save(batch_size))) { + LOG_WARN("save vector results failed", K(ret)); + } else { + backuped_size_ = batch_size; + } + return ret; +} + +int ObWindowFunctionVecOp::restore_child_vectors() +{ + int ret = OB_SUCCESS; + if (backuped_size_ <= 0) { + } else if (OB_FAIL(all_expr_vector_copy_.restore())) { + LOG_WARN("restore vector results failed", K(ret)); + } else { + backuped_size_ = 0; + } + return ret; +} + +int ObWindowFunctionVecOp::init_mem_context() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(mem_context_)) { + ObSQLSessionInfo *sess = ctx_.get_my_session(); + uint64_t tenant_id = sess->get_effective_tenant_id(); + lib::ContextParam param; + param.set_mem_attr(tenant_id, ObModIds::OB_SQL_WINDOW_ROW_STORE, ObCtxIds::WORK_AREA) + .set_properties(lib::USE_TL_PAGE_OPTIONAL); + if (OB_FAIL(CURRENT_CONTEXT->CREATE_CONTEXT(mem_context_, param))) { + LOG_WARN("create entity failed", K(ret)); + } else if (OB_ISNULL(mem_context_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("null memory entity returned", K(ret)); + } + } + return ret; +} + +void ObWindowFunctionVecOp::destroy_mem_context() +{ + if (OB_NOT_NULL(mem_context_)) { + DESTROY_CONTEXT(mem_context_); + mem_context_ = nullptr; + } +} + +struct __mem_pred_op +{ + const static int64_t UPDATE_MEM_SIZE_PERIODIC_CNT = 1024; + __mem_pred_op(ObSqlMemMgrProcessor &p, int64_t &amm_p_cnt): p_(p), amm_p_cnt_(amm_p_cnt) {} + OB_INLINE bool operator()(int64_t cur_cnt) + { + UNUSED(cur_cnt); + p_.set_periodic_cnt(1024); + return 0 == ((++amm_p_cnt_) % UPDATE_MEM_SIZE_PERIODIC_CNT); + } +private: + ObSqlMemMgrProcessor &p_; + int64_t &amm_p_cnt_; +}; + +struct __mem_extend_op +{ + __mem_extend_op(ObSqlMemMgrProcessor &p): p_(p) {} + OB_INLINE bool operator()(int64_t max_memory_size) + { + return p_.get_data_size() > max_memory_size; + } +private: + ObSqlMemMgrProcessor &p_; +}; +int ObWindowFunctionVecOp::update_mem_limit_version_periodically() +{ + int ret = OB_SUCCESS; + // update global mem bound every 1024 rows + // use total_stored_row_cnt of wf op instead of row_cnt of each ra_rs_ here + // because total_stored_row_cnt is monotone increasing + bool updated = false; + bool need_inc_version = false; + if (!GCONF.is_sql_operator_dump_enabled()) { + } else if (OB_FAIL(sql_mem_processor_.update_max_available_mem_size_periodically( + &mem_context_->get_malloc_allocator(), + __mem_pred_op(sql_mem_processor_, amm_periodic_cnt_), updated))) { + LOG_WARN("failed to update max available memory size periodically", K(ret)); + } else if (updated || need_dump()) { + if (OB_FAIL(sql_mem_processor_.extend_max_memory_size( + &mem_context_->get_malloc_allocator(), __mem_extend_op(sql_mem_processor_), + need_inc_version, sql_mem_processor_.get_data_size()))) { + LOG_WARN("failed to extend max memory size", K(ret), K(updated), K(need_dump())); + } + } + if (OB_FAIL(ret)) { + } else if (need_inc_version) { + ++global_mem_limit_version_; + } + return ret; +} + +// ================ WinFuncColExpr +int WinFuncColExpr::init_aggregate_ctx(const int64_t tenant_id) +{ + // only aggr expr need agg_ctx_ + using info_array = ObFixedArray; + int ret = OB_SUCCESS; + ObIAllocator &local_allocator = *op_.local_allocator_; + info_array *aggr_infos = OB_NEWx(info_array, &local_allocator, local_allocator, 1); + void *aggr_row_buf = nullptr, *res_row_buf = nullptr; + int32_t aggr_row_buf_sz = 0, res_row_buf_sz = 0; + if (OB_ISNULL(aggr_infos)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(aggr_infos->push_back(wf_info_.aggr_info_))) { + LOG_WARN("push back element failed"); + } else { + winfunc::AggrExpr *agg_expr = static_cast(wf_expr_); + agg_expr->aggr_processor_ = OB_NEWx(aggregate::Processor, &local_allocator, + op_.eval_ctx_, *aggr_infos, + ObModIds::OB_SQL_WINDOW_LOCAL, + op_.op_monitor_info_, tenant_id); + if (OB_ISNULL(agg_expr->aggr_processor_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(agg_expr->aggr_processor_->init())) { + LOG_WARN("processor init failed", K(ret), K(wf_info_.aggr_info_)); + } else if (FALSE_IT(agg_expr->aggr_processor_->set_support_fast_single_row_agg(true))) { + } else if (FALSE_IT(agg_ctx_ = agg_expr->aggr_processor_->get_rt_ctx())) { + } else if (FALSE_IT(aggr_row_buf_sz = op_.spec_.max_batch_size_ * agg_ctx_->row_meta().row_size_)) { + // do nothing + } else if (OB_ISNULL(aggr_row_buf = local_allocator.alloc(aggr_row_buf_sz))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + agg_ctx_->win_func_agg_ = true; + MEMSET(aggr_row_buf, 0, aggr_row_buf_sz); + aggr_rows_ = (aggregate::AggrRowPtr *)local_allocator.alloc(sizeof(aggregate::AggrRowPtr) + * op_.spec_.max_batch_size_); + if (OB_ISNULL(aggr_rows_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + int32_t offset = 0, step = agg_ctx_->row_meta().row_size_; + for (int i = 0; i < op_.spec_.max_batch_size_; i++, offset += step) { + aggr_rows_[i] = (char *)aggr_row_buf + offset; + } + } + } + } + return ret; +} + +int32_t WinFuncColExpr::non_aggr_reserved_row_size() const +{ + int32_t ret_size = 0; + VecValueTypeClass vec_tc = wf_info_.expr_->get_vec_value_tc(); + if (!wf_expr_->is_aggregate_expr()) { + if (is_fixed_length_vec(vec_tc) || vec_tc == VEC_TC_NUMBER) { + ret_size = ObDatum::get_reserved_size( + ObDatum::get_obj_datum_map_type(wf_info_.expr_->datum_meta_.type_)); + } else { + ret_size = sizeof(char *) + sizeof(uint32_t); // + } + } + return ret_size; +} +int WinFuncColExpr::init_non_aggregate_ctx() +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(wf_expr_->is_aggregate_expr())) { + // do nothing + } else { + int64_t batch_size = op_.get_spec().max_batch_size_; + ObIAllocator &local_alloc = *op_.local_allocator_; + int64_t word_cnt = ObBitVector::word_count(batch_size); + int32_t reserved_row_size = non_aggr_reserved_row_size(); + void *data_buf = local_alloc.alloc(reserved_row_size * batch_size); + null_nonaggr_results_ = (ObBitVector *)local_alloc.alloc(word_cnt * ObBitVector::BYTES_PER_WORD); + if (OB_ISNULL(data_buf) || OB_ISNULL(null_nonaggr_results_)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMSET(data_buf, 0, reserved_row_size * batch_size); + MEMSET(null_nonaggr_results_, 0, word_cnt * ObBitVector::BYTES_PER_WORD); + int64_t offset = 0; + non_aggr_results_ = (char *)data_buf; + } + } + return ret; +} + +int WinFuncColExpr::init_res_rows(const int64_t tenant_id) +{ + int ret = OB_SUCCESS; + ObIAllocator &local_allocator = *op_.local_allocator_; + wf_res_row_meta_.set_allocator(&local_allocator); + void *res_row_ptr_buf = nullptr; + int32_t res_row_ptr_buf_sz = 0; + ObSEArray all_exprs; + if (OB_FAIL(all_exprs.push_back(wf_info_.expr_))) { + LOG_WARN("push back elements failed", K(ret)); + } else if (OB_FAIL(wf_res_row_meta_.init(all_exprs, 0))) { + LOG_WARN("init compact row meta failed", K(ret)); + } else if (FALSE_IT(res_row_ptr_buf_sz = op_.spec_.max_batch_size_ * sizeof(const ObCompactRow *))) { + } else if (OB_ISNULL(res_row_ptr_buf = local_allocator.alloc(res_row_ptr_buf_sz))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMSET(res_row_ptr_buf, 0, res_row_ptr_buf_sz); + res_rows_ = (const ObCompactRow **)res_row_ptr_buf; + } + return ret; +} + +int WinFuncColExpr::reset_for_partition(const int64_t batch_size, const ObBitVector &skip) +{ + int ret = OB_SUCCESS; + UNUSED(skip); + OB_ASSERT(batch_size <= op_.spec_.max_batch_size_); + if (wf_expr_->is_aggregate_expr()) { + agg_ctx_->reuse(); + agg_ctx_->removal_info_.reset(); + aggregate::Processor *processor = static_cast(wf_expr_)->aggr_processor_; + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + if (OB_FAIL(processor->add_one_aggregate_row(aggr_rows_[i], agg_ctx_->row_meta().row_size_))) { + LOG_WARN("setup rt info failed", K(ret)); + } + } + } else { + int64_t word_cnt = ObBitVector::word_count(batch_size); + MEMSET(null_nonaggr_results_, 0, ObBitVector::BYTES_PER_WORD * word_cnt); + } + return ret; +} + +void WinFuncColExpr::reset() +{ + if (wf_expr_ != nullptr) { + wf_expr_->destroy(); + } + if (res_ != nullptr) { + res_->reset(); + } + agg_ctx_ = nullptr; + wf_res_row_meta_.reset(); +} + +void WinFuncColExpr::reset_for_scan() +{ + part_first_row_idx_ = 0; + MEMSET(pby_row_mapped_idxes_, 0, op_.max_pby_col_cnt_ * sizeof(int32_t)); +} +// ================ ObWindownFunctionVecSpec +int ObWindowFunctionVecSpec::register_to_datahub(ObExecContext &ctx) const +{ + int ret = OB_SUCCESS; + if (single_part_parallel_) { + if (OB_ISNULL(ctx.get_sqc_handler())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null sqc handler", K(ret)); + } else { + SPWinFuncPXWholeMsg::WholeMsgProvider *provider = + OB_NEWx(SPWinFuncPXWholeMsg::WholeMsgProvider, &ctx.get_allocator()); + if (OB_ISNULL(provider)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + ObSqcCtx &sqc_ctx = ctx.get_sqc_handler()->get_sqc_ctx(); + if (OB_FAIL(sqc_ctx.add_whole_msg_provider(get_id(), dtl::DH_SP_WINFUNC_PX_WHOLE_MSG, + *provider))) { + LOG_WARN("add whole msg provider failed", K(ret)); + } + } + } + } else if (range_dist_parallel_) { + RDWinFuncPXWholeMsg::WholeMsgProvider *provider = OB_NEWx(RDWinFuncPXWholeMsg::WholeMsgProvider, &ctx.get_allocator()); + if (OB_ISNULL(provider)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_ISNULL(ctx.get_sqc_handler())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null sqc handler", K(ret)); + } else if (OB_FAIL(ctx.get_sqc_handler()->get_sqc_ctx().add_whole_msg_provider( + get_id(), dtl::DH_RD_WINFUNC_PX_WHOLE_MSG, *provider))) { + LOG_WARN("add whole msg failed", K(ret)); + } + } else if (is_participator()) { + if (OB_ISNULL(ctx.get_sqc_handler())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null sqc handler", K(ret)); + } else { + ObReportingWFWholeMsg::WholeMsgProvider *provider = + OB_NEWx(ObReportingWFWholeMsg::WholeMsgProvider, &ctx.get_allocator()); + if (OB_ISNULL(provider)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + ObSqcCtx &sqc_ctx = ctx.get_sqc_handler()->get_sqc_ctx(); + if (OB_FAIL(sqc_ctx.add_whole_msg_provider( + get_id(), dtl::DH_SECOND_STAGE_REPORTING_WF_WHOLE_MSG, *provider))) { + LOG_WARN("add whole msg failed", K(ret)); + } + } + } + } + return ret; +} + +int ObWindowFunctionVecSpec::rd_sort_cmp(RowMeta &row_meta, const ObCompactRow *l_row, + const ObCompactRow *r_row, const int64_t begin, + const int64_t end, int &cmp_ret) const +{ + int ret = OB_SUCCESS; + cmp_ret = 0; + const char *l_payload = nullptr, *r_payload = nullptr; + int32_t l_len = 0, r_len = 0; + bool l_isnull = false, r_isnull = false; + if (l_row == nullptr && r_row == nullptr) { + cmp_ret = 0; + } else if (l_row == nullptr) { + cmp_ret = 1; + } else if (r_row == nullptr) { + cmp_ret = -1; + } else { + for (int64_t i = begin; cmp_ret == 0 && OB_SUCC(ret) && i < end; i++) { + ObObjMeta &obj_meta = rd_coord_exprs_.at(i)->obj_meta_; + l_row->get_cell_payload(row_meta, i, l_payload, l_len); + l_isnull = l_row->is_null(i); + r_row->get_cell_payload(row_meta, i, r_payload, r_len); + r_isnull = r_row->is_null(i); + if (OB_FAIL(rd_sort_cmp_funcs_.at(i).row_cmp_func_(obj_meta, obj_meta, l_payload, l_len, + l_isnull, r_payload, r_len, r_isnull, + cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if(!rd_sort_collations_.at(i).is_ascending_) { + cmp_ret = cmp_ret * (-1); + } + } + } + return ret; +} + +struct __pby_oby_sort_op +{ + __pby_oby_sort_op(const ObWindowFunctionVecSpec &spec): spec_(spec) {} + // sort by (PBY, OBY, SQC_ID, THREAD_ID) + OB_INLINE int operator()(RDWinFuncPXPartialInfo *l, RDWinFuncPXPartialInfo *r) + { + int cmp_ret = 0; + (void)spec_.rd_pby_oby_cmp(l->row_meta_, l->first_row_, r->first_row_, cmp_ret); + if (cmp_ret == 0) { + (void)spec_.rd_pby_oby_cmp(l->row_meta_, l->last_row_, r->last_row_, cmp_ret); + } + return (cmp_ret == 0) ? + (std::tie(l->sqc_id_, l->thread_id_) < std::tie(r->sqc_id_, r->thread_id_)) : + (cmp_ret < 0); + }; +private: + const ObWindowFunctionVecSpec &spec_; +}; + +int ObWindowFunctionVecSpec::rd_generate_patch(RDWinFuncPXPieceMsgCtx &msg_ctx, ObEvalCtx &eval_ctx) const +{ + int ret = OB_SUCCESS; + std::sort(msg_ctx.infos_.begin(), msg_ctx.infos_.end(), __pby_oby_sort_op(*this)); +#ifndef NDEBUG + for (int i = 0; i < msg_ctx.infos_.count(); i++) { + RDWinFuncPXPartialInfo *info = msg_ctx.infos_.at(i); + if (info->first_row_ == nullptr) { break; } + CompactRow2STR first_row(info->row_meta_, *info->first_row_, &rd_coord_exprs_); + CompactRow2STR last_row(info->row_meta_, *info->last_row_, &rd_coord_exprs_); + int64_t first_row_extra = *reinterpret_cast(info->first_row_->get_extra_payload(info->row_meta_)); + int64_t last_row_extra = *reinterpret_cast(info->last_row_->get_extra_payload(info->row_meta_)); + LOG_INFO("generating patch", K(i), K(first_row), K(last_row), K(first_row_extra), K(last_row_extra)); + } +#endif + // first generate frame offset + RDWinFuncPXPartialInfo *prev = nullptr; + int cmp_ret = 0; + for (int i = 0; OB_SUCC(ret) && i < msg_ctx.infos_.count(); i++) { + RDWinFuncPXPartialInfo *cur = msg_ctx.infos_.at(i); + if (cur->first_row_ == NULL) { + break; + } + bool prev_same_part = (nullptr != prev); + if (prev_same_part) { + if (OB_FAIL(rd_pby_cmp(cur->row_meta_, prev->last_row_, cur->first_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else { + prev_same_part = (cmp_ret == 0); + } + } + if (OB_FAIL(ret)) { + } else if (prev_same_part) { + cur->first_row_frame_offset() = prev->last_row_frame_offset() + 1; + if (OB_FAIL(rd_pby_cmp(cur->row_meta_, cur->first_row_, cur->last_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret == 0) { + cur->last_row_frame_offset() += prev->last_row_frame_offset() + 1; + } + } + prev = cur; + } // end for + + // second: generate patch info for each window function + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx); + // indexes: + // 0: first_row's patch + // 1: last_row's patch + // 2: first_row's ranking after patching + // 3: last_row's ranking after patching + guard.set_batch_size(4); + const char *payload = nullptr; + int32_t len = 0; + bool null_payload; + using patch_pair = std::pair; + + ObSEArray patch_pairs; + LastCompactRow first_row_patch(msg_ctx.arena_alloc_); + LastCompactRow last_row_patch(msg_ctx.arena_alloc_); + __PartialResult part_res(eval_ctx, msg_ctx.arena_alloc_); + // use uniform/uniform_const format as data format + // PX Coordinator may not supported vectorization 2.0, + // in this case, if vector headers are initilized as default formats (discrete/fixed_length formats) + // but expr data are filled with datums, unexpected errors will happen. + for (int i = 0; OB_SUCC(ret) && i < rd_coord_exprs_.count(); i++) { + VectorFormat default_fmt = rd_coord_exprs_.at(i)->get_default_res_format(); + if (OB_FAIL(rd_coord_exprs_.at(i)->init_vector_for_write( + eval_ctx, rd_coord_exprs_.at(i)->is_const_expr() ? VEC_UNIFORM_CONST : VEC_UNIFORM, 4))) { + LOG_WARN("init vector failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(first_row_patch.init_row_meta(rd_coord_exprs_, sizeof(int64_t), false))) { + LOG_WARN("init row meta failed", K(ret)); + } else if (OB_FAIL(last_row_patch.init_row_meta(rd_coord_exprs_, sizeof(int64_t), false))) { + LOG_WARN("init row meta failed", K(ret)); + } + } + for (int idx = 0; OB_SUCC(ret) && idx < msg_ctx.infos_.count(); idx++) { + RDWinFuncPXPartialInfo *cur = msg_ctx.infos_.at(idx); + if (cur->first_row_ == nullptr) { + break; + } + guard.set_batch_idx(0); + if (OB_FAIL(first_row_patch.save_store_row(*cur->first_row_))) { + LOG_WARN("save first row failed", K(ret)); + } else if (OB_FAIL(first_row_patch.to_expr(rd_coord_exprs_, eval_ctx))) { + LOG_WARN("to expr failed", K(ret)); + } else if (OB_FAIL(last_row_patch.save_store_row(*cur->last_row_))) { + LOG_WARN("save last row failed", K(ret)); + } else if (FALSE_IT(guard.set_batch_idx(1))) { + } else if (OB_FAIL(last_row_patch.to_expr(rd_coord_exprs_, eval_ctx))) { + LOG_WARN("to expr failed", K(ret)); + } else {// do nothing + } + for (int i = 0; OB_SUCC(ret) && i < rd_wfs_.count(); i++) { + const WinFuncInfo &wf_info = wf_infos_.at(rd_wfs_.at(i)); + const bool is_rank = (wf_info.func_type_ == T_WIN_FUN_RANK); + const bool is_dense_rank = (wf_info.func_type_ == T_WIN_FUN_DENSE_RANK); + const bool is_range_frame = (wf_info.win_type_ == WINDOW_RANGE); + int64_t res_idx = i + rd_sort_collations_.count(); + ObExpr *patch_expr = rd_coord_exprs_.at(res_idx); + if (FALSE_IT(patch_expr->get_vector(eval_ctx)->set_null(0))) { + } else if (FALSE_IT(patch_expr->get_vector(eval_ctx)->set_null(1))) { + // do nothing + } else if (is_rank || is_dense_rank) { + prev = nullptr; + int64_t prev_idx = idx - 1; + bool prev_same_part = (prev_idx >= 0); + if (prev_same_part) { + prev = msg_ctx.infos_.at(prev_idx); + if (OB_FAIL(rd_pby_cmp(cur->row_meta_, prev->last_row_, cur->first_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else { + prev_same_part = (cmp_ret == 0); + } + } + // patch first_row + guard.set_batch_idx(0); + if (OB_FAIL(ret)) { + } else if (!prev_same_part) { + // do nothing + } else if (OB_FAIL(rd_oby_cmp(cur->row_meta_, prev->last_row_, cur->first_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret == 0) { // prev same order + // prev last row + patch_expr->get_vector(eval_ctx)->get_payload(3, payload, len); + ObCompactRow *prev_last_row_patch = patch_pairs.at(prev_idx).second; + if (OB_FAIL(part_res.add_rank(wf_info, false, payload, len, -1))) { + LOG_WARN("add rank failed", K(ret)); + } + } else if (is_rank + && OB_FAIL(part_res.add_rank(wf_info, true, nullptr, 0, + cur->first_row_frame_offset()))) { + LOG_WARN("add rank failed", K(ret)); + } else if (is_dense_rank) { + patch_expr->get_vector(eval_ctx)->get_payload(3, payload, len); + if (OB_FAIL(part_res.add_rank(wf_info, false, payload, len, 0))) { + LOG_WARN("add rank failed", K(ret)); + } + } + // if first_row & last_row in different partition, patch is not needed for last_row + // if first_row & last_row in same partition, patch_first will patch value into last row, no need patching for last row as well + + // store rank results + patch_expr->get_vector(eval_ctx)->set_null(2); + patch_expr->get_vector(eval_ctx)->set_null(3); + ObIVector *patch = patch_expr->get_vector(eval_ctx); + guard.set_batch_idx(2); + if (FALSE_IT(cur->first_row_->get_cell_payload(cur->row_meta_, res_idx, payload, len))) { + } else if (OB_FAIL(part_res.add_rank(wf_info, cur->first_row_->is_null(res_idx), + payload, len, 0))) { + LOG_WARN("add rank failed", K(ret)); + } else if (FALSE_IT(patch->get_payload(0, payload, len))) { + } else if (OB_FAIL( + part_res.add_rank(wf_info, patch->is_null(0), payload, len, 0))) { + LOG_WARN("add rank failed", K(ret)); + } + guard.set_batch_idx(3); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(rd_pby_cmp(cur->row_meta_, cur->first_row_, cur->last_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret == 0) { + // first_row & last_row have same order, add first_row's patch into last_row + // else add first row's frame_offset into last_row + if (OB_FAIL(rd_oby_cmp(cur->row_meta_, cur->first_row_, cur->last_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret == 0 || is_dense_rank) { + if (OB_FAIL( + part_res.add_rank(wf_info, patch->is_null(0), payload, len, 0))) { + LOG_WARN("add rank failed", K(ret)); + } + } else if (cmp_ret != 0 + && OB_FAIL(part_res.add_rank(wf_info, true, nullptr, 0, + cur->first_row_frame_offset()))) { + LOG_WARN("add rank failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + LOG_WARN("add rank failed", K(ret)); + } else if (FALSE_IT(cur->get_cell(res_idx, false, payload, len))) { + } else if (OB_FAIL(part_res.add_rank(wf_info, cur->is_null(res_idx, false), + payload, len, 0))) { + LOG_WARN("add rank failed", K(ret)); + } + } else { // aggregation function + cmp_ret = 0; + // coordinator will patch partial results in following steps: + // 1. for [0...part_cnt], patch first_row's patch into each rows in partition + // 2. for [0...part_cnt], patch last_row's patch into rows which have same order as last_row in partition + + // hence, first_row's patch is sum of previous partial results with same partition + // last_row's patch is sum of following partial results with same partition and same order + + // first row's patch + guard.set_batch_idx(0); + for (int prev_idx = idx - 1; cmp_ret == 0 && OB_SUCC(ret) && prev_idx >= 0; prev_idx--) { + RDWinFuncPXPartialInfo *prev = msg_ctx.infos_.at(prev_idx); + if (OB_FAIL(rd_pby_cmp(cur->row_meta_, prev->last_row_, cur->first_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret == 0) { + prev->get_cell(res_idx, false, payload, len); + null_payload = prev->is_null(res_idx, false); + if (OB_FAIL(part_res.merge(wf_info, null_payload, payload, len))) { + LOG_WARN("merge result failed", K(ret)); + } + } + } // end for + + // last row's patch + guard.set_batch_idx(1); + cmp_ret = 0; + for (int post_idx = idx + 1; + is_range_frame && cmp_ret == 0 && OB_SUCC(ret) && post_idx < msg_ctx.infos_.count(); + post_idx++) { + RDWinFuncPXPartialInfo *post = msg_ctx.infos_.at(post_idx); + if (OB_FAIL(rd_pby_oby_cmp(cur->row_meta_, post->first_row_, cur->last_row_, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret == 0) { + post->get_cell(res_idx, true, payload, len); + null_payload = post->is_null(res_idx, true); + if (OB_FAIL(part_res.merge(wf_info, null_payload, payload, len))) { + LOG_WARN("merge result failed", K(ret)); + } + } + } + } + if (OB_SUCC(ret)) { + patch_expr->set_evaluated_projected(eval_ctx); + } + } // end iter of wf_infos + if (OB_SUCC(ret)) { + int64_t mock_skip = 0; + ObBatchRows tmp_brs; + tmp_brs.size_ = 2; + tmp_brs.skip_ = to_bit_vector(&mock_skip); + tmp_brs.end_ = false; + guard.set_batch_idx(0); + patch_pair tmp_pair; + if (OB_FAIL(first_row_patch.save_store_row(rd_coord_exprs_, tmp_brs, eval_ctx, sizeof(int64_t), false))) { + LOG_WARN("save store row failed", K(ret)); + } else if (FALSE_IT(guard.set_batch_idx(1))) { + } else if (OB_FAIL( + last_row_patch.save_store_row(rd_coord_exprs_, tmp_brs, eval_ctx, sizeof(int64_t), false))) { + LOG_WARN("save store row failed", K(ret)); + } else { + *reinterpret_cast(first_row_patch.compact_row_->get_extra_payload( + cur->row_meta_)) = cur->first_row_frame_offset(); + *reinterpret_cast(last_row_patch.compact_row_->get_extra_payload( + cur->row_meta_)) = cur->last_row_frame_offset(); + int32_t buf_size = first_row_patch.compact_row_->get_row_size() + + last_row_patch.compact_row_->get_row_size(); + char *buf = (char *)msg_ctx.arena_alloc_.alloc(buf_size); + if (OB_ISNULL(buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(buf, first_row_patch.compact_row_, first_row_patch.compact_row_->get_row_size()); + tmp_pair.first = reinterpret_cast(buf); + buf += first_row_patch.compact_row_->get_row_size(); + MEMCPY(buf, last_row_patch.compact_row_, last_row_patch.compact_row_->get_row_size()); + tmp_pair.second = reinterpret_cast(buf); + if (OB_FAIL(patch_pairs.push_back(tmp_pair))) { LOG_WARN("push back failed", K(ret)); } + } + } + } + } + for (int i = 0; OB_SUCC(ret) && i < msg_ctx.infos_.count(); i++) { + if (i >= patch_pairs.count()) { + msg_ctx.infos_.at(i)->first_row_ = nullptr; + msg_ctx.infos_.at(i)->last_row_ = nullptr; + } else { + msg_ctx.infos_.at(i)->first_row_ = patch_pairs.at(i).first; + msg_ctx.infos_.at(i)->last_row_ = patch_pairs.at(i).second; + } + } + return ret; +} + +// ================ ObWindowFunctionVecOpInput +int ObWindowFunctionVecOpInput::init_wf_participator_shared_info(ObIAllocator &alloc, int64_t task_cnt) +{ + int ret = OB_SUCCESS; + ObWFParticipatorSharedInfo *shared_info = nullptr; + if (OB_ISNULL(shared_info = reinterpret_cast( + alloc.alloc(sizeof(ObWFParticipatorSharedInfo))))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + wf_participator_shared_info_ = reinterpret_cast(shared_info); + shared_info->sqc_thread_count_ = task_cnt; + shared_info->process_cnt_ = 0; + shared_info->ret_ = OB_SUCCESS; + new (&shared_info->cond_)common::SimpleCond(common::ObWaitEventIds::SQL_WF_PARTICIPATOR_COND_WAIT); + new (&shared_info->lock_)ObSpinLock(common::ObLatchIds::SQL_WF_PARTICIPATOR_COND_LOCK); + } + return ret; +} + +int ObWindowFunctionVecOpInput::sync_wait(ObExecContext &ctx, ObReportingWFWholeMsg::WholeMsgProvider *msg_provider) +{ + int ret = OB_SUCCESS; + ObWFParticipatorSharedInfo *shared_info = + reinterpret_cast(wf_participator_shared_info_); + if (OB_ISNULL(shared_info) || OB_ISNULL(msg_provider)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null arguments", K(ret), K(shared_info), K(msg_provider)); + } else { + int64_t exit_cnt = shared_info->sqc_thread_count_; + int64_t &sync_cnt = shared_info->process_cnt_; + bool synced_once = false; + int64_t loop = 0; + while (OB_SUCC(ret)) { + ++loop; + if (!synced_once) { + ObSpinLockGuard guard(shared_info->lock_); + synced_once = true; + if (0 == ATOMIC_AAF(&sync_cnt, 1) % exit_cnt) { + shared_info->cond_.signal(); + LOG_DEBUG("debug sync_cnt", K(ret), K(sync_cnt), K(lbt())); + break; + } + } + int tmp_ret = ATOMIC_LOAD(&shared_info->ret_); + if (OB_FAIL(tmp_ret)) { + ret = tmp_ret; + } else if (0 == loop % 8 && OB_UNLIKELY(IS_INTERRUPTED())) { + ObInterruptCode code = GET_INTERRUPT_CODE(); + ret = code.code_; // overwrite ret + LOG_WARN("received a interrupt", K(code), K(ret)); + } else if (0 == loop % 16 && OB_FAIL(ctx.fast_check_status())) { + LOG_WARN("failed to check status", K(ret)); + } else if (0 == ATOMIC_LOAD(&sync_cnt) % exit_cnt) { + LOG_DEBUG("debug sunc_cnt", K(ret), K(sync_cnt), K(loop), K(exit_cnt), K(lbt())); + break; + } else { + uint32_t key = shared_info->cond_.get_key(); + shared_info->cond_.wait(key, 1000); // wait for 1000 us per loop + } + } // end while + } + return ret; +} + +} // end sql +} // end oceanbase \ No newline at end of file diff --git a/src/sql/engine/window_function/ob_window_function_vec_op.h b/src/sql/engine/window_function/ob_window_function_vec_op.h new file mode 100644 index 0000000000..f40fb88b4d --- /dev/null +++ b/src/sql/engine/window_function/ob_window_function_vec_op.h @@ -0,0 +1,545 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_WINDOW_FUNCTION_VEC_OP_H_ +#define OCEANBASE_WINDOW_FUNCTION_VEC_OP_H_ + +#include "lib/container/ob_se_array.h" +#include "sql/engine/ob_operator.h" +#include "sql/engine/window_function/row_store.h" +#include "sql/engine/window_function/win_expr.h" +#include "sql/engine/basic/ob_vector_result_holder.h" +#include "sql/engine/window_function/ob_window_function_op.h" +#include "sql/engine/px/datahub/components/ob_dh_second_stage_reporting_wf.h" + +namespace oceanbase +{ +namespace sql +{ +namespace winfunc +{ + template + class WinExprWrapper; +} // end winfunc + +class ObWindowFunctionVecOp; +class SPWinFuncPXWholeMsg; + +class RDWinFuncPXPieceMsgCtx; +class RDWinFuncPXPartialInfo; +// copy from ObWindowFunctionSpec +class ObWindowFunctionVecSpec: public ObWindowFunctionSpec +{ + OB_UNIS_VERSION_V(1); +public: + ObWindowFunctionVecSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type) : + ObWindowFunctionSpec(alloc, type) + {} + + virtual int register_to_datahub(ObExecContext &ctx) const override; + + int rd_generate_patch(RDWinFuncPXPieceMsgCtx &msg_ctx, ObEvalCtx &eval_ctx) const; + + int rd_sort_cmp(RowMeta &row_meta, const ObCompactRow *l_row, const ObCompactRow *r_row, const int64_t begin, + const int64_t end, int &cmp_ret) const; + + int rd_pby_oby_cmp(RowMeta &row_meta, const ObCompactRow *l_row, const ObCompactRow *r_row, + int &cmp_ret) const + { + return rd_sort_cmp(row_meta, l_row, r_row, 0, rd_sort_collations_.count(), cmp_ret); + } + + int rd_pby_cmp(RowMeta &row_meta, const ObCompactRow *l_row, const ObCompactRow *r_row, + int &cmp_ret) const + { + return rd_sort_cmp(row_meta, l_row, r_row, 0, rd_pby_sort_cnt_, cmp_ret); + } + + int rd_oby_cmp(RowMeta &row_meta, const ObCompactRow *l_row, const ObCompactRow *r_row, int &cmp_ret) const + { + return rd_sort_cmp(row_meta, l_row, r_row, rd_pby_sort_cnt_, rd_sort_collations_.count(), + cmp_ret); + } +private: + DISALLOW_COPY_AND_ASSIGN(ObWindowFunctionVecSpec); +}; + +class WinFuncColExpr : public common::ObDLinkBase +{ +public: + WinFuncColExpr(WinFuncInfo &wf_info, ObWindowFunctionVecOp &op, const int64_t wf_idx) : + wf_info_(wf_info), op_(op), wf_idx_(wf_idx), part_first_row_idx_(-1), wf_expr_(nullptr), + res_(nullptr), pby_row_mapped_idxes_(nullptr), reordered_pby_row_idx_(nullptr), wf_res_row_meta_(), + res_rows_(nullptr), agg_ctx_(nullptr), aggr_rows_(nullptr), non_aggr_results_(nullptr), + null_nonaggr_results_(nullptr) + {} + void destroy() { reset(); } + void reset(); + void reset_for_scan(); + ~WinFuncColExpr() { destroy(); } + // init agg_ctx_ & alloc aggr_rows_ + int init_aggregate_ctx(const int64_t tenant_id); + int init_non_aggregate_ctx(); + int init_res_rows(const int64_t tenant_id); + int32_t non_aggr_reserved_row_size() const; + // need reset agg_ctx.allocator_ + int reset_for_partition(const int64_t batch_size, const ObBitVector &skip); + WinFuncInfo &wf_info_; + ObWindowFunctionVecOp &op_; + int64_t wf_idx_; + int64_t part_first_row_idx_; + // LastCompactRow pby_row_; + winfunc::IWinExpr *wf_expr_; + // results of window function + winfunc::RowStores *res_; + int32_t *pby_row_mapped_idxes_; + int32_t *reordered_pby_row_idx_; + RowMeta wf_res_row_meta_; + const ObCompactRow **res_rows_; // tmp results rows pointers + // only valid for aggregate functions + aggregate::RuntimeContext *agg_ctx_; + aggregate::AggrRowPtr *aggr_rows_; + // only valid for non-aggregate functions + char *non_aggr_results_; + ObBitVector *null_nonaggr_results_; + + TO_STRING_KV(K_(wf_info), K_(wf_idx), K_(part_first_row_idx)); +}; + +// copy from ObWindowFunctionOpInput +class ObWindowFunctionVecOpInput : public ObOpInput +{ + OB_UNIS_VERSION_V(1); +public: + ObWindowFunctionVecOpInput(ObExecContext &ctx, const ObOpSpec &spec) : + ObOpInput(ctx, spec), local_task_count_(1), total_task_count_(1), + wf_participator_shared_info_(0) + {} + virtual int init(ObTaskInfo &task_info) override + { + int ret = OB_SUCCESS; + return ret; + } + + virtual void reset() override + { + local_task_count_ = 1; + total_task_count_ = 1; + } + + int64_t get_total_task_count() const { return total_task_count_; } + + void set_total_task_count(int64_t total_count) { total_task_count_ = total_count; } + + void set_local_task_count(int64_t task_count) { local_task_count_ = task_count; } + + int64_t get_local_task_count() const { return local_task_count_; } + + ObWFParticipatorSharedInfo *get_wf_participator_shared_info() + { + return reinterpret_cast(wf_participator_shared_info_); + } + + int64_t &get_sqc_thread_count() + { + return get_wf_participator_shared_info()->sqc_thread_count_; + } + + int64_t &get_process_cnt() + { + return get_wf_participator_shared_info()->process_cnt_; + } + + void set_error_code(int ret_code) + { + ATOMIC_SET(&(get_wf_participator_shared_info()->ret_), ret_code); + } + + int init_wf_participator_shared_info(ObIAllocator &alloc, int64_t task_cnt); + + int sync_wait(ObExecContext &ctx, ObReportingWFWholeMsg::WholeMsgProvider *msg_provider); +private: + DISALLOW_COPY_AND_ASSIGN(ObWindowFunctionVecOpInput); + +public: + int64_t local_task_count_; + int64_t total_task_count_; + uint64_t wf_participator_shared_info_; +}; + +class ObWindowFunctionVecOp: public ObOperator +{ +private: + using WinFuncColExprList = common::ObDList; + using PbyHashValueArray = common::ObSArray; +private: + enum class ProcessStatus + { + PARTIAL, + COORDINATE, + FINAL + }; + +public: + ObWindowFunctionVecOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input): + ObOperator(exec_ctx, spec, input), + local_allocator_(nullptr), + stat_(ProcessStatus::PARTIAL), + input_stores_(), + wf_list_(), + all_expr_vector_copy_(), + backuped_size_(0), + last_output_row_idx_(common::OB_INVALID_INDEX), + child_iter_end_(false), + iter_end_(false), + dir_id_(-1), + first_part_saved_(false), + last_part_saved_(false), + rd_patch_(nullptr), + first_part_outputed_(false), + patch_first_(false), + patch_last_(false), + last_computed_part_rows_(0), + rd_coord_row_meta_(nullptr), + last_aggr_status_(0), + pby_set_count_(0), + next_wf_pby_expr_cnt_to_transmit_(common::OB_INVALID_COUNT), + pby_expr_cnt_idx_array_(), + pby_hash_values_(), + participator_whole_msg_array_(), + pby_hash_values_sets_(), + input_row_meta_(), + max_pby_col_cnt_(0), + pby_row_mapped_idx_arr_(nullptr), + last_row_idx_arr_(nullptr), + all_part_exprs_(), + batch_ctx_(), + sp_merged_row_(nullptr), + all_wf_res_row_meta_(nullptr), + mem_context_(nullptr), + profile_(ObSqlWorkAreaType::HASH_WORK_AREA), + sql_mem_processor_(profile_, op_monitor_info_), + global_mem_limit_version_(0), + amm_periodic_cnt_(0), + store_it_age_() + {} + + virtual ~ObWindowFunctionVecOp() {} + virtual int inner_open() override; + virtual int inner_close() override; + virtual int inner_rescan() override; + virtual int inner_get_next_batch(const int64_t max_row_cnt) override; + virtual int inner_get_next_row() override + { + return OB_NOT_IMPLEMENT; + } + virtual void destroy() override; +public: + int get_part_end_idx() { return input_stores_.cur_->count(); } + const RowMeta &get_input_row_meta() const { return input_row_meta_; } + inline ObExprPtrIArray &get_all_expr() + { + return const_cast(MY_SPEC.all_expr_); + } + + static bool all_supported_winfuncs(const ObIArray &win_exprs); +private: + struct cell_info + { + bool is_null_; + int32_t len_; + const char *payload_; + cell_info(bool is_null, int32_t len, const char *payload) : + is_null_(is_null), len_(len), payload_(payload) + {} + cell_info(): is_null_(true), len_(0), payload_(nullptr) {} + TO_STRING_KV(K_(is_null), K_(len), KP_(payload)); + }; + int init(); + + int create_stores(const int64_t tenant_id); + + void reset_stores(); + + void destroy_stores(); + + int reset_for_scan(const int64_t tenant_id); + + int build_pby_hash_values_for_transmit(); + + int build_participator_whole_msg_array(); + + int setup_participator_pby_hash_sets(WFInfoFixedArray &wf_infos, + ObWindowFunctionVecOpInput *op_input); + + int get_next_batch_from_child(int64_t batch_size, const ObBatchRows *&child_brs); + + int mapping_pby_row_to_idx_arr(const ObBatchRows &child_brs, const ObCompactRow *last_row); + template + int mapping_pby_col_to_idx_arr(int32_t col_id, const ObExpr &part_expr, const ObBatchRows &brs, + const cell_info *last_part_res); + int eval_prev_part_exprs(const ObCompactRow *last_row, ObIAllocator &alloc, + const ObExprPtrIArray &part_exprs, + common::ObIArray &last_part_infos); + + OB_INLINE int save_pby_row_for_wf(WinFuncColExpr *end_wf, const int64_t batch_idx); + + int detect_and_report_aggr_status(const ObBatchRows &child_brs, const int64_t start_idx, + const int64_t end_idx); + + OB_INLINE int update_part_first_row_idx(WinFuncColExpr *end); + + int partial_next_batch(const int64_t max_row_cnt); + + int do_partial_next_batch(const int64_t max_row_cnt, bool &do_output); + + int coordinate(); + + int final_next_batch(const int64_t max_row_cnt); + + int get_next_partition(int64_t &check_times); + + int output_batch_rows(const int64_t output_row_cnt); + + int add_aggr_res_row_for_participator(WinFuncColExpr *end, winfunc::RowStore &store); + + int compute_wf_values(WinFuncColExpr *end, int64_t &check_times); + + int set_null_results_of_wf(WinFuncColExpr &wf, const int64_t batch_size, + const ObBitVector &nullres_skip); + + int calc_bypass_pushdown_rows_of_wf(WinFuncColExpr &wf, const int64_t batch_size, + const ObBitVector &pushdown_skip); + + int process_child_batch(const int64_t batch_idx, const ObBatchRows *kchild_brs, + int64_t &check_times); + + int check_same_partition(WinFuncColExpr &wf_col, bool &same); + + int find_same_partition_of_wf(WinFuncColExpr *&end_wf); + + int detect_nullres_or_pushdown_rows(WinFuncColExpr &wf, ObBitVector &nullres_skip, + ObBitVector &pushdown_skip, ObBitVector &wf_skip); + + // output rows stored in input_stores_ + int output_stored_rows(const int64_t out_processed_cnt, const int64_t out_cur_cnt, + winfunc::RowStores &store, int64_t &output_cnt); + + int output_stored_rows(const int64_t out_processed_cnt, const int64_t out_cur_cnt, + WinFuncColExpr &wf_col, int64_t &outputed_cnt); + + int attach_rows_to_output(const ObCompactRow **rows, int64_t row_cnt); + + int attach_row_to_output(const ObCompactRow *row); + + int output_wf_rows(WinFuncColExpr &wf, const ObCompactRow **stored_rows); + + int get_last_input_row_of_prev_batch(const ObCompactRow *&last_row); + + // for single partition parallel execution, collect partition aggr results and do merging. + int collect_sp_partial_results(); + + int collect_wf_res_row(const int64_t batch_idx, const int64_t stored_row_idx, ObCompactRow *&res_row); + + // send piece msg and wait for whole response + int sp_get_whole_msg(bool is_empty, SPWinFuncPXWholeMsg &msg, ObCompactRow *sending_row); + + int sp_merge_partial_results(SPWinFuncPXWholeMsg &msg); + + // rwf stands for reporting window function + int rwf_get_whole_msg(const PbyHashValueArray *hash_value_arr, ObReportingWFWholeMsg &who_msg); + + int rwf_participator_coordinate(const int64_t pushdown_wf_idx); + + int rwf_calc_pby_row_hash(const ObBatchRows &child_brs, const ObIArray &pby_exprs, + uint64_t &hash_value); + + int rwf_update_aggr_status_code(const int64_t start_idx, const int64_t end_idx); + + int rwf_send_empty_piece_data(); + + // rd stands for range distribution + int rd_fetch_patch(); + + typedef winfunc::RowStore* winfunc::RowStores::*StoreMemberPtr; + + int rd_build_partial_info_row(int64_t idx, bool is_first_part, + ObIAllocator &alloc, ObCompactRow *&build_row); + + int rd_output_final_batch(const int64_t max_row_cnt); + + int rd_apply_patches(const int64_t max_row_cnt); + + int rd_patch_result(const int64_t idx, bool patch_first, bool patch_last); + + int rd_find_first_row_upper_bound(int64_t batch_size, int64_t &upper_bound); + int rd_find_last_row_lower_bound(int64_t batch_size, int64_t &lower_bound); + + template + int rd_merge_result(PartialMerge &part_res, WinFuncInfo &info, int64_t rd_col_id, + int64_t first_row_same_order_upper_bound, + int64_t last_row_same_order_lower_bound); + + + + OB_INLINE int64_t next_nonskip_row_index(int64_t cur_idx, const ObBatchRows &brs); + + int init_batch_ctx(); + + int restore_child_vectors(); + + int backup_child_vectors(int64_t batch_size); + + inline bool need_dump() const + { + return sql_mem_processor_.get_data_size() + local_mem_used() > sql_mem_processor_.get_mem_bound(); + } + + int init_mem_context(); + + void destroy_mem_context(); + + int update_mem_limit_version_periodically(); + + inline double get_input_rows_mem_bound_ratio() const + { + return MY_SPEC.input_rows_mem_bound_ratio_; + } + int64_t local_mem_used() const + { + return local_allocator_->used(); + } +public: + struct OpBatchCtx { // values used to help batch-calculation + const ObCompactRow **stored_rows_; + ObBitVector *nullres_skip_; + ObBitVector *pushdown_skip_; + ObBitVector *calc_wf_skip_; + // used for window function evaluations + // or tmp expr evaluation + ObBitVector *bound_eval_skip_; + int64_t *upper_pos_arr_; + int64_t *lower_pos_arr_; + LastCompactRow *tmp_wf_res_row_; + LastCompactRow *tmp_input_row_; + void *all_exprs_backup_buf_; // memory for backup/restore during compute_wf_values + int32_t all_exprs_backup_buf_len_; + + OpBatchCtx() : + stored_rows_(nullptr), nullres_skip_(nullptr), pushdown_skip_(nullptr), + calc_wf_skip_(nullptr), bound_eval_skip_(nullptr), upper_pos_arr_(nullptr), + lower_pos_arr_(nullptr), tmp_wf_res_row_(nullptr), tmp_input_row_(nullptr), + all_exprs_backup_buf_(nullptr), all_exprs_backup_buf_len_(0) + {} + + void reset() + { + if (tmp_wf_res_row_ != nullptr) { + tmp_wf_res_row_->reset(); + } + if (tmp_input_row_ != nullptr) { + tmp_input_row_->reset(); + } + } + }; +public: + ObWindowFunctionVecOp::OpBatchCtx &get_batch_ctx() { return batch_ctx_; } +private: + friend class WinFuncColExpr; + template friend class winfunc::WinExprWrapper; + friend class winfunc::RowStores; + + friend class winfunc::StoreGuard; +private: + common::ObArenaAllocator *local_allocator_; + // this allocator will be reset in rescan + common::ObArenaAllocator rescan_alloc_; + ProcessStatus stat_; + + // only `cur_` is used for non-batch execution. + winfunc::RowStores input_stores_; + WinFuncColExprList wf_list_; + ObVectorsResultHolder all_expr_vector_copy_; + int64_t backuped_size_; + + int64_t last_output_row_idx_; // TODO: useless, remove this + bool child_iter_end_; + bool iter_end_; + int64_t dir_id_; + + // Members for range distribution parallel execution + // `rd`: is abbreviation for range distribution + bool first_part_saved_; + bool last_part_saved_; + RDWinFuncPXPartialInfo *rd_patch_; + common::ObArenaAllocator patch_alloc_; // TODO: maybe not used + + bool first_part_outputed_; + bool patch_first_; + bool patch_last_; + + int64_t last_computed_part_rows_; + RowMeta *rd_coord_row_meta_; + // row store iteration age to prevent output row datum released during the same batch + + // Members for reporting wf push down, use for pushdown paricipator transmit pieces to datahub begin + int64_t last_aggr_status_; // aggr_status of last input row for participator + + // Use for wf participator, the count of different pby_set in wf op + // ( pby1(c1, c2, c3), pby2(c1, c2), pby3(c1, c2), pby4(c1), pby5(c1)) is 3 + int64_t pby_set_count_; + + // next part expr count of pieces to send to datahub for wf pushdown participator + int64_t next_wf_pby_expr_cnt_to_transmit_; + // Use for wf participator, the idx of different pby_set of pushdown wf + // index of array : wf_idx - 1 (because wf_idx is start from 1) + // value of array : the idx of different pby_set of pushdown wf (value is -1 if isn't pushdown wf) + ObArray pby_expr_cnt_idx_array_; + // Use for wf participator, to transmit pieces to datahub + ObArray pby_hash_values_; + // Use to store msg recieved from datahub + ObArray participator_whole_msg_array_; + // Use to decide whether compute or bypass, generated from ObReportingWFWholeMsg + ObArray pby_hash_values_sets_; + // Members for reporting wf push down, use for pushdown paricipator transmit pieces to datahub end + + // row meta for input row + RowMeta input_row_meta_; + // used for mapping pby rows to idx array + int64_t max_pby_col_cnt_; + int32_t *pby_row_mapped_idx_arr_; + int32_t *last_row_idx_arr_; + ObSEArray all_part_exprs_; + + OpBatchCtx batch_ctx_; + + LastCompactRow *sp_merged_row_; // for single partition parallel execution + RowMeta *all_wf_res_row_meta_; + // for auto memory management + lib::MemoryContext mem_context_; + ObSqlWorkAreaProfile profile_; + ObSqlMemMgrProcessor sql_mem_processor_; + + // Each RowsStore may trigger a new mem_limit fetch, this records newest mem_limit version + // Synchronize this version to others to let them update the mem_limit + int64_t global_mem_limit_version_; + // Only increase, not decrease, used for update_max_available_mem_size_periodically + // Means the total count of rows which have been added to the each ra datum store + int64_t amm_periodic_cnt_; + + ObTempBlockStore::IterationAge store_it_age_; +}; + +} // end sql +} // end oceanbase + +#endif // OCEANBASE_WINDOW_FUNCTION_VEC_OP_H_ \ No newline at end of file diff --git a/src/sql/engine/window_function/row_store.cpp b/src/sql/engine/window_function/row_store.cpp new file mode 100644 index 0000000000..9e457472e0 --- /dev/null +++ b/src/sql/engine/window_function/row_store.cpp @@ -0,0 +1,297 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#define USING_LOG_PREFIX SQL_ENG + +#include "row_store.h" +#include "sql/engine/basic/ob_temp_row_store.h" +#include "sql/engine/window_function/ob_window_function_vec_op.h" + +namespace oceanbase +{ +namespace sql +{ + +namespace winfunc +{ + +StoreGuard::StoreGuard(ObWindowFunctionVecOp &op): op_(op) +{ + + op_.input_stores_.set_it_age(&op_.store_it_age_); + for (WinFuncColExpr *it = op_.wf_list_.get_first(); it != op_.wf_list_.get_header(); it = it->get_next()) { + it->res_->set_it_age(&op_.store_it_age_); + } + op_.store_it_age_.inc(); +} + +int RowStore::init(const int64_t max_batch_size, const RowMeta &row_meta, + const lib::ObMemAttr &mem_attr, const int64_t mem_limit, bool enable_dump) +{ + int ret = OB_SUCCESS; + void *ptr_buf = nullptr; + if (OB_FAIL(ra_rs_.init(row_meta, max_batch_size, mem_attr, mem_limit, enable_dump, NONE_COMPRESSOR))) { + LOG_WARN("init ra temp row store failed", K(ret)); + } else if (OB_FAIL(ra_reader_.init(&ra_rs_))) { + LOG_WARN("init ra reader failed", K(ret)); + } else if (OB_ISNULL(ptr_buf = allocator_->alloc(sizeof(ObCompactRow *) * max_batch_size))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + row_ptrs_ = static_cast(ptr_buf); + } + return ret; +} +int RowStore::add_batch_rows(const ObIArray &exprs, const RowMeta &row_meta, + ObEvalCtx &eval_ctx, const EvalBound &bound, const ObBitVector &skip, + bool add_row_cnt, ObCompactRow **stored_rows /*null*/, bool is_input) +{ + int ret = OB_SUCCESS; + bool need_addding = (skip.accumulate_bit_cnt(bound) < bound.range_size()); + if (OB_SUCC(ret) && need_addding) { + int64_t stored_row_cnt = 0; + int added_cnt = bound.range_size() - skip.accumulate_bit_cnt(bound); + if (OB_FAIL(ra_rs_.add_batch(exprs, eval_ctx, bound, skip, stored_row_cnt, + (stored_rows == nullptr ? row_ptrs_ : stored_rows)))) { + SQL_ENG_LOG(WARN, "add batch rows failed", K(ret)); + } else if (OB_UNLIKELY(stored_row_cnt != added_cnt)) { + ret = OB_ERR_UNEXPECTED; + SQL_ENG_LOG(WARN, "unexpected added row count", K(stored_row_cnt), K(added_cnt)); + } else if (FALSE_IT(stored_row_cnt_ += added_cnt)) { + // do nothing + } else if (add_row_cnt) { + row_cnt_ += added_cnt; + } + // process dump after adding batch rows + if (OB_SUCC(ret) && OB_FAIL(store_set_.process_dump(is_input))) { + LOG_WARN("process dump failed", K(ret)); + } + } + return ret; +} + +int RowStore::get_batch_rows(const int64_t start_idx, const int64_t end_idx, + const ObCompactRow **stored_rows) +{ + int ret = OB_SUCCESS; + int64_t read_rows = 0; + if (OB_UNLIKELY(end_idx - start_idx == 0)) { + //do nothing + } else if (OB_FAIL(ra_reader_.get_batch_rows(start_idx, end_idx, read_rows, stored_rows))) { + SQL_ENG_LOG(WARN, "get batch rows failed", K(ret), K(start_idx), K(end_idx)); + } else if (OB_UNLIKELY(read_rows != end_idx - start_idx)) { + SQL_ENG_LOG(WARN, "unexpected read rows", K(ret), K(read_rows), K(end_idx), K(start_idx)); + } + return ret; +} + +int RowStore::attach_rows(const ObIArray &exprs, const RowMeta &row_meta, + ObEvalCtx &eval_ctx, const int64_t start_idx, const int64_t end_idx, + bool use_reserve_buf) +{ + int ret = OB_SUCCESS; + ObSEArray vec_ptrs; + if (OB_UNLIKELY(end_idx - start_idx == 0)) { + // do nothing + } else { + const ObCompactRow **stored_rows = const_cast(row_ptrs_); + if (OB_FAIL(get_batch_rows(start_idx, end_idx, stored_rows))) { + LOG_WARN("get batch rows failed", K(ret)); + } + for (int i = 0; OB_SUCC(ret) && i < exprs.count(); i++) { + if (exprs.at(i)->is_const_expr()) { // do nothing + } else if (OB_FAIL(exprs.at(i)->init_vector(eval_ctx, exprs.at(i)->get_default_res_format(), + end_idx - start_idx, use_reserve_buf))) { + LOG_WARN("init vector for write failed", K(ret)); + } else if (OB_UNLIKELY(!is_valid_format(exprs.at(i)->get_format(eval_ctx)))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid format", K(ret)); + } else if (OB_FAIL(exprs.at(i)->get_vector(eval_ctx)->from_rows(row_meta, stored_rows, + end_idx - start_idx, i))) { + LOG_WARN("from rows failed", K(ret)); + } + } + // set evaluated and projected after attaching rows + for (int i = 0; OB_SUCC(ret) && i < exprs.count(); i++) { + exprs.at(i)->set_evaluated_projected(eval_ctx); + } + } + return ret; +} + +int RowStore::attach_rows(ObExpr *expr, const RowMeta &row_meta, ObEvalCtx &eval_ctx, + const int64_t start_idx, const int64_t end_idx, bool use_reserve_buf) +{ + int ret = OB_SUCCESS; + ObSEArray tmp_exprs; + if (OB_FAIL(tmp_exprs.push_back(expr))) { + LOG_WARN("push back element failed", K(ret)); + } else if (OB_FAIL( + attach_rows(tmp_exprs, row_meta, eval_ctx, start_idx, end_idx, use_reserve_buf))) { + LOG_WARN("attach rows failed", K(ret)); + } + return ret; +} + +int RowStore::process_dump(const int64_t target_size, const int64_t g_mem_limit_versiom, + int64_t &dumped_size) +{ + int ret = OB_SUCCESS; + local_mem_limit_version_ = g_mem_limit_versiom; + int64_t mem_used = ra_rs_.get_mem_used(); + dumped_size = 0; + if (ra_rs_.is_empty_save_block_cnt()) { + LOG_DEBUG("no need dumping"); // CHANGE TO DEBUG + } else if (OB_FAIL(ra_rs_.dump(false, target_size))) { + LOG_WARN("dump store failed", K(ret)); + } else { + dumped_size = mem_used - ra_rs_.get_mem_used(); + // reset reader after dumping + ra_reader_.reset(); + LOG_TRACE("dumped store", K(*this), K(dumped_size), K(local_mem_limit_version_), K(target_size)); + } + return ret; +} + +// int RowStore::add_row(const ObCompactRow *row, ObCompactRow *&stored_row, bool inc_row_cnt, bool is_input /* false */) +// { +// int ret = OB_SUCCESS; + +// if (OB_FAIL(ra_rs_.add_row(row, stored_row))) { +// LOG_WARN("add row failed", K(ret)); +// } else if (OB_FAIL(store_set_.process_dump(is_input))) { +// LOG_WARN("process dump failed", K(ret)); +// } else { +// row_cnt_ += inc_row_cnt; +// stored_row_cnt_ += 1; +// } +// return ret; +// } + +// ================ RowStores ================ +int RowStores::process_dump(bool is_input) +{ +#define LOG_STORE(store) \ + if (store != nullptr) { LOG_DEBUG("store info", K(*store)); } +#define CALC_DUMP_SIZE(store) \ + if (store != nullptr) { target_dump_size += store->ra_rs_.get_mem_used(); } + +#define DUMP_STORE(store) \ + do { \ + if (OB_FAIL(ret)) { \ + } else if (target_dump_size > 0 && store != nullptr && store->ra_rs_.get_mem_used() > 0) { \ + if (OB_FAIL( \ + store->process_dump(target_dump_size, op_->global_mem_limit_version_, dumped_size))) { \ + LOG_WARN("process dump failed", K(ret)); \ + } else { \ + target_dump_size -= dumped_size; \ + } \ + } \ + } while (0) + + int ret = OB_SUCCESS; + bool need_dump = false; + int dump_errsim = OB_E(EventTable::EN_FORCE_WINFUNC_STORE_DUMP) OB_SUCCESS; + bool force_dump = (dump_errsim == OB_ALLOCATE_MEMORY_FAILED); + bool force_no_dump = (dump_errsim != OB_SUCCESS) && !force_dump; + int64_t target_dump_size = 0; + if (OB_UNLIKELY(force_no_dump)) { + // do nothing + } else if (OB_UNLIKELY(force_dump)) { + LST_DO_CODE(CALC_DUMP_SIZE, first_, processed_, cur_); // dump all + } else if (OB_ISNULL(op_)) { + ret = OB_NOT_INIT; + LOG_WARN("row stores not inited", K(ret)); + } else if (OB_FAIL(op_->update_mem_limit_version_periodically())) { + LOG_WARN("update global memory limit version failed", K(ret)); + } else if (need_check_dump(op_->global_mem_limit_version_)) { + // op_local_mem is needed for winfunc operator to work properly, it wont't change after opening operator. + int64_t op_local_mem = op_->local_mem_used(); + if (is_input) { + const static double MEM_DISCOUNT_FOR_PART_END = 0.8; + const double mem_bound_ratio = + op_->get_input_rows_mem_bound_ratio() * MEM_DISCOUNT_FOR_PART_END; + target_dump_size = -(op_->sql_mem_processor_.get_mem_bound() * mem_bound_ratio - op_local_mem); + LST_DO_CODE(CALC_DUMP_SIZE, first_, processed_, cur_); + } else { // for result + // dump BIG_BLOCK_SIZE * 2 at least, because dump is already needed at this time, + // ensure the minimum amount of memory for each dump. + const static int64_t MIN_DUMP_SIZE = (256L << 10) * 2; + target_dump_size = (op_->sql_mem_processor_.get_data_size() - op_->sql_mem_processor_.get_mem_bound()); + target_dump_size = MAX(target_dump_size, MIN_DUMP_SIZE); + } + } + // TODO: chang log level + LST_DO_CODE(LOG_STORE, first_, processed_, cur_, last_); + if (OB_UNLIKELY(target_dump_size > 0)) { + LOG_TRACE("dumping winfunc store", K(target_dump_size), K(force_dump), K(force_no_dump), + K(op_->global_mem_limit_version_), K(dump_errsim), + K(op_->sql_mem_processor_.get_data_size()), + K(op_->sql_mem_processor_.get_mem_bound()), K(op_->local_mem_used())); + // dumping order: first_, processed_, cur_ + int64_t dumped_size = 0; + LST_DO_CODE(DUMP_STORE, first_, processed_, cur_); + if (OB_SUCC(ret)) { op_->sql_mem_processor_.set_number_pass(1); } + } + return ret; + +#undef LOG_STORE +#undef DUMP_STORE +#undef CALC_DUMP_SIZE +} + +bool RowStores::need_check_dump(int64_t g_mem_limit_version) +{ +#define CHECK_VERSION(store)\ + if (store != nullptr) {\ + ret |= (store->local_mem_limit_version_ != g_mem_limit_version);\ + } + + bool ret = false; + LST_DO_CODE(CHECK_VERSION, first_, last_, cur_, processed_); + return ret; + +#undef CHECK_VERSION +} + +int RowStores::setup_mem_mgr() +{ +#define SETUP_STORE(store) \ + if (store != nullptr) { \ + store->ra_rs_.set_dir_id(op_->sql_mem_processor_.get_dir_id()); \ + store->ra_rs_.set_mem_stat(&(op_->sql_mem_processor_)); \ + store->ra_rs_.set_io_event_observer(&(op_->io_event_observer_)); \ + } + + int ret = OB_SUCCESS; + if (OB_ISNULL(op_)) { + ret = OB_NOT_INIT; + LOG_WARN("empty operator", K(ret)); + } else { + LST_DO_CODE(SETUP_STORE, first_, last_, processed_, cur_); + LOG_TRACE("trace init sql mem mgr for window function", + K(op_->profile_.get_cache_size()), K(op_->profile_.get_expect_size()), K(ret)); + } + +#undef SETUP_STORE + return ret; +} + +void RowStores::set_it_age(ObTempBlockStore::IterationAge *age) +{ + if (cur_ != nullptr) { cur_->ra_reader_.set_iteration_age(age); } + if (processed_ != nullptr) { processed_->ra_reader_.set_iteration_age(age); } + if (first_ != nullptr) { first_->ra_reader_.set_iteration_age(age); } + if (last_ != nullptr) { last_->ra_reader_.set_iteration_age(age); } +} +} // end winfunc +} // end sql +} // end oceanbase \ No newline at end of file diff --git a/src/sql/engine/window_function/row_store.h b/src/sql/engine/window_function/row_store.h new file mode 100644 index 0000000000..c296bed6a4 --- /dev/null +++ b/src/sql/engine/window_function/row_store.h @@ -0,0 +1,239 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_WINDOW_FUNCTION_ROWSTORE_H_ +#define OCEANBASE_WINDOW_FUNCTION_ROWSTORE_H_ + +#include "lib/container/ob_2d_array.h" +#include "sql/engine/basic/ob_compact_row.h" +#include "sql/engine/basic/ob_temp_row_store.h" + +namespace oceanbase +{ +namespace sql +{ + +class ObWindowFunctionVecOp; + +namespace winfunc +{ +class RowStores; + +class StoreGuard +{ +public: + StoreGuard(ObWindowFunctionVecOp &op); + ~StoreGuard() {} +private: + ObWindowFunctionVecOp &op_; +}; + +struct RowStore +{ +public: + RowStore(const int64_t tenant_id, ObIAllocator *store_alloc, ObIAllocator *arena_alloc, RowStores &store_set) : + store_set_(store_set), + allocator_(arena_alloc), + ra_rs_(store_alloc), + ra_reader_(), + row_cnt_(0), + stored_row_cnt_(0), + output_row_idx_(0), + row_ptrs_(nullptr), + local_mem_limit_version_(0) + {} + + ~RowStore() { destroy(); } + + int init(const int64_t max_batch_size, const RowMeta &row_meta, const lib::ObMemAttr &mem_attr, + const int64_t mem_limit, bool enable_dump); + + void destroy() + { + reset(); + ra_rs_.~ObRATempRowStore(); + allocator_ = nullptr; + } + + void reset() + { + row_cnt_ = 0; + stored_row_cnt_ = 0; + output_row_idx_ = 0; + ra_reader_.reset(); + ra_rs_.reset(); + local_mem_limit_version_ = 0; + } + + inline int64_t count() const { return row_cnt_; } + // computed but not outputed row cnt + inline int64_t to_output_rows() const { return row_cnt_ - output_row_idx_; } + // not computed row cnt + inline int64_t to_compute_rows() const { return stored_row_cnt_ - row_cnt_; } + inline bool is_empty() const { return stored_row_cnt_ == 0; } + + OB_INLINE int get_row(const int64_t row_idx, const ObCompactRow *&sr) + { + return ra_reader_.get_row(row_idx, sr); + } + + int add_batch_rows(const ObIArray &exprs, const RowMeta &row_meta, ObEvalCtx &eval_ctx, + const EvalBound &bound, const ObBitVector &skip, bool add_row_cnt, + ObCompactRow **stored_rows = nullptr, bool is_input = false); + int attach_rows(const ObIArray &exprs, const RowMeta &row_meta, ObEvalCtx &eval_ctx, + const int64_t start_idx, const int64_t end_idx, bool use_reserv_buf); + + int attach_rows(ObExpr *expr, const RowMeta &row_meta, ObEvalCtx &eval_ctx, + const int64_t start_idx, const int64_t end_idx, bool use_reserve_buf); + + int get_batch_rows(const int64_t start_idx, const int64_t end_idx, + const ObCompactRow **stored_rows); + + int process_dump(const int64_t target_size, const int64_t g_mem_limit_versiom, int64_t &dumped_size); + +public: + RowStores &store_set_; + ObIAllocator *allocator_; + sql::ObRATempRowStore ra_rs_; + sql::ObRATempRowStore::RAReader ra_reader_; + // record begin idx of current partition. always zero for rows_store_ TODO: maybe useless + // int64_t begin_idx_; + // cnt of rows computed + // `ObWinfowFunctionVecOp::compute_wf_values` uses row_cnt_ to inlcude rows participating computing + int64_t row_cnt_; + /* In non-batch execution, rows of only one partition is stored in rows_store_. + * If get row of next partition from child, store it in next_row_ and compute current_part. + * While in batch execution, we get a batch of rows from child, and it may contain multiple parts. + * All these rows are stored in a rows_store, but we only compute one partition one time. + * row_cnt_ is the last index of the partition we compute currently. + * stored_row_cnt_ is the count of all rows stored in this rows_store. + */ + int64_t stored_row_cnt_; + // [begin_idx_, output_row_idx_) => rows output already + // [output_row_idx_, row_cnt_) => rows computed already but not output + // [row_cnt_, stored_row_cnt_) => rows not computed yet + int64_t output_row_idx_; + ObCompactRow **row_ptrs_; + int64_t local_mem_limit_version_; + + TO_STRING_KV(K_(row_cnt), K_(stored_row_cnt), K_(output_row_idx), K(ra_rs_.get_mem_used()), + K(ra_rs_.get_mem_hold()), K_(local_mem_limit_version)); +}; + +struct RowStores +{ + RowStores() : op_(nullptr),processed_(NULL), cur_(NULL), first_(NULL), last_(NULL) + {} + + void set_operator(ObWindowFunctionVecOp *op) { op_ = op; } + int init(const int64_t max_batch_size, const RowMeta &row_meta, const lib::ObMemAttr &mem_attr, + const int64_t mem_limit, bool enable_dump) + { + int ret = OB_SUCCESS; + if (OB_ISNULL(processed_) || OB_ISNULL(cur_)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "unexpected null store", K(processed_), K(cur_)); + } else if (OB_FAIL(processed_->init(max_batch_size, row_meta, mem_attr, mem_limit, enable_dump))) { + SQL_LOG(WARN, "init store failed", K(ret)); + } else if (OB_FAIL(cur_->init(max_batch_size, row_meta, mem_attr, mem_limit, enable_dump))) { + SQL_LOG(WARN, "init store failed", K(ret)); + } else if (OB_NOT_NULL(first_) || OB_NOT_NULL(last_)) { + if (OB_ISNULL(first_) || OB_ISNULL(last_)) { + ret = OB_ERR_UNEXPECTED; + SQL_LOG(WARN, "invalid null store", K(first_), K(last_)); + } else if (OB_FAIL(first_->init(max_batch_size, row_meta, mem_attr, mem_limit, enable_dump))) { + SQL_LOG(WARN, "init store failed", K(ret)); + } else if (OB_FAIL(last_->init(max_batch_size, row_meta, mem_attr, mem_limit, enable_dump))) { + SQL_LOG(WARN, "init store failed", K(ret)); + } + } + if (OB_SUCC(ret) && OB_FAIL(setup_mem_mgr())) { + SQL_LOG(WARN, "setup memory manager failed", K(ret)); + } + return ret; + } + + int process_dump(bool is_input); + + virtual ~RowStores() + { + destroy(); + } + + void swap_cur_processed() + { + std::swap(cur_, processed_); + } + + void swap_first_processed() + { + std::swap(first_, processed_); + } + + void swap_first_cur() + { + std::swap(first_, cur_); + } + + void swap_last_cur() + { + std::swap(last_, cur_); + } + + void set_it_age(ObTempBlockStore::IterationAge *age); + + void reset() + { +#define RESET_STORE(store) \ + do { \ + if (store != nullptr) { store->reset(); } \ + } while (false) + + LST_DO_CODE(RESET_STORE, processed_, cur_, first_, last_); +#undef RESET_STORE + } + + void destroy() + { +#define DESTROY_STORE(store) \ + do { \ + if ((store) != nullptr) { store->destroy(); } \ + } while (false) + + LST_DO_CODE(DESTROY_STORE, processed_, cur_, first_, last_); + processed_ = nullptr; + cur_ = nullptr; + first_ = nullptr; + last_ = nullptr; + op_ = nullptr; +#undef DESTROY_STORE + } + + TO_STRING_KV(K(processed_), K(first_), K(last_), K(cur_)); +private: + int setup_mem_mgr(); + + bool need_check_dump(int64_t g_mem_limit_version); +public: + ObWindowFunctionVecOp *op_; + // `processed_` is rows calculated but not outputed, only used in vectorized execution + RowStore *processed_; + // current operation rows + RowStore *cur_; + // first and last partition for range distribute window function + RowStore *first_; + RowStore *last_; +}; +} // end winfunc +} // end sql +} // end oceanbase +#endif // OCEANBASE_WINDOW_FUNCTION_ROWSTORE_H_ \ No newline at end of file diff --git a/src/sql/engine/window_function/win_expr.cpp b/src/sql/engine/window_function/win_expr.cpp new file mode 100644 index 0000000000..d5246773bf --- /dev/null +++ b/src/sql/engine/window_function/win_expr.cpp @@ -0,0 +1,2506 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ +#define USING_LOG_PREFIX SQL_ENG + +#include "win_expr.h" +#include "share/aggregate/iaggregate.h" +#include "sql/engine/window_function/ob_window_function_vec_op.h" +#include "sql/engine/expr/ob_expr_truncate.h" +#include "lib/timezone/ob_time_convert.h" + +namespace oceanbase +{ +namespace sql +{ +namespace winfunc +{ +static int eval_bound_exprs(WinExprEvalCtx &ctx, const int64_t row_start, const int64_t batch_size, + const ObBitVector &skip, const bool is_upper); + +static int calc_borders_for_current_row(WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t batch_size, const ObBitVector &skip, + const bool is_upper); + +static int eval_and_check_between_literal(WinExprEvalCtx &ctx, ObBitVector &eval_skip, + const ObExpr *between_expr, const int64_t batch_size, + int64_t *pos_arr); +static int calc_borders_for_rows_between(WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t batch_size, const ObBitVector &eval_skip, + const ObExpr *between_expr, const bool is_preceding, + const bool is_upper, int64_t *pos_arr); + +static int calc_borders_for_no_sort_expr(WinExprEvalCtx &ctx, + const int64_t batch_size, const ObBitVector &eval_skip, + const ObExpr *bound_expr, const bool is_upper, + int64_t *pos_arr); + +static int calc_borders_for_sort_expr(WinExprEvalCtx &ctx, const ObExpr *bound_expr, + const int64_t batch_size, const int64_t row_start, + ObBitVector &eval_skip, const bool is_upper, + int64_t *pos_arr); + +static int cmp_prev_row(WinExprEvalCtx &ctx, const int64_t cur_idx, int &cmp_ret); + +template +struct int_trunc +{ + static int get(const char *payload, const int32_t len,const ObDatumMeta &meta, int64_t &value); +}; + +// WinExprHelper +template +int WinExprWrapper::process_partition(WinExprEvalCtx &ctx, const int64_t part_start, + const int64_t part_end, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(part_start > row_start || part_end < row_end)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid partition", K(part_start), K(part_end), K(row_start), K(row_end)); + } else if (OB_UNLIKELY(part_start >= part_end || row_start >= row_end)) { + LOG_DEBUG("empty partition", K(part_start), K(part_end), K(row_start), K(row_end)); + } else { + const ObCompactRow *prev_row = nullptr, *cur_row = nullptr; + Frame prev_frame, cur_frame; + bool whole_frame = true, valid_frame = true; + ObEvalCtx::BatchInfoScopeGuard guard(ctx.win_col_.op_.get_eval_ctx()); + guard.set_batch_size(row_end - row_start); + if (OB_FAIL(eval_bound_exprs(ctx, row_start, row_end - row_start, skip, true))) { + LOG_WARN("eval upper bound failed", K(ret)); + } else if (OB_FAIL(eval_bound_exprs(ctx, row_start, row_end-row_start, skip, false))) { + LOG_WARN("eval lower bound failed", K(ret)); + } else if (is_aggregate_expr()) { + AggrExpr *agg_expr = reinterpret_cast(this); + prev_frame = agg_expr->last_valid_frame_; + if (row_start > part_start) { + ctx.win_col_.agg_ctx_->removal_info_ = agg_expr->last_removal_info_; + } + // TODO: maybe prefetch agg rows is a good idea + int prev_calc_idx = -1; + for (int row_idx = row_start; OB_SUCC(ret) && row_idx < row_end; row_idx++) { + int32_t batch_idx = row_idx - row_start; + if (skip.at(batch_idx)) { + continue; + } + guard.set_batch_idx(batch_idx); + aggregate::AggrRowPtr agg_row = ctx.win_col_.aggr_rows_[batch_idx]; + bool is_null = false; // useless for aggregation function + if (OB_FAIL(update_frame(ctx, prev_frame, cur_frame, batch_idx, row_start, whole_frame, + valid_frame))) { + LOG_WARN("update frame failed", K(ret)); + } else if (OB_UNLIKELY(!valid_frame)) { + if (OB_FAIL(AggrExpr::set_result_for_invalid_frame(ctx, agg_row))) { + LOG_WARN("set result for invalid frame failed", K(ret)); + } + } else if (prev_frame == cur_frame) { + // for aggregate function, same frame means same results + // just copy aggr row + char *copied_row = (prev_calc_idx == -1 ? agg_expr->last_aggr_row_ : + ctx.win_col_.aggr_rows_[prev_calc_idx]); + if (OB_FAIL(copy_aggr_row(ctx, copied_row, agg_row))) { + LOG_WARN("copy aggr row failed", K(ret)); + } + } else if (whole_frame) { + ctx.win_col_.agg_ctx_->removal_info_.reset_for_new_frame(); + if (OB_FAIL(static_cast(this)->process_window(ctx, cur_frame, row_idx, agg_row, is_null))) { + LOG_WARN("eval aggregate function failed", K(ret)); + } + } else if (OB_FAIL(static_cast(this)->accum_process_window( + ctx, cur_frame, prev_frame, row_idx, agg_row, is_null))) { + LOG_WARN("increase evaluation function failed", K(ret)); + } + if (OB_FAIL(ret)) { + } else { + prev_frame = cur_frame; + prev_calc_idx = batch_idx; + } + } // end for + if (OB_SUCC(ret) && prev_calc_idx != -1) { + agg_expr->last_valid_frame_ = prev_frame; + if (row_end < part_end) { + agg_expr->last_removal_info_ = ctx.win_col_.agg_ctx_->removal_info_; + } + int32_t row_size = ctx.win_col_.agg_ctx_->row_meta().row_size_; + void *tmp_buf = nullptr; + if (agg_expr->last_aggr_row_ != nullptr) { + if (OB_FAIL(copy_aggr_row(ctx, ctx.win_col_.aggr_rows_[prev_calc_idx], agg_expr->last_aggr_row_))) { + LOG_WARN("copy aggr row failed", K(ret)); + } + } else if (OB_ISNULL(tmp_buf = ctx.reserved_buf(row_size))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else if (OB_FAIL(copy_aggr_row(ctx, ctx.win_col_.aggr_rows_[prev_calc_idx], (char *)tmp_buf))) { + LOG_WARN("copy aggr row failed", K(ret)); + } else { + agg_expr->last_aggr_row_ = (char *)tmp_buf; + } + } + } else { + void *extra = nullptr; + int32_t non_aggr_row_size = ctx.win_col_.non_aggr_reserved_row_size(); + bool is_null = false; + if (OB_FAIL(static_cast(this)->generate_extra(ctx.allocator_, extra))) { + LOG_WARN("generate extra data failed", K(ret)); + } else { + MEMSET(ctx.win_col_.non_aggr_results_, 0, non_aggr_row_size * (row_end - row_start)); + ctx.extra_ = extra; + } + for (int row_idx = row_start; OB_SUCC(ret) && row_idx < row_end; row_idx++) { + int32_t batch_idx = row_idx - row_start; + if (skip.at(batch_idx)) { + continue; + } + guard.set_batch_idx(batch_idx); + is_null = false; + char *non_aggr_res = ctx.win_col_.non_aggr_results_ + non_aggr_row_size * batch_idx; + if (OB_FAIL(update_frame(ctx, prev_frame, cur_frame, batch_idx, row_start, whole_frame, + valid_frame))) { + LOG_WARN("update frame failed", K(ret)); + } else if (OB_UNLIKELY(!valid_frame)) { + is_null = true; + } else if (OB_FAIL(static_cast(this)->process_window(ctx, cur_frame, row_idx, + non_aggr_res, is_null))) { + LOG_WARN("process window failed", K(ret)); + } + if (OB_FAIL(ret)) { + } else if (is_null) { + ctx.win_col_.null_nonaggr_results_->set(batch_idx); + } + } // end for + } + // collect partition results + if (OB_SUCC(ret)) { + ObExpr *wf_expr = ctx.win_col_.wf_info_.expr_; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + if (OB_FAIL(ret)) { + } else if (OB_FAIL(static_cast(this)->collect_part_results(ctx, row_start, + row_end, skip))) { + LOG_WARN("collect partition results failed", K(ret)); + } + } + } + return ret; +} + +template +int WinExprWrapper::update_frame(WinExprEvalCtx &ctx, const Frame &prev_frame, + Frame &new_frame, const int64_t idx, + const int64_t row_start, bool &whole_frame, + bool &valid_frame) +{ + int ret = OB_SUCCESS; + int64_t part_first_idx = ctx.win_col_.part_first_row_idx_; + int64_t part_end_idx = ctx.win_col_.op_.get_part_end_idx(); + int64_t *upper_pos_arr = ctx.win_col_.op_.batch_ctx_.upper_pos_arr_; + int64_t *lower_pos_arr = ctx.win_col_.op_.batch_ctx_.lower_pos_arr_; + Frame part_frame(part_first_idx, part_end_idx); + new_frame.head_ = upper_pos_arr[idx]; + new_frame.tail_ = lower_pos_arr[idx]; + valid_frame = true; + whole_frame = true; + const ObWindowFunctionVecSpec &spec = static_cast(ctx.win_col_.op_.get_spec()); + if (OB_UNLIKELY(new_frame.head_ == INT64_MAX || new_frame.tail_ == INT64_MAX)) { + LOG_DEBUG("invalid frame", K(new_frame)); + valid_frame = false; + } else if (FALSE_IT(valid_frame = Frame::valid_frame(part_frame, new_frame))) { + } else if (!valid_frame) { + } else if (spec.single_part_parallel_) { + // whole frame, no need to update + } else { + Frame::prune_frame(part_frame, new_frame); + if (static_cast(this)->is_aggregate_expr()) { + bool can_inv = (ctx.win_col_.wf_info_.remove_type_ != common::REMOVE_INVALID); + aggregate::Processor *processor = reinterpret_cast(this)->aggr_processor_; + if (prev_frame.is_valid() + && !Frame::need_restart_aggr(can_inv, prev_frame, new_frame, + ctx.win_col_.agg_ctx_->removal_info_, + ctx.win_col_.wf_info_.remove_type_)) { + whole_frame = false; + } + } + } + LOG_DEBUG("update frame", K(ret), K(valid_frame), K(prev_frame), K(new_frame), K(idx), + K(whole_frame), K(ctx.win_col_.wf_info_.remove_type_)); + return ret; +} + +template +int WinExprWrapper::copy_aggr_row(WinExprEvalCtx &ctx, const char *src_row, char *dst_row) +{ + int ret = OB_SUCCESS; + aggregate::RuntimeContext *agg_ctx = ctx.win_col_.agg_ctx_; + MEMCPY(dst_row, src_row, ctx.win_col_.agg_ctx_->row_meta().row_size_); + if (!agg_ctx->row_meta().is_var_len(0)) {// do nothing + } else { + int32_t cell_len = agg_ctx->row_meta().get_cell_len(0, dst_row); + int64_t &payload_addr = + *reinterpret_cast(agg_ctx->row_meta().locate_cell_payload(0, dst_row)); + const char *payload = reinterpret_cast(payload_addr); + bool is_not_null = agg_ctx->row_meta().locate_notnulls_bitmap(dst_row).at(0); + + if (OB_LIKELY(is_not_null && cell_len > 0)) { + void *tmp_buf = ctx.reserved_buf(cell_len); + if (OB_ISNULL(tmp_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(tmp_buf, payload, cell_len); + payload_addr = reinterpret_cast(tmp_buf); + } + } else { + payload_addr = 0; + } + } + return ret; +} + +int NonAggrWinExpr::eval_param_int_value(ObExpr *param, ObEvalCtx &ctx, const bool need_check_valid, + const bool need_nmb, ParamStatus &status) +{ + int ret = OB_SUCCESS; + int64_t mock_skip_data = 0; + ObBitVector *skip = to_bit_vector(&mock_skip_data); + ObEvalCtx::BatchInfoScopeGuard guard(ctx); + guard.set_batch_size(1); + guard.set_batch_idx(0); + EvalBound bound(1, true); + bool is_valid_param = true; + bool is_null = false; + int64_t val = 0; + if (OB_FAIL(param->eval_vector(ctx, *skip, bound))) { + LOG_WARN("eval failed", K(ret)); + } else if (param->get_vector(ctx)->is_null(0)) { + is_null = true; + is_valid_param = !need_check_valid; + status.is_null_ = true; + status.calculated_ = true; + } else if (need_nmb || param->obj_meta_.is_number() || param->obj_meta_.is_number_float()) { + number::ObNumber result_nmb(param->get_vector(ctx)->get_number(0)); + is_valid_param = !need_check_valid || !result_nmb.is_negative(); + if (OB_FAIL(result_nmb.extract_valid_int64_with_trunc(val))) { + LOG_WARN("extract int64_t value failed", K(ret)); + } + } else if (param->obj_meta_.is_decimal_int()) { + const ObDecimalInt *decint = param->get_vector(ctx)->get_decimal_int(0); + int32_t in_bytes = param->get_vector(ctx)->get_length(0); + ObDecimalIntBuilder trunc_res_val; + const int16_t in_prec = param->datum_meta_.precision_; + const int16_t in_scale = param->datum_meta_.scale_; + const int16_t out_scale = 0; + is_valid_param = !need_check_valid || !wide::is_negative(decint, in_bytes); + ObDatum in_datum; + in_datum.ptr_ = reinterpret_cast(decint); + in_datum.len_ = in_bytes; + if (in_scale == out_scale) { + trunc_res_val.from(decint, in_bytes); + } else if (OB_FAIL(ObExprTruncate::do_trunc_decimalint(in_prec, in_scale, in_prec, out_scale, + out_scale, in_datum, trunc_res_val))) { + LOG_WARN("truncate decimal int failed", K(ret)); + } + bool is_in_val_valid = false; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(wide::check_range_valid_int64(trunc_res_val.get_decimal_int(), + trunc_res_val.get_int_bytes(), is_in_val_valid, + val))) { + LOG_WARN("check valid int64 failed", K(ret)); + } else if (!is_in_val_valid) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("res val is not a valid int64", K(ret)); + } + } else if (ob_is_int_tc(param->datum_meta_.type_)) { + val = param->get_vector(ctx)->get_int(0); + is_valid_param = !need_check_valid || val >= 0; + } else if (ob_is_uint_tc(param->datum_meta_.type_)) { + uint64_t tmp_val = param->get_vector(ctx)->get_uint(0); + is_valid_param = !need_check_valid || static_cast(tmp_val) >= 0; + if (tmp_val > INT64_MAX && is_valid_param) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret), K(tmp_val), K(INT64_MAX)); + } else { + val = static_cast(tmp_val); + } + } else if (ob_is_float_tc(param->datum_meta_.type_)) { + float tmp_val = param->get_vector(ctx)->get_float(0); + is_valid_param = !need_check_valid || tmp_val >= 0; + if (tmp_val > INT64_MAX && is_valid_param) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret), K(tmp_val)); + } else { + val = static_cast(tmp_val); + } + } else if (ob_is_double_tc(param->datum_meta_.type_)) { + double tmp_val = param->get_vector(ctx)->get_double(0); + is_valid_param = !need_check_valid || tmp_val >= 0; + if (tmp_val > INT64_MAX && is_valid_param) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret), K(tmp_val)); + } else { + val = static_cast(tmp_val); + } + } else if (ob_is_bit_tc(param->datum_meta_.type_)) { + uint64_t tmp_val = param->get_vector(ctx)->get_bit(0); + is_valid_param = !need_check_valid || static_cast(tmp_val) >= 0; + if (tmp_val > INT64_MAX && is_valid_param) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret), K(tmp_val), K(INT64_MAX)); + } else { + val = static_cast(tmp_val); + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not support type", K(ret), K(*param)); + } + if (OB_SUCC(ret) && !status.is_null_) { + status.calculated_ = true; + status.int_val_ = val; + } + if (OB_SUCC(ret) && !is_valid_param) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), K(val)); + } + return ret; +} + +#define SET_NON_AGG_DATAS(fmt) \ + do { \ + fmt *data = static_cast(non_agg_expr->get_vector(eval_ctx)); \ + int32_t offset = 0, step = ctx.win_col_.non_aggr_reserved_row_size(); \ + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++, offset += step) { \ + if (skip.at(i)) { continue; } \ + guard.set_batch_idx(i); \ + char *res_row = ctx.win_col_.non_aggr_results_ + offset; \ + if (!ctx.win_col_.null_nonaggr_results_->at(i)) { \ + if (is_fixed_len_data) { \ + payload = res_row; \ + len = ctx.win_col_.non_aggr_reserved_row_size(); \ + data->set_payload(i, payload, len); \ + } else if (vec_tc == VEC_TC_NUMBER) { \ + data->set_number(i, *reinterpret_cast(res_row)); \ + } else { \ + payload = reinterpret_cast(*reinterpret_cast(res_row)); \ + len = *reinterpret_cast(res_row + sizeof(char *)); \ + char *res_buf = non_agg_expr->get_str_res_mem(eval_ctx, len); \ + if (OB_ISNULL(res_buf)) { \ + ret = OB_ALLOCATE_MEMORY_FAILED; \ + LOG_WARN("allocate memory failed", K(ret)); \ + } else { \ + MEMCPY(res_buf, payload, len); \ + data->set_payload_shallow(i, res_buf, len); \ + } \ + } \ + } else { \ + data->set_null(i); \ + } \ + } \ + } while (false) + +#define SET_NON_AGG_FIXED_DATAS(vec_tc) \ + case (vec_tc): { \ + SET_NON_AGG_DATAS(ObFixedLengthFormat>); \ + } break + +int NonAggrWinExpr::collect_part_results(WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) +{ + int ret = OB_SUCCESS; + ObExpr *non_agg_expr = ctx.win_col_.wf_info_.expr_; + int64_t batch_size = row_end - row_start; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + VectorFormat fmt = non_agg_expr->get_format(eval_ctx); + VecValueTypeClass vec_tc = non_agg_expr->get_vec_value_tc(); + bool is_fixed_len_data = is_fixed_length_vec(vec_tc); + const char *payload = nullptr; + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx); + guard.set_batch_size(batch_size); + int32_t len = 0; + switch (fmt) { + case common::VEC_UNIFORM: { + SET_NON_AGG_DATAS(ObUniformFormat); + break; + } + case common::VEC_FIXED: { + switch (vec_tc) { + LST_DO_CODE(SET_NON_AGG_FIXED_DATAS, FIXED_VEC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected tc", K(vec_tc)); + } + } + break; + } + case common::VEC_DISCRETE: { + SET_NON_AGG_DATAS(ObDiscreteFormat); + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret)); + } + } + if (OB_FAIL(ret)) { + LOG_WARN("collect part results failed", K(ret)); + } else { + non_agg_expr->set_evaluated_projected(eval_ctx); + } + return ret; +} + +template +int RankLikeExpr::process_window(WinExprEvalCtx &ctx, const Frame &frame, + const int64_t row_idx, char *res, bool &is_null) +{ + int ret = OB_SUCCESS; + bool equal_with_prev_row = false; + is_null = false; + if (row_idx != frame.head_) { + int cmp_ret = 0; + if (OB_FAIL(cmp_prev_row(ctx, row_idx, cmp_ret))) { + LOG_WARN("compare previous row failed", K(ret)); + } else { + equal_with_prev_row = (cmp_ret == 0); + } + } else { + // reset rank + rank_of_prev_row_ = 0; + } + if (OB_SUCC(ret)) { + int64_t rank = -1; + if (equal_with_prev_row) { + rank = rank_of_prev_row_; + } else if (rank_op == T_WIN_FUN_RANK || rank_op == T_WIN_FUN_PERCENT_RANK) { + rank = row_idx - frame.head_ + 1; + } else if (rank_op == T_WIN_FUN_DENSE_RANK) { + rank = rank_of_prev_row_ + 1; + } + LOG_DEBUG("calculate rank result", K(rank_op), K(rank), K(frame)); + if (rank_op == T_WIN_FUN_PERCENT_RANK) { + if (ob_is_number_tc(ctx.win_col_.wf_info_.expr_->datum_meta_.type_)) { + // in mysql mode, percent rank may return double + if (0 == frame.tail_ - frame.head_ - 1) { + number::ObNumber zero_nmb; + zero_nmb.set_zero(); + MEMCPY(res, &(zero_nmb.d_), sizeof(ObNumberDesc)); + } else { + number::ObNumber numerator; + number::ObNumber denominator; + number::ObNumber res_nmb; + ObNumStackAllocator<3> tmp_alloc; + if (OB_FAIL(numerator.from(rank - 1, tmp_alloc))) { + LOG_WARN("failed to build number from int64_t", K(ret)); + } else if (OB_FAIL(denominator.from(frame.tail_ - frame.head_ - 1, tmp_alloc))) { + LOG_WARN("failed to build number from int64_t", K(ret)); + } else if (OB_FAIL(numerator.div(denominator, res_nmb, tmp_alloc))) { + LOG_WARN("failed to div number", K(ret)); + } else { + number::ObCompactNumber *res_cnum = reinterpret_cast(res); + res_cnum->desc_ = res_nmb.d_; + MEMCPY(&(res_cnum->digits_[0]), res_nmb.get_digits(), sizeof(uint32_t) * res_nmb.d_.len_); + } + } + } else if (ObDoubleType == ctx.win_col_.wf_info_.expr_->datum_meta_.type_) { + if (0 == frame.tail_ - frame.head_ - 1) { + *reinterpret_cast(res) = 0; + } else { + double numerator = static_cast(rank - 1); + double denominator= static_cast(frame.tail_ - frame.head_ - 1); + *reinterpret_cast(res) = (numerator / denominator); + } + } + } else if (lib::is_oracle_mode()) { + number::ObNumber res_nmb; + ObNumStackAllocator<1> tmp_alloc; + if (OB_FAIL(res_nmb.from(rank, tmp_alloc))) { + LOG_WARN("failed to build number from int64_t", K(ret)); + } else { + MEMCPY(res, &(res_nmb.d_), sizeof(ObNumberDesc)); + MEMCPY(res + sizeof(ObNumberDesc), res_nmb.get_digits(), + sizeof(uint32_t) * res_nmb.d_.len_); + } + } else { + *reinterpret_cast(res) = rank; + } + if (OB_SUCC(ret)) { + rank_of_prev_row_ = rank; + } + } + return ret; +} + +template +int RankLikeExpr::generate_extra(ObIAllocator &allocator, void *&extra) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(extra = allocator.alloc(sizeof(int64_t)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + *reinterpret_cast(extra) = 0; + } + return ret; +} + +int Ntile::process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, char *res, bool &is_null) +{ + int ret = OB_SUCCESS; + ParamStatus *param_status = reinterpret_cast(ctx.extra_); + is_null = false; + if (OB_UNLIKELY(!param_status->calculated_)) { + // calculated bucket number + const ObExprPtrIArray ¶ms = ctx.win_col_.wf_info_.param_exprs_; + ObExpr *param = nullptr; + int64_t bucket_num = 0; + bool is_null = false; + if (OB_UNLIKELY(params.count() != 1)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("The number of arguments of NTILE should be 1", K(params.count()), K(ret)); + } else if (OB_ISNULL(param = params.at(0))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null param expr", K(ret)); + } else if (!is_oracle_mode() && !param->obj_meta_.is_numeric_type()) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("invalid argument", K(ret), K(param->obj_meta_)); + } else if (OB_FAIL(NonAggrWinExpr::eval_param_int_value(param, ctx.win_col_.op_.get_eval_ctx(), + lib::is_mysql_mode(), false, *param_status))) { + if (ret == OB_ERR_WINDOW_FRAME_ILLEGAL) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Incorrect arguments to ntile", K(ret)); + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "ntile"); + } else { + LOG_WARN("get_param_int_value failed", K(ret)); + } + } + } + if (OB_FAIL(ret)) { + } else if (param_status->is_null_) { + // do nothing + is_null = true; + } else if (param_status->int_val_ <= 0) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("bucket number is invalid", K(ret), K(param_status->int_val_)); + } else { + int64_t bucket_num = param_status->int_val_; + int64_t total = frame.tail_ - frame.head_; + int64_t x = total / bucket_num; + int64_t y = total % bucket_num; + const int64_t f_row_idx = row_idx - frame.head_; + int64_t result = 0; + LOG_DEBUG("print ntile param", K(total), K(x), K(y), K(f_row_idx)); + if (0 == x) { + result = f_row_idx + 1; + } else { + if (f_row_idx < (y * (x + 1))) { + result = f_row_idx / (x + 1) + 1; + } else { + result = (f_row_idx - y * (x + 1)) / x + y + 1; + } + } + if (ctx.win_col_.wf_info_.expr_->datum_meta_.type_ == ObNumberType) { + ObNumStackOnceAlloc tmp_alloc; + number::ObNumber result_num; + if (OB_FAIL(result_num.from(result, tmp_alloc))) { + LOG_WARN("number from int failed", K(ret)); + } else { + MEMCPY(res, &(result_num.d_), sizeof(ObNumberDesc)); + MEMCPY(res + sizeof(ObNumberDesc), result_num.get_digits(), sizeof(uint32_t) * result_num.d_.len_); + } + } else { + *reinterpret_cast(res) = result; + } + } + return ret; +} + +int Ntile::generate_extra(ObIAllocator &allocator, void *&extra) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(extra = allocator.alloc(sizeof(ParamStatus)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + new(extra)ParamStatus(); + } + return ret; +} + +static int memcpy_results(WinExprEvalCtx &ctx, VecValueTypeClass res_tc, char *res_buf, + const char *src, int32_t len) +{ + int ret = OB_SUCCESS; + char *data_buf = nullptr; + int32_t data_len = len; + ObExpr *win_expr = ctx.win_col_.wf_info_.expr_; + if (is_fixed_length_vec(res_tc) || res_tc == VEC_TC_NUMBER) { + MEMCPY(res_buf, src, len); + } else if (OB_UNLIKELY(len == 0)) { + *reinterpret_cast(res_buf) = 0; + *reinterpret_cast(res_buf + sizeof(char *)) = len; + } else if (OB_ISNULL(data_buf = ctx.reserved_buf(len))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(data_buf, src, len); + *reinterpret_cast(res_buf) = reinterpret_cast(data_buf); + *reinterpret_cast(res_buf + sizeof(char *)) = len; + } + return ret; +} + +int NthValue::process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, char *res, bool &is_null) +{ + int ret = OB_SUCCESS; + is_null = false; + const ObExprPtrIArray ¶ms = ctx.win_col_.wf_info_.param_exprs_; + bool is_param_null = false; + int64_t nth_val = 0; + const RowMeta &input_row_meta = ctx.win_col_.op_.get_input_row_meta(); + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + ObIArray &all_exprs = ctx.win_col_.op_.get_all_expr(); + // TODO: second param of nth_value in mysql mode is a const expr, optimize calculating. + ctx.win_col_.op_.clear_evaluated_flag(); + ParamStatus param_status; + if (OB_FAIL(ctx.input_rows_.attach_rows(all_exprs, input_row_meta, eval_ctx, row_idx, row_idx + 1, + false))) { + LOG_WARN("attach rows failed", K(ret)); + } else if (OB_UNLIKELY(params.count() != 2)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid number of params", K(ret), K(params.count()), K(ret)); + } else if (OB_FAIL( + NonAggrWinExpr::eval_param_int_value(params.at(1), ctx.win_col_.op_.get_eval_ctx(), + lib::is_mysql_mode(), false, param_status))) { + if (ret == OB_ERR_WINDOW_FRAME_ILLEGAL) { + if (param_status.is_null_) { + ret = OB_SUCCESS; + is_null = true; + } else { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("Incorrect arguments to nth_value", K(ret)); + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "nth_value"); + } + } else { + LOG_WARN("get_param_int_value failed", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else { + is_param_null = param_status.is_null_; + nth_val = param_status.int_val_; + } + if (OB_SUCC(ret) && !is_null) { + ObWindowFunctionVecOp &op = ctx.win_col_.op_; + if (OB_UNLIKELY(lib::is_oracle_mode() && (is_param_null || nth_val <= 0))) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("invalid argument", K(ret), K(is_param_null), K(nth_val)); + } else if (OB_UNLIKELY(lib::is_mysql_mode() + && (!params.at(1)->obj_meta_.is_integer_type() || nth_val == 0))) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arguments to nth_value", K(ret), K(nth_val), K(params.at(1)->obj_meta_)); + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "nth_value"); + } else { + bool is_ignore_null = ctx.win_col_.wf_info_.is_ignore_null_; + bool is_from_first = ctx.win_col_.wf_info_.is_from_first_; + + int64_t k = 0, cur_idx = (is_from_first ? frame.head_ : frame.tail_ - 1); + bool is_calc_nth = false; + const RowMeta &input_row_meta = ctx.win_col_.op_.get_input_row_meta(); + LOG_DEBUG("nth value params", K(is_param_null), K(nth_val), K(is_from_first), K(is_ignore_null), K(frame)); + while (OB_SUCC(ret) && k < nth_val) { + op.clear_evaluated_flag(); + int64_t batch_size = std::min(nth_val - k, ctx.win_col_.op_.get_spec().max_batch_size_); + batch_size = std::min((is_from_first ? (frame.tail_ - cur_idx) : (cur_idx - frame.head_ + 1)), + batch_size); + + int64_t start_idx = (is_from_first ? cur_idx : cur_idx - batch_size + 1); + int64_t end_idx = (is_from_first ? cur_idx + batch_size : cur_idx + 1); + if (start_idx >= end_idx) { break; } + int64_t word_cnt = ObBitVector::word_count(op.get_spec().max_batch_size_); + MEMSET(op.get_batch_ctx().bound_eval_skip_, 0, ObBitVector::BYTES_PER_WORD * word_cnt); + int64_t step = (is_from_first ? 1 : -1); + EvalBound bound(batch_size, true); + VecValueTypeClass res_tc = params.at(0)->get_vec_value_tc(); + VectorFormat param_fmt = VEC_INVALID; + if (OB_FAIL(ret)) { + } else if (OB_FAIL(ctx.input_rows_.attach_rows(op.get_all_expr(), input_row_meta, + op.get_eval_ctx(), start_idx, end_idx, + false))) { + LOG_WARN("attach rows failed", K(ret)); + } else if (OB_FAIL(params.at(0)->eval_vector( + op.get_eval_ctx(), *op.get_batch_ctx().bound_eval_skip_, bound))) { + LOG_WARN("eval vector failed", K(ret)); + } else if (FALSE_IT(param_fmt = params.at(0)->get_format(op.get_eval_ctx()))) { + } else if (param_fmt != VEC_UNIFORM && param_fmt != VEC_UNIFORM_CONST) { + ObIVector *data = params.at(0)->get_vector(op.get_eval_ctx()); + ObBitmapNullVectorBase *param_nulls = static_cast(data); + for (int i = 0, idx = (is_from_first ? 0 : batch_size - 1); + !is_calc_nth && i < batch_size; i++, idx += step) { + if ((!param_nulls->is_null(idx) || !is_ignore_null) && ++k == nth_val) { + is_calc_nth = true; + if (param_nulls->is_null(idx)) { + is_null = true; + } else { + ret = memcpy_results(ctx, res_tc, res, data->get_payload(idx), data->get_length(idx)); + } + } + } + } else if (param_fmt == VEC_UNIFORM) { + ObUniformFormat *data = static_cast *>(params.at(0)->get_vector(op.get_eval_ctx())); + for (int j = 0, idx = (is_from_first ? 0 : batch_size - 1); + !is_calc_nth && j < batch_size; j++, idx += step) { + if ((!data->is_null(idx) || !is_ignore_null) && ++k == nth_val) { + is_calc_nth = true; + if (data->is_null(idx)) { + is_null = true; + } else { + ret = memcpy_results(ctx, res_tc, res, data->get_payload(idx), data->get_length(idx)); + } + } + } + } else if (param_fmt == VEC_UNIFORM_CONST) { + ObUniformFormat *data = static_cast *>(params.at(0)->get_vector(op.get_eval_ctx())); + for (int j = 0, idx = (is_from_first ? 0 : batch_size - 1); + !is_calc_nth && j < batch_size; j++, idx += step) { + if ((!data->is_null(idx) || !is_ignore_null) && ++k == nth_val) { + is_calc_nth = true; + if (data->is_null(idx)) { + is_null = true; + } else { + ret = memcpy_results(ctx, res_tc, res, data->get_payload(idx), data->get_length(idx)); + } + } + } + } + if (OB_SUCC(ret)) { + (is_from_first ? (cur_idx += batch_size) : (cur_idx -= batch_size)); + if ((is_from_first && cur_idx >= frame.tail_) || (!is_from_first && cur_idx < frame.head_)) { + break; + } + } + } // end while + if (!is_calc_nth) { + is_null = true; + } + } + } + return ret; +} + +int NthValue::generate_extra(ObIAllocator &allocator, void *&extra) +{ + int ret = OB_SUCCESS; + extra = nullptr; + if (lib::is_mysql_mode()) { + void *buf = allocator.alloc(sizeof(ParamStatus)); + if (OB_ISNULL(buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + new (buf) ParamStatus(); + extra = buf; + } + } + return ret; +} + +int LeadOrLag::process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) +{ + int ret = OB_SUCCESS; + enum LeadLagParamType + { + VALUE_EXPR = 0, + OFFSET = 1, + DEFAULT_VALUE = 2, + NUM_LEAD_LAG_PARAMS + }; + const bool is_lead = (T_WIN_FUN_LEAD == ctx.win_col_.wf_info_.func_type_); + int lead_lag_offset_direction = (is_lead ? 1 : -1); + // if not specified, the default offset is 1. + bool is_lead_lag_offset_used = false; + const ObIArray ¶ms = ctx.win_col_.wf_info_.param_exprs_; + const ObCompactRow *a_row = nullptr; + ObIArray &all_exprs = ctx.win_col_.op_.get_all_expr(); + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + const RowMeta &input_row_meta = ctx.win_col_.op_.get_input_row_meta(); + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx); + guard.set_batch_size(1); + guard.set_batch_idx(0); + sql::EvalBound eval_bound(1, true); + int64_t mock_skip_data = 0; + ObBitVector *mock_skip = to_bit_vector(&mock_skip_data); + char *default_val = nullptr; + int32_t default_val_len = 0; + if (OB_UNLIKELY(params.count() > NUM_LEAD_LAG_PARAMS || params.count() <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid number of params", K(ret), K(params.count())); + } else if (FALSE_IT(ctx.win_col_.op_.clear_evaluated_flag())) { + } else if (OB_FAIL(ctx.input_rows_.attach_rows(all_exprs, input_row_meta, eval_ctx, row_idx, + row_idx + 1, false))) { + LOG_WARN("attach rows failed", K(ret)); + } else { + for (int j = 0; OB_SUCC(ret) && j < params.count(); j++) { + if (OB_ISNULL(params.at(j))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid null param", K(ret), K(j)); + } else if (OB_FAIL(params.at(j)->eval_vector(eval_ctx, *mock_skip, eval_bound))) { + LOG_WARN("eval vector failed", K(ret)); + } else if (j == DEFAULT_VALUE && !params.at(j)->get_vector(eval_ctx)->is_null(0)) { + const char *payload = nullptr; + int32_t len = 0; + params.at(j)->get_vector(eval_ctx)->get_payload(0, payload, len); + if (OB_UNLIKELY(len == 0)) { + default_val_len = len; + default_val = nullptr; + } else if (OB_ISNULL(default_val = (char *)ctx.allocator_.alloc(len))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(default_val, payload, len); + default_val_len = len; + } + } else { + is_lead_lag_offset_used |= (j == OFFSET); + } + } + } + int64_t offset = 0; + if (OB_FAIL(ret)) { + } else if (is_lead_lag_offset_used) { + NonAggrWinExpr::ParamStatus param_status; + if (OB_FAIL(NonAggrWinExpr::eval_param_int_value(params.at(OFFSET), eval_ctx, false, false, + param_status))) { + LOG_WARN("eval param int value failed", K(ret)); + } else if (OB_UNLIKELY(param_status.is_null_ || param_status.int_val_ < 0 + || (lib::is_oracle_mode() && ctx.win_col_.wf_info_.is_ignore_null_ + && param_status.int_val_ == 0))) { + ret = OB_ERR_ARGUMENT_OUT_OF_RANGE; + if (!param_status.is_null_) { + LOG_USER_ERROR(OB_ERR_ARGUMENT_OUT_OF_RANGE, param_status.int_val_); + } + LOG_WARN("lead/lag argument is out of range", K(ret), K(param_status.is_null_), + K(param_status.int_val_)); + } else { + offset = param_status.int_val_; + } + } else { + offset = 1; // default to 1 + } + LOG_DEBUG("lead/lag expr", K(is_lead_lag_offset_used), K(offset), K(lead_lag_offset_direction)); + if (OB_SUCC(ret)) { + // FIXME: opt this code + int64_t step = 0; + bool found = false; + const char *src = nullptr; + int32_t src_len = 0; + bool src_isnull = false; + for (int64_t i = row_idx; OB_SUCC(ret) && !found && i >= frame.head_ && i < frame.tail_; + i += lead_lag_offset_direction) { + ctx.win_col_.op_.clear_evaluated_flag(); + if (OB_FAIL( + ctx.input_rows_.attach_rows(all_exprs, input_row_meta, eval_ctx, i, i + 1, false))) { + LOG_WARN("attach rows failed", K(ret)); + } else if (OB_FAIL(params.at(0)->eval_vector(eval_ctx, *mock_skip, eval_bound))) { + LOG_WARN("eval vector failed", K(ret)); + } else if (ctx.win_col_.wf_info_.is_ignore_null_ + && params.at(0)->get_vector(eval_ctx)->is_null(0)) { + step = (i == row_idx) ? step + 1 : step; + } else if (step++ == offset) { + src_isnull = params.at(0)->get_vector(eval_ctx)->is_null(0); + params.at(0)->get_vector(eval_ctx)->get_payload(0, src, src_len); + found = true; + } + } + VecValueTypeClass res_tc = params.at(0)->get_vec_value_tc(); + if (OB_SUCC(ret)) { + if (!found) { + if (default_val != nullptr) { + if (OB_FAIL(memcpy_results(ctx, res_tc, res, default_val, default_val_len))) { + LOG_WARN("copy results failed", K(ret)); + } + } else { + is_null = true; + } + } else if (src_isnull) { + is_null = true; + } else if (OB_FAIL(memcpy_results(ctx, res_tc, res, src, src_len))) { + LOG_WARN("copy results failed", K(ret)); + } + } + } + return ret; +} + +int LeadOrLag::generate_extra(ObIAllocator &allocator, void *&extra) +{ + int ret = OB_SUCCESS; + extra = nullptr; + return ret; +} + +int CumeDist::process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) +{ + int ret = OB_SUCCESS; + int64_t same_idx = row_idx; + ObIArray &all_exprs = ctx.win_col_.op_.get_all_expr(); + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + LastCompactRow ref_row(ctx.allocator_); + const ObCompactRow *iter_row = nullptr; + if (OB_FAIL(ref_row.init_row_meta(all_exprs, 0, true))) { + LOG_WARN("init row meta failed", K(ret)); + } else if (OB_FAIL(ctx.input_rows_.get_row(row_idx, iter_row))) { + LOG_WARN("get row failed", K(ret)); + } else if (OB_FAIL(ref_row.save_store_row(*iter_row))) { + LOG_WARN("save store row failed", K(ret)); + } + bool should_continue = true; + ExprFixedArray &sort_cols = ctx.win_col_.wf_info_.sort_exprs_; + ObSortCollations &sort_collations = ctx.win_col_.wf_info_.sort_collations_; + ObSortFuncs &sort_cmp_funcs = ctx.win_col_.wf_info_.sort_cmp_funcs_; + const RowMeta &input_row_meta = ctx.win_col_.op_.get_input_row_meta(); + while (should_continue && OB_SUCC(ret) && same_idx + 1 < frame.tail_) { + if (OB_FAIL(ctx.input_rows_.get_row(same_idx + 1, iter_row))) { + LOG_WARN("get row failed", K(ret)); + } else { + int cmp_ret = 0; + const char *l_data = nullptr, *r_data = nullptr; + int32_t l_len = 0, r_len = 0; + bool l_isnull = false, r_isnull = false; + for (int i = 0; OB_SUCC(ret) && should_continue && i < sort_cols.count(); i++) { + ObObjMeta &obj_meta = sort_cols.at(i)->obj_meta_; + int64_t field_idx = sort_collations.at(i).field_idx_; + sql::NullSafeRowCmpFunc cmp_fn = sort_cmp_funcs.at(i).row_cmp_func_; + iter_row->get_cell_payload(input_row_meta, field_idx, l_data, l_len); + l_isnull = iter_row->is_null(field_idx); + ref_row.compact_row_->get_cell_payload(input_row_meta, field_idx, r_data, r_len); + r_isnull = ref_row.compact_row_->is_null(field_idx); + if (OB_FAIL(cmp_fn(obj_meta, obj_meta, l_data, l_len, l_isnull, r_data, r_len, r_isnull, + cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else if (cmp_ret != 0) { + should_continue = false; + } + } + if (OB_SUCC(ret) && should_continue) { + same_idx++; + } + } + } + if (OB_SUCC(ret)) { + if (ob_is_number_tc(ctx.win_col_.wf_info_.expr_->datum_meta_.type_)) { + number::ObNumber numerator; + number::ObNumber denominator; + number::ObNumber res_nmb; + + ObNumStackAllocator<3> tmp_alloc; + if (OB_FAIL(numerator.from(same_idx - frame.head_ + 1, tmp_alloc))) { + LOG_WARN("number::from failed", K(ret)); + } else if (OB_FAIL(denominator.from(frame.tail_ - frame.head_, tmp_alloc))) { + LOG_WARN("number::from failed", K(ret)); + } else if (OB_FAIL(numerator.div(denominator, res_nmb, tmp_alloc))) { + LOG_WARN("failed to div number", K(ret)); + } else { + number::ObCompactNumber *res_cnum = reinterpret_cast(res); + res_cnum->desc_ = res_nmb.d_; + MEMCPY(&(res_cnum->digits_[0]), res_nmb.get_digits(), sizeof(uint32_t) * res_nmb.d_.len_); + is_null = false; + } + } else if (ObDoubleType == ctx.win_col_.wf_info_.expr_->datum_meta_.type_) { + double numerator, denominator; + numerator = static_cast(same_idx - frame.head_ + 1); + denominator = static_cast(frame.tail_ - frame.head_); + *reinterpret_cast(res) = (numerator / denominator); + is_null = false; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("he result type of window function is unexpected", K(ret), K(ctx.win_col_.wf_info_)); + } + } + return ret; +} + +int CumeDist::generate_extra(ObIAllocator &allocator, void *&extra) +{ + int ret = OB_SUCCESS; + extra = nullptr; + return ret; +} + +int RowNumber::process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) +{ + int ret = OB_SUCCESS; + int64_t row_nmb = row_idx - frame.head_ + 1; + if (lib::is_oracle_mode()) { + number::ObNumber res_nmb; + ObNumStackOnceAlloc tmp_alloc; + if (OB_FAIL(res_nmb.from(row_nmb, tmp_alloc))) { + LOG_WARN("number::from failed", K(ret)); + } else { + number::ObCompactNumber *cnum = reinterpret_cast(res); + cnum->desc_ = res_nmb.d_; + MEMCPY(&(cnum->digits_[0]), res_nmb.get_digits(), res_nmb.d_.len_ * sizeof(uint32_t)); + } + } else { + *reinterpret_cast(res) = row_nmb; + } + return ret; +} + +int RowNumber::generate_extra(ObIAllocator &allocator, void *&extra) +{ + int ret = OB_SUCCESS; + extra = nullptr; + return ret; +} + +int AggrExpr::process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *agg_row, bool &is_null) +{ + int ret = OB_SUCCESS; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + ObExpr *agg_expr = ctx.win_col_.wf_info_.expr_; + ObWindowFunctionVecOp &op = ctx.win_col_.op_; + int64_t total_size = frame.tail_ - frame.head_; + const RowMeta &input_row_meta = op.get_input_row_meta(); + ObBitVector &eval_skip = *op.get_batch_ctx().bound_eval_skip_; + ObEvalCtx::BatchInfoScopeGuard guard(op.get_eval_ctx()); + int64_t row_start = frame.head_; + ObBatchRows tmp_brs; + aggregate::RemovalInfo &removal_info = ctx.win_col_.agg_ctx_->removal_info_; + LOG_DEBUG("aggregate expr process window", K(frame), K(removal_info), K(row_start)); + char *res_buf = nullptr; + int total_calc_size = 0, calc_cnt = 0; + while (OB_SUCC(ret) && total_size > 0) { + op.clear_evaluated_flag(); + int64_t batch_size = std::min(total_size, op.get_spec().max_batch_size_); + guard.set_batch_size(batch_size); + tmp_brs.size_ = batch_size; + tmp_brs.end_ = false; + tmp_brs.skip_ = &eval_skip; + total_calc_size += batch_size; + calc_cnt += 1; + if (OB_FAIL(ctx.input_rows_.attach_rows(op.get_all_expr(), input_row_meta, eval_ctx, row_start, + row_start + batch_size, false))) { + LOG_WARN("attach rows failed", K(ret)); + } else if (OB_FAIL(calc_pushdown_skips(ctx, batch_size, eval_skip, tmp_brs.all_rows_active_))) { + LOG_WARN("calc pushdown skips failed", K(ret)); + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(aggr_processor_->eval_aggr_param_batch(tmp_brs))) { + LOG_WARN("eval aggr params failed", K(ret)); + } else if (OB_FAIL(aggr_processor_->add_batch_rows(0, 1, agg_row, tmp_brs, (uint16_t)0, + batch_size))) { + LOG_WARN("add batch rows failed", K(ret)); + } else if (ctx.win_col_.wf_info_.remove_type_ == REMOVE_EXTRENUM + && removal_info.is_max_min_idx_changed_) { + removal_info.max_min_index_ += row_start; + removal_info.is_max_min_idx_changed_ = false; + } + if (OB_SUCC(ret)) { + total_size -= batch_size; + row_start += batch_size; + } + // if result is variable-length type, address stored in agg_row maybe invalid after `attach_rows` + // thus we copy results into res_buf and store corresponding address instread. + bool is_res_not_null = ctx.win_col_.agg_ctx_->row_meta().locate_notnulls_bitmap(agg_row).at(0); + if (ctx.win_col_.agg_ctx_->row_meta().is_var_len(0) && is_res_not_null) { + int64_t addr_val = *reinterpret_cast(ctx.win_col_.agg_ctx_->row_meta().locate_cell_payload(0, agg_row)); + int32_t val_len = ctx.win_col_.agg_ctx_->row_meta().get_cell_len(0, agg_row); + const char *val = reinterpret_cast(addr_val); + if (val != res_buf && val_len > 0) { // new value + res_buf = ctx.reserved_buf(val_len); + if (OB_ISNULL(res_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(res_buf, val, val_len); + addr_val = reinterpret_cast(res_buf); + *reinterpret_cast(ctx.win_col_.agg_ctx_->row_meta().locate_cell_payload(0, agg_row)) = addr_val; + } + } + } + } // end while + if (OB_FAIL(ret)) { + } else if (aggregate::agg_res_not_null(ctx.win_col_.wf_info_.func_type_)) { + ctx.win_col_.agg_ctx_->row_meta().locate_notnulls_bitmap(agg_row).set(0); + } else if (removal_info.enable_removal_opt_ && !frame.is_accum_frame_) { + if (removal_info.null_cnt_ == frame.tail_ - frame.head_) { + ctx.win_col_.agg_ctx_->row_meta().locate_notnulls_bitmap(agg_row).unset(0); + } else { + ctx.win_col_.agg_ctx_->row_meta().locate_notnulls_bitmap(agg_row).set(0); + } + } + return ret; +} + +int AggrExpr::set_result_for_invalid_frame(WinExprEvalCtx &ctx, char *agg_row) +{ + int ret = OB_SUCCESS; + ObExprOperatorType fun_type = ctx.win_col_.wf_info_.func_type_; + aggregate::RuntimeContext &agg_ctx = *ctx.win_col_.agg_ctx_; + aggregate::NotNullBitVector ¬_nulls = agg_ctx.row_meta().locate_notnulls_bitmap(agg_row); + switch(fun_type) + { + case T_FUN_COUNT: { + *reinterpret_cast(agg_ctx.row_meta().locate_cell_payload(0, agg_row)) = 0; + not_nulls.set(0); + break; + } + case T_FUN_SYS_BIT_AND: + case T_FUN_SYS_BIT_OR: + case T_FUN_SYS_BIT_XOR: { + uint64_t res_val = (fun_type == T_FUN_SYS_BIT_AND ? UINT_MAX_VAL[ObUInt64Type] : 0); + *reinterpret_cast(agg_ctx.row_meta().locate_cell_payload(0, agg_row)) = res_val; + not_nulls.set(0); + break; + } + default: { + not_nulls.unset(0); + break; + } + } + return ret; +} + +int AggrExpr::calc_pushdown_skips(WinExprEvalCtx &ctx, const int64_t batch_size, + sql::ObBitVector &skip, bool &all_active) +{ + int ret = OB_SUCCESS; + skip.unset_all(0, batch_size); + const ObWindowFunctionVecSpec &spec = static_cast(ctx.win_col_.op_.get_spec()); + ObWindowFunctionVecOp &op = ctx.win_col_.op_; + all_active = true; + if (spec.is_push_down()) { + VectorFormat fmt = spec.wf_aggr_status_expr_->get_format(op.get_eval_ctx()); + VecValueTypeClass tc = spec.wf_aggr_status_expr_->get_vec_value_tc(); + if (OB_UNLIKELY(fmt != VEC_FIXED || tc != VEC_TC_INTEGER)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format & type class", K(ret), K(fmt), K(tc)); + } else { + if (spec.is_participator()) { + // participator's status code is filled by window function, data format must be VEC_FIXED + if (OB_UNLIKELY(spec.wf_aggr_status_expr_->get_format(op.get_eval_ctx()) != VEC_FIXED)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret), + K(spec.wf_aggr_status_expr_->get_format(op.get_eval_ctx()))); + } else { + ObFixedLengthFormat *data = static_cast *>( + spec.wf_aggr_status_expr_->get_vector(op.get_eval_ctx())); + for (int i = 0; i < batch_size; i++) { + int64_t status = *reinterpret_cast(data->get_payload(i)); + if (status < 0) { + skip.set(i); + all_active = false; + } + } + } + } else if (spec.is_consolidator()) { + int64_t wf_idx = ctx.win_col_.wf_idx_; + VectorFormat status_fmt = spec.wf_aggr_status_expr_->get_format(op.get_eval_ctx()); + if (status_fmt == VEC_FIXED) { + ObFixedLengthFormat *data = static_cast *>( + spec.wf_aggr_status_expr_->get_vector(op.get_eval_ctx())); + for (int i = 0; i < batch_size; i++) { + int64_t status = *reinterpret_cast(data->get_payload(i)); + if ((status < 0 && -status != wf_idx) || (status >= 0 && status < wf_idx)) { + skip.set(i); + all_active = false; + } + } + } else if (status_fmt == VEC_UNIFORM) { + ObUniformFormat *data = static_cast *>( + spec.wf_aggr_status_expr_->get_vector(op.get_eval_ctx())); + for (int i = 0; i < batch_size; i++) { + int64_t status = *reinterpret_cast(data->get_payload(i)); + if ((status < 0 && -status != wf_idx) || (status >= 0 && status < wf_idx)) { + skip.set(i); + all_active = false; + } + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret), K(status_fmt)); + } + } + } + } + return ret; +} + +int AggrExpr::accum_process_window(WinExprEvalCtx &ctx, const Frame &cur_frame, + const Frame &prev_frame, const int64_t row_idx, char *agg_row, + bool &is_null) +{ + int ret = OB_SUCCESS; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + ObExpr *agg_expr = ctx.win_col_.wf_info_.expr_; + ObWindowFunctionVecOp &op = ctx.win_col_.op_; + int64_t head_l = std::min(cur_frame.head_, prev_frame.head_); + int64_t head_r = std::max(cur_frame.head_, prev_frame.head_); + int64_t tail_l = std::min(cur_frame.tail_, prev_frame.tail_); + int64_t tail_r = std::max(cur_frame.tail_, prev_frame.tail_); + int64_t total_size = (head_r - head_l) + (tail_r - tail_l); + Frame new_frame(head_l, head_r, true), new_frame2(tail_l, tail_r, true); + aggregate::RemovalInfo &removal_info = ctx.win_col_.agg_ctx_->removal_info_; + ctx.win_col_.agg_ctx_->set_inverse_agg(prev_frame.head_ < cur_frame.head_); + int64_t cur_idx = eval_ctx.get_batch_idx(); + if (OB_UNLIKELY(cur_idx < 0 || cur_idx >= eval_ctx.get_batch_size())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected idx", K(ret), K(cur_idx)); + } else { + char *prev_row = (cur_idx == 0 ? last_aggr_row_ : ctx.win_col_.aggr_rows_[cur_idx - 1]); + if (OB_FAIL(copy_aggr_row(ctx, prev_row, agg_row))) { + LOG_WARN("copy aggr row failed", K(ret)); + } + } + if (OB_FAIL(ret)) { // TODO: if frame size is small, should `add_one_row` for `process_window` @optimize + } else if (!new_frame.is_empty() + && OB_FAIL(process_window(ctx, new_frame, row_idx, agg_row, is_null))) { + LOG_WARN("process window failed", K(ret)); + } else if (FALSE_IT(ctx.win_col_.agg_ctx_->set_inverse_agg(cur_frame.tail_ < prev_frame.tail_))) { + } else if (!new_frame2.is_empty() + && OB_FAIL(process_window(ctx, new_frame2, row_idx, agg_row, is_null))) { + LOG_WARN("process window failed", K(ret)); + } + if (OB_SUCC(ret) && !aggregate::agg_res_not_null(ctx.win_col_.wf_info_.func_type_) + && removal_info.enable_removal_opt_) { + if (removal_info.null_cnt_ == cur_frame.tail_ - cur_frame.head_) { + ctx.win_col_.agg_ctx_->row_meta().locate_notnulls_bitmap(agg_row).unset(0); + } else { + ctx.win_col_.agg_ctx_->row_meta().locate_notnulls_bitmap(agg_row).set(0); + } + } + return ret; +} + +void AggrExpr::destroy() +{ + if (aggr_processor_ != nullptr) { + aggr_processor_->destroy(); + } +} + +int AggrExpr::collect_part_results(WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) +{ + int ret = OB_SUCCESS; + aggregate::IAggregate *iagg = aggr_processor_->get_aggregates().at(0); + aggregate::RuntimeContext &agg_ctx = *ctx.win_col_.agg_ctx_; + int64_t batch_size = row_end - row_start; + int32_t output_size = 0; + if (OB_FAIL(iagg->collect_batch_group_results(agg_ctx, 0, 0, 0, batch_size, output_size, &skip))) { + LOG_WARN("collect batch group results failed", K(ret)); + } + return ret; +} + +template +int AggrExpr::set_payload(WinExprEvalCtx &ctx, ColumnFmt *columns, const int64_t idx, + const char *payload, int32_t len) +{ + int ret = OB_SUCCESS; + ObExpr *res_expr = ctx.win_col_.wf_info_.expr_; + VecValueTypeClass vec_tc = res_expr->get_vec_value_tc(); + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + const ObWindowFunctionVecSpec &spec = static_cast(ctx.win_col_.op_.get_spec()); + // guard used for `get_str_res_mem` + ObEvalCtx::BatchInfoScopeGuard guard(eval_ctx); + guard.set_batch_idx(idx); + // count function in consolidator is T_FUN_COUNT, not T_FUN_COUNT_SUM!!! + bool is_count_sum = (T_FUN_COUNT == ctx.win_col_.wf_info_.func_type_ && spec.is_consolidator()); + if (T_FUN_COUNT != ctx.win_col_.wf_info_.func_type_ || is_count_sum || lib::is_mysql_mode()) { + if (is_fixed_length_vec(vec_tc)) { + columns->set_payload(idx, payload, len); + } else if (vec_tc == VEC_TC_NUMBER) { + columns->set_number(idx, *reinterpret_cast(payload)); + } else { + char *res_buf = res_expr->get_str_res_mem(eval_ctx, len); + if (OB_ISNULL(res_buf)) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory failed", K(ret)); + } else { + MEMCPY(res_buf, payload, len); + columns->set_payload_shallow(idx, res_buf, len); + } + } + } else { + number::ObNumber res_nmb; + if (OB_FAIL(res_nmb.from(*reinterpret_cast(payload), ctx.allocator_))) { + LOG_WARN("cast to number failed", K(ret)); + } else { + columns->set_number(idx, res_nmb); + } + } + return ret; +} +// >>>>>>>> helper functions +int eval_bound_exprs(WinExprEvalCtx &ctx, const int64_t row_start, const int64_t batch_size, + const ObBitVector &skip, const bool is_upper) +{ + int ret = OB_SUCCESS; + OB_ASSERT(batch_size <= ctx.win_col_.op_.get_spec().max_batch_size_); + WinFuncInfo &wf_info = ctx.win_col_.wf_info_; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + ObBitVector *eval_skip = ctx.win_col_.op_.get_batch_ctx().bound_eval_skip_; + int64_t *pos_arr = (is_upper ? ctx.win_col_.op_.get_batch_ctx().upper_pos_arr_ : + ctx.win_col_.op_.get_batch_ctx().lower_pos_arr_); + bool is_rows = (wf_info.win_type_ == WINDOW_ROWS); + bool is_preceding = (is_upper ? wf_info.upper_.is_preceding_ : wf_info.lower_.is_preceding_); + bool is_unbounded = (is_upper ? wf_info.upper_.is_unbounded_ : wf_info.lower_.is_unbounded_); + bool is_nmb_literal = (is_upper ? wf_info.upper_.is_nmb_literal_ : wf_info.lower_.is_nmb_literal_); + ObExpr *between_value_expr = + (is_upper ? wf_info.upper_.between_value_expr_ : wf_info.lower_.between_value_expr_); + ObExpr *bound_expr = + (is_upper ? wf_info.upper_.range_bound_expr_ : wf_info.lower_.range_bound_expr_); + + eval_skip->deep_copy(skip, batch_size); + MEMSET(pos_arr, -1, batch_size * sizeof(int64_t)); + LOG_DEBUG("eval bound exprs", K(is_rows), K(is_upper), K(is_preceding), K(is_unbounded), + K(is_nmb_literal), K(batch_size), KPC(between_value_expr), KPC(bound_expr), + K(wf_info.sort_exprs_), K(row_start), K(ctx.win_col_.part_first_row_idx_), + K(ctx.win_col_.op_.get_part_end_idx())); + bool is_finished = false; + + if (NULL == between_value_expr && is_unbounded) { + // no care rows if range, no need to evaluated; + if (is_preceding) { + for (int i = 0; i < batch_size; i++) { + if (eval_skip->at(i)) { continue; } + pos_arr[i] = ctx.win_col_.part_first_row_idx_; + } + } else { + for (int i = 0; i < batch_size; i++) { + if (eval_skip->at(i)) { continue; } + pos_arr[i] = ctx.win_col_.op_.get_part_end_idx(); + } + } + is_finished = true; + } else if (NULL == between_value_expr && !is_unbounded) { + // current row by rows/range, no need to evaluate bound exprs + if (OB_FAIL(calc_borders_for_current_row(ctx, row_start, batch_size, *eval_skip, is_upper))) { + LOG_WARN("calc borders for current_row failed", K(ret)); + } else { + is_finished = true; + } + } else if (is_nmb_literal) { + if (OB_ISNULL(between_value_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null between value expr", K(ret)); + } else if (OB_UNLIKELY(lib::is_mysql_mode() && is_rows + && !between_value_expr->obj_meta_.is_integer_type())) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or non-integral type", K(ret), + K(between_value_expr->obj_meta_)); + } else if (OB_FAIL(eval_and_check_between_literal(ctx, *eval_skip, between_value_expr, + batch_size, pos_arr))) { + LOG_WARN("eval and check between literal is failed", K(ret)); + } + } + // between ... and ... + if (OB_FAIL(ret)) { + } else if (is_finished) { + } else if (is_rows) { + if (OB_FAIL(calc_borders_for_rows_between(ctx, row_start, batch_size, *eval_skip, + between_value_expr, is_preceding, is_upper, pos_arr))) { + LOG_WARN("calculate borders for `rows between ... and ...` failed", K(ret)); + } + } else if (OB_ISNULL(bound_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null bound expr", K(ret)); + } else if (wf_info.sort_exprs_.count() == 0) { + if (OB_FAIL(calc_borders_for_no_sort_expr(ctx, batch_size, *eval_skip, bound_expr, is_upper, + pos_arr))) { + LOG_WARN("calc borders failed", K(ret)); + } + } else if (wf_info.sort_exprs_.count() != 1) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("only need one sort expr", K(ret)); + } else if (OB_FAIL(calc_borders_for_sort_expr(ctx, bound_expr, batch_size, row_start, *eval_skip, + is_upper, pos_arr))) { + LOG_WARN("calc borders failed", K(ret)); + } + return ret; +} + +struct __data_tuple +{ + const char *data_; + int32_t len_; + bool is_null_; + + __data_tuple(const char *data, int32_t len, bool null): data_(data), len_(len), is_null_(null) {} + __data_tuple(): data_(), len_(), is_null_(true) {} + + __data_tuple(const __data_tuple &other) : + data_(other.data_), len_(other.len_), is_null_(other.is_null_) + {} + + TO_STRING_KV(KP_(data), K_(len), K_(is_null)); +}; + +int calc_borders_for_current_row(winfunc::WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t batch_size, const ObBitVector &skip, + const bool is_upper) +{ + int ret = OB_SUCCESS; + WinFuncInfo &wf_info = ctx.win_col_.wf_info_; + bool is_rows = (wf_info.win_type_ == WINDOW_ROWS); + int64_t *pos_arr = (is_upper ? ctx.win_col_.op_.get_batch_ctx().upper_pos_arr_: + ctx.win_col_.op_.get_batch_ctx().lower_pos_arr_); + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + if (is_rows) { + for (int i = 0; i < batch_size; i++) { + if (skip.at(i)) { continue; } + pos_arr[i] = row_start + i + (is_upper ? 0 : 1); + } + } else { + // range + // for current row, it's no sense for is_preceding + // we should jump to detect step by step(for case that the sort columns has very small ndv) + + // Exponential detection + int32_t step = 1; + int pos = row_start, prev_row_pos = -1; + ObSortCollations &sort_collations = wf_info.sort_collations_; + ObSortFuncs &sort_cmp_funcs = wf_info.sort_cmp_funcs_; + ObExprPtrIArray &all_exprs = ctx.win_col_.op_.get_all_expr(); + const char *l_payload = nullptr, *r_payload = nullptr; + int32_t l_len = 0, r_len = 0; + int cmp_ret = 0; + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + if (skip.at(i)) { continue; } + int32_t cur_idx = i; + if (prev_row_pos != -1) { + if (OB_FAIL(cmp_prev_row(ctx, i + row_start, cmp_ret))) { + LOG_WARN("compare previous row failed", K(ret)); + } else if (cmp_ret == 0) { // same as before + pos_arr[i] = prev_row_pos; + continue; + } else if (is_upper) { + // cur_row != prev_row, cur_row's upper border equals to cur_idx + pos_arr[i] = cur_idx + row_start; + prev_row_pos = pos_arr[i]; + continue; + } + } + if (OB_FAIL(ret)) { + } else { + const ObCompactRow *a_row = nullptr; + int32_t step = 1; + int32_t pos = cur_idx + row_start; + ObSEArray<__data_tuple, 16> cur_row_tuple; + for (int i = 0; OB_SUCC(ret) && i < sort_collations.count(); i++) { + const char *payload = nullptr; + int32_t len = 0; + bool is_cur_null = false; + int64_t field_idx = sort_collations.at(i).field_idx_; + ObIVector *data = all_exprs.at(field_idx)->get_vector(eval_ctx); + data->get_payload(cur_idx, payload, len); + is_cur_null = data->is_null(cur_idx); + if (OB_FAIL(cur_row_tuple.push_back(__data_tuple(payload, len, is_cur_null)))) { + LOG_WARN("push back element failed", K(ret)); + } + } + + while (OB_SUCC(ret)) { + bool found_border = false; + is_upper ? (pos -= step) : (pos += step); + bool overflow = (is_upper ? (pos < ctx.win_col_.part_first_row_idx_) : + (pos >= ctx.win_col_.op_.get_part_end_idx())); + if (overflow) { + found_border = true; + } else if (OB_FAIL(ctx.input_rows_.get_row(pos, a_row))) { + LOG_WARN("get stored row failed", K(ret)); + } else { + cmp_ret = 0; + for (int j = 0; OB_SUCC(ret) && !found_border && j < sort_collations.count(); j++) { + const int64_t field_idx = sort_collations.at(j).field_idx_; + sql::NullSafeRowCmpFunc cmp_fn = sort_cmp_funcs.at(j).row_cmp_func_; + l_payload = cur_row_tuple.at(j).data_; + l_len = cur_row_tuple.at(j).len_; + bool l_isnull = cur_row_tuple.at(j).is_null_; + a_row->get_cell_payload(ctx.win_col_.op_.get_input_row_meta(), field_idx, r_payload, r_len); + bool r_isnull = a_row->is_null(field_idx); + if (OB_ISNULL(cmp_fn)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null compare function", K(ret)); + } else if (OB_FAIL(cmp_fn(all_exprs.at(field_idx)->obj_meta_, + all_exprs.at(field_idx)->obj_meta_, + l_payload, l_len, l_isnull, + r_payload, r_len, r_isnull, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else { + found_border = (cmp_ret != 0); + } + } + } + if (OB_FAIL(ret)) { + } else if (found_border) { + is_upper ? (pos += step) : (pos -= step); + if (step == 1) { + break; + } else { + step = 1; + } + } else { + step *= 2; + } + } // end inner while + if (OB_SUCC(ret)) { + if (pos < ctx.win_col_.part_first_row_idx_ || pos >= ctx.win_col_.op_.get_part_end_idx()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid position", K(ret), K(pos)); + } else { + pos_arr[i] = pos + (is_upper ? 0 : 1); + prev_row_pos = pos_arr[i]; + } + } + } + } // end for + } + return ret; +} + +template +static int _check_betweenn_value(const ObExpr *expr, ObEvalCtx &ctx, const ObBitVector &skip, + const EvalBound &bound) +{ + int ret = OB_SUCCESS; + Fmt *columns = static_cast(expr->get_vector(ctx)); + const char *payload = nullptr; + int32_t len = 0; + int64_t value = 0; + for (int i = bound.start(); OB_SUCC(ret) && i < bound.end(); i++) { + if (skip.at(i) || columns->is_null(i)) { continue; } + columns->get_payload(i, payload, len); + ret = int_trunc::get(payload, len, expr->datum_meta_, value); + if (OB_FAIL(ret)) { + LOG_WARN("truncate integer failed", K(ret)); + } else if (OB_UNLIKELY(value < 0)) { + if (lib::is_mysql_mode()) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("rame start or end is negative, NULL or of non-integral type", K(ret), K(value)); + } else { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("invaid argument", K(ret), K(value)); + } + } + } + return ret; +} +// if border is_nmb_literal == true, check value of between_expr is_valid +// in mysql mode, null value is invalid with error reporting +// in oracle mode, null value is invalid without error, just set invalid frame +#define CHECK_BTW_FIXED_VAL(vec_tc) \ + case (vec_tc): { \ + ret = _check_betweenn_value>, vec_tc>( \ + between_expr, eval_ctx, eval_skip, eval_bound); \ + } break + +#define CHECK_BTW_UNI_VAL(vec_tc) \ + case (vec_tc): { \ + ret = _check_betweenn_value, vec_tc>(between_expr, eval_ctx, eval_skip, \ + eval_bound); \ + } break + +#define CHECK_BTW_CONST_VAL(vec_tc) \ + case (vec_tc): { \ + ret = _check_betweenn_value, vec_tc>(between_expr, eval_ctx, eval_skip, \ + eval_bound); \ + } break + +int eval_and_check_between_literal(winfunc::WinExprEvalCtx &ctx, ObBitVector &eval_skip, + const ObExpr *between_expr, const int64_t batch_size, int64_t *pos_arr) +{ + int ret = OB_SUCCESS; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + EvalBound eval_bound(batch_size, false); + if (OB_ISNULL(between_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null between expr", K(ret)); + } else if (OB_FAIL(between_expr->eval_vector(eval_ctx, eval_skip, eval_bound))) { + LOG_WARN("expr evaluation failed", K(ret)); + } else { + VectorFormat fmt = between_expr->get_format(eval_ctx); + VecValueTypeClass vec_tc = between_expr->get_vec_value_tc(); + switch (fmt) { + case common::VEC_DISCRETE: + case common::VEC_CONTINUOUS: + case common::VEC_FIXED: { + ObBitmapNullVectorBase *data = static_cast(between_expr->get_vector(eval_ctx)); + if (lib::is_mysql_mode()) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + if (eval_skip.at(i)) { continue; } + if (data->is_null(i)) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or non-integral type", K(ret)); + } + } + } else { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + if (eval_skip.at(i)) { continue; } + if (data->is_null(i)) { + // frame of current must be invalid, + // we set pos_arr[i] to INT64_MAX to represent invalid frame border + pos_arr[i] = INT64_MAX; + } + } + } + } break; + case common::VEC_UNIFORM: { + ObUniformFormat *data = static_cast *>(between_expr->get_vector(eval_ctx)); + if (lib::is_mysql_mode()) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + if (eval_skip.at(i)) { continue; } + if (data->is_null(i)) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is_negative, NULL or non-integral type", K(ret)); + } + } + } else { + for (int i = 0; i < batch_size; i++) { + if (eval_skip.at(i)) { continue; } + if (data->is_null(i)) { + pos_arr[i] = INT64_MAX; + } + } + } + } break; + case common::VEC_UNIFORM_CONST: { + ObUniformFormat *data = static_cast *>(between_expr->get_vector(eval_ctx)); + bool has_null = false; + for (int i = 0; i < batch_size; i++) { + if (eval_skip.at(i)) {continue; } + has_null = data->is_null(i); + break; + } + if (has_null) { + if (lib::is_mysql_mode()) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or non-integral type", K(ret)); + } else { + for (int i = 0; i < batch_size; i++) { + if (eval_skip.at(i)) { continue; } + pos_arr[i] = INT64_MAX; + } + } + } + } break; + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected format", K(ret), K(fmt)); + } + } + if (OB_SUCC(ret)) { + // check interval value valid + switch(fmt) { + case common::VEC_DISCRETE: { + ret = _check_betweenn_value(between_expr, eval_ctx, eval_skip, eval_bound); + break; + } + case common::VEC_FIXED: { + switch (vec_tc) { + LST_DO_CODE(CHECK_BTW_FIXED_VAL, VEC_TC_INTEGER, VEC_TC_UINTEGER, VEC_TC_FLOAT, + VEC_TC_DOUBLE, VEC_TC_FIXED_DOUBLE, VEC_TC_BIT, VEC_TC_DEC_INT32, + VEC_TC_DEC_INT64, VEC_TC_DEC_INT128, VEC_TC_DEC_INT256, VEC_TC_DEC_INT512); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not supported vec type class", K(vec_tc)); + } + } + break; + } + case common::VEC_UNIFORM: { + switch (vec_tc) { + LST_DO_CODE(CHECK_BTW_UNI_VAL, VEC_TC_NUMBER, VEC_TC_INTEGER, VEC_TC_UINTEGER, + VEC_TC_FLOAT, VEC_TC_DOUBLE, VEC_TC_FIXED_DOUBLE, VEC_TC_BIT, + VEC_TC_DEC_INT32, VEC_TC_DEC_INT64, VEC_TC_DEC_INT128, VEC_TC_DEC_INT256, + VEC_TC_DEC_INT512); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not supported vec type class", K(vec_tc)); + } + } + break; + } + case common::VEC_UNIFORM_CONST: { + switch (vec_tc) { + LST_DO_CODE(CHECK_BTW_CONST_VAL, VEC_TC_NUMBER, VEC_TC_INTEGER, VEC_TC_UINTEGER, + VEC_TC_FLOAT, VEC_TC_DOUBLE, VEC_TC_FIXED_DOUBLE, VEC_TC_BIT, + VEC_TC_DEC_INT32, VEC_TC_DEC_INT64, VEC_TC_DEC_INT128, VEC_TC_DEC_INT256, + VEC_TC_DEC_INT512); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not supported vec type class", K(vec_tc)); + } + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not supported format", K(ret), K(fmt)); + } + } + if (OB_FAIL(ret)) { + LOG_WARN("check between value faile", K(ret)); + } + } + } + return ret; +} + +static OB_INLINE int check_interval_valid(const int64_t row_idx, const int64_t interval, + const bool is_preceding) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(interval < 0)) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("invalid interval", K(ret), K(interval)); + } else if (OB_UNLIKELY(!is_preceding && static_cast(row_idx + interval) > INT64_MAX)) { + if (lib::is_mysql_mode()) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), + K(row_idx + interval)); + } else { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret), K(row_idx + interval)); + } + } + return ret; +} +template +int _calc_borders_for_rows_between(winfunc::WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t batch_size, const ObBitVector &eval_skip, + ColumnFmt *between_data, const ObDatumMeta &meta, + const bool is_preceding, const bool is_upper, int64_t *pos_arr) +{ + int ret = OB_SUCCESS; + const char *payload = nullptr; + int32_t len = 0; + int64_t interval = 0; + if (ob_is_number_tc(meta.type_)) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + int64_t row_idx = row_start + i; + if (eval_skip.at(i) || between_data->is_null(i)) { continue; } + between_data->get_payload(i, payload, len); + const number::ObCompactNumber *cnum = reinterpret_cast(payload); + number::ObNumber result_nmb(*cnum); + if (lib::is_mysql_mode()) { + if (OB_UNLIKELY(result_nmb.is_negative())) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), K(result_nmb)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(result_nmb.extract_valid_int64_with_trunc(interval))) { + LOG_WARN("extract invalid int64 failed", K(ret)); + } else if (OB_FAIL(check_interval_valid(row_idx, interval, is_preceding))) { + LOG_WARN("check interval valid failed", K(ret)); + } else { + pos_arr[i] = (is_preceding ? row_idx - interval : row_idx + interval); + pos_arr[i] += (is_upper ? 0 : 1); + } + } + } else if (ob_is_decimal_int(meta.type_)) { + int16_t in_prec = meta.precision_; + int16_t in_scale = meta.scale_; + int16_t out_scale = 0; + ObDecimalIntBuilder trunc_res_val; + ObDatum in_datum; + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + int64_t row_idx = row_start + i; + if (eval_skip.at(i) || between_data->is_null(i)) { continue; } + between_data->get_payload(i, payload, len); + if (lib::is_mysql_mode()) { + if (OB_UNLIKELY(wide::is_negative(reinterpret_cast(payload), len))) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret)); + } + } else if (in_scale == out_scale) { + trunc_res_val.from(reinterpret_cast(payload), len); + } else { + in_datum.ptr_ = payload; + in_datum.len_ = len; + if (OB_FAIL(sql::ObExprTruncate::do_trunc_decimalint(in_prec, in_scale, in_prec, out_scale, + out_scale, in_datum, trunc_res_val))) { + LOG_WARN("trunc decimal int failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + bool is_in_val_valid = false; + if (OB_FAIL(wide::check_range_valid_int64(trunc_res_val.get_decimal_int(), + trunc_res_val.get_int_bytes(), is_in_val_valid, + interval))) { + LOG_WARN("check range valid int64 failed", K(ret)); + } else if (!is_in_val_valid) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("res_val is not valid int64", K(ret)); + } else if (OB_FAIL(check_interval_valid(row_idx, interval, is_preceding))) { + LOG_WARN("invalid interval", K(ret)); + } else { + pos_arr[i] = (is_preceding ? row_idx - interval : row_idx + interval); + pos_arr[i] += (is_upper ? 0 : 1); + } + } + } + } else if (ob_is_int_tc(meta.type_)) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + int64_t row_idx = row_start + i; + if (eval_skip.at(i) || between_data->is_null(i)) { continue; } + between_data->get_payload(i, payload, len); + interval = *reinterpret_cast(payload); + if (lib::is_mysql_mode() && OB_UNLIKELY(interval < 0)) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), K(interval)); + } else if (OB_FAIL(check_interval_valid(row_idx, interval, is_preceding))) { + LOG_WARN("invalid interval", K(ret)); + } else { + pos_arr[i] = (is_preceding ? row_idx - interval : row_idx + interval); + pos_arr[i] += (is_upper ? 0 : 1); + } + } + } else if (ob_is_uint_tc(meta.type_)) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + int64_t row_idx = row_start + i; + if (eval_skip.at(i) || between_data->is_null(i)) { continue; } + between_data->get_payload(i, payload, len); + uint64_t tmp_val = *reinterpret_cast(payload); + if (lib::is_mysql_mode() && static_cast(tmp_val) < 0) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), K(tmp_val)); + } else if (tmp_val > INT64_MAX) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret), K(tmp_val)); + } else { + interval = static_cast(tmp_val); + if (OB_FAIL(check_interval_valid(row_idx, interval, is_preceding))) { + LOG_WARN("check interval failed", K(ret)); + } else { + pos_arr[i] = (is_preceding ? row_idx - interval : row_idx + interval); + pos_arr[i] += (is_upper ? 0 : 1); + } + } + } + } else if (ob_is_float_tc(meta.type_)) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + int64_t row_idx = row_start + i; + if (eval_skip.at(i) || between_data->is_null(i)) { continue; } + between_data->get_payload(i, payload, len); + float tmp_val = *reinterpret_cast(payload); + if (lib::is_mysql_mode() && tmp_val < 0) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), K(tmp_val)); + } else if (tmp_val > INT64_MAX) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("out of range", K(ret), K(tmp_val)); + } else { + interval = static_cast(tmp_val); + if (OB_FAIL(check_interval_valid(row_idx, interval, is_preceding))) { + LOG_WARN("check interval failed", K(ret)); + } else { + pos_arr[i] = (is_preceding ? row_idx - interval : row_idx + interval); + pos_arr[i] += (is_upper ? 0 : 1); + } + } + } + } else if (ob_is_double_tc(meta.type_)) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + int64_t row_idx = row_start + i; + if (eval_skip.at(i) || between_data->is_null(i)) { continue; } + between_data->get_payload(i, payload, len); + double tmp_val = *reinterpret_cast(payload); + if (lib::is_mysql_mode() && tmp_val < 0) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), K(tmp_val)); + } else if (tmp_val > INT64_MAX) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("out of range", K(ret), K(tmp_val)); + } else { + interval = static_cast(tmp_val); + if (OB_FAIL(check_interval_valid(row_idx, interval, is_preceding))) { + LOG_WARN("check interval failed", K(ret)); + } else { + pos_arr[i] = (is_preceding ? row_idx - interval : row_idx + interval); + pos_arr[i] += (is_upper ? 0 : 1); + } + } + } + } else if (ob_is_bit_tc(meta.type_)) { + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + int64_t row_idx = row_start + i; + if (eval_skip.at(i) || between_data->is_null(i)) { continue; } + between_data->get_payload(i, payload, len); + uint64_t tmp_val = *reinterpret_cast(payload); + if (lib::is_mysql_mode() && static_cast(tmp_val) < 0) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), K(tmp_val)); + } else if (tmp_val > INT64_MAX) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("out of range", K(ret), K(tmp_val)); + } else { + interval = static_cast(tmp_val); + if (OB_FAIL(check_interval_valid(row_idx, interval, is_preceding))) { + LOG_WARN("check interval failed", K(ret)); + } else { + pos_arr[i] = (is_preceding ? row_idx - interval : row_idx + interval); + pos_arr[i] += (is_upper ? 0 : 1); + } + } + } + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not support type", K(ret), K(meta)); + } + return ret; +} + +int calc_borders_for_rows_between(winfunc::WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t batch_size, const ObBitVector &eval_skip, + const ObExpr *between_expr, const bool is_preceding, const bool is_upper, + int64_t *pos_arr) +{ +#define CALC_BORDER(fmt) \ + ret = _calc_borders_for_rows_between(ctx, row_start, batch_size, eval_skip, \ + static_cast(data), meta, is_preceding, is_upper, \ + pos_arr) +#define CALC_FIXED_TYPE_BORDER(vec_tc) \ + case (vec_tc): { CALC_BORDER(ObFixedLengthFormat>); } break + + int ret = OB_SUCCESS; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + VectorFormat fmt = between_expr->get_format(eval_ctx); + const ObDatumMeta &meta = between_expr->datum_meta_; + VecValueTypeClass vec_tc = get_vec_value_tc(meta.type_, meta.scale_, meta.precision_); + ObIVector *data = between_expr->get_vector(eval_ctx); + switch(fmt) { + case common::VEC_UNIFORM: { + CALC_BORDER(ObUniformFormat); + break; + } + case common::VEC_UNIFORM_CONST: { + CALC_BORDER(ObUniformFormat); + break; + } + case common::VEC_DISCRETE: { + CALC_BORDER(ObDiscreteFormat); + break; + } + case common::VEC_CONTINUOUS: { + CALC_BORDER(ObContinuousFormat); + break; + } + case common::VEC_FIXED: { + switch(vec_tc) { + LST_DO_CODE(CALC_FIXED_TYPE_BORDER, + VEC_TC_INTEGER, VEC_TC_UINTEGER, VEC_TC_FLOAT, + VEC_TC_DOUBLE, VEC_TC_FIXED_DOUBLE, VEC_TC_BIT, VEC_TC_DEC_INT32, + VEC_TC_DEC_INT64, VEC_TC_DEC_INT128, VEC_TC_DEC_INT256, VEC_TC_DEC_INT512); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected vector tc", K(ret), K(vec_tc)); + } + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret), K(fmt)); + } + } + return ret; +#undef CALC_BORDER +#undef CALC_FIXED_TYPE_BORDER +} + +int calc_borders_for_no_sort_expr(WinExprEvalCtx &ctx, const int64_t batch_size, + const ObBitVector &eval_skip, const ObExpr *bound_expr, + const bool is_upper, int64_t *pos_arr) +{ + int ret = OB_SUCCESS; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + if (OB_UNLIKELY(!ob_is_integer_type(bound_expr->datum_meta_.type_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid bound value type", K(ret)); + } else if (OB_FAIL(bound_expr->eval_vector(eval_ctx, eval_skip, EvalBound(batch_size, false)))) { + LOG_WARN("expr eval_vector failed", K(ret)); + } else { + VectorFormat fmt = bound_expr->get_format(eval_ctx); + VecValueTypeClass vec_tc = bound_expr->get_vec_value_tc(); + switch(fmt) { + case common::VEC_UNIFORM: { + ObUniformFormat *data = static_cast *>(bound_expr->get_vector(eval_ctx)); + int64_t part_first_idx = ctx.win_col_.part_first_row_idx_; + int64_t part_end_idx = ctx.win_col_.op_.get_part_end_idx(); + for (int i = 0; i < batch_size; i++) { + if (eval_skip.at(i) || pos_arr[i] == INT64_MAX) { continue; } + if (data->is_null(i) || data->get_bool(i)) { + pos_arr[i] = (is_upper ? part_first_idx : part_end_idx); + } else { + pos_arr[i] = (is_upper ? part_end_idx : part_first_idx - 1); + } + } + break; + } + case common::VEC_UNIFORM_CONST: { + ObUniformFormat *data = static_cast *>(bound_expr->get_vector(eval_ctx)); + int64_t part_first_idx = ctx.win_col_.part_first_row_idx_; + int64_t part_end_idx = ctx.win_col_.op_.get_part_end_idx(); + for (int i = 0; i < batch_size; i++) { + if (eval_skip.at(i) || pos_arr[i] == INT64_MAX) { continue; } + if (data->is_null(i) || data->get_bool(i)) { + pos_arr[i] = (is_upper ? part_first_idx : part_end_idx); + } else { + pos_arr[i] = (is_upper ? part_end_idx : part_first_idx - 1); + } + } + break; + } + case common::VEC_FIXED: { + if (vec_tc != VEC_TC_INTEGER && vec_tc != VEC_TC_UINTEGER) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected vec tc", K(ret), K(vec_tc)); + } else { + // just use int64_t as RTCType, results are same for int64_t/uint64_t + ObFixedLengthFormat *data = + static_cast *>(bound_expr->get_vector(eval_ctx)); + int64_t part_first_idx = ctx.win_col_.part_first_row_idx_; + int64_t part_end_idx = ctx.win_col_.op_.get_part_end_idx(); + for (int i = 0; i < batch_size; i++) { + if (eval_skip.at(i) || pos_arr[i] == INT64_MAX) { continue; } + if (data->is_null(i) || data->get_bool(i)) { + pos_arr[i] = (is_upper ? part_first_idx : part_end_idx); + } else { + pos_arr[i] = (is_upper ? part_end_idx : part_first_idx - 1); + } + } + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected data format", K(ret)); + } + } + } + return ret; +} + +template +static int _batch_check_datetime_interval(IntervalFmt *interval_data, UnitFmt *unit_data, + const int64_t batch_size, const ObBitVector &eval_skip) +{ + int ret = OB_SUCCESS; + ObString interval_val; + int64_t unit_value; + ObInterval interval_time; + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + if (eval_skip.at(i)) { + } else if (interval_data->is_null(i) || unit_data->is_null(i)) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), + K(interval_data->is_null(i)), K(unit_data->is_null(i))); + } else { + interval_val = interval_data->get_string(i); + unit_value = unit_data->get_int(i); + ObDateUnitType unit_val = static_cast(unit_value); + if (OB_FAIL(ObTimeConverter::str_to_ob_interval(interval_val, unit_val, interval_time))) { + if (OB_UNLIKELY(OB_INVALID_DATE_VALUE == ret)) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to convert string to ob interval", K(ret)); + } + } else { + bool is_valid = !(DT_MODE_NEG & interval_time.mode_); + if (!is_valid) { + ret = OB_ERR_WINDOW_FRAME_ILLEGAL; + LOG_WARN("frame start or end is negative, NULL or of non-integral type", K(ret), + K(interval_val), K(unit_value)); + } + } + } + } + return ret; +} +static int _check_datetime_interval_valid(ObEvalCtx &eval_ctx, const ObExpr *bound_expr, + const int64_t batch_size, const ObBitVector &eval_skip) +{ + int ret = OB_SUCCESS; + if (bound_expr->type_ == T_FUN_SYS_DATE_ADD || bound_expr->type_ == T_FUN_SYS_DATE_SUB) { + ObExpr *interval_expr = bound_expr->args_[1]; + ObExpr *unit_expr = bound_expr->args_[2]; + if (OB_FAIL(interval_expr->eval_vector(eval_ctx, eval_skip, EvalBound(batch_size, false))) + || OB_FAIL(unit_expr->eval_vector(eval_ctx, eval_skip, EvalBound(batch_size, false)))) { + LOG_WARN("expr evaluation failed", K(ret)); + } else if (OB_LIKELY(interval_expr->get_format(eval_ctx) == VEC_DISCRETE + && unit_expr->get_format(eval_ctx) == VEC_FIXED)) { + ObDiscreteFormat *interval_data = static_cast(interval_expr->get_vector(eval_ctx)); + ObFixedLengthFormat *unit_data = static_cast *>(unit_expr->get_vector(eval_ctx)); + if (OB_FAIL(_batch_check_datetime_interval(interval_data, unit_data, batch_size, eval_skip))) { + LOG_WARN("check failed", K(ret)); + } + } else { + ObIVector *interval_data = interval_expr->get_vector(eval_ctx); + ObIVector *unit_data = unit_expr->get_vector(eval_ctx); + if (OB_FAIL(_batch_check_datetime_interval(interval_data, unit_data, batch_size, eval_skip))) { + LOG_WARN("check failed", K(ret)); + } + } + } + return ret; +} + +template +int _calc_borders_for_sort_expr(WinExprEvalCtx &ctx, const int64_t batch_size, + const int64_t row_start, const ObBitVector &eval_skip, + const bool is_upper, ColumnFmt *bound_data, + const ObExpr *bound_expr, int64_t *pos_arr) +{ + int ret = OB_SUCCESS; + WinFuncInfo &wf_info = ctx.win_col_.wf_info_; + int64_t cell_idx = wf_info.sort_collations_.at(0).field_idx_; + bool is_ascending = wf_info.sort_collations_.at(0).is_ascending_; + bool is_preceding = (is_upper ? wf_info.upper_.is_preceding_ : wf_info.lower_.is_preceding_); + ObExpr *sort_expr = wf_info.sort_exprs_.at(0); + sql::NullSafeRowCmpFunc sort_cmp_fn = wf_info.sort_cmp_funcs_.at(0).row_cmp_func_; + ObObjMeta sort_obj_meta = sort_expr->obj_meta_; + ObObjMeta bound_obj_meta = bound_expr->obj_meta_; + const RowMeta &input_row_meta = ctx.win_col_.op_.get_input_row_meta(); + // range between ... and ... + // copy from `ObWindowFunctionOp::get_pos` + const static int L = 1; + const static int LE = 1 << 1; + const static int G = 1 << 2; + const static int GE = 1 << 3; + const static int ROLL = L | G; + + int64_t prev_row_border = -1; + const char *l_ptr = nullptr, *r_ptr = nullptr; + int32_t l_len = 0, r_len = 0; + int cmp_ret = 0; + bool l_isnull = false, r_isnull = false; + int64_t part_first_idx = ctx.win_col_.part_first_row_idx_; + int64_t part_end_idx = ctx.win_col_.op_.get_part_end_idx(); + for (int i = 0; OB_SUCC(ret) && i < batch_size; i++) { + if (eval_skip.at(i) || pos_arr[i] == INT64_MAX) { continue; } + int64_t row_idx = row_start + i; + bool found = false; + if (prev_row_border != -1) { + if (OB_FAIL(cmp_prev_row(ctx, row_idx, cmp_ret))) { + LOG_WARN("compare previous row failed", K(ret)); + } else if (cmp_ret == 0) { + found = true; + pos_arr[i] = prev_row_border; + } + } + if (OB_FAIL(ret)) { + } else if (!found) { + int64_t pos = row_idx; + bool re_direction = false; + int cmp_mode = !(is_preceding ^ is_ascending) ? L : G; + if (is_preceding ^ is_upper) { + cmp_mode = cmp_mode << 1; // why??? + } + int step = (cmp_mode & ROLL) ? 1 : 0; + int cmp_times = 0; + r_isnull = bound_data->is_null(i); + bound_data->get_payload(i, r_ptr, r_len); + while (OB_SUCC(ret)) { + cmp_times++; + bool match = false; + const ObCompactRow *a_row = nullptr; + (is_preceding ^ re_direction) ? (pos -= step) : (pos += step); + const bool overflow = + (is_preceding ^ re_direction) ? (pos < part_first_idx) : (pos >= part_end_idx); + if (overflow) { + match = true; + } else if (OB_FAIL(ctx.input_rows_.get_row(pos , a_row))) { + LOG_WARN("get row failed", K(ret)); + } else { + l_isnull = a_row->is_null(cell_idx); + a_row->get_cell_payload(input_row_meta, cell_idx, l_ptr, l_len); + if (OB_FAIL(sort_cmp_fn(sort_obj_meta, sort_obj_meta, + l_ptr, l_len, l_isnull, + r_ptr, r_len, r_isnull, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } else { + match = ((cmp_mode & L) && cmp_ret < 0) + || ((cmp_mode & LE) && cmp_ret <= 0) + || ((cmp_mode & G) && cmp_ret > 0) + || ((cmp_mode & GE) && cmp_ret >= 0); + LOG_DEBUG("cmp_result", K(ret), K(cmp_mode), K(match), K(pos), K(row_start), + K(cmp_times), K(l_isnull), K(r_isnull), K(is_preceding), K(is_ascending)); + } + } + if (OB_SUCC(ret)) { + if (match) { + if (pos == row_idx && !(cmp_mode & ROLL)) { + // for LE/GE, if equal to current row, + // change cmp_mode to search opposite direction. + if (LE == cmp_mode) { + cmp_mode = G; + } else if (GE == cmp_mode) { + cmp_mode = L; + } + re_direction = true; + step = 1; + } else if (step <= 1) { + if (cmp_mode & ROLL) { + (is_preceding ^ re_direction) ? (pos += step) : (pos -= step); + } + break; + } else { + (is_preceding ^ re_direction) ? (pos += step) : (pos -= step); + step = 1; + } + } else { + step = (0 == step ? 1 : (2 * step)); + } + } + } // end inner while + if (OB_FAIL(ret)) { + } else { + pos_arr[i] = pos + (is_upper ? 0 : 1); + prev_row_border = pos_arr[i]; + } + } + } // end for + return ret; +} +int calc_borders_for_sort_expr(WinExprEvalCtx &ctx, const ObExpr *bound_expr, + const int64_t batch_size, const int64_t row_start, + ObBitVector &eval_skip, const bool is_upper, int64_t *pos_arr) +{ +#define CALC_BORDER(fmt) \ + ret = _calc_borders_for_sort_expr(ctx, batch_size, row_start, eval_skip, is_upper, \ + static_cast(data), bound_expr, pos_arr) + +#define CALC_FIXED_TYPE_BORDER(vec_tc) \ + case (vec_tc): { CALC_BORDER(ObFixedLengthFormat>); } break + + int ret = OB_SUCCESS; + ObEvalCtx &eval_ctx = ctx.win_col_.op_.get_eval_ctx(); + bool is_nmb_literal = (is_upper ? ctx.win_col_.wf_info_.upper_.is_nmb_literal_ : + ctx.win_col_.wf_info_.lower_.is_nmb_literal_); + if (OB_FAIL(bound_expr->eval_vector( + eval_ctx, eval_skip, + EvalBound(batch_size, eval_skip.accumulate_bit_cnt(batch_size) == 0)))) { + LOG_WARN("eval vector failed", K(ret)); + } else if (lib::is_mysql_mode() && !is_nmb_literal + && ob_is_temporal_type(bound_expr->datum_meta_.type_)) { + if (OB_FAIL(_check_datetime_interval_valid(eval_ctx, bound_expr, batch_size, eval_skip))) { + LOG_WARN("invalid datetime interval", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else { + ObIVector *data = bound_expr->get_vector(eval_ctx); + VectorFormat fmt = bound_expr->get_format(eval_ctx); + VecValueTypeClass vec_tc = bound_expr->get_vec_value_tc(); + switch(fmt) { + case common::VEC_UNIFORM: { + CALC_BORDER(ObUniformFormat); + break; + } + case common::VEC_UNIFORM_CONST: { + CALC_BORDER(ObUniformFormat); + break; + } + case common::VEC_DISCRETE: { + CALC_BORDER(ObDiscreteFormat); + break; + } + case common::VEC_CONTINUOUS: { + CALC_BORDER(ObContinuousFormat); + break; + } + case common::VEC_FIXED: { + switch(vec_tc) { + // TODO: check supported types of bound expr + LST_DO_CODE(CALC_FIXED_TYPE_BORDER, FIXED_VEC_LIST); + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unsupported type", K(ret), K(vec_tc)); + } + } + break; + } + default: { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unsupport data format", K(ret), K(fmt), K(*bound_expr)); + } + } + } + return ret; +} + +int cmp_prev_row(WinExprEvalCtx &ctx, const int64_t cur_idx, int &cmp_ret) +{ + int ret = OB_SUCCESS; + const ObCompactRow *stored_rows[2] = {nullptr}; + if (OB_FAIL(ctx.input_rows_.get_batch_rows(cur_idx - 1, cur_idx + 1, stored_rows))) { + LOG_WARN("get stored rows failed", K(ret)); + } + cmp_ret = 0; + ObSortCollations &sort_collations = ctx.win_col_.wf_info_.sort_collations_; + ObSortFuncs &sort_cmp_funcs = ctx.win_col_.wf_info_.sort_cmp_funcs_; + ObIArray &sort_exprs = ctx.win_col_.wf_info_.sort_exprs_; + const char *l_data = nullptr, *r_data = nullptr; + int32_t l_len = 0, r_len = 0; + bool l_isnull = false, r_isnull = false; + const RowMeta &input_row_meta = ctx.win_col_.op_.get_input_row_meta(); + for (int i = 0; OB_SUCC(ret) && cmp_ret == 0 && i < sort_collations.count(); i++) { + int64_t field_idx = sort_collations.at(i).field_idx_; + ObObjMeta &obj_meta = sort_exprs.at(i)->obj_meta_; + sql::NullSafeRowCmpFunc cmp_fn = sort_cmp_funcs.at(i).row_cmp_func_; + stored_rows[0]->get_cell_payload(input_row_meta, field_idx, l_data, l_len); + l_isnull = stored_rows[0]->is_null(field_idx); + stored_rows[1]->get_cell_payload(input_row_meta, field_idx, r_data, r_len); + r_isnull = stored_rows[1]->is_null(field_idx); + if (OB_FAIL( + cmp_fn(obj_meta, obj_meta, l_data, l_len, l_isnull, r_data, r_len, r_isnull, cmp_ret))) { + LOG_WARN("compare failed", K(ret)); + } + } + return ret; +} + +template<> +struct int_trunc +{ + static int get(const char *payload, const int32_t len,const ObDatumMeta &meta, int64_t &value) + { + int ret = OB_SUCCESS; + number::ObNumber res_nmb(*reinterpret_cast(payload)); + if (OB_FAIL(res_nmb.extract_valid_int64_with_trunc(value))) { + LOG_WARN("truncate integer failed", K(ret)); + } + return ret; + } +}; + +template<> +struct int_trunc +{ + static int get(const char *payload, const int32_t len,const ObDatumMeta &meta, int64_t &value) + { + int ret = OB_SUCCESS; + value = *reinterpret_cast(payload); + return ret; + } +}; + +template <> +struct int_trunc +{ + static int get(const char *payload, const int32_t len, const ObDatumMeta &meta, int64_t &value) + { + int ret = OB_SUCCESS; + uint64_t tmp_value = *reinterpret_cast(payload); + bool is_valid_param = !lib::is_mysql_mode() || static_cast(tmp_value) >= 0; + if (tmp_value > INT64_MAX && is_valid_param) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret)); + } else { + value = static_cast(tmp_value); + } + return ret; + } +}; + +template<> +struct int_trunc: public int_trunc {}; + +template<> +struct int_trunc +{ + static int get(const char *payload, const int32_t len,const ObDatumMeta &meta, int64_t &value) + { + int ret = OB_SUCCESS; + const float tmp_value = *reinterpret_cast(payload); + if (tmp_value > INT64_MAX) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret)); + } else { + value = static_cast(tmp_value); + } + return ret; + } +}; + +template<> +struct int_trunc +{ + static int get(const char *payload, const int32_t len,const ObDatumMeta &meta, int64_t &value) + { + int ret = OB_SUCCESS; + const double tmp_value = *reinterpret_cast(payload); + if (tmp_value > INT64_MAX) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret)); + } else { + value = static_cast(tmp_value); + } + return ret; + } +}; + +template<> +struct int_trunc: public int_trunc {}; + +template +struct dec_int_trunc +{ + static int get(const char *payload, const int32_t len,const ObDatumMeta &meta, int64_t &value) + { + int ret = OB_SUCCESS; + ObScale out_scale = 0; + ObDecimalIntBuilder trunc_res_val; + ObDatum in_datum; + in_datum.ptr_ = payload; + in_datum.len_ = len; + bool is_valid = false; + if (meta.scale_ == out_scale) { + trunc_res_val.from(reinterpret_cast(payload), len); + } else if (OB_FAIL(ObExprTruncate::do_trunc_decimalint(meta.precision_, meta.scale_, + meta.precision_, out_scale, out_scale, + in_datum, trunc_res_val))) { + LOG_WARN("truncate decimal int failed", K(ret)); + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(wide::check_range_valid_int64(trunc_res_val.get_decimal_int(), + trunc_res_val.get_int_bytes(), is_valid, + value))) { + LOG_WARN("check valid int64 failed", K(ret)); + } else if (OB_UNLIKELY(!is_valid)) { + ret = OB_DATA_OUT_OF_RANGE; + LOG_WARN("int64 out of range", K(ret)); + } + return ret; + } +}; + +template<> +struct int_trunc: public dec_int_trunc{}; + +template<> +struct int_trunc: public dec_int_trunc{}; + +template<> +struct int_trunc: public dec_int_trunc{}; + +template<> +struct int_trunc: public dec_int_trunc{}; + +template<> +struct int_trunc: public dec_int_trunc{}; +} // end winfunc +} // end sql +} // end oceanbase \ No newline at end of file diff --git a/src/sql/engine/window_function/win_expr.h b/src/sql/engine/window_function/win_expr.h new file mode 100644 index 0000000000..37aaa97c5b --- /dev/null +++ b/src/sql/engine/window_function/win_expr.h @@ -0,0 +1,336 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_WINDOW_FUNCTION_EXPR_H_ +#define OCEANBASE_WINDOW_FUNCTION_EXPR_H_ + +#include "share/ob_define.h" +#include "share/aggregate/processor.h" + +namespace oceanbase +{ +namespace sql +{ +class ObCompactRow; +class WinFuncColExpr; + +namespace winfunc +{ +using namespace share; +class RowStore; + +// copy from` ob_aggregate_processor.h` +struct RemovalInfo +{ + RemovalInfo() + : max_min_index_(-1), + is_index_change_(false), + is_inv_aggr_(false), + null_cnt_(0), + is_out_of_range_(false) + { + } + ~RemovalInfo() {} + void reset() { + max_min_index_ = -1; + is_index_change_ = false; + is_inv_aggr_ = false; + null_cnt_ = 0; + is_out_of_range_ = false; + } + void max_min_update(const int64_t max_min_index) { + if (is_index_change_) { + max_min_index_ = max_min_index; + is_index_change_ = false; + } + } + TO_STRING_KV(K_(max_min_index), K_(is_index_change), K_(is_inv_aggr)); + int64_t max_min_index_; // extreme index position + bool is_index_change_; // whether the extreme value index position changes + bool is_inv_aggr_; // whether the aggregate function support single line inverse + int64_t null_cnt_; // count of null in frame for calculating sum + bool is_out_of_range_; // whether out of range when calculateing +}; + +// copy from `ObWindowFunctionOp::Frame` +struct Frame +{ + Frame(const int64_t head = -1, const int64_t tail = -1, bool is_accum_frame = false) : + head_(head), tail_(tail), is_accum_frame_(is_accum_frame) + {} + Frame(const Frame &other): head_(other.head_), tail_(other.tail_), is_accum_frame_(other.is_accum_frame_) {} + bool operator==(const Frame &other) const + { + return same_frame(*this, other); + } + static bool valid_frame(const Frame &part_frame, const Frame &frame) + { + return frame.head_ < frame.tail_ && frame.head_ < part_frame.tail_ + && frame.tail_ > part_frame.head_; + } + static bool same_frame(const Frame &left, const Frame &right) + { + return left.head_ == right.head_ && left.tail_ == right.tail_; + } + static void prune_frame(const Frame &part_frame, Frame &frame) + { + // it's caller's responsibility for invoking valid_frame() first + if (frame.head_ < part_frame.head_) { frame.head_ = part_frame.head_; } + if (frame.tail_ > part_frame.tail_) { frame.tail_ = part_frame.tail_; } + } + static bool need_restart_aggr(const bool can_inv, const Frame &last_valid_frame, + const Frame &new_frame, const aggregate::RemovalInfo &removal_info, + const uint64_t &remove_type) + { + bool need = false; + if (-1 == last_valid_frame.head_ || -1 == last_valid_frame.tail_) { + need = true; + } else { + const int64_t inc_cost = std::abs(last_valid_frame.head_ - new_frame.head_) + + std::abs(last_valid_frame.tail_ - new_frame.tail_); + const int64_t restart_cost = new_frame.tail_ - new_frame.head_; + if (inc_cost > restart_cost) { + need = true; + } else if (!can_inv) { + // has sliding-out row + if (new_frame.head_ > last_valid_frame.head_ || new_frame.tail_ < last_valid_frame.tail_) { + need = true; + } + } else if (common::REMOVE_EXTRENUM == remove_type) { + // max_min index miss from calculation range + if (removal_info.max_min_index_ < new_frame.head_ + || removal_info.max_min_index_ > new_frame.tail_) { + need = true; + } + } + } + return need; + } + bool is_valid() const + { + return head_ < tail_ + && head_ != -1 && head_ != INT64_MAX + && tail_ != -1 && tail_ != INT64_MAX; + } + bool is_empty() const + { + return tail_ <= head_; + } + void reset() + { + head_ = tail_ = -1; + } + TO_STRING_KV(K(head_), K(tail_)); + + int64_t head_; + int64_t tail_; // !!! not included + bool is_accum_frame_; +}; + +struct WinExprEvalCtx +{ + WinExprEvalCtx(RowStore &input_rows, WinFuncColExpr &win_col, const int64_t tenant_id) : + input_rows_(input_rows), win_col_(win_col), + allocator_(ObModIds::OB_SQL_WINDOW_LOCAL, OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id, + ObCtxIds::WORK_AREA), + extra_(nullptr) + {} + + char *reserved_buf(int32_t len) + { + return (char *)allocator_.alloc(len); + } + ~WinExprEvalCtx() + { + extra_ = nullptr; + allocator_.reset(); + } + RowStore &input_rows_; + sql::WinFuncColExpr &win_col_; + // used for tmp memory allocating during partition process. + common::ObArenaAllocator allocator_; + void *extra_; // maybe useless +}; + +class IWinExpr +{ +public: + virtual int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) = 0; + virtual int collect_part_results(WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) = 0; + virtual int accum_process_window(WinExprEvalCtx &ctx, const Frame &cur_frame, + const Frame &prev_frame, const int64_t row_idx, char *res, + bool &is_null) = 0; + virtual int process_partition(WinExprEvalCtx &ctx, const int64_t part_start, + const int64_t part_end, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) = 0; + // used to generate extra ctx for expr evaluation + virtual int generate_extra(ObIAllocator &allocator, void *&extra) = 0; + + virtual bool is_aggregate_expr() const = 0; + virtual void destroy() = 0; +}; + + +template +class WinExprWrapper: public IWinExpr +{ +public: + virtual int process_partition(WinExprEvalCtx &ctx, const int64_t part_start, + const int64_t part_end, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override + { + return OB_NOT_IMPLEMENT; + } + virtual void destroy() override + { // do nothing + return; + } +protected: + int copy_aggr_row(WinExprEvalCtx &ctx, const char *src_row, char *dst_row); +private: + int update_frame(WinExprEvalCtx &ctx, const Frame &prev_frame, Frame &new_frame, + const int64_t idx, const int64_t row_start, bool &whole_frame, + bool &valid_frame); +}; + +// TODO: adjust inheritance +class NonAggrWinExpr: public WinExprWrapper +{ +protected: + struct ParamStatus + { + ParamStatus() : flags_(0), int_val_(0) + {} + union + { + struct + { + uint32_t calculated_ : 1; + uint32_t is_null_ : 1; + uint32_t reserved_ : 30; + }; + uint32_t flags_; + }; + int64_t int_val_; + }; + int eval_param_int_value(ObExpr *param, ObEvalCtx &ctx, const bool need_check_valid, + const bool need_nmb, ParamStatus &status); + +public: + virtual int accum_process_window(WinExprEvalCtx &ctx, const Frame &cur_frame, + const Frame &prev_frame, const int64_t row_idx, + char *res, bool &is_null) override final + { + int ret = OB_NOT_IMPLEMENT; + return ret; + } + virtual bool is_aggregate_expr() const override final { return false; } + + virtual int collect_part_results(WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) override final; +}; + +template +class RankLikeExpr final: public NonAggrWinExpr +{ +public: + RankLikeExpr():NonAggrWinExpr(), rank_of_prev_row_(0) {} + + virtual int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override; +private: + int64_t rank_of_prev_row_; +}; + +class RowNumber final: public NonAggrWinExpr +{ +public: + virtual int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override; +}; + +class Ntile final: public NonAggrWinExpr +{ +public: + virtual int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override; +}; + +class NthValue final: public NonAggrWinExpr +{ +public: + virtual int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override; +}; + +class LeadOrLag final: public NonAggrWinExpr +{ +public: + virtual int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override; +}; + +class CumeDist final: public NonAggrWinExpr +{ +public: + virtual int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override; +}; + +class AggrExpr final: public WinExprWrapper +{ +public: + AggrExpr(): aggr_processor_(nullptr), last_valid_frame_(), last_aggr_row_(nullptr) {} + int process_window(WinExprEvalCtx &ctx, const Frame &frame, const int64_t row_idx, + char *res, bool &is_null) override; + + int accum_process_window(WinExprEvalCtx &ctx, const Frame &cur_frame, const Frame &prev_frame, + const int64_t row_idx, char *res, bool &is_null) override; + bool is_aggregate_expr() const override { return true; } + virtual int collect_part_results(WinExprEvalCtx &ctx, const int64_t row_start, + const int64_t row_end, const ObBitVector &skip) override; + virtual int generate_extra(ObIAllocator &allocator, void *&extra) override + { + return OB_SUCCESS; + } + + static int set_result_for_invalid_frame(WinExprEvalCtx &ctx, char *agg_row); + + virtual void destroy() override; + +private: + int calc_pushdown_skips(WinExprEvalCtx &ctx, const int64_t batch_size, sql::ObBitVector &skip, bool &all_active); + + template + int set_payload(WinExprEvalCtx &ctx, ColumnFmt *columns, const int64_t idx, + const char *payload, int32_t len); + +public: + aggregate::Processor *aggr_processor_; + Frame last_valid_frame_; + aggregate::RemovalInfo last_removal_info_; + char *last_aggr_row_; +}; + +} // end winfunc +} // end sql +} // end oceanbase +#endif // OCEANBASE_WINDOW_FUNCTION_EXPR_H_ \ No newline at end of file diff --git a/src/sql/optimizer/ob_log_sort.cpp b/src/sql/optimizer/ob_log_sort.cpp index e4d504cce3..9a10893e68 100644 --- a/src/sql/optimizer/ob_log_sort.cpp +++ b/src/sql/optimizer/ob_log_sort.cpp @@ -21,9 +21,14 @@ #include "sql/optimizer/ob_join_order.h" #include "share/ob_order_perserving_encoder.h" #include "common/ob_smart_call.h" +#include "sql/engine/px/p2p_datahub/ob_p2p_dh_mgr.h" +#include "sql/optimizer/ob_log_table_scan.h" + using namespace oceanbase::sql; using namespace oceanbase::common; +#define MIN_TSC_OUTPUT_ROWS_FOR_PD_TOPN_FILTER 256 + int ObLogSort::set_sort_keys(const common::ObIArray &order_keys) { int ret = OB_SUCCESS; @@ -536,6 +541,237 @@ int ObLogSort::is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) return ret; } +int ObLogSort::try_allocate_pushdown_topn_runtime_filter() +{ + int ret = OB_SUCCESS; + ObLogicalOperator *node = nullptr; + double tsc_output_rows = 0; + ObOpRawExpr *pushdown_topn_filter_expr = nullptr; + ObSQLSessionInfo *session_info = nullptr; + uint64_t table_id = OB_INVALID_ID; + int64_t p2p_sequence_id = OB_INVALID_ID; + common::ObSEArray candidate_sk_exprs; + int64_t effective_sk_cnt = 0; + bool tsc_has_exchange = false; + bool tsc_has_px_coord = false; + bool can_allocate = false; + uint64_t min_cluster_version = GET_MIN_CLUSTER_VERSION(); + int ecode = EventTable::EN_PX_DISABLE_PD_TOPN_FILTER; + if (min_cluster_version < DATA_VERSION_4_3_2_0) { + can_allocate = false; + } else if (OB_SUCCESS != ecode) { + can_allocate = false; + LOG_TRACE("[TopN Filter]disable push down topn filter by tracepoint"); + } else if (OB_ISNULL(get_plan())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("log_plan unexpected null"); + } else if (OB_ISNULL(session_info = get_plan()->get_optimizer_context().get_session_info())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("session_info unexpected null"); + } else if (OB_ISNULL(topn_expr_)) { + // not topn scene + can_allocate = false; + } else if (prefix_pos_ > 0) { + // For index (c1, c2 c3), if order by c1, c3, its prefix sort scene + // if all the date of c1 is unique, topn runtime filter is useless, + // otherwise if most of the date of c1 is unique, the topn runtime filter + // can filter on c3. + // Since we can't distinguish these two scene accurately, topn runtime filter + // is forbiddened in the prefix sort scene. Enable it by judging condition + // prefix_pos_ < effective_sk_cnt. + can_allocate = false; + LOG_TRACE("[TopN Filter]can not pushdown when prefix sort ", K(prefix_pos_), K(sort_keys_.count())); + } else if (OB_FAIL(get_candidate_pushdown_sort_keys(table_id, candidate_sk_exprs))) { + LOG_WARN("failed to get_candidate_pushdown_sort_keys"); + } else if (OB_INVALID_ID == table_id) { + // such as order by sqrt(3.3), the sort key is a const expr, can not pushdown + can_allocate = false; + } else if (0 == candidate_sk_exprs.count()) { + // such as order by t1.c1 + t2.c1, the sort key is related to 2 tables, can not pushdown + can_allocate = false; + LOG_TRACE("[TopN Filter]the first sort key from different tables"); + } else if (OB_FAIL(check_sort_key_can_pushdown_to_tsc(get_child(first_child), candidate_sk_exprs, + table_id, node, tsc_has_exchange, + tsc_has_px_coord, effective_sk_cnt))) { + LOG_WARN("failed to find check check_sort_key_can_pushdown_to_tsc", K(ret)); + } else if (0 == effective_sk_cnt) { + can_allocate = false; + LOG_TRACE("[TopN Filter]no effective_sk"); + } else if (OB_ISNULL(node)) { + // tsc node is in block path, or not tsc node + // can not pushdown topn filter + can_allocate = false; + LOG_TRACE("[TopN Filter]block path, or no tsc node"); + } else if (tsc_has_px_coord) { + // not support + can_allocate = false; + } else if (tsc_has_exchange) { + // TODO XUNSI: global topn filter, impl topn filter in neighbour dfos scenes. + can_allocate = false; + LOG_TRACE("[TopN Filter]global topn filter is not support now"); + } else { + const OptTableMetas &table_metas = get_plan()->get_basic_table_metas(); + const OptTableMeta *table_meta = nullptr; + int ecode = EventTable::EN_PX_PD_TOPN_FILTER_IGNORE_TABLE_CARD; + if (OB_ISNULL(table_meta = table_metas.get_table_meta_by_table_id(table_id))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table_meta unexpected null"); + } else if (FALSE_IT(tsc_output_rows = table_meta->get_rows())) { + } else if (ecode != OB_SUCCESS) { + can_allocate = true; + LOG_TRACE("[TopN Filter] force enable topn filter ignore less card", K(tsc_output_rows)); + } else if (MIN_TSC_OUTPUT_ROWS_FOR_PD_TOPN_FILTER > tsc_output_rows) { + // table scan output first batch rows after a batch prepared, after that + // the topn sort operator can get the first batch data, if the data num + // of the table is too small, the topn filter will be invalid + can_allocate = false; + LOG_TRACE("[TopN Filter] disable topn filter because of less rows", K(tsc_output_rows)); + } else { + can_allocate = true; + } + } + + if (OB_SUCC(ret) && can_allocate) { + ObRawExprFactory &expr_factory = get_plan()->get_optimizer_context().get_expr_factory(); + if (OB_FAIL( + expr_factory.create_raw_expr(T_OP_PUSHDOWN_TOPN_FILTER, pushdown_topn_filter_expr))) { + LOG_WARN("fail to create raw expr", K(ret)); + } else { + bool only_white_filter = + T_REF_COLUMN == candidate_sk_exprs.at(0)->get_expr_type() && effective_sk_cnt > 1; + if (only_white_filter) { + LOG_TRACE("multi sort keys, and the first column can be pushdown as white"); + effective_sk_cnt = 1; + } + for (int64_t i = 0; i < effective_sk_cnt && OB_SUCC(ret); ++i) { + ObRawExpr *sort_key = candidate_sk_exprs.at(i); + if (OB_FAIL(pushdown_topn_filter_expr->add_param_expr(sort_key))) { + LOG_WARN("fail to add param expr", K(ret)); + } + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(pushdown_topn_filter_expr->formalize(session_info))) { + LOG_WARN("fail to formalize expr", K(ret)); + } else if (OB_FAIL(node->get_filter_exprs().push_back(pushdown_topn_filter_expr))) { + LOG_WARN("fail to push back expr", K(ret)); + } else if (OB_FAIL(PX_P2P_DH.generate_p2p_dh_id(p2p_sequence_id))) { + LOG_WARN("fail to generate p2p dh id", K(ret)); + } else { + (void)topn_filter_info_.init(p2p_sequence_id, pushdown_topn_filter_expr, effective_sk_cnt, + tsc_has_exchange); + } + } + + LOG_TRACE("[TopN Filter] allocate pushdown topn filter expr", K(can_allocate), + K(sort_keys_.count()), K(prefix_pos_), K(candidate_sk_exprs.count()), K(table_id), + K(node), K(tsc_has_px_coord), K(tsc_has_exchange), K(tsc_output_rows), + K(topn_filter_info_)); + return ret; +} + +// get the front sort keys from same table, topn filter can pushdown to this table +int ObLogSort::get_candidate_pushdown_sort_keys( + uint64_t &table_id, + common::ObSEArray &candidate_sk_exprs) +{ + int ret = OB_SUCCESS; + bool can_expr_pushdown = true; + for (int64_t i = 0; i < sort_keys_.count() && OB_SUCC(ret) && can_expr_pushdown; ++i) { + ObRawExpr *sort_key = sort_keys_.at(i).expr_; + if (OB_ISNULL(sort_key)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr"); + } else if (OB_FAIL(check_expr_can_pushdown(sort_key, table_id, can_expr_pushdown))) { + LOG_WARN("failed to check_sort_key_can_pushdown"); + } else if (can_expr_pushdown && OB_FAIL(candidate_sk_exprs.push_back(sort_key))) { + LOG_WARN("failed to pushback"); + } + } + return ret; +} + +int ObLogSort::check_expr_can_pushdown(ObRawExpr *expr, uint64_t &table_id, bool &can_push_down) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr"); + } else if (expr->is_const_expr()) { + // skip check const expr and its children + } else if (OB_FAIL(is_expr_in_pushdown_whitelist(expr, can_push_down))) { + LOG_WARN("failed to check expr_in_pushdown_whitelist"); + } else if (!can_push_down) { + // not in whitelist, can not pushdown + } else if (expr->is_column_ref_expr()) { + // check if all columns from the same table + ObColumnRefRawExpr *col_ref_expr = static_cast(expr); + if (OB_INVALID_ID == table_id) { + table_id = col_ref_expr->get_table_id(); + } else if (table_id != col_ref_expr->get_table_id()) { + // column from different tables, this sort key and the succeed sort key can not pushdown + can_push_down = false; + } + } else { + for (int64_t i = 0; i < expr->get_param_count() && OB_SUCC(ret) && can_push_down; ++i) { + ObRawExpr *child_expr = expr->get_param_expr(i); + if (OB_FAIL(SMART_CALL(check_expr_can_pushdown(child_expr, table_id, can_push_down)))) { + LOG_WARN("failed to do check_expr_can_pushdown"); + } + } + } + return ret; +} + +int ObLogSort::is_expr_in_pushdown_whitelist(ObRawExpr *expr, bool &in_pushdown_whitelist) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr"); + } else if (expr->is_column_ref_expr() || expr->is_const_raw_expr() || expr->is_op_expr()) { + // const expr, column ref, or +-*/ op + in_pushdown_whitelist = true; + } else { + // some system functions can also be pushdown + switch (expr->get_expr_type()) { + case T_OP_ABS: + case T_FUN_SYS_CAST: + case T_FUN_SYS_SUBSTR: + case T_FUN_SYS_LENGTH: + case T_FUN_SYS_UPPER: + case T_FUN_SYS_LOWER: + case T_FUN_SYS_ROUND: + case T_FUN_SYS_FLOOR: + case T_FUN_SYS_CEIL: + case T_FUN_SYS_SQRT: + case T_FUN_SYS_EXP: + case T_FUN_SYS_POWER: + case T_FUN_SYS_LN: + case T_FUN_SYS_LOG: + case T_FUN_SYS_ASIN: + case T_FUN_SYS_ACOS: + case T_FUN_SYS_ATAN: + case T_FUN_SYS_ATAN2: + case T_FUN_SYS_COS: + case T_FUN_SYS_TAN: + case T_FUN_SYS_SIN: + case T_FUN_SYS_SINH: + case T_FUN_SYS_COSH: + case T_FUN_SYS_TANH: { + in_pushdown_whitelist = true; + break; + } + default: { + in_pushdown_whitelist = false; + LOG_TRACE("this expr can not be pushdown", K(expr->get_expr_type())); + break; + } + } + } + return ret; +} + int ObLogSort::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) { int ret = OB_SUCCESS; diff --git a/src/sql/optimizer/ob_log_sort.h b/src/sql/optimizer/ob_log_sort.h index 1ac93021ff..7fefb8a473 100644 --- a/src/sql/optimizer/ob_log_sort.h +++ b/src/sql/optimizer/ob_log_sort.h @@ -19,108 +19,158 @@ namespace oceanbase { namespace sql { - class ObRawExpr; - class ObLogSort; - class ObLogSort : public ObLogicalOperator +class ObRawExpr; +class ObLogSort; + +struct ObTopNFilterInfo +{ +public: + ObTopNFilterInfo() + : enabled_(false), p2p_sequence_id_(OB_INVALID_ID), pushdown_topn_filter_expr_(nullptr), + effective_sk_cnt_(0), is_shuffle_(false) + {} + inline void init(int64_t p2p_sequence_id, ObRawExpr *pushdown_topn_filter_expr, + int64_t effective_sk_cnt, bool is_shuffle) { - public: - ObLogSort(ObLogPlan &plan) - : ObLogicalOperator(plan), - hash_sortkey_(), - sort_keys_(), - encode_sortkeys_(), - topn_expr_(NULL), - minimum_row_count_(0), - topk_precision_(0), - prefix_pos_(0), - is_local_merge_sort_(false), - topk_limit_expr_(NULL), - topk_offset_expr_(NULL), - is_fetch_with_ties_(false), - part_cnt_(0), - sort_key_width_(0.0) - {} - virtual ~ObLogSort() - {} - virtual int est_cost() override; - virtual int est_width() override; - virtual int do_re_est_cost(EstimateCostInfo ¶m, double &card, double &op_cost, double &cost) override; - int inner_est_cost(const int64_t parallel, double child_card, double &topn_count, double &op_cost); - const OrderItem &get_hash_sortkey() const { return hash_sortkey_; } - OrderItem &get_hash_sortkey() { return hash_sortkey_; } - inline void set_hash_sortkey(const OrderItem &hash_sortkey) { hash_sortkey_ = hash_sortkey; } - const common::ObIArray &get_sort_keys() const { return sort_keys_; } - common::ObIArray &get_sort_keys() { return sort_keys_; } - const common::ObIArray &get_encode_sortkeys() const { return encode_sortkeys_; } - common::ObIArray &get_encode_sortkeys() { return encode_sortkeys_; } - int get_sort_output_exprs(ObIArray &output_exprs); - int create_encode_sortkey_expr(const common::ObIArray &order_keys); - int get_sort_exprs(common::ObIArray &sort_exprs); + p2p_sequence_id_ = p2p_sequence_id; + pushdown_topn_filter_expr_ = pushdown_topn_filter_expr; + effective_sk_cnt_ = effective_sk_cnt; + is_shuffle_ = is_shuffle; + enabled_ = true; + } + TO_STRING_KV(K_(enabled), K_(p2p_sequence_id), KP_(pushdown_topn_filter_expr), + K_(effective_sk_cnt), K_(is_shuffle)); - inline void set_topn_expr(ObRawExpr *expr) { topn_expr_ = expr; } - inline void set_prefix_pos(int64_t prefix_pos) { prefix_pos_ = prefix_pos; } - inline void set_local_merge_sort(bool is_local_merge_sort) { is_local_merge_sort_ = is_local_merge_sort; } - inline void set_fetch_with_ties(bool is_fetch_with_ties) { is_fetch_with_ties_ = is_fetch_with_ties; } - inline void set_part_cnt(uint64_t part_cnt) { part_cnt_ = part_cnt; } +public: + bool enabled_; + int64_t p2p_sequence_id_; + ObRawExpr *pushdown_topn_filter_expr_; + // the sort key may from different tables, we can only use the front keys from the same table + int64_t effective_sk_cnt_; + // the topn sort op and the tsc op not in same dfo, need shuffle + bool is_shuffle_; +}; + +class ObLogSort : public ObLogicalOperator +{ +public: + ObLogSort(ObLogPlan &plan) + : ObLogicalOperator(plan), + hash_sortkey_(), + sort_keys_(), + encode_sortkeys_(), + topn_expr_(NULL), + minimum_row_count_(0), + topk_precision_(0), + prefix_pos_(0), + is_local_merge_sort_(false), + topk_limit_expr_(NULL), + topk_offset_expr_(NULL), + is_fetch_with_ties_(false), + part_cnt_(0), + sort_key_width_(0.0) + {} + virtual ~ObLogSort() + {} + virtual int est_cost() override; + virtual int est_width() override; + virtual int do_re_est_cost(EstimateCostInfo ¶m, double &card, double &op_cost, double &cost) override; + int inner_est_cost(const int64_t parallel, double child_card, double &topn_count, double &op_cost); + const OrderItem &get_hash_sortkey() const { return hash_sortkey_; } + OrderItem &get_hash_sortkey() { return hash_sortkey_; } + inline void set_hash_sortkey(const OrderItem &hash_sortkey) { hash_sortkey_ = hash_sortkey; } + const common::ObIArray &get_sort_keys() const { return sort_keys_; } + common::ObIArray &get_sort_keys() { return sort_keys_; } + const common::ObIArray &get_encode_sortkeys() const { return encode_sortkeys_; } + common::ObIArray &get_encode_sortkeys() { return encode_sortkeys_; } + int get_sort_output_exprs(ObIArray &output_exprs); + int create_encode_sortkey_expr(const common::ObIArray &order_keys); + int get_sort_exprs(common::ObIArray &sort_exprs); + + inline void set_topn_expr(ObRawExpr *expr) { topn_expr_ = expr; } + inline void set_prefix_pos(int64_t prefix_pos) { prefix_pos_ = prefix_pos; } + inline void set_local_merge_sort(bool is_local_merge_sort) { is_local_merge_sort_ = is_local_merge_sort; } + inline void set_fetch_with_ties(bool is_fetch_with_ties) { is_fetch_with_ties_ = is_fetch_with_ties; } + inline void set_part_cnt(uint64_t part_cnt) { part_cnt_ = part_cnt; } + + // check if the current sort is a pushed down + inline bool is_prefix_sort() const { return prefix_pos_ != 0; } + inline bool is_local_merge_sort() const { return is_local_merge_sort_; } + inline bool is_part_sort() const { return part_cnt_ != 0; } + inline bool is_fetch_with_ties() const { return is_fetch_with_ties_; } + inline bool enable_encode_sortkey_opt() const { return encode_sortkeys_.count()!=0; } + inline int64_t get_part_cnt() const { return part_cnt_; } + inline int64_t get_prefix_pos() const { return prefix_pos_; } + inline ObRawExpr *get_topn_expr() const { return topn_expr_; } + inline void set_topk_limit_expr(ObRawExpr *top_limit_expr) + { + topk_limit_expr_ = top_limit_expr; + } + inline ObRawExpr *get_topk_limit_expr() { return topk_limit_expr_; } + inline void set_topk_offset_expr(ObRawExpr *topk_offset_expr) + { + topk_offset_expr_ = topk_offset_expr; + } + inline ObRawExpr *get_topk_offset_expr() { return topk_offset_expr_; } + int set_sort_keys(const common::ObIArray &order_keys); + virtual int get_op_exprs(ObIArray &all_exprs) override; + virtual int is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) override; + virtual uint64_t hash(uint64_t seed) const override; + virtual const char *get_name() const; + inline void set_minimal_row_count(int64_t minimum_row_count) + { + minimum_row_count_ = minimum_row_count; + } + inline int64_t get_minimum_row_count() const {return minimum_row_count_;} + inline void set_topk_precision(int64_t topk_precision) + { + topk_precision_ = topk_precision; + } + inline int64_t get_topk_precision() const {return topk_precision_;} + inline double get_sort_key_width() const { return sort_key_width_; } + virtual bool is_block_op() const override { return !is_prefix_sort(); } + virtual int compute_op_ordering() override; + virtual int get_plan_item_info(PlanText &plan_text, + ObSqlPlanItem &plan_item) override; + inline bool enable_pd_topn_filter() const { return topn_filter_info_.enabled_; } + inline int64_t get_p2p_sequence_id() const { return topn_filter_info_.p2p_sequence_id_; } + inline ObRawExpr *get_pushdown_topn_filter_expr() const + { + return topn_filter_info_.pushdown_topn_filter_expr_; + } + inline int64_t get_effective_sk_cnt_of_topn_filter() const + { + return topn_filter_info_.effective_sk_cnt_; + } + inline bool is_shuffle_pd_topn_filter() { return topn_filter_info_.is_shuffle_; } + int try_allocate_pushdown_topn_runtime_filter(); + int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index); +protected: + virtual int inner_replace_op_exprs(ObRawExprReplacer &replacer); + int est_sort_key_width(); +private: + int get_candidate_pushdown_sort_keys( + uint64_t &table_id, + common::ObSEArray &candidate_sk_exprs); + int check_expr_can_pushdown(ObRawExpr *expr, uint64_t &table_id, bool &can_push_down); + int is_expr_in_pushdown_whitelist(ObRawExpr *expr, bool &in_pushdown_whitelist); +private: + OrderItem hash_sortkey_; + common::ObSEArray sort_keys_; + common::ObSEArray encode_sortkeys_; + ObRawExpr *topn_expr_; + int64_t minimum_row_count_; + int64_t topk_precision_; + int64_t prefix_pos_; // for prefix_sort + bool is_local_merge_sort_; // is used for final sort operator + ObRawExpr *topk_limit_expr_; + ObRawExpr *topk_offset_expr_; + bool is_fetch_with_ties_; + int64_t part_cnt_; + double sort_key_width_; + ObTopNFilterInfo topn_filter_info_; //top n filter can be pushdown to table scan +}; - // check if the current sort is a pushed down - inline bool is_prefix_sort() const { return prefix_pos_ != 0; } - inline bool is_local_merge_sort() const { return is_local_merge_sort_; } - inline bool is_part_sort() const { return part_cnt_ != 0; } - inline bool is_fetch_with_ties() const { return is_fetch_with_ties_; } - inline bool enable_encode_sortkey_opt() const { return encode_sortkeys_.count()!=0; } - inline int64_t get_part_cnt() const { return part_cnt_; } - inline int64_t get_prefix_pos() const { return prefix_pos_; } - inline ObRawExpr *get_topn_expr() const { return topn_expr_; } - inline void set_topk_limit_expr(ObRawExpr *top_limit_expr) - { - topk_limit_expr_ = top_limit_expr; - } - inline ObRawExpr *get_topk_limit_expr() { return topk_limit_expr_; } - inline void set_topk_offset_expr(ObRawExpr *topk_offset_expr) - { - topk_offset_expr_ = topk_offset_expr; - } - inline ObRawExpr *get_topk_offset_expr() { return topk_offset_expr_; } - int set_sort_keys(const common::ObIArray &order_keys); - virtual int get_op_exprs(ObIArray &all_exprs) override; - virtual int is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) override; - virtual uint64_t hash(uint64_t seed) const override; - virtual const char *get_name() const; - inline void set_minimal_row_count(int64_t minimum_row_count) - { - minimum_row_count_ = minimum_row_count; - } - inline int64_t get_minimum_row_count() const {return minimum_row_count_;} - inline void set_topk_precision(int64_t topk_precision) - { - topk_precision_ = topk_precision; - } - inline int64_t get_topk_precision() const {return topk_precision_;} - inline double get_sort_key_width() const { return sort_key_width_; } - virtual bool is_block_op() const override { return !is_prefix_sort(); } - virtual int compute_op_ordering() override; - virtual int get_plan_item_info(PlanText &plan_text, - ObSqlPlanItem &plan_item) override; - virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; - protected: - virtual int inner_replace_op_exprs(ObRawExprReplacer &replacer); - int est_sort_key_width(); - private: - OrderItem hash_sortkey_; - common::ObSEArray sort_keys_; - common::ObSEArray encode_sortkeys_; - ObRawExpr *topn_expr_; - int64_t minimum_row_count_; - int64_t topk_precision_; - int64_t prefix_pos_; // for prefix_sort - bool is_local_merge_sort_; // is used for final sort operator - ObRawExpr *topk_limit_expr_; - ObRawExpr *topk_offset_expr_; - bool is_fetch_with_ties_; - int64_t part_cnt_; - double sort_key_width_; - }; } // end of namespace sql } // end of namespace oceanbase diff --git a/src/sql/optimizer/ob_log_table_scan.h b/src/sql/optimizer/ob_log_table_scan.h index b714e04c4d..c0f7c3fbd3 100644 --- a/src/sql/optimizer/ob_log_table_scan.h +++ b/src/sql/optimizer/ob_log_table_scan.h @@ -576,6 +576,10 @@ public: inline uint64_t get_doc_id_index_table_id() const { return doc_id_table_id_; } virtual int get_card_without_filter(double &card) override; inline ObRawExpr *get_identify_seq_expr() { return identify_seq_expr_; } + inline int has_exec_param(bool &bool_ret) const + { + return est_cost_info_ == NULL ? common::OB_SUCCESS : est_cost_info_->has_exec_param(bool_ret); + } void set_identify_seq_expr(ObRawExpr *expr) { identify_seq_expr_ = expr; } const ObIArray& get_filter_monotonicity() const diff --git a/src/sql/optimizer/ob_log_window_function.cpp b/src/sql/optimizer/ob_log_window_function.cpp index d7ec3c86c9..03037b8c67 100644 --- a/src/sql/optimizer/ob_log_window_function.cpp +++ b/src/sql/optimizer/ob_log_window_function.cpp @@ -185,6 +185,66 @@ int ObLogWindowFunction::get_plan_item_info(PlanText &plan_text, return ret; } +int ObLogWindowFunction::est_input_rows_mem_bound_ratio() +{ + int ret = OB_SUCCESS; + double input_width = 0.0; + double wf_res_width = 0.0; + ObLogicalOperator *first_child = nullptr; + if (OB_ISNULL(get_plan()) || OB_ISNULL(first_child = get_child(ObLogicalOperator::first_child))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("first child is null", K(ret), K(first_child)); + } else { + input_width = first_child->get_width(); + for (int64_t i = 0; i < get_window_exprs().count(); i++) { + wf_res_width += ObOptEstCost::get_estimate_width_from_type(get_window_exprs().at(i)->get_result_type()); + } + const double input_mem_bound_ratio = input_width / (input_width + wf_res_width); + input_rows_mem_bound_ratio_ = input_mem_bound_ratio; + LOG_TRACE("est_input_rows_mem_bound_ration", K(input_width), K(wf_res_width), + K(input_mem_bound_ratio)); + } + return ret; +} + +int ObLogWindowFunction::est_window_function_part_cnt() +{ + int ret = OB_SUCCESS; + double estimated_part_cnt = 1.0; + ObLogicalOperator *first_child = get_child(ObLogicalOperator::first_child); + if (OB_ISNULL(first_child)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null first child", K(ret)); + } else if (get_window_exprs().count() > 0 && get_window_exprs().at(0)->get_partition_exprs().count() > 0) { + // FIME: @zongmei.zzm, modify the first partition columns to all the partition columns + // after @jiangxiu.wt support more accurate method to calculate NDV with multi columns + ObSEArray partition_exprs; + if (OB_FAIL(partition_exprs.push_back(get_window_exprs().at(0)->get_partition_exprs().at(0)))) { + LOG_WARN("push back element failed", K(ret)); + } else if (OB_FAIL(ObOptSelectivity::calculate_distinct( + get_plan()->get_update_table_metas(), get_plan()->get_selectivity_ctx(), + partition_exprs, first_child->get_card(), estimated_part_cnt))) { + LOG_WARN("calculate ndv failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + estimated_part_cnt_ = MAX(1.0, estimated_part_cnt); + LOG_TRACE("est_window_function_part_cnt success", K(ret), K(estimated_part_cnt)); + } + return ret; +} + +int ObLogWindowFunction::compute_property() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(est_window_function_part_cnt())) { + LOG_WARN("fail to est_window_function_part_cnt", K(ret)); + } else if (OB_FAIL(ObLogicalOperator::compute_property())) { + LOG_WARN("failed to compute property", K(ret)); + } + return ret; +} + int ObLogWindowFunction::est_width() { int ret = OB_SUCCESS; @@ -201,8 +261,10 @@ int ObLogWindowFunction::est_width() output_exprs, width))) { LOG_WARN("failed to estimate width for output winfunc exprs", K(ret)); + } else if (FALSE_IT(set_width(width))) { + } else if (OB_FAIL(est_input_rows_mem_bound_ratio())) { + LOG_WARN("estimate input rows mem bound ratio failed", K(ret)); } else { - set_width(width); LOG_TRACE("est_width for winfunc", K(output_exprs), K(width)); } return ret; diff --git a/src/sql/optimizer/ob_log_window_function.h b/src/sql/optimizer/ob_log_window_function.h index 73440717db..4c24acdc40 100644 --- a/src/sql/optimizer/ob_log_window_function.h +++ b/src/sql/optimizer/ob_log_window_function.h @@ -38,7 +38,9 @@ namespace sql rd_sort_keys_cnt_(0), rd_pby_sort_cnt_(0), wf_aggr_status_expr_(NULL), - origin_sort_card_(0.0) + origin_sort_card_(0.0), + input_rows_mem_bound_ratio_(0.0), + estimated_part_cnt_(0.0) {} virtual ~ObLogWindowFunction() {} virtual int get_explain_name_internal(char *buf, @@ -113,6 +115,11 @@ namespace sql int add_win_dist_options(const ObLogicalOperator *op, const ObIArray &all_win_funcs, ObWindowDistHint &win_dist_hint); + double get_input_rows_mem_bound_ratio() const { return input_rows_mem_bound_ratio_; } + double get_estimated_part_cnt() const { return estimated_part_cnt_; } + int est_input_rows_mem_bound_ratio(); + int est_window_function_part_cnt(); + virtual int compute_property() override; virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; private: ObSEArray win_exprs_; @@ -156,6 +163,9 @@ namespace sql //for est_cost when use topn sort double origin_sort_card_; + // for auto memory management + double input_rows_mem_bound_ratio_; + double estimated_part_cnt_; }; } } diff --git a/src/sql/optimizer/ob_logical_operator.cpp b/src/sql/optimizer/ob_logical_operator.cpp index 039cb8c9f2..5e621e305f 100644 --- a/src/sql/optimizer/ob_logical_operator.cpp +++ b/src/sql/optimizer/ob_logical_operator.cpp @@ -1609,6 +1609,12 @@ int ObLogicalOperator::do_post_traverse_operation(const TraverseOp &op, void *ct AllocBloomFilterContext *alloc_bf_ctx = static_cast(ctx); CK( OB_NOT_NULL(alloc_bf_ctx)); OC( (allocate_runtime_filter_for_hash_join)(*alloc_bf_ctx)); + if (OB_FAIL(ret)) { + } else if (LOG_SORT == get_type() + && OB_FAIL(static_cast(this) + ->try_allocate_pushdown_topn_runtime_filter())) { + LOG_WARN("failed to allocate topn runtime filter for sort"); + } break; } case ALLOC_OP: { @@ -5218,6 +5224,241 @@ int ObLogicalOperator::find_table_scan(ObLogicalOperator* root_op, return ret; } +int ObLogicalOperator::check_sort_key_can_pushdown_to_tsc_detail( + ObLogicalOperator *op, ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, bool &table_scan_has_exchange, + bool &has_px_coord) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(op)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect null logical operator", K(ret)); + } else if (op->is_block_op()) { + } else { + switch (op->get_type()) { + case LOG_GROUP_BY: { + if (OB_FAIL(check_sort_key_can_pushdown_to_tsc_for_gby( + op, candidate_sk_expr, table_id, scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord))) { + LOG_WARN("failed to check group by"); + } + break; + } + + case LOG_WINDOW_FUNCTION: { + if (OB_FAIL(check_sort_key_can_pushdown_to_tsc_for_winfunc( + op, candidate_sk_expr, table_id, scan_op, find_table_scan, table_scan_has_exchange, + has_px_coord))) { + LOG_WARN("failed to check window function"); + } + break; + } + + case LOG_JOIN: { + if (OB_FAIL(check_sort_key_can_pushdown_to_tsc_for_join( + op, candidate_sk_expr, table_id, scan_op, find_table_scan, table_scan_has_exchange, + has_px_coord))) { + LOG_WARN("failed to check join"); + } + break; + } + + case LOG_DISTINCT: + case LOG_EXCHANGE: { + ObLogicalOperator *child = op->get_child(first_child); + if (op->is_block_input(first_child)) { + } else if (OB_FAIL(SMART_CALL(check_sort_key_can_pushdown_to_tsc_detail( + child, candidate_sk_expr, table_id, scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord)))) { + LOG_WARN("failed to check", K(ret)); + } + break; + } + case LOG_TABLE_SCAN: { + ObLogTableScan *scan = static_cast(op); + if (scan->get_table_id() == table_id) { + bool has_exec_param = false; + if (scan->use_das()) { + LOG_TRACE("[TopN Filter]can not pushdown to das table scan"); + } else if (OB_FAIL(scan->has_exec_param(has_exec_param))) { + LOG_WARN("failed to has_exec_param"); + } else if (has_exec_param) { + LOG_TRACE("[TopN Filter]can not pushdown to tsc with exec param"); + } else { + scan_op = op; + find_table_scan = true; + } + } + break; + } + case LOG_TEMP_TABLE_ACCESS: { + ObLogTempTableAccess *scan = static_cast(op); + if (scan->get_table_id() == table_id) { + scan_op = op; + find_table_scan = true; + } + break; + } + case LOG_SORT: + case LOG_MATERIAL: { + LOG_TRACE("[TopN Filter]can not pushdown across sort or material"); + break; + } + default: { + break; + } + } + } + if (OB_SUCC(ret) && find_table_scan && log_op_def::LOG_EXCHANGE == op->get_type()) { + table_scan_has_exchange = true; + ObLogExchange *exch_op = static_cast(op); + if (exch_op->is_px_coord()) { + has_px_coord = true; + } + } + return ret; +} + +int ObLogicalOperator::check_sort_key_can_pushdown_to_tsc_for_gby( + ObLogicalOperator *op, ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, bool &table_scan_has_exchange, + bool &has_px_coord) +{ + int ret = OB_SUCCESS; + ObLogGroupBy *group_by = static_cast(op); + const common::ObIArray &group_by_exprs = group_by->get_group_by_exprs(); + // only the sort key is in group by expr, topn filter can pushdown + if (!is_contain(group_by_exprs, candidate_sk_expr)) { + } else { + ObLogicalOperator *child = group_by->get_child(first_child); + if (group_by->is_block_input(first_child)) { + } else if (OB_FAIL(SMART_CALL(check_sort_key_can_pushdown_to_tsc_detail( + child, candidate_sk_expr, table_id, scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord)))) { + LOG_WARN("failed to check", K(ret)); + } + } + return ret; +} + +int ObLogicalOperator::check_sort_key_can_pushdown_to_tsc_for_winfunc( + ObLogicalOperator *op, ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, bool &table_scan_has_exchange, + bool &has_px_coord) +{ + int ret = OB_SUCCESS; + ObLogWindowFunction *log_win_func = static_cast(op); + ObSEArray partition_exprs; + for (int64_t i = 0; OB_SUCC(ret) && i < log_win_func->get_window_exprs().count(); ++i) { + ObWinFunRawExpr *win_expr = log_win_func->get_window_exprs().at(i); + if (OB_ISNULL(win_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("window function expr is null", K(ret)); + } else if (i == 0) { + if (OB_FAIL(partition_exprs.assign(win_expr->get_partition_exprs()))) { + LOG_WARN("failed to assign partition exprs", K(ret)); + } + } else if (OB_FAIL(ObOptimizerUtil::intersect_exprs( + partition_exprs, win_expr->get_partition_exprs(), partition_exprs))) { + LOG_WARN("failed to intersect expr array", K(ret)); + } else if (partition_exprs.empty()) { + break; + } + } + if (OB_FAIL(ret)) { + } else if (candidate_sk_expr->has_flag(CNT_WINDOW_FUNC)) { + LOG_TRACE("[TopN Filter]countain win func, can not pushdown"); + } else if (!is_contain(partition_exprs, candidate_sk_expr)) { + LOG_TRACE("[TopN Filter]contain none partition by expr, can not pushdown"); + } else { + ObLogicalOperator *child = log_win_func->get_child(first_child); + if (log_win_func->is_block_input(first_child)) { + } else if (OB_FAIL(SMART_CALL(check_sort_key_can_pushdown_to_tsc_detail( + child, candidate_sk_expr, table_id, scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord)))) { + LOG_WARN("failed to check", K(ret)); + } + } + return ret; +} + +int ObLogicalOperator::check_sort_key_can_pushdown_to_tsc_for_join( + ObLogicalOperator *op, ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, bool &table_scan_has_exchange, + bool &has_px_coord) +{ + int ret = OB_SUCCESS; + ObLogJoin *log_join = static_cast(op); + ObJoinType join_type = log_join->get_join_type(); + if (FULL_OUTER_JOIN == join_type || CONNECT_BY_JOIN == join_type) { + // can not pushdown + LOG_TRACE("[TopN Filter]can not pushdown across full outer join and connnect by join"); + } else if (LEFT_OUTER_JOIN == join_type || LEFT_SEMI_JOIN == join_type + || LEFT_ANTI_JOIN == join_type) { + // output in left + ObLogicalOperator *child = log_join->get_child(first_child); + if (log_join->is_block_input(first_child)) { + LOG_TRACE("[TopN Filter]can not pushdown across left join but block left"); + } else if (OB_FAIL(SMART_CALL(check_sort_key_can_pushdown_to_tsc_detail( + child, candidate_sk_expr, table_id, scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord)))) { + LOG_WARN("failed to check", K(ret)); + } + } else if (RIGHT_OUTER_JOIN == join_type || RIGHT_ANTI_JOIN == join_type + || RIGHT_SEMI_JOIN == join_type) { + // output in right + ObLogicalOperator *child = log_join->get_child(second_child); + if (log_join->is_block_input(second_child)) { + LOG_TRACE("[TopN Filter]can not pushdown across right join but block right"); + } else if (OB_FAIL(SMART_CALL(check_sort_key_can_pushdown_to_tsc_detail( + child, candidate_sk_expr, table_id, scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord)))) { + LOG_WARN("failed to check", K(ret)); + } + } else if (INNER_JOIN == join_type) { + for (int64_t i = 0; OB_SUCC(ret) && nullptr == scan_op && i < log_join->get_num_of_child(); + ++i) { + ObLogicalOperator *child = log_join->get_child(i); + if (log_join->is_block_input(i)) { + continue; + } else if (OB_FAIL(SMART_CALL(check_sort_key_can_pushdown_to_tsc_detail( + child, candidate_sk_expr, table_id, scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord)))) { + LOG_WARN("failed to check", K(ret)); + } + } + } + return ret; +} + +int ObLogicalOperator::check_sort_key_can_pushdown_to_tsc( + ObLogicalOperator *root_op, + common::ObSEArray &candidate_sk_exprs, + uint64_t table_id, ObLogicalOperator *&scan_op, bool &table_scan_has_exchange, + bool &has_px_coord, int64_t &effective_sk_cnt) +{ + int ret = OB_SUCCESS; + scan_op = nullptr; + bool find_table_scan = true; + if (OB_ISNULL(root_op)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect null logical operator", K(ret)); + } + for (int64_t i = 0; i < candidate_sk_exprs.count() && OB_SUCC(ret) && find_table_scan; ++i) { + // for every expr, set to false and check + find_table_scan = false; + ObRawExpr *candidate_sk_expr = candidate_sk_exprs.at(i); + if (OB_FAIL(check_sort_key_can_pushdown_to_tsc_detail(root_op, candidate_sk_expr, table_id, + scan_op, find_table_scan, + table_scan_has_exchange, has_px_coord))) { + LOG_WARN("failed to check_sort_key_can_pushdown_to_tsc_detail"); + } else if (find_table_scan) { + effective_sk_cnt = i + 1; + } + } + return ret; +} + int ObLogicalOperator::allocate_partition_join_filter(const ObIArray &infos, int64_t &filter_id) { @@ -6243,7 +6484,6 @@ int ObLogicalOperator::find_max_px_resource_child(OPEN_PX_RESOURCE_ANALYZE_DECLA return ret; } - int ObLogicalOperator::check_op_orderding_used_by_parent(bool &used) { int ret = OB_SUCCESS; diff --git a/src/sql/optimizer/ob_logical_operator.h b/src/sql/optimizer/ob_logical_operator.h index c6225074fa..f7dd913418 100644 --- a/src/sql/optimizer/ob_logical_operator.h +++ b/src/sql/optimizer/ob_logical_operator.h @@ -1773,6 +1773,19 @@ protected: void clear_dist_flag(uint64_t &flags) const { flags = 0; } + + int find_table_scan(ObLogicalOperator* root_op, + uint64_t table_id, + ObLogicalOperator* &scan_op, + bool& table_scan_has_exchange, + bool &has_px_coord); + + int check_sort_key_can_pushdown_to_tsc( + ObLogicalOperator *op, + common::ObSEArray &effective_sk_exprs, + uint64_t table_id, ObLogicalOperator *&scan_op, bool &table_scan_has_exchange, + bool &has_px_coord, int64_t &effective_sk_cnt); + protected: log_op_def::ObLogOpType type_; @@ -1821,11 +1834,6 @@ private: ObLogicalOperator *op_sort, bool &need_remove, bool global_order); - int find_table_scan(ObLogicalOperator* root_op, - uint64_t table_id, - ObLogicalOperator* &scan_op, - bool& table_scan_has_exchange, - bool &has_px_coord); //private function, just used for allocating join filter node. int allocate_partition_join_filter(const ObIArray &infos, int64_t &filter_id); @@ -1866,6 +1874,23 @@ private: ObLogicalOperator *scan_node, const JoinFilterInfo &info); + int check_sort_key_can_pushdown_to_tsc_detail(ObLogicalOperator *op, + ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, + bool &table_scan_has_exchange, bool &has_px_coord); + int check_sort_key_can_pushdown_to_tsc_for_gby(ObLogicalOperator *op, + ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, + bool &table_scan_has_exchange, bool &has_px_coord); + int check_sort_key_can_pushdown_to_tsc_for_winfunc(ObLogicalOperator *op, + ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, + bool &table_scan_has_exchange, bool &has_px_coord); + int check_sort_key_can_pushdown_to_tsc_for_join(ObLogicalOperator *op, + ObRawExpr *candidate_sk_expr, uint64_t table_id, + ObLogicalOperator *&scan_op, bool &find_table_scan, + bool &table_scan_has_exchange, bool &has_px_coord); + /* manual set dop for each dfo */ int refine_dop_by_hint(); diff --git a/src/sql/optimizer/ob_opt_est_cost_model.cpp b/src/sql/optimizer/ob_opt_est_cost_model.cpp index b4481f1634..4576d817a4 100644 --- a/src/sql/optimizer/ob_opt_est_cost_model.cpp +++ b/src/sql/optimizer/ob_opt_est_cost_model.cpp @@ -128,6 +128,34 @@ void ObTableMetaInfo::assign(const ObTableMetaInfo &table_meta_info) table_type_ = table_meta_info.table_type_; } +int ObCostTableScanInfo::has_exec_param(const ObIArray &exprs, bool &bool_ret) const +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; i < exprs.count() && OB_SUCC(ret) && !bool_ret; ++i) { + if (OB_FAIL(exprs.at(i)->has_exec_param(bool_ret))) { + LOG_WARN("failed to has_exec_param"); + } + } + return ret; +} + +int ObCostTableScanInfo::has_exec_param(bool &bool_ret) const +{ + int ret = OB_SUCCESS; + bool_ret = false; + if (OB_FAIL(has_exec_param(prefix_filters_, bool_ret))) { + } else if (OB_FAIL(has_exec_param(pushdown_prefix_filters_, bool_ret))) { + LOG_WARN("failed to has_exec_param"); + } else if (OB_FAIL(has_exec_param(ss_postfix_range_filters_, bool_ret))) { + LOG_WARN("failed to has_exec_param"); + } else if (OB_FAIL(has_exec_param(postfix_filters_, bool_ret))) { + LOG_WARN("failed to has_exec_param"); + } else if (OB_FAIL(has_exec_param(table_filters_, bool_ret))) { + LOG_WARN("failed to has_exec_param"); + } + return ret; +} + double ObTableMetaInfo::get_micro_block_numbers() const { double ret = 0.0; diff --git a/src/sql/optimizer/ob_opt_est_cost_model.h b/src/sql/optimizer/ob_opt_est_cost_model.h index ccd74a4142..182a1a2be8 100644 --- a/src/sql/optimizer/ob_opt_est_cost_model.h +++ b/src/sql/optimizer/ob_opt_est_cost_model.h @@ -244,6 +244,8 @@ struct ObCostTableScanInfo { } int assign(const ObCostTableScanInfo &other_est_cost_info); + int has_exec_param(bool &bool_ret) const; + int has_exec_param(const ObIArray &exprs, bool &bool_ret) const; TO_STRING_KV(K_(table_id), K_(ref_table_id), K_(index_id), K_(table_meta_info), K_(index_meta_info), diff --git a/src/sql/resolver/dml/ob_hint.cpp b/src/sql/resolver/dml/ob_hint.cpp index a9c4c1dd15..c9454004f2 100644 --- a/src/sql/resolver/dml/ob_hint.cpp +++ b/src/sql/resolver/dml/ob_hint.cpp @@ -838,6 +838,18 @@ bool ObOptParamHint::is_param_val_valid(const OptParamType param_type, const ObO || 0 == val.get_varchar().case_compare("false")); break; } + case SPILL_COMPRESSION_CODEC: { + is_valid = val.is_varchar(); + if (is_valid) { + bool exist_valid_codec = false; + for (int i = 0; i < ARRAYSIZEOF(common::sql_temp_store_compress_funcs) && !exist_valid_codec; ++i) { + if (0 == ObString::make_string(sql_temp_store_compress_funcs[i]).case_compare(val.get_varchar())) { + exist_valid_codec = true; + } + } + is_valid = exist_valid_codec; + } + } case INLIST_REWRITE_THRESHOLD: { is_valid = val.is_int() && (0 < val.get_int()); break; diff --git a/src/sql/resolver/dml/ob_hint.h b/src/sql/resolver/dml/ob_hint.h index e49bb61905..53872240c2 100644 --- a/src/sql/resolver/dml/ob_hint.h +++ b/src/sql/resolver/dml/ob_hint.h @@ -156,6 +156,7 @@ struct ObOptParamHint DEF(ENABLE_RICH_VECTOR_FORMAT,) \ DEF(_ENABLE_STORAGE_CARDINALITY_ESTIMATION,) \ DEF(PRESERVE_ORDER_FOR_PAGINATION,) \ + DEF(SPILL_COMPRESSION_CODEC,) \ DEF(INLIST_REWRITE_THRESHOLD,) \ DECLARE_ENUM(OptParamType, opt_param, OPT_PARAM_TYPE_DEF, static); diff --git a/src/sql/resolver/expr/ob_raw_expr.cpp b/src/sql/resolver/expr/ob_raw_expr.cpp index 1fd03570e8..bd68560a84 100644 --- a/src/sql/resolver/expr/ob_raw_expr.cpp +++ b/src/sql/resolver/expr/ob_raw_expr.cpp @@ -1107,6 +1107,23 @@ int ObRawExpr::extract_local_session_vars_recursively(ObIArrayhas_exec_param(bool_ret)))) { + LOG_WARN("failed to has_exec_param"); + } + } + } + return ret; +} + //////////////////////////////////////////////////////////////// int ObConstRawExpr::assign(const ObRawExpr &other) { @@ -2887,6 +2904,25 @@ int ObOpRawExpr::get_name_internal(char *buf, const int64_t buf_len, int64_t &po if (OB_FAIL(BUF_PRINTF("BM25(k1=1.2, b=0.75, epsilon=0.25)"))) { LOG_WARN("fail to BUF_PRINTF", K(ret)); } + } else if (T_OP_PUSHDOWN_TOPN_FILTER == get_expr_type()) { + if (OB_FAIL(BUF_PRINTF("TOPN_FILTER("))) { + LOG_WARN("fail to BUF_PRINTF", K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < get_param_count() ; ++i) { + if (OB_ISNULL(get_param_expr(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("param_expr is NULL", K(i), K(ret)); + } else if (OB_FAIL(get_param_expr(i)->get_name(buf, buf_len, pos, type))) { + LOG_WARN("fail to get_name", K(i), K(ret)); + } else if (i < get_param_count() - 1) { + if (OB_FAIL(BUF_PRINTF(", "))) { + LOG_WARN("fail to BUF_PRINTF", K(i), K(ret)); + } + } else if (OB_FAIL(BUF_PRINTF(")"))) { + LOG_WARN("fail to BUF_PRINTF", K(i), K(ret)); + } + } + } } else { if (OB_FAIL(BUF_PRINTF("(%s", get_type_name(get_expr_type())))) { LOG_WARN("fail to BUF_PRINTF", K(ret)); @@ -2990,6 +3026,15 @@ bool ObOpRawExpr::is_white_runtime_filter_expr() const break; } } + // sort is compare in vectorize format, so only one column can pushdown as + // white filter + } else if (T_OP_PUSHDOWN_TOPN_FILTER == type_ && 1 == exprs_.count() + && T_REF_COLUMN == exprs_.at(0)->get_expr_type()) { + // FIXME: @zhouhaiyu.zhy + // for now, storage pushdown filter can not process both a < 10 and a is null in one filter + // so disable white topn runtime filter + // LOG_TRACE("[TopN Filter] push topn filter as white filter"); + bool_ret = false; } else { bool_ret = false; } diff --git a/src/sql/resolver/expr/ob_raw_expr.h b/src/sql/resolver/expr/ob_raw_expr.h index 7797f316d3..7ec267c53e 100644 --- a/src/sql/resolver/expr/ob_raw_expr.h +++ b/src/sql/resolver/expr/ob_raw_expr.h @@ -2029,6 +2029,8 @@ public: void set_local_session_var_id(int64_t idx) { local_session_var_id_ = idx; } int64_t get_local_session_var_id() { return local_session_var_id_; } + int has_exec_param(bool &bool_ret) const; + private: const ObRawExpr *get_same_identify(const ObRawExpr *e, const ObExprEqualCheckContext *check_ctx) const; diff --git a/src/storage/column_store/ob_co_sstable_row_scanner.cpp b/src/storage/column_store/ob_co_sstable_row_scanner.cpp index 685d426b76..ebd53f82ce 100644 --- a/src/storage/column_store/ob_co_sstable_row_scanner.cpp +++ b/src/storage/column_store/ob_co_sstable_row_scanner.cpp @@ -746,7 +746,9 @@ int ObCOSSTableRowScanner::update_continuous_range( if (nullptr != result_bitmap) { group_is_true = result_bitmap->is_all_true(); } - if (group_is_true) { + bool filter_tree_can_continuous = + rows_filter_ == nullptr ? true : rows_filter_->can_continuous_filter(); + if (group_is_true && filter_tree_can_continuous) { // current group is true, continue do filter if not reach end if (reverse_scan_) { continuous_end_row_id = current_start_row_id; diff --git a/src/storage/column_store/ob_co_sstable_rows_filter.cpp b/src/storage/column_store/ob_co_sstable_rows_filter.cpp index 084c57d862..4ecdc1ea49 100644 --- a/src/storage/column_store/ob_co_sstable_rows_filter.cpp +++ b/src/storage/column_store/ob_co_sstable_rows_filter.cpp @@ -17,6 +17,7 @@ #include "ob_co_where_optimizer.h" #include "storage/access/ob_block_row_store.h" #include "storage/access/ob_table_access_context.h" +#include "common/ob_smart_call.h" namespace oceanbase { @@ -37,7 +38,8 @@ ObCOSSTableRowsFilter::ObCOSSTableRowsFilter() filter_iters_(), iter_filter_node_(), bitmap_buffer_(), - pd_filter_info_() + pd_filter_info_(), + can_continuous_filter_(true) { } @@ -78,6 +80,8 @@ int ObCOSSTableRowsFilter::init( } else if (FALSE_IT(depth = nullptr == context.sample_filter_ ? depth : depth + 1)) { } else if (OB_FAIL(init_bitmap_buffer(depth))) { LOG_WARN("Failed to init bitmap buffer", K(ret), K(depth)); + } else if (OB_FAIL(filter_tree_can_continuous_filter(filter_, can_continuous_filter_))) { + LOG_WARN("failed to filter_tree_can_continuous_filter", K(ret)); } else { is_inited_ = true; } @@ -877,5 +881,27 @@ int ObCOSSTableRowsFilter::switch_context_for_cg_iter( return ret; } +int ObCOSSTableRowsFilter::filter_tree_can_continuous_filter(sql::ObPushdownFilterExecutor *filter, + bool &can_continuous_filter) const +{ + int ret = OB_SUCCESS; + if (nullptr == filter) { + can_continuous_filter = true; + } else if (!filter->filter_can_continuous_filter()) { + can_continuous_filter = false; + } else { + for (int64_t i = 0; i < filter->get_child_count(); ++i) { + sql::ObPushdownFilterExecutor *child = nullptr; + (void)filter->get_child(i, child); + if (OB_FAIL(SMART_CALL(filter_tree_can_continuous_filter(child, can_continuous_filter)))) { + LOG_WARN("failed to filter_tree_can_continuous_filter"); + } else if (!can_continuous_filter) { + break; + } + } + } + return ret; +} + } } diff --git a/src/storage/column_store/ob_co_sstable_rows_filter.h b/src/storage/column_store/ob_co_sstable_rows_filter.h index 56e27b48c4..212794f3f6 100644 --- a/src/storage/column_store/ob_co_sstable_rows_filter.h +++ b/src/storage/column_store/ob_co_sstable_rows_filter.h @@ -64,6 +64,7 @@ public: common::ObIArray &cg_params, const bool col_cnt_changed, ObICGIterator *&cg_iter); + inline bool can_continuous_filter() const { return can_continuous_filter_; } TO_STRING_KV(K_(is_inited), K_(subtree_filter_iter_to_locate), K_(batch_size), KPC_(iter_param), KP_(access_ctx), KP_(co_sstable), K_(filter), K_(filter_iters), K_(iter_filter_node), K_(bitmap_buffer), K_(pd_filter_info)); @@ -124,6 +125,8 @@ private: const sql::ObCommonFilterTreeStatus status_two); static void set_status_of_filter_tree(sql::ObPushdownFilterExecutor *filter); static void clear_filter_state(sql::ObPushdownFilterExecutor *filter); + int filter_tree_can_continuous_filter(sql::ObPushdownFilterExecutor *filter, + bool &can_continuous_filter) const; private: bool is_inited_; bool prepared_; @@ -139,6 +142,7 @@ private: ObSEArray iter_filter_node_; ObSEArray bitmap_buffer_; sql::PushdownFilterInfo pd_filter_info_; + bool can_continuous_filter_; }; } } diff --git a/src/storage/column_store/ob_co_where_optimizer.cpp b/src/storage/column_store/ob_co_where_optimizer.cpp index 639ed7da68..24137f5091 100644 --- a/src/storage/column_store/ob_co_where_optimizer.cpp +++ b/src/storage/column_store/ob_co_where_optimizer.cpp @@ -137,6 +137,14 @@ uint64_t ObCOWhereOptimizer::estimate_execution_cost(sql::ObPushdownFilterExecut uint64_t execution_cost; if (!filter.is_filter_white_node()) { execution_cost = UINT64_MAX; + } else if (filter.is_filter_dynamic_node()) { + ObDynamicFilterExecutor &dynamic_filter = static_cast(filter); + if (dynamic_filter.get_filter_node().get_dynamic_filter_type() + == DynamicFilterType::PD_TOPN_FILTER) { + execution_cost = 1; + } else { + execution_cost = UINT64_MAX; + } } else { ObWhiteFilterExecutor &white_filter = static_cast(filter); switch (white_filter.get_op_type()) { diff --git a/tools/deploy/mysql_test/test_suite/groupby/r/mysql/group_by_basic.result b/tools/deploy/mysql_test/test_suite/groupby/r/mysql/group_by_basic.result index dd0dee8114..e79c3abe43 100644 --- a/tools/deploy/mysql_test/test_suite/groupby/r/mysql/group_by_basic.result +++ b/tools/deploy/mysql_test/test_suite/groupby/r/mysql/group_by_basic.result @@ -449,9 +449,9 @@ Outputs & filters: range_key([t29_25.__pk_increment]), range(MIN ; MAX)always true 3 - output([t29_25.c1]), filter(nil), rowset=256 sort_keys([t29_25.c1, DESC]), topn(1) - 4 - output([t29_25.c1]), filter(nil), rowset=256 + 4 - output([t29_25.c1]), filter([TOPN_FILTER(t29_25.c1)]), rowset=256 access([t29_25.c1]), partitions(p0) - is_index_back=false, is_global_index=false, + is_index_back=false, is_global_index=false, filter_before_indexback[false], range_key([t29_25.__pk_increment]), range(MIN ; MAX)always true create table t2 as SELECT t29_25.c1, t29_25.c8 FROM (SELECT (SELECT t29_25.c1 diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result index 1f3895b9d2..0c69f7a815 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result @@ -213,6 +213,7 @@ server_check_interval server_cpu_quota_max server_cpu_quota_min server_permanent_offline_time +spill_compression_codec sql_login_thread_count sql_net_thread_count sql_protocol_min_tls_version diff --git a/unittest/sql/engine/basic/test_ra_temp_row_store.cfg b/unittest/sql/engine/basic/test_ra_temp_row_store.cfg new file mode 100644 index 0000000000..0936c78606 --- /dev/null +++ b/unittest/sql/engine/basic/test_ra_temp_row_store.cfg @@ -0,0 +1,8 @@ +digit_data_format=4 +string_data_format=4 +data_range_level=0 +skips_probability=10 +nulls_probability=30 +round=3 +batch_size=256 +output_result_to_file=0 \ No newline at end of file diff --git a/unittest/sql/engine/basic/test_ra_temp_row_store.cpp b/unittest/sql/engine/basic/test_ra_temp_row_store.cpp new file mode 100644 index 0000000000..159a5eb060 --- /dev/null +++ b/unittest/sql/engine/basic/test_ra_temp_row_store.cpp @@ -0,0 +1,446 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +// #define USING_LOG_PREFIX SQL_ENGINE +#define USING_LOG_PREFIX COMMON +#include +#include +#include "sql/engine/test_op_engine.h" +#include "sql/engine/ob_test_config.h" +#include "sql/engine/basic/ob_temp_row_store.h" +#include +#include +#include +#include "lib/alloc/ob_malloc_allocator.h" +#include "lib/allocator/ob_malloc.h" +#include "storage/blocksstable/ob_data_file_prepare.h" +#include "storage/blocksstable/ob_tmp_file.h" +#include "sql/engine/basic/ob_temp_row_store.h" +#include "sql/engine/basic/ob_ra_row_store.h" +#include "common/row/ob_row_store.h" +#include "share/config/ob_server_config.h" +#include "sql/ob_sql_init.h" +#include "share/datum/ob_datum.h" +#include "sql/engine/expr/ob_expr.h" +#include "share/ob_simple_mem_limit_getter.h" +#include "share/vector/ob_fixed_length_vector.h" + +using namespace ::oceanbase::sql; + +namespace test +{ +namespace sql +{ +using namespace common; +static ObSimpleMemLimitGetter getter; + +#define CALL(func, ...) func(__VA_ARGS__); ASSERT_FALSE(HasFatalFailure()); + +struct MyAllocator : public DefaultPageAllocator +{ + void *alloc(const int64_t sz, const ObMemAttr &attr) override + { + int64_t size = sz + 8; + uint64_t *mem = (uint64_t *)DefaultPageAllocator::alloc(size, attr); + if (NULL != mem) { + *mem = sz; + total_ += sz; + return mem + 1; + } + return NULL; + } + + void free(void *p) + { + if (NULL != p) { + uint64_t *mem = ((uint64_t *)p - 1); + total_ -= *mem; + memset(p, 0xAA, *mem); + DefaultPageAllocator::free(mem); + } + } + + int64_t total_ = 0; +}; + +class TestRATempRowStore : public TestOpEngine +{ +public: + TestRATempRowStore(); + virtual ~TestRATempRowStore(); + virtual void SetUp(); + virtual void TearDown(); + +private: + // disallow copy + DISALLOW_COPY_AND_ASSIGN(TestRATempRowStore); + +protected: + // function members + void init_exprs() { + int ret = OB_SUCCESS; + std::string test_file_path = ObTestOpConfig::get_instance().test_filename_prefix_ + ".test"; + std::ifstream if_tests(test_file_path); + ASSERT_EQ(if_tests.is_open(), true); + std::string line; + bool first_line = true; + while (std::getline(if_tests, line)) { + // handle query + if (line.size() <= 0) continue; + if (line.at(0) == '#') continue; + + if (first_line) { + ObOperator *op = NULL; + ObExecutor exector; + ASSERT_EQ(OB_SUCCESS, get_tested_op_from_string(line, true, op, exector)); + int round = 1; + const int64_t max_row_cnt = 256; + const ObBatchRows *child_brs = nullptr; + while (!op->brs_.end_) { + ASSERT_EQ(OB_SUCCESS, op->get_next_batch(max_row_cnt, child_brs)); + if (cells_.count_ == 0 && op->brs_.size_ > 0) { + FOREACH_CNT_X(e, op->spec_.output_, OB_SUCC(ret)) + { + ASSERT_EQ(OB_SUCCESS, cells_.push_back(*e)); + } + eval_ctx_ = new ObEvalCtx(op->eval_ctx_); + } + } + first_line = false; + } else { + ObOperator *op = NULL; + ObExecutor exector; + ASSERT_EQ(OB_SUCCESS, get_tested_op_from_string(line, true, op, exector, true)); + int round = 1; + const int64_t max_row_cnt = 256; + const ObBatchRows *child_brs = nullptr; + while (!op->brs_.end_) { + ASSERT_EQ(OB_SUCCESS, op->get_next_batch(max_row_cnt, child_brs)); + if (ver_cells_.count_ == 0 && op->brs_.size_ > 0) { + FOREACH_CNT_X(e, op->spec_.output_, OB_SUCC(ret)) + { + ASSERT_EQ(OB_SUCCESS, ver_cells_.push_back(*e)); + } + ver_eval_ctx_ = new ObEvalCtx(op->eval_ctx_); + } + } + } + exec_ctx_.~ObExecContext(); + new (&exec_ctx_) ObExecContext(allocator_); + exec_ctx_.set_sql_ctx(&sql_ctx_); + exec_ctx_.set_my_session(&session_info_); + exec_ctx_.create_physical_plan_ctx(); + + vec_2_exec_ctx_.~ObExecContext(); + new (&vec_2_exec_ctx_) ObExecContext(allocator_); + vec_2_exec_ctx_.set_sql_ctx(&sql_ctx_); + vec_2_exec_ctx_.set_my_session(&session_info_); + vec_2_exec_ctx_.create_physical_plan_ctx(); + } + if_tests.close(); + } + + void gen_row(int64_t row_id, int64_t idx = 0) + { + ObIVector *vector_0 = cells_.at(0)->get_vector(*eval_ctx_); + vector_0->set_int(idx, row_id); + ObIVector *vector_1 = cells_.at(1)->get_vector(*eval_ctx_); + vector_1->set_null(idx); + + int64_t size = 10 + random() % BUF_SIZE; + ObIVector *vector_2 = cells_.at(2)->get_vector(*eval_ctx_); + vector_2->set_string(idx, str_buf_, (int)size); + } + + void verify_row(ObRATempRowStore::RAReader& it, int64_t n, bool verify_all = false) + { + int ret = it.get_row(n, stored_row_); + ASSERT_EQ(OB_SUCCESS, ret); + verify_row_data(n, verify_all); + } + + void verify_row_data(int64_t n, bool verify_all = false, int64_t idx = 0, bool batch_verify = false) + { + ObIVector *vector_0 = ver_cells_.at(0)->get_vector(*ver_eval_ctx_); + ObIVector *vector_1 = ver_cells_.at(1)->get_vector(*ver_eval_ctx_); + ObIVector *vector_2 = ver_cells_.at(2)->get_vector(*ver_eval_ctx_); + if (!batch_verify) { + vector_0->from_row(row_meta_, stored_row_, idx, 0); + vector_1->from_row(row_meta_, stored_row_, idx, 1); + vector_2->from_row(row_meta_, stored_row_, idx, 2); + } + int64_t v = vector_0->get_int(idx); + if (n >= 0) { + ASSERT_EQ(n, v); + } + if (verify_all) { + ASSERT_EQ(true, vector_1->is_null(idx)); + ASSERT_EQ(0, strncmp(str_buf_, vector_2->get_payload(idx), vector_2->get_length(idx))); + } + } + + void verify_n_rows(ObRATempRowStore &rs, ObRATempRowStore::RAReader &it, int64_t n, + bool verify_all = false, int64_t chunk_size = 0, int64_t start = 0) + { + for (int64_t i = start; i < n; i++) { + verify_row(it, i, verify_all); + } + } + + void verify_n_rows(int64_t n, bool verify_all = false) + { + return verify_n_rows(rs_, reader_, n, verify_all); + } + + void append_rows(ObRATempRowStore &rs, int64_t cnt) + { + int64_t ret = OB_SUCCESS; + int64_t base = rs.get_row_cnt(); + ObCompactRow *sr = nullptr; + for (int64_t i = 0; i < cnt; i++) { + gen_row(base + i); + ret = rs.add_row(cells_, *eval_ctx_, sr); + ASSERT_EQ(OB_SUCCESS, ret); + } + ASSERT_EQ(base + cnt, rs.get_row_cnt()); + } + + void batch_append_rows(int64_t cnt) + { + int64_t ret = 0; + int64_t base = rs_.get_row_cnt(); + skip_->reset(batch_size_); + for (int64_t i = 0; i < cnt;) { + int64_t bcnt = std::min(cnt - i, batch_size_); + for (int64_t j = 0; j < bcnt; j++, i++) { + gen_row(base + i, j); + } + int64_t stored_row_cnt = 0; + ObCompactRow **srs = nullptr; + ObBatchRows *br = new ObBatchRows(); + br->skip_ = skip_; + br->all_rows_active_ = true; + br->size_ = bcnt; + ret = rs_.add_batch(cells_, *eval_ctx_, *br, stored_row_cnt, srs); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(stored_row_cnt, bcnt); + } + ASSERT_EQ(base + cnt, rs_.get_row_cnt()); + } + + void batch_verify_all(int64_t row_cnt) + { + reader_.reset(); + int64_t read_cnt = 0; + int ret = OB_SUCCESS; + while (OB_SUCC(ret) && read_cnt < row_cnt) { + int64_t cnt = 0; + int64_t bcnt = std::min(batch_size_, row_cnt - read_cnt); + ret = reader_.get_batch_rows(ver_cells_, *ver_eval_ctx_, read_cnt, read_cnt + bcnt, cnt, stored_rows_); + if (OB_SUCC(ret)) { + ASSERT_GT(cnt, 0); + for (int64_t i = 0; i < cnt; i++) { + CALL(verify_row_data, read_cnt + i, true, i, true); + } + read_cnt += cnt; + } + } + if (row_cnt > rs_.get_row_cnt()) { + ASSERT_EQ(read_cnt, rs_.get_row_cnt()); + } else { + ASSERT_EQ(read_cnt, row_cnt); + } + } + void append_rows(int64_t cnt) + { + return append_rows(rs_, cnt); + } + + int init_tenant_mgr(); + +protected: + // data members + const static int64_t COLS = 3; + int64_t cell_cnt_; + MyAllocator rs_alloc_; + ObRATempRowStore rs_; + ObRATempRowStore::RAReader reader_; + const ObCompactRow *stored_row_; + const ObCompactRow **stored_rows_; + RowMeta &row_meta_; + ObBitVector *skip_; + int64_t tenant_id_ = OB_SYS_TENANT_ID; + const static int64_t BUF_SIZE = 2 << 7; + char str_buf_[BUF_SIZE]; + int64_t batch_size_ = 256; + ObSEArray cells_; + ObSEArray ver_cells_; + ObEvalCtx *eval_ctx_; + ObEvalCtx *ver_eval_ctx_; +}; + +TestRATempRowStore::TestRATempRowStore() : +rs_(), +reader_(), +row_meta_(rs_.row_meta_), +eval_ctx_(NULL), +ver_eval_ctx_(NULL) +{ + std::string schema_filename = ObTestOpConfig::get_instance().test_filename_prefix_ + ".schema"; + strcpy(schema_file_path_, schema_filename.c_str()); +} + +TestRATempRowStore::~TestRATempRowStore() +{} + +int TestRATempRowStore::init_tenant_mgr() +{ + int ret = OB_SUCCESS; + ObAddr self; + oceanbase::rpc::frame::ObReqTransport req_transport(NULL, NULL); + oceanbase::obrpc::ObSrvRpcProxy rpc_proxy; + oceanbase::obrpc::ObCommonRpcProxy rs_rpc_proxy; + oceanbase::share::ObRsMgr rs_mgr; + int64_t tenant_id = OB_SYS_TENANT_ID; + self.set_ip_addr("127.0.0.1", 8086); + ret = getter.add_tenant(tenant_id, + 2L * 1024L * 1024L * 1024L, 4L * 1024L * 1024L * 1024L); + EXPECT_EQ(OB_SUCCESS, ret); + const int64_t ulmt = 128LL << 30; + const int64_t llmt = 128LL << 30; + ret = getter.add_tenant(OB_SERVER_TENANT_ID, + ulmt, + llmt); + EXPECT_EQ(OB_SUCCESS, ret); + oceanbase::lib::set_memory_limit(128LL << 32); + return ret; +} + + +void TestRATempRowStore::SetUp() +{ + TestOpEngine::SetUp(); + int ret = OB_SUCCESS; + ASSERT_EQ(OB_SUCCESS, init_tenant_mgr()); + ASSERT_EQ(OB_SUCCESS, ret); + + cell_cnt_ = COLS; + + skip_ = (ObBitVector *)vec_2_alloc_.alloc(ObBitVector::memory_size(batch_size_)); + + init_exprs(); + //mem limit 1M + rs_.set_allocator(rs_alloc_); + ObMemAttr attr(tenant_id_, "TestTmpRStore", ObCtxIds::WORK_AREA); + const int64_t extra_size = sizeof(uint64_t); // for hash value + ret = rs_.init(cells_, 1L << 10, attr, 1L << 20, true, extra_size); + row_meta_ = rs_.row_meta_; + ASSERT_EQ(OB_SUCCESS, ret); + ret = reader_.init(&rs_); + ASSERT_EQ(OB_SUCCESS, ret); + ASSERT_EQ(OB_SUCCESS, rs_.alloc_dir_id()); + + memset(str_buf_, 'a', BUF_SIZE); + for (int64_t i = 0; i < BUF_SIZE; i++) { + str_buf_[i] += i % 26; + } + LOG_INFO("setup finished"); +} + +void TestRATempRowStore::TearDown() +{ + destroy(); + reader_.reset(); + rs_.reset(); + rs_.~ObRATempRowStore(); + + blocksstable::ObTmpFileManager::get_instance().destroy(); + LOG_INFO("TearDown finished", K_(rs)); +} + +TEST_F(TestRATempRowStore, basic) +{ + int ret = OB_SUCCESS; + LOG_WARN("starting basic test: append 200 rows"); + CALL(append_rows, 300); // no need to dump + CALL(verify_n_rows, rs_.get_row_cnt(), true); + LOG_WARN("basic test: varified rows", K(rs_.get_row_cnt())); + reader_.reset(); + rs_.reset(); + + LOG_WARN("starting basic test: append 4000 rows"); + CALL(append_rows, 40000); //need to dump + ASSERT_EQ(40000, rs_.get_row_cnt()); + LOG_WARN("starting basic test: verify rows"); + CALL(verify_n_rows, rs_.get_row_cnt() - 1, true); + + ret = reader_.get_row(rs_.get_row_cnt() - 1, stored_row_); + ASSERT_EQ(OB_SUCCESS, ret); + ret = reader_.get_row(rs_.get_row_cnt(), stored_row_); + ASSERT_EQ(OB_INDEX_OUT_OF_RANGE, ret); + reader_.reset(); + + ret = reader_.get_row(0, stored_row_); + ASSERT_EQ(OB_SUCCESS, ret); + LOG_WARN("first row"); + reader_.reset(); + rs_.reset(); + + LOG_INFO("============== start to test batch ==============="); + CALL(batch_append_rows, 300); // no need to dump + CALL(verify_n_rows, rs_.get_row_cnt(), true); + reader_.reset(); + CALL(batch_verify_all, rs_.get_row_cnt()); + reader_.reset(); + rs_.reset(); + ASSERT_EQ(0, rs_alloc_.total_); + + LOG_INFO("============= start to test batch dump ==========="); + CALL(batch_append_rows, 30000); // need dump + ASSERT_EQ(30000, rs_.get_row_cnt()); + CALL(batch_verify_all, 0); + CALL(batch_verify_all, 100); + CALL(batch_verify_all, 30000); + CALL(batch_verify_all, 50000); + reader_.reset(); + rs_.reset(); + ASSERT_EQ(0, rs_alloc_.total_); + + reader_.reset(); + rs_.reset(); + ASSERT_EQ(0, rs_alloc_.total_); +} + +} // namespace test +} // namespace test + +int main(int argc, char **argv) +{ + ObTestOpConfig::get_instance().test_filename_prefix_ = "test_ra_temp_row_store"; + for (int i = 1; i < argc; i++) { + if (strcmp(argv[i], "-bg") == 0) { + ObTestOpConfig::get_instance().test_filename_prefix_ += "_bg"; + ObTestOpConfig::get_instance().run_in_background_ = true; + } + } + ObTestOpConfig::get_instance().init(); + + system(("rm -f " + ObTestOpConfig::get_instance().test_filename_prefix_ + ".log").data()); + system(("rm -f " + ObTestOpConfig::get_instance().test_filename_prefix_ + ".log.*").data()); + oceanbase::common::ObClockGenerator::init(); + observer::ObReqTimeGuard req_timeinfo_guard; + OB_LOGGER.set_log_level("INFO"); + OB_LOGGER.set_file_name((ObTestOpConfig::get_instance().test_filename_prefix_ + ".log").data(), true); + init_sql_factories(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} \ No newline at end of file diff --git a/unittest/sql/engine/basic/test_ra_temp_row_store.schema b/unittest/sql/engine/basic/test_ra_temp_row_store.schema new file mode 100644 index 0000000000..162929f770 --- /dev/null +++ b/unittest/sql/engine/basic/test_ra_temp_row_store.schema @@ -0,0 +1,4 @@ +#create database opt; +#use opt; +create table t1(c1 int, c2 int, c3 varchar(128)); +create table t2(c1 int, c2 int, c3 varchar(128)); \ No newline at end of file diff --git a/unittest/sql/engine/basic/test_ra_temp_row_store.test b/unittest/sql/engine/basic/test_ra_temp_row_store.test new file mode 100644 index 0000000000..d8cb1a709c --- /dev/null +++ b/unittest/sql/engine/basic/test_ra_temp_row_store.test @@ -0,0 +1,3 @@ +# Please ensure that there are only two sql, one sql for storage and one sql for verification +select c1, c2, c3 from t1 +select c1, c2, c3 from t2; diff --git a/unittest/sql/engine/px/CMakeLists.txt b/unittest/sql/engine/px/CMakeLists.txt index a4f48ba3b8..2c945dc80e 100644 --- a/unittest/sql/engine/px/CMakeLists.txt +++ b/unittest/sql/engine/px/CMakeLists.txt @@ -1,3 +1,4 @@ sql_unittest(test_random_affi) #sql_unittest(test_slice_calc) +sql_unittest(test_adaptive_slide_window) sql_unittest(test_ob_small_hashset) diff --git a/unittest/sql/engine/px/test_adaptive_slide_window.cpp b/unittest/sql/engine/px/test_adaptive_slide_window.cpp new file mode 100644 index 0000000000..e2631ec6ab --- /dev/null +++ b/unittest/sql/engine/px/test_adaptive_slide_window.cpp @@ -0,0 +1,93 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#include + +#define private public +#include "sql/engine/expr/ob_expr_operator.h" + +#define ADAPTIVE_SLIDE_WINDOW_SIZE 4096 + +using namespace std; +namespace oceanbase +{ +namespace sql +{ +class AdapitveSlideWindowTest : public ::testing::Test +{ +public: + AdapitveSlideWindowTest() = default; + virtual ~AdapitveSlideWindowTest() = default; + virtual void SetUp(){}; + virtual void TearDown(){}; + +public: + int64_t total_count_{0}; + ObAdaptiveFilterSlideWindow slide_window_{total_count_}; + +private: + DISALLOW_COPY_AND_ASSIGN(AdapitveSlideWindowTest); +}; + +static void mock_filter(ObAdaptiveFilterSlideWindow &slide_window, int64_t &total_count) +{ + bool reopen_flag = false; + int64_t partial_total_count = common::ObRandom::rand(1, ADAPTIVE_SLIDE_WINDOW_SIZE); + int64_t partial_filter_count = common::ObRandom::rand(0, partial_total_count); + total_count += partial_total_count; + if (slide_window.dynamic_disable()) { + if (slide_window.cur_pos_ >= slide_window.next_check_start_pos_) { + reopen_flag = true; + } + partial_filter_count = 0; + slide_window.update_slide_window_info(partial_filter_count, partial_total_count); + EXPECT_EQ(reopen_flag, !slide_window.dynamic_disable()); + EXPECT_EQ(0, slide_window.partial_filter_count_); + EXPECT_EQ(0, slide_window.partial_total_count_); + } else { + int64_t acc_partial_total_count = partial_total_count + slide_window.partial_total_count_; + int64_t acc_partial_filter_count = partial_filter_count + slide_window.partial_filter_count_; + double filter_rate = acc_partial_filter_count / (double)acc_partial_total_count; + bool need_check_filter_rate = + total_count >= slide_window.next_check_start_pos_ + ADAPTIVE_SLIDE_WINDOW_SIZE; + slide_window.update_slide_window_info(partial_filter_count, partial_total_count); + if (need_check_filter_rate) { + if (filter_rate >= 0.5) { + EXPECT_EQ(false, slide_window.dynamic_disable()); + EXPECT_EQ(0, slide_window.window_cnt_); + EXPECT_EQ(total_count, slide_window.next_check_start_pos_); + } else { + EXPECT_EQ(true, slide_window.dynamic_disable()); + } + EXPECT_EQ(0, slide_window.partial_filter_count_); + EXPECT_EQ(0, slide_window.partial_total_count_); + } else { + EXPECT_EQ(acc_partial_filter_count, slide_window.partial_filter_count_); + EXPECT_EQ(acc_partial_total_count, slide_window.partial_total_count_); + } + } +} + +TEST_F(AdapitveSlideWindowTest, test_adaptive_slide_window) +{ + slide_window_.start_to_work(); + for (int64_t i = 0; i < 100; ++i) { mock_filter(slide_window_, total_count_); } +} + +} // namespace sql +} // namespace oceanbase +int main(int argc, char **argv) +{ + OB_LOGGER.set_log_level("TRACE"); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/unittest/sql/engine/test_op_engine.cpp b/unittest/sql/engine/test_op_engine.cpp index c542a84ea4..5edb34ef64 100644 --- a/unittest/sql/engine/test_op_engine.cpp +++ b/unittest/sql/engine/test_op_engine.cpp @@ -293,7 +293,7 @@ ObOperator *TestOpEngine::subtitude_table_scan_to_fake(ObOperator *root) } int TestOpEngine::get_tested_op_from_string(const std::string &sql, bool vector_2, ObOperator *&op, - ObExecutor &executor) + ObExecutor &executor, bool use_old_ctx) { int ret = OB_SUCCESS; ObStmt *stmt = NULL; @@ -302,7 +302,7 @@ int TestOpEngine::get_tested_op_from_string(const std::string &sql, bool vector_ ObArenaAllocator *p_alloc = NULL; ObExecContext *p_exec_ctx = NULL; - if (vector_2) { + if (vector_2 || !use_old_ctx) { p_alloc = &vec_2_alloc_; p_exec_ctx = &vec_2_exec_ctx_; } else { diff --git a/unittest/sql/engine/test_op_engine.h b/unittest/sql/engine/test_op_engine.h index e0f6107a05..bc2c88e622 100644 --- a/unittest/sql/engine/test_op_engine.h +++ b/unittest/sql/engine/test_op_engine.h @@ -79,7 +79,8 @@ protected: int test_phy_plan(ObPhysicalPlan &plan); ObOperator *subtitude_table_scan_to_fake(ObOperator *root); - int get_tested_op_from_string(const std::string &sql, bool vector_2, ObOperator *&op, ObExecutor &executor); + int get_tested_op_from_string(const std::string &sql, bool vector_2, ObOperator *&op, + ObExecutor &executor, bool use_old_ctx = false); int generate_physical_plan(ObLogPlan *log_plan, ObPhysicalPlan &phy_plan, ObExecContext &exec_ctx, bool enable_rich_format); int open_and_get_op(ObExecContext &exec_ctx, ObExecutor &ob_exe, ObPhysicalPlan &phy_plan, ObOperator *&root);